public interface FileMergingSnapshotManager extends Closeable
| Modifier and Type | Interface and Description |
|---|---|
static class |
FileMergingSnapshotManager.SpaceStat
Space usage statistics of a managed directory.
|
static class |
FileMergingSnapshotManager.SubtaskKey
A key identifies a subtask.
|
| Modifier and Type | Method and Description |
|---|---|
boolean |
couldReusePreviousStateHandle(StreamStateHandle stateHandle)
Check whether previous state handles could further be reused considering the space
amplification.
|
FileMergingCheckpointStateOutputStream |
createCheckpointStateOutputStream(FileMergingSnapshotManager.SubtaskKey subtaskKey,
long checkpointId,
CheckpointedStateScope scope)
Create a new
FileMergingCheckpointStateOutputStream. |
org.apache.flink.core.fs.Path |
getManagedDir(FileMergingSnapshotManager.SubtaskKey subtaskKey,
CheckpointedStateScope scope)
Get the managed directory of the file-merging snapshot manager, created in
initFileSystem(org.apache.flink.core.fs.FileSystem, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, int) or registerSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey). |
DirectoryStreamStateHandle |
getManagedDirStateHandle(FileMergingSnapshotManager.SubtaskKey subtaskKey,
CheckpointedStateScope scope)
Get the
DirectoryStreamStateHandle of the managed directory, created in initFileSystem(org.apache.flink.core.fs.FileSystem, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, int) or registerSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey). |
void |
initFileSystem(org.apache.flink.core.fs.FileSystem fileSystem,
org.apache.flink.core.fs.Path checkpointBaseDir,
org.apache.flink.core.fs.Path sharedStateDir,
org.apache.flink.core.fs.Path taskOwnedStateDir,
int writeBufferSize)
Initialize the file system, recording the checkpoint path the manager should work with.
|
static boolean |
isFileMergingHandle(StreamStateHandle handle) |
void |
notifyCheckpointAborted(FileMergingSnapshotManager.SubtaskKey subtaskKey,
long checkpointId)
This method is called as a notification once a distributed checkpoint has been aborted.
|
void |
notifyCheckpointComplete(FileMergingSnapshotManager.SubtaskKey subtaskKey,
long checkpointId)
Notifies the manager that the checkpoint with the given
checkpointId completed and
was committed. |
void |
notifyCheckpointStart(FileMergingSnapshotManager.SubtaskKey subtaskKey,
long checkpointId)
Notifies the manager that the checkpoint with the given
checkpointId has been
started. |
void |
notifyCheckpointSubsumed(FileMergingSnapshotManager.SubtaskKey subtaskKey,
long checkpointId)
This method is called as a notification once a distributed checkpoint has been subsumed.
|
void |
registerSubtaskForSharedStates(FileMergingSnapshotManager.SubtaskKey subtaskKey)
Register a subtask and create the managed directory for shared states.
|
void |
restoreStateHandles(long checkpointId,
FileMergingSnapshotManager.SubtaskKey subtaskKey,
Stream<SegmentFileStateHandle> stateHandles)
Restore and re-register the SegmentFileStateHandles into FileMergingSnapshotManager.
|
void |
reusePreviousStateHandle(long checkpointId,
Collection<? extends StreamStateHandle> stateHandles)
A callback method which is called when previous state handles are reused by following
checkpoint(s).
|
void |
unregisterSubtask(FileMergingSnapshotManager.SubtaskKey subtaskKey)
Unregister a subtask.
|
void initFileSystem(org.apache.flink.core.fs.FileSystem fileSystem,
org.apache.flink.core.fs.Path checkpointBaseDir,
org.apache.flink.core.fs.Path sharedStateDir,
org.apache.flink.core.fs.Path taskOwnedStateDir,
int writeBufferSize)
throws IllegalArgumentException
The layout of checkpoint directory:
/user-defined-checkpoint-dir
/{job-id} (checkpointBaseDir)
|
+ --shared/
|
+ --subtask-1/
+ -- merged shared state files
+ --subtask-2/
+ -- merged shared state files
+ --taskowned/
+ -- merged private state files
+ --chk-1/
+ --chk-2/
+ --chk-3/
The reason why initializing directories in this method instead of the constructor is that
the FileMergingSnapshotManager itself belongs to the TaskStateManager, which is
initialized when receiving a task, while the base directories for checkpoint are created by
FsCheckpointStorageAccess when the state backend initializes per subtask. After the
checkpoint directories are initialized, the managed subdirectories are initialized here.
Note: This method may be called several times, the implementation should ensure
idempotency, and throw IllegalArgumentException when any of the path in params change
across function calls.
fileSystem - The filesystem to write to.checkpointBaseDir - The base directory for checkpoints.sharedStateDir - The directory for shared checkpoint data.taskOwnedStateDir - The name of the directory for state not owned/released by the
master, but by the TaskManagers.writeBufferSize - The buffer size for writing files to the file system.IllegalArgumentException - thrown if these three paths are not deterministic across
calls.void registerSubtaskForSharedStates(FileMergingSnapshotManager.SubtaskKey subtaskKey)
subtaskKey - the subtask key identifying a subtask.for layout information.void unregisterSubtask(FileMergingSnapshotManager.SubtaskKey subtaskKey)
subtaskKey - the subtask key identifying a subtask.FileMergingCheckpointStateOutputStream createCheckpointStateOutputStream(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId, CheckpointedStateScope scope)
FileMergingCheckpointStateOutputStream. According to the file merging
strategy, the streams returned by multiple calls to this function may share the same
underlying physical file, and each stream writes to a segment of the physical file.subtaskKey - The subtask key identifying the subtask.checkpointId - ID of the checkpoint.scope - The state's scope, whether it is exclusive or shared.org.apache.flink.core.fs.Path getManagedDir(FileMergingSnapshotManager.SubtaskKey subtaskKey, CheckpointedStateScope scope)
initFileSystem(org.apache.flink.core.fs.FileSystem, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, int) or registerSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey).subtaskKey - the subtask key identifying the subtask.scope - the checkpoint scope.DirectoryStreamStateHandle getManagedDirStateHandle(FileMergingSnapshotManager.SubtaskKey subtaskKey, CheckpointedStateScope scope)
DirectoryStreamStateHandle of the managed directory, created in initFileSystem(org.apache.flink.core.fs.FileSystem, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path, int) or registerSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey).subtaskKey - the subtask key identifying the subtask.scope - the checkpoint scope.DirectoryStreamStateHandle for one subtask in specified checkpoint scope.void notifyCheckpointStart(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId)
checkpointId has been
started.subtaskKey - the subtask key identifying the subtask.checkpointId - The ID of the checkpoint that has been started.void notifyCheckpointComplete(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
checkpointId completed and
was committed.subtaskKey - the subtask key identifying the subtask.checkpointId - The ID of the checkpoint that has been completed.Exception - thrown if anything goes wrong with the listener.void notifyCheckpointAborted(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
subtaskKey - the subtask key identifying the subtask.checkpointId - The ID of the checkpoint that has been completed.Exception - thrown if anything goes wrong with the listener.void notifyCheckpointSubsumed(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
subtaskKey - the subtask key identifying the subtask.checkpointId - The ID of the checkpoint that has been completed.Exception - thrown if anything goes wrong with the listener.boolean couldReusePreviousStateHandle(StreamStateHandle stateHandle)
stateHandle - the handle to be reused.void reusePreviousStateHandle(long checkpointId,
Collection<? extends StreamStateHandle> stateHandles)
checkpointId - the checkpoint that reuses the handles.stateHandles - the handles to be reused.void restoreStateHandles(long checkpointId,
FileMergingSnapshotManager.SubtaskKey subtaskKey,
Stream<SegmentFileStateHandle> stateHandles)
checkpointId - the restored checkpoint id.subtaskKey - the subtask key identifying the subtask.stateHandles - the restored segment file handles.static boolean isFileMergingHandle(StreamStateHandle handle)
Copyright © 2014–2025 The Apache Software Foundation. All rights reserved.