-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38804][runtime] draft fix #27373
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 |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import org.apache.flink.api.common.TaskInfoImpl; | ||
| import org.apache.flink.api.common.cache.DistributedCache; | ||
| import org.apache.flink.api.common.time.Deadline; | ||
| import org.apache.flink.configuration.CheckpointingOptions; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.configuration.TaskManagerOptions; | ||
| import org.apache.flink.core.fs.AutoCloseableRegistry; | ||
|
|
@@ -42,6 +43,9 @@ | |
| import org.apache.flink.runtime.checkpoint.CheckpointOptions; | ||
| import org.apache.flink.runtime.checkpoint.CheckpointStoreUtil; | ||
| import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestExecutorFactory; | ||
| import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; | ||
| import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl; | ||
| import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager; | ||
| import org.apache.flink.runtime.clusterframework.types.AllocationID; | ||
| import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; | ||
| import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; | ||
|
|
@@ -78,11 +82,19 @@ | |
| import org.apache.flink.runtime.query.TaskKvStateRegistry; | ||
| import org.apache.flink.runtime.shuffle.ShuffleEnvironment; | ||
| import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; | ||
| import org.apache.flink.runtime.state.CheckpointStorage; | ||
| import org.apache.flink.runtime.state.CheckpointStorageAccess; | ||
| import org.apache.flink.runtime.state.CheckpointStorageLoader; | ||
| import org.apache.flink.runtime.state.CheckpointStorageWorkerView; | ||
| import org.apache.flink.runtime.state.StateBackend; | ||
| import org.apache.flink.runtime.state.StateBackendLoader; | ||
| import org.apache.flink.runtime.state.TaskStateManager; | ||
| import org.apache.flink.runtime.state.filesystem.FsMergingCheckpointStorageAccess; | ||
| import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; | ||
| import org.apache.flink.runtime.taskexecutor.KvStateService; | ||
| import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; | ||
| import org.apache.flink.runtime.taskexecutor.slot.TaskSlotPayload; | ||
| import org.apache.flink.streaming.api.graph.StreamConfig; | ||
| import org.apache.flink.types.Either; | ||
| import org.apache.flink.util.ExceptionUtils; | ||
| import org.apache.flink.util.FatalExitExceptionHandler; | ||
|
|
@@ -97,6 +109,7 @@ | |
| import org.apache.flink.util.WrappingRuntimeException; | ||
| import org.apache.flink.util.concurrent.FutureUtils; | ||
| import org.apache.flink.util.function.RunnableWithException; | ||
| import org.apache.flink.util.function.SupplierWithException; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
@@ -311,6 +324,8 @@ public class Task | |
| */ | ||
| private UserCodeClassLoader userCodeClassLoader; | ||
|
|
||
| private volatile ChannelStateWriter channelStateWriter; | ||
|
|
||
| /** | ||
| * <b>IMPORTANT:</b> This constructor may not start any work that would need to be undone in the | ||
| * case of a failing task deployment. | ||
|
|
@@ -698,6 +713,7 @@ private void doRun() { | |
| TaskKvStateRegistry kvStateRegistry = | ||
| kvStateService.createKvStateTaskRegistry(jobId, getJobVertexId()); | ||
|
|
||
| final StateBackend stateBackend = createStateBackend(); | ||
| Environment env = | ||
| new RuntimeEnvironment( | ||
| jobId, | ||
|
|
@@ -730,7 +746,21 @@ private void doRun() { | |
| this, | ||
| externalResourceInfoProvider, | ||
| channelStateExecutorFactory, | ||
| taskManagerActions); | ||
| taskManagerActions, | ||
| stateBackend); | ||
|
|
||
| final CheckpointStorage checkpointStorage = createCheckpointStorage(stateBackend); | ||
| CheckpointStorageAccess checkpointStorageAccess = | ||
| checkpointStorage.createCheckpointStorage(this.jobId); | ||
| checkpointStorageAccess = | ||
| tryApplyFileMergingCheckpoint( | ||
| checkpointStorageAccess, | ||
| this.taskStateManager.getFileMergingSnapshotManager(), | ||
| postFailureCleanUpRegistry, | ||
| env); | ||
| this.channelStateWriter = createChannelStateWriter(checkpointStorage, checkpointStorageAccess); | ||
| env.setCheckpointStorageAccess(checkpointStorageAccess); | ||
| env.setChannelStateWriter(channelStateWriter); | ||
|
Member
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.
From the PR, the writer is set in Task. IIUC, we do not need to change lot of code if it is still set in StreamTask, right? If so, the |
||
|
|
||
| // Make sure the user code classloader is accessible thread-locally. | ||
| // We are setting the correct context class loader before instantiating the invokable | ||
|
|
@@ -1011,6 +1041,14 @@ private void releaseResources() { | |
| } | ||
| closeAllResultPartitions(); | ||
| closeAllInputGates(); | ||
| if (this.channelStateWriter != null) { | ||
| try { | ||
| this.channelStateWriter.close(); | ||
| } catch (IOException e) { | ||
| LOG.error( | ||
| "Failed to close channelStateWriter for task {}.", taskNameWithSubtask, e); | ||
| } | ||
| } | ||
|
|
||
| try { | ||
| taskStateManager.close(); | ||
|
|
@@ -1558,6 +1596,110 @@ public void deliverOperatorEvent(OperatorID operator, SerializedValue<OperatorEv | |
| } | ||
| } | ||
|
|
||
|
|
||
| // ------------------------------------------------------------------------ | ||
| // State backend | ||
| // ------------------------------------------------------------------------ | ||
|
|
||
| private StateBackend createStateBackend() throws Exception { | ||
| final StreamConfig configuration = new StreamConfig(this.taskConfiguration); | ||
| final StateBackend fromApplication = | ||
| configuration.getStateBackend(userCodeClassLoader.asClassLoader()); | ||
|
|
||
| return StateBackendLoader.fromApplicationOrConfigOrDefault( | ||
| fromApplication, | ||
| getJobConfiguration(), | ||
| taskManagerConfig.getConfiguration(), | ||
| userCodeClassLoader.asClassLoader(), | ||
| LOG); | ||
| } | ||
|
|
||
| private CheckpointStorage createCheckpointStorage(StateBackend backend) throws Exception { | ||
| final StreamConfig configuration = new StreamConfig(this.taskConfiguration); | ||
| final CheckpointStorage fromApplication = | ||
| configuration.getCheckpointStorage(userCodeClassLoader.asClassLoader()); | ||
| return CheckpointStorageLoader.load( | ||
| fromApplication, | ||
| backend, | ||
| getJobConfiguration(), | ||
| taskManagerConfig.getConfiguration(), | ||
| userCodeClassLoader.asClassLoader(), | ||
| LOG); | ||
| } | ||
|
|
||
|
|
||
| private CheckpointStorageAccess tryApplyFileMergingCheckpoint( | ||
| CheckpointStorageAccess checkpointStorageAccess, | ||
| @Nullable FileMergingSnapshotManager fileMergingSnapshotManager, | ||
| AutoCloseableRegistry cleanUpRegistry, | ||
| Environment environment) { | ||
| if (fileMergingSnapshotManager == null) { | ||
| return checkpointStorageAccess; | ||
| } | ||
| try { | ||
| CheckpointStorageAccess mergingCheckpointStorageAccess = | ||
| (CheckpointStorageAccess) | ||
| checkpointStorageAccess.toFileMergingStorage( | ||
| fileMergingSnapshotManager, environment); | ||
| mergingCheckpointStorageAccess.initializeBaseLocationsForCheckpoint(); | ||
| if (mergingCheckpointStorageAccess instanceof FsMergingCheckpointStorageAccess) { | ||
| cleanUpRegistry.registerCloseable( | ||
| () -> | ||
| ((FsMergingCheckpointStorageAccess) mergingCheckpointStorageAccess) | ||
| .close()); | ||
| } | ||
| return mergingCheckpointStorageAccess; | ||
| } catch (IOException e) { | ||
| LOG.warn( | ||
| "Initiating FsMergingCheckpointStorageAccess failed " | ||
| + "with exception: {}, falling back to original checkpoint storage access {}.", | ||
| e.getMessage(), | ||
| checkpointStorageAccess.getClass(), | ||
| e); | ||
| return checkpointStorageAccess; | ||
| } | ||
| } | ||
|
|
||
| private ChannelStateWriter createChannelStateWriter(CheckpointStorage checkpointStorage, CheckpointStorageAccess checkpointStorageAccess) throws Exception { | ||
| return CheckpointingOptions.isUnalignedCheckpointEnabled(getJobConfiguration()) | ||
| ? this.openChannelStateWriter( | ||
| // Note: don't pass checkpointStorageAccess directly to channel | ||
| // state writer. | ||
| // The fileSystem of checkpointStorageAccess may be an instance | ||
| // of SafetyNetWrapperFileSystem, which close all held streams | ||
| // when thread exits. Channel state writers are invoked in other | ||
| // threads instead of task thread, therefore channel state | ||
| // writer cannot share file streams directly, otherwise | ||
| // conflicts will occur on job exit. | ||
| () -> { | ||
| if (checkpointStorageAccess | ||
| instanceof FsMergingCheckpointStorageAccess) { | ||
| // FsMergingCheckpointStorageAccess using unguarded | ||
| // fileSystem, which can be shared. | ||
| return checkpointStorageAccess; | ||
| } else { | ||
| // Other checkpoint storage access should be lazily | ||
| // initialized to avoid sharing. | ||
| return checkpointStorage.createCheckpointStorage( | ||
| this.jobId); | ||
| } | ||
| }) | ||
| : ChannelStateWriter.NO_OP; | ||
| } | ||
|
|
||
| public ChannelStateWriter openChannelStateWriter(SupplierWithException<CheckpointStorageWorkerView, ? extends IOException> | ||
| checkpointStorageWorkerView) { | ||
| return new ChannelStateWriterImpl( | ||
| this.getJobVertexId(), | ||
| this.taskNameWithSubtask, | ||
| this.taskInfo.getIndexOfThisSubtask(), | ||
| checkpointStorageWorkerView, | ||
| this.channelStateExecutorFactory, | ||
| jobConfiguration.get( | ||
| CheckpointingOptions | ||
| .UNALIGNED_MAX_SUBTASKS_PER_CHANNEL_STATE_FILE)); | ||
| } | ||
|
|
||
| // ------------------------------------------------------------------------ | ||
| // Utilities | ||
| // ------------------------------------------------------------------------ | ||
|
|
||
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.
From the code, stateBackend never be null, it is initialized in constructor instead of setter.