You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2018/07/23 22:06:03 UTC

[06/11] flink git commit: [FLINK-9908][scheduling] Do not cancel individual scheduling future

[FLINK-9908][scheduling] Do not cancel individual scheduling future

Since the individual scheduling futures contain logic to release the slot if it cannot
be assigned to the Execution, we must not cancel them. Otherwise we might risk that
slots are not returned to the SlotPool leaving it in an inconsistent state.

This closes #6383.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c897471d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c897471d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c897471d

Branch: refs/heads/master
Commit: c897471ddbe25ed85b9f9ec6b15fdbd11cf0cec5
Parents: 19d39ec
Author: Till Rohrmann <tr...@apache.org>
Authored: Sun Jul 22 20:05:05 2018 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Jul 24 00:05:39 2018 +0200

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionGraph.java  |   8 --
 .../ExecutionGraphSchedulingTest.java           | 111 ++++++++++++++++++-
 .../executiongraph/TestingSlotProvider.java     |  82 ++++++++++++++
 .../jobmanager/slots/TestingSlotOwner.java      |   2 +-
 4 files changed, 193 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c897471d/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 22c11ef..acb1e16 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -989,14 +989,6 @@ public class ExecutionGraph implements AccessExecutionGraph {
 					throw new CompletionException(resultThrowable);
 				});
 
-		currentSchedulingFuture.whenComplete(
-			(Void ignored, Throwable throwable) -> {
-				if (throwable instanceof CancellationException) {
-					// cancel the individual allocation futures
-					allAllocationsFuture.cancel(false);
-				}
-			});
-
 		return currentSchedulingFuture;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c897471d/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
index e43137b..6092f52 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.executiongraph;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.blob.VoidBlobWriter;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
@@ -39,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
 import org.apache.flink.runtime.jobmanager.slots.DummySlotOwner;
 import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
 import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner;
@@ -46,6 +48,7 @@ import org.apache.flink.runtime.jobmaster.LogicalSlot;
 import org.apache.flink.runtime.jobmaster.SlotOwner;
 import org.apache.flink.runtime.jobmaster.SlotRequestId;
 import org.apache.flink.runtime.jobmaster.TestingLogicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
 import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
@@ -62,14 +65,19 @@ import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import java.net.InetAddress;
+import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -504,6 +512,89 @@ public class ExecutionGraphSchedulingTest extends TestLogger {
 		assertThat(executionGraph.getTerminationFuture().get(), is(JobStatus.FAILED));
 	}
 
+	/**
+	 * Tests that all slots are being returned to the {@link SlotOwner} if the
+	 * {@link ExecutionGraph} is being cancelled. See FLINK-9908
+	 */
+	@Test
+	public void testCancellationOfIncompleteScheduling() throws Exception {
+		final int parallelism = 10;
+
+		final JobVertex jobVertex = new JobVertex("Test job vertex");
+		jobVertex.setInvokableClass(NoOpInvokable.class);
+		jobVertex.setParallelism(parallelism);
+
+		final JobGraph jobGraph = new JobGraph(jobVertex);
+		jobGraph.setAllowQueuedScheduling(true);
+		jobGraph.setScheduleMode(ScheduleMode.EAGER);
+
+		final TestingSlotOwner slotOwner = new TestingSlotOwner();
+		final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway();
+
+		final ConcurrentMap<SlotRequestId, Integer> slotRequestIds = new ConcurrentHashMap<>(parallelism);
+		final CountDownLatch requestedSlotsLatch = new CountDownLatch(parallelism);
+
+		final TestingSlotProvider slotProvider = new TestingSlotProvider(
+			(SlotRequestId slotRequestId) -> {
+				slotRequestIds.put(slotRequestId, 1);
+				requestedSlotsLatch.countDown();
+				return new CompletableFuture<>();
+			});
+
+
+		final ExecutionGraph executionGraph = createExecutionGraph(jobGraph, slotProvider);
+
+		executionGraph.scheduleForExecution();
+
+		// wait until we have requested all slots
+		requestedSlotsLatch.await();
+
+		final Set<SlotRequestId> slotRequestIdsToReturn = ConcurrentHashMap.newKeySet(slotRequestIds.size());
+		slotRequestIdsToReturn.addAll(slotRequestIds.keySet());
+		final CountDownLatch countDownLatch = new CountDownLatch(slotRequestIds.size());
+
+		slotOwner.setReturnAllocatedSlotConsumer(logicalSlot -> {
+			slotRequestIdsToReturn.remove(logicalSlot.getSlotRequestId());
+			countDownLatch.countDown();
+		});
+		slotProvider.setSlotCanceller(slotRequestId -> {
+			slotRequestIdsToReturn.remove(slotRequestId);
+			countDownLatch.countDown();
+		});
+
+		final OneShotLatch slotRequestsBeingFulfilled = new OneShotLatch();
+
+		// start completing the slot requests asynchronously
+		executor.execute(
+			() -> {
+				slotRequestsBeingFulfilled.trigger();
+
+				for (SlotRequestId slotRequestId : slotRequestIds.keySet()) {
+					final SingleLogicalSlot singleLogicalSlot = createSingleLogicalSlot(slotOwner, taskManagerGateway, slotRequestId);
+					slotProvider.complete(slotRequestId, singleLogicalSlot);
+				}
+			});
+
+		// make sure that we complete cancellations of deployed tasks
+		taskManagerGateway.setCancelConsumer(
+			(ExecutionAttemptID executionAttemptId) -> {
+				final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttemptId);
+
+				// if the execution was cancelled in state SCHEDULING, then it might already have been removed
+				if (execution != null) {
+					execution.cancelingComplete();
+				}
+			}
+		);
+
+		slotRequestsBeingFulfilled.await();
+
+		executionGraph.cancel();
+
+		countDownLatch.await();
+		assertThat(slotRequestIdsToReturn, is(empty()));
+	}
+
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
@@ -548,11 +639,29 @@ public class ExecutionGraphSchedulingTest extends TestLogger {
 		return new SimpleSlot(slot, slotOwner, 0);
 	}
 
