Zakelly commented on code in PR #24640:
URL: https://github.com/apache/flink/pull/24640#discussion_r1574334314
##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java:
##########
@@ -830,6 +832,8 @@ public CompletableFuture<Acknowledge> submitTask(
channelStateExecutorFactoryManager.getOrCreateExecutorFactory(jobId));
taskMetricGroup.gauge(MetricNames.IS_BACK_PRESSURED,
task::isBackPressured);
+ registerTaskRestoreInfoToFileMergingManager(
+ fileMergingSnapshotManager, task.getTaskInfo(),
taskRestore);
Review Comment:
I suggest move this part into
`StreamTaskStateInitializerImpl#streamOperatorStateContext`, where the state
backends are performing restoration.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java:
##########
@@ -575,8 +576,51 @@ private void createManagedDirectory(Path managedPath) {
@Override
public void close() throws IOException {}
+ // ------------------------------------------------------------------------
+ // restore
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void restoreStateHandles(
+ long checkpointId, SubtaskKey subtaskKey,
Stream<SegmentFileStateHandle> stateHandles) {
+
+ Set<LogicalFile> uploadedLogicalFiles;
+ synchronized (lock) {
+ uploadedLogicalFiles =
+ uploadedStates.computeIfAbsent(checkpointId, id -> new
HashSet<>());
+ }
+
+ stateHandles.forEach(
+ fileHandle -> {
+ PhysicalFile physicalFile =
+ new PhysicalFile(
+ null,
+ fileHandle.getFilePath(),
+ physicalFileDeleter,
Review Comment:
So for now will this `FileMergingSnapshotManager` delete underlying files
after rescaling?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java:
##########
@@ -575,8 +576,51 @@ private void createManagedDirectory(Path managedPath) {
@Override
public void close() throws IOException {}
+ // ------------------------------------------------------------------------
+ // restore
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void restoreStateHandles(
+ long checkpointId, SubtaskKey subtaskKey,
Stream<SegmentFileStateHandle> stateHandles) {
+
+ Set<LogicalFile> uploadedLogicalFiles;
+ synchronized (lock) {
+ uploadedLogicalFiles =
+ uploadedStates.computeIfAbsent(checkpointId, id -> new
HashSet<>());
+ }
+
+ stateHandles.forEach(
+ fileHandle -> {
+ PhysicalFile physicalFile =
+ new PhysicalFile(
+ null,
+ fileHandle.getFilePath(),
+ physicalFileDeleter,
+ fileHandle.getScope());
Review Comment:
Should we have shared `PhysicalFile` instance for multiple `LogicalFile`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]