Interface FileMergingSnapshotManager
-
- All Superinterfaces:
AutoCloseable
,Closeable
- All Known Implementing Classes:
AcrossCheckpointFileMergingSnapshotManager
,FileMergingSnapshotManagerBase
,WithinCheckpointFileMergingSnapshotManager
public interface FileMergingSnapshotManager extends Closeable
FileMergingSnapshotManager provides an interface to manage files and meta information for checkpoint files with merging checkpoint files enabled. It manages the files for ONE single task in TM, including all subtasks of this single task that is running in this TM. There is one FileMergingSnapshotManager for each job per task manager.
-
-
Nested Class Summary
Nested Classes Modifier and Type Interface Description static class
FileMergingSnapshotManager.SpaceStat
Space usage statistics of a managed directory.static class
FileMergingSnapshotManager.SubtaskKey
A key identifies a subtask.
-
Method Summary
-
-
-
Method Detail
-
initFileSystem
void initFileSystem(FileSystem fileSystem, Path checkpointBaseDir, Path sharedStateDir, Path taskOwnedStateDir, int writeBufferSize) throws IllegalArgumentException
Initialize the file system, recording the checkpoint path the manager should work with.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 byFsCheckpointStorageAccess
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.- Parameters:
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.- Throws:
IllegalArgumentException
- thrown if these three paths are not deterministic across calls.
-
registerSubtaskForSharedStates
void registerSubtaskForSharedStates(FileMergingSnapshotManager.SubtaskKey subtaskKey)
Register a subtask and create the managed directory for shared states.- Parameters:
subtaskKey
- the subtask key identifying a subtask.- See Also:
for layout information.
-
unregisterSubtask
void unregisterSubtask(FileMergingSnapshotManager.SubtaskKey subtaskKey)
Unregister a subtask.- Parameters:
subtaskKey
- the subtask key identifying a subtask.
-
createCheckpointStateOutputStream
FileMergingCheckpointStateOutputStream createCheckpointStateOutputStream(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId, CheckpointedStateScope scope)
Create a newFileMergingCheckpointStateOutputStream
. 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.- Parameters:
subtaskKey
- The subtask key identifying the subtask.checkpointId
- ID of the checkpoint.scope
- The state's scope, whether it is exclusive or shared.- Returns:
- An output stream that writes state for the given checkpoint.
-
getManagedDir
Path getManagedDir(FileMergingSnapshotManager.SubtaskKey subtaskKey, CheckpointedStateScope scope)
Get the managed directory of the file-merging snapshot manager, created ininitFileSystem(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)
orregisterSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey)
.- Parameters:
subtaskKey
- the subtask key identifying the subtask.scope
- the checkpoint scope.- Returns:
- the managed directory for one subtask in specified checkpoint scope.
-
getManagedDirStateHandle
DirectoryStreamStateHandle getManagedDirStateHandle(FileMergingSnapshotManager.SubtaskKey subtaskKey, CheckpointedStateScope scope)
Get theDirectoryStreamStateHandle
of the managed directory, created ininitFileSystem(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)
orregisterSubtaskForSharedStates(org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey)
.- Parameters:
subtaskKey
- the subtask key identifying the subtask.scope
- the checkpoint scope.- Returns:
- the
DirectoryStreamStateHandle
for one subtask in specified checkpoint scope.
-
notifyCheckpointStart
void notifyCheckpointStart(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId)
Notifies the manager that the checkpoint with the givencheckpointId
has been started.- Parameters:
subtaskKey
- the subtask key identifying the subtask.checkpointId
- The ID of the checkpoint that has been started.
-
notifyCheckpointComplete
void notifyCheckpointComplete(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
Notifies the manager that the checkpoint with the givencheckpointId
completed and was committed.- Parameters:
subtaskKey
- the subtask key identifying the subtask.checkpointId
- The ID of the checkpoint that has been completed.- Throws:
Exception
- thrown if anything goes wrong with the listener.
-
notifyCheckpointAborted
void notifyCheckpointAborted(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
This method is called as a notification once a distributed checkpoint has been aborted.- Parameters:
subtaskKey
- the subtask key identifying the subtask.checkpointId
- The ID of the checkpoint that has been completed.- Throws:
Exception
- thrown if anything goes wrong with the listener.
-
notifyCheckpointSubsumed
void notifyCheckpointSubsumed(FileMergingSnapshotManager.SubtaskKey subtaskKey, long checkpointId) throws Exception
This method is called as a notification once a distributed checkpoint has been subsumed.- Parameters:
subtaskKey
- the subtask key identifying the subtask.checkpointId
- The ID of the checkpoint that has been completed.- Throws:
Exception
- thrown if anything goes wrong with the listener.
-
couldReusePreviousStateHandle
boolean couldReusePreviousStateHandle(StreamStateHandle stateHandle)
Check whether previous state handles could further be reused considering the space amplification.- Parameters:
stateHandle
- the handle to be reused.
-
reusePreviousStateHandle
void reusePreviousStateHandle(long checkpointId, Collection<? extends StreamStateHandle> stateHandles)
A callback method which is called when previous state handles are reused by following checkpoint(s).- Parameters:
checkpointId
- the checkpoint that reuses the handles.stateHandles
- the handles to be reused.
-
restoreStateHandles
void restoreStateHandles(long checkpointId, FileMergingSnapshotManager.SubtaskKey subtaskKey, Stream<SegmentFileStateHandle> stateHandles)
Restore and re-register the SegmentFileStateHandles into FileMergingSnapshotManager.- Parameters:
checkpointId
- the restored checkpoint id.subtaskKey
- the subtask key identifying the subtask.stateHandles
- the restored segment file handles.
-
isFileMergingHandle
static boolean isFileMergingHandle(StreamStateHandle handle)
-
-