+	@Nonnull
+	private SingleLogicalSlot createSingleLogicalSlot(TestingSlotOwner slotOwner, SimpleAckingTaskManagerGateway taskManagerGateway, SlotRequestId slotRequestId) {
+		TaskManagerLocation location = new TaskManagerLocation(
+			ResourceID.generate(), InetAddress.getLoopbackAddress(), 12345);
+
+		SimpleSlotContext slotContext = new SimpleSlotContext(
+			new AllocationID(),
+			location,
+			0,
+			taskManagerGateway);
+
+		return new SingleLogicalSlot(
+			slotRequestId,
+			slotContext,
+			null,
+			Locality.LOCAL,
+			slotOwner);
+	}
+
 	private static TaskManagerGateway createTaskManager() {
 		TaskManagerGateway tm = mock(TaskManagerGateway.class);
 		when(tm.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)))
 				.thenReturn(CompletableFuture.completedFuture(Acknowledge.get()));
-
 		return tm;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c897471d/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java
new file mode 100644
index 0000000..ed8fe13
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java
@@ -0,0 +1,82 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * {@link SlotProvider} implementation for testing purposes.
+ */
+final class TestingSlotProvider implements SlotProvider {
+
+	private final ConcurrentMap<SlotRequestId, CompletableFuture<LogicalSlot>> slotFutures;
+
+	private final Function<SlotRequestId, CompletableFuture<LogicalSlot>> slotFutureCreator;
+
+	private volatile Consumer<SlotRequestId> slotCanceller = ignored -> {};
+
+	TestingSlotProvider(Function<SlotRequestId, CompletableFuture<LogicalSlot>> slotFutureCreator) {
+		this.slotFutureCreator = slotFutureCreator;
+		this.slotFutures = new ConcurrentHashMap<>(4);
+	}
+
+	public void setSlotCanceller(Consumer<SlotRequestId> slotCanceller) {
+		this.slotCanceller = slotCanceller;
+	}
+
+	@Override
+	public CompletableFuture<LogicalSlot> allocateSlot(SlotRequestId slotRequestId, ScheduledUnit task, boolean allowQueued, SlotProfile slotProfile, Time timeout) {
+		Preconditions.checkState(!slotFutures.containsKey(slotRequestId));
+		final CompletableFuture<LogicalSlot> slotFuture = slotFutureCreator.apply(slotRequestId);
+
+		slotFutures.put(slotRequestId, slotFuture);
+
+		return slotFuture;
+	}
+
+	@Override
+	public CompletableFuture<Acknowledge> cancelSlotRequest(SlotRequestId slotRequestId, @Nullable SlotSharingGroupId slotSharingGroupId, Throwable cause) {
+		final CompletableFuture<LogicalSlot> slotFuture = slotFutures.remove(slotRequestId);
+		slotFuture.cancel(false);
+
+		slotCanceller.accept(slotRequestId);
+
+		return CompletableFuture.completedFuture(Acknowledge.get());
+	}
+
+	public void complete(SlotRequestId slotRequestId, LogicalSlot logicalSlot) {
+		slotFutures.get(slotRequestId).complete(logicalSlot);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c897471d/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java
index 727c0b5..b922204 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java
@@ -37,7 +37,7 @@ public class TestingSlotOwner implements SlotOwner {
 
 	@Override
 	public CompletableFuture<Boolean> returnAllocatedSlot(LogicalSlot logicalSlot) {
-		final Consumer<LogicalSlot> currentReturnAllocatedSlotConsumer = this.returnAllocatedSlotConsumer;
+		final Consumer<LogicalSlot> currentReturnAllocatedSlotConsumer = returnAllocatedSlotConsumer;
 
 		if (currentReturnAllocatedSlotConsumer != null) {
 			currentReturnAllocatedSlotConsumer.accept(logicalSlot);