-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-12667][runtime] Add JobID to TaskExecutorGateway#releasePartitions #8630
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -640,7 +640,7 @@ public CompletableFuture<Acknowledge> updatePartitions( | |
| } | ||
|
|
||
| @Override | ||
| public void releasePartitions(Collection<ResultPartitionID> partitionIds) { | ||
| public void releasePartitions(JobID jobId, Collection<ResultPartitionID> partitionIds) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. a bit weird that we will leave
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it's not gonna be big, but this part is simply already done. |
||
| try { | ||
| shuffleEnvironment.releasePartitions(partitionIds); | ||
| } catch (Throwable t) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,12 +19,16 @@ | |
| package org.apache.flink.runtime.executiongraph; | ||
|
|
||
| import org.apache.flink.api.common.JobID; | ||
| import org.apache.flink.api.java.tuple.Tuple2; | ||
| import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; | ||
| import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; | ||
| import org.apache.flink.runtime.execution.ExecutionState; | ||
| import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; | ||
| import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; | ||
| import org.apache.flink.runtime.instance.SimpleSlot; | ||
| import org.apache.flink.runtime.io.network.partition.ResultPartitionID; | ||
| import org.apache.flink.runtime.io.network.partition.ResultPartitionType; | ||
| import org.apache.flink.runtime.jobgraph.DistributionPattern; | ||
| import org.apache.flink.runtime.jobgraph.JobVertex; | ||
| import org.apache.flink.runtime.jobgraph.JobVertexID; | ||
| import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; | ||
|
|
@@ -54,6 +58,7 @@ | |
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.function.Consumer; | ||
|
|
||
| import static org.hamcrest.Matchers.containsInAnyOrder; | ||
| import static org.hamcrest.Matchers.equalTo; | ||
|
|
@@ -64,6 +69,7 @@ | |
| import static org.hamcrest.Matchers.sameInstance; | ||
| import static org.junit.Assert.assertEquals; | ||
| import static org.junit.Assert.assertFalse; | ||
| import static org.junit.Assert.assertNotNull; | ||
| import static org.junit.Assert.assertThat; | ||
| import static org.junit.Assert.assertTrue; | ||
|
|
||
|
|
@@ -293,6 +299,82 @@ public void testSlotAllocationCancellationWhenExecutionCancelled() throws Except | |
| assertThat(canceledSlotRequests, equalTo(slotRequests)); | ||
| } | ||
|
|
||
| /** | ||
| * Tests that the partitions are released in case of an execution cancellation after the execution is already finished. | ||
| */ | ||
| @Test | ||
| public void testPartitionReleaseOnCancelingAfterBeingFinished() throws Exception { | ||
| testPartitionReleaseAfterFinished(Execution::cancel); | ||
| } | ||
|
|
||
| /** | ||
| * Tests that the partitions are released in case of an execution suspension after the execution is already finished. | ||
| */ | ||
| @Test | ||
| public void testPartitionReleaseOnSuspendingAfterBeingFinished() throws Exception { | ||
| testPartitionReleaseAfterFinished(Execution::suspend); | ||
| } | ||
|
|
||
| private void testPartitionReleaseAfterFinished(Consumer<Execution> postFinishedExecutionAction) throws Exception { | ||
| final Tuple2<JobID, Collection<ResultPartitionID>> releasedPartitions = Tuple2.of(null, null); | ||
| final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); | ||
| taskManagerGateway.setReleasePartitionsConsumer(releasedPartitions::setFields); | ||
|
|
||
| final JobVertex producerVertex = createNoOpJobVertex(); | ||
| final JobVertex consumerVertex = createNoOpJobVertex(); | ||
| consumerVertex.connectNewDataSetAsInput(producerVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); | ||
|
|
||
| final SimpleSlot slot = new SimpleSlot( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we could provide an utility for creating
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. such a utility already exists in #createProgrammedSlotProvider, will update the test to use that instead |
||
| new SingleSlotTestingSlotOwner(), | ||
| new LocalTaskManagerLocation(), | ||
| 0, | ||
| taskManagerGateway); | ||
|
|
||
| final ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(1); | ||
| slotProvider.addSlot(producerVertex.getID(), 0, CompletableFuture.completedFuture(slot)); | ||
|
|
||
| ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( | ||
| new JobID(), | ||
| slotProvider, | ||
| new NoRestartStrategy(), | ||
| producerVertex, | ||
| consumerVertex); | ||
|
|
||
| executionGraph.start(TestingComponentMainThreadExecutorServiceAdapter.forMainThread()); | ||
|
|
||
| ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(producerVertex.getID()); | ||
| ExecutionVertex executionVertex = executionJobVertex.getTaskVertices()[0]; | ||
|
|
||
| final Execution execution = executionVertex.getCurrentExecutionAttempt(); | ||
|
|
||
| execution.allocateResourcesForExecution( | ||
| slotProvider, | ||
| false, | ||
| LocationPreferenceConstraint.ALL, | ||
| Collections.emptySet(), | ||
| TestingUtils.infiniteTime()); | ||
|
|
||
| execution.deploy(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we could further provide an utility for creating Execution like below: Then we could get Execution from future, and further get ExecutionVertex and ExecutionGraph from Execution. Then this helper could be reused for many existing tests.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree that we could de-duplicate some code here. I'm concerned that we're baking in a few assumptions (vertex parallelism should be 1, first vertex in the array is special), and there a few subtle differences in tests that where I don't know whether they are significant or not. (For example, |
||
| execution.switchToRunning(); | ||
|
|
||
| // simulate a case where a cancel/suspend call is too slow and the task is already finished | ||
| // in this case we have to explicitly release the finished partition | ||
| // if the task were canceled properly the TM would release the partition automatically | ||
| execution.markFinished(); | ||
| postFinishedExecutionAction.accept(execution); | ||
|
|
||
| assertEquals(executionGraph.getJobID(), releasedPartitions.f0); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe use |
||
| assertEquals(executionVertex.getProducedPartitions().size(), releasedPartitions.f1.size()); | ||
| for (ResultPartitionID partitionId : releasedPartitions.f1) { | ||
| // ensure all IDs of released partitions are actually valid | ||
| IntermediateResultPartition intermediateResultPartition = executionVertex | ||
| .getProducedPartitions() | ||
| .get(partitionId.getPartitionId()); | ||
| assertNotNull(intermediateResultPartition); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this assert might be not necessary
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this check is done to ensure that the ids of all released partitions are actually valid. without it the test would pass even if completely random partitions were passed to the task executor. I'll add a comment to clarify this; I had to think for a bit myself as to what we're checking here. |
||
| assertEquals(execution.getAttemptId(), partitionId.getProducerId()); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Tests that all preferred locations are calculated. | ||
| */ | ||
|
|
@@ -417,7 +499,7 @@ public void testTaskRestoreStateIsNulledAfterDeployment() throws Exception { | |
| slotProvider, | ||
| new NoRestartStrategy(), | ||
| jobVertex); | ||
|
|
||
| ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(jobVertexId); | ||
|
|
||
| ExecutionVertex executionVertex = executionJobVertex.getTaskVertices()[0]; | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.