-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-11630] Wait for the termination of all running Tasks when shutting down TaskExecutor #9072
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 |
|---|---|---|
|
|
@@ -132,6 +132,8 @@ | |
| import java.util.Set; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.CompletionException; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.TimeoutException; | ||
| import java.util.function.BiConsumer; | ||
| import java.util.stream.Collectors; | ||
|
|
@@ -182,6 +184,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { | |
| /** The kvState registration service in the task manager. */ | ||
| private final KvStateService kvStateService; | ||
|
|
||
| private final TaskCompletionTracker taskCompletionTracker; | ||
|
|
||
| // --------- job manager connections ----------- | ||
|
|
||
| private final Map<ResourceID, JobManagerConnection> jobManagerConnections; | ||
|
|
@@ -273,6 +277,7 @@ public TaskExecutor( | |
| this.currentRegistrationTimeoutId = null; | ||
|
|
||
| this.stackTraceSampleService = new StackTraceSampleService(rpcService.getScheduledExecutor()); | ||
| this.taskCompletionTracker = new TaskCompletionTracker(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -333,31 +338,46 @@ private void handleStartTaskExecutorServicesException(Exception e) throws Except | |
| public CompletableFuture<Void> onStop() { | ||
| log.info("Stopping TaskExecutor {}.", getAddress()); | ||
|
|
||
| Throwable throwable = null; | ||
| Throwable jobManagerDisconnectThrowable = null; | ||
|
|
||
| if (resourceManagerConnection != null) { | ||
| resourceManagerConnection.close(); | ||
| } | ||
|
|
||
| FlinkException cause = new FlinkException("The TaskExecutor is shutting down."); | ||
| for (JobManagerConnection jobManagerConnection : jobManagerConnections.values()) { | ||
| try { | ||
| disassociateFromJobManager(jobManagerConnection, new FlinkException("The TaskExecutor is shutting down.")); | ||
| disassociateFromJobManager(jobManagerConnection, cause); | ||
| } catch (Throwable t) { | ||
| throwable = ExceptionUtils.firstOrSuppressed(t, throwable); | ||
| jobManagerDisconnectThrowable = ExceptionUtils.firstOrSuppressed(t, jobManagerDisconnectThrowable); | ||
| } | ||
| } | ||
|
|
||
| try { | ||
| stopTaskExecutorServices(); | ||
| } catch (Exception e) { | ||
| throwable = ExceptionUtils.firstOrSuppressed(e, throwable); | ||
| final Throwable throwableBeforeTasksCompletion = jobManagerDisconnectThrowable; | ||
|
|
||
| return FutureUtils | ||
| .runAfterwards( | ||
| taskCompletionTracker.failIncompleteTasksAndGetTerminationFuture(), | ||
| this::stopTaskExecutorServices) | ||
| .handle((ignored, throwable) -> { | ||
| handleOnStopException(throwableBeforeTasksCompletion, throwable); | ||
| return null; | ||
| }); | ||
| } | ||
|
|
||
| private void handleOnStopException(Throwable throwableBeforeTasksCompletion, Throwable throwableAfterTasksCompletion) { | ||
| final Throwable throwable; | ||
|
|
||
| if (throwableBeforeTasksCompletion != null) { | ||
| throwable = ExceptionUtils.firstOrSuppressed(throwableBeforeTasksCompletion, throwableAfterTasksCompletion); | ||
| } else { | ||
| throwable = throwableAfterTasksCompletion; | ||
| } | ||
|
|
||
| if (throwable != null) { | ||
| return FutureUtils.completedExceptionally(new FlinkException("Error while shutting the TaskExecutor down.", throwable)); | ||
| throw new CompletionException(new FlinkException("Error while shutting the TaskExecutor down.", throwable)); | ||
| } else { | ||
| log.info("Stopped TaskExecutor {}.", getAddress()); | ||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -596,6 +616,7 @@ public CompletableFuture<Acknowledge> submitTask( | |
|
|
||
| if (taskAdded) { | ||
| task.startTaskThread(); | ||
| taskCompletionTracker.trackTaskCompletion(task); | ||
|
|
||
| setupResultPartitionBookkeeping(tdd, task.getTerminationFuture()); | ||
| return CompletableFuture.completedFuture(Acknowledge.get()); | ||
|
|
@@ -1826,4 +1847,30 @@ public SlotReport retrievePayload(ResourceID resourceID) { | |
| return taskSlotTable.createSlotReport(getResourceID()); | ||
| } | ||
| } | ||
|
|
||
| private static class TaskCompletionTracker { | ||
| private final Map<ExecutionAttemptID, Task> incompleteTasks; | ||
|
|
||
| private TaskCompletionTracker() { | ||
| incompleteTasks = new ConcurrentHashMap<>(8); | ||
| } | ||
|
|
||
| void trackTaskCompletion(Task task) { | ||
| incompleteTasks.put(task.getExecutionId(), task); | ||
| task.getTerminationFuture().thenRun(() -> incompleteTasks.remove(task.getExecutionId())); | ||
|
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. could we also rely on
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 is also my concern. We already have
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. Andrey explained to me that we call
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. Yes, Andrey has explained to me before. |
||
| } | ||
|
|
||
| CompletableFuture<Void> failIncompleteTasksAndGetTerminationFuture() { | ||
| FlinkException cause = new FlinkException("The TaskExecutor is shutting down."); | ||
| return FutureUtils.waitForAll( | ||
| incompleteTasks | ||
| .values() | ||
| .stream() | ||
| .map(task -> { | ||
| task.failExternally(cause); | ||
| return task.getTerminationFuture(); | ||
| }) | ||
| .collect(Collectors.toList())); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,157 @@ | ||
| /* | ||
| * 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.deployment; | ||
|
|
||
| import org.apache.flink.api.common.JobID; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; | ||
| import org.apache.flink.runtime.clusterframework.types.AllocationID; | ||
| import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor.MaybeOffloaded; | ||
| import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor.NonOffloaded; | ||
| import org.apache.flink.runtime.executiongraph.DummyJobInformation; | ||
| import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; | ||
| import org.apache.flink.runtime.executiongraph.JobInformation; | ||
| import org.apache.flink.runtime.executiongraph.TaskInformation; | ||
| import org.apache.flink.runtime.jobgraph.JobVertexID; | ||
| import org.apache.flink.util.SerializedValue; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Collection; | ||
| import java.util.Collections; | ||
|
|
||
| /** | ||
| * Builder for {@link TaskDeploymentDescriptor}. | ||
| */ | ||
| public class TaskDeploymentDescriptorBuilder { | ||
| private JobID jobId; | ||
| private MaybeOffloaded<JobInformation> serializedJobInformation; | ||
| private MaybeOffloaded<TaskInformation> serializedTaskInformation; | ||
| private ExecutionAttemptID executionId; | ||
| private AllocationID allocationId; | ||
| private int subtaskIndex; | ||
| private int attemptNumber; | ||
| private Collection<ResultPartitionDeploymentDescriptor> producedPartitions; | ||
| private Collection<InputGateDeploymentDescriptor> inputGates; | ||
| private int targetSlotNumber; | ||
|
|
||
| @Nullable | ||
| private JobManagerTaskRestore taskRestore; | ||
|
|
||
| private TaskDeploymentDescriptorBuilder(JobID jobId, String invokableClassName) throws IOException { | ||
| TaskInformation taskInformation = new TaskInformation( | ||
| new JobVertexID(), | ||
| "test task", | ||
| 1, | ||
| 1, | ||
| invokableClassName, | ||
| new Configuration()); | ||
|
|
||
| this.jobId = jobId; | ||
| this.serializedJobInformation = | ||
| new NonOffloaded<>(new SerializedValue<>(new DummyJobInformation(jobId, "DummyJob"))); | ||
| this.serializedTaskInformation = new NonOffloaded<>(new SerializedValue<>(taskInformation)); | ||
| this.executionId = new ExecutionAttemptID(); | ||
| this.allocationId = new AllocationID(); | ||
| this.subtaskIndex = 0; | ||
| this.attemptNumber = 0; | ||
| this.producedPartitions = Collections.emptyList(); | ||
| this.inputGates = Collections.emptyList(); | ||
| this.targetSlotNumber = 0; | ||
| this.taskRestore = null; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setSerializedJobInformation( | ||
| MaybeOffloaded<JobInformation> serializedJobInformation) { | ||
| this.serializedJobInformation = serializedJobInformation; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setSerializedTaskInformation( | ||
| MaybeOffloaded<TaskInformation> serializedTaskInformation) { | ||
| this.serializedTaskInformation = serializedTaskInformation; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setJobId(JobID jobId) { | ||
| this.jobId = jobId; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setExecutionId(ExecutionAttemptID executionId) { | ||
| this.executionId = executionId; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setAllocationId(AllocationID allocationId) { | ||
| this.allocationId = allocationId; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setSubtaskIndex(int subtaskIndex) { | ||
| this.subtaskIndex = subtaskIndex; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setAttemptNumber(int attemptNumber) { | ||
| this.attemptNumber = attemptNumber; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setProducedPartitions( | ||
| Collection<ResultPartitionDeploymentDescriptor> producedPartitions) { | ||
| this.producedPartitions = producedPartitions; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setInputGates(Collection<InputGateDeploymentDescriptor> inputGates) { | ||
| this.inputGates = inputGates; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setTargetSlotNumber(int targetSlotNumber) { | ||
| this.targetSlotNumber = targetSlotNumber; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptorBuilder setTaskRestore(@Nullable JobManagerTaskRestore taskRestore) { | ||
| this.taskRestore = taskRestore; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskDeploymentDescriptor build() { | ||
| return new TaskDeploymentDescriptor( | ||
| jobId, | ||
| serializedJobInformation, | ||
| serializedTaskInformation, | ||
| executionId, | ||
| allocationId, | ||
| subtaskIndex, | ||
| attemptNumber, | ||
| targetSlotNumber, | ||
| taskRestore, | ||
| producedPartitions, | ||
| inputGates); | ||
| } | ||
|
|
||
| public static TaskDeploymentDescriptorBuilder newBuilder(JobID jobId, Class<?> invokableClass) throws IOException { | ||
| return new TaskDeploymentDescriptorBuilder(jobId, invokableClass.getName()); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need a
ConcurrentHashMap? Where does the concurrency come from?