Zakelly commented on code in PR #22890:
URL: https://github.com/apache/flink/pull/22890#discussion_r1251641209


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorFileMergingManager.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.state;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager;
+import 
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.ShutdownHookUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * There is one {@link FileMergingSnapshotManager} for each job per task 
manager. This class holds
+ * the all {@link FileMergingSnapshotManager} objects for a task executor 
(manager).
+ */
+public class TaskExecutorFileMergingManager {
+    /** Logger for this class. */
+    private static final Logger LOG = 
LoggerFactory.getLogger(TaskExecutorFileMergingManager.class);
+
+    /**
+     * This map holds all FileMergingSnapshotManager for tasks running on the 
task manager /
+     * executor that own the instance of this.
+     */
+    @GuardedBy("lock")
+    private final Map<JobID, FileMergingSnapshotManager> 
fileMergingSnapshotManagerByJobId;
+
+    @GuardedBy("lock")
+    private boolean closed;
+
+    private final Object lock = new Object();
+
+    /** shutdown hook for this manager. */
+    private final Thread shutdownHook;
+
+    public TaskExecutorFileMergingManager() {
+        this.fileMergingSnapshotManagerByJobId = new HashMap<>();
+        this.closed = false;
+        this.shutdownHook =
+                ShutdownHookUtil.addShutdownHook(this::shutdown, 
getClass().getSimpleName(), LOG);
+    }
+
+    public FileMergingSnapshotManager fileMergingSnapshotManagerForJob(

Review Comment:
   Maybe some JavaDocs for `fileMergingSnapshotManagerForJob` and 
`releaseMergingSnapshotManagerForJob`, explaining when they will be called.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java:
##########
@@ -510,6 +517,38 @@ protected StreamTask(
         }
     }
 
+    private CheckpointStorageAccess applyFileMergingCheckpoint(
+            CheckpointStorageAccess checkpointStorageAccess,
+            FileMergingSnapshotManager fileMergingSnapshotManager) {
+        if (fileMergingSnapshotManager == null
+                || fileMergingSnapshotManager instanceof 
NonFileMergingSnapshotManager) {
+            return checkpointStorageAccess;
+        }
+
+        if (!(checkpointStorageAccess instanceof FsCheckpointStorageAccess)) {
+            LOG.warn(
+                    "Unsupported checkpoint storage access for file merging: 
{}. "
+                            + "Falling back to original checkpoint storage 
access.",
+                    checkpointStorageAccess.getClass());
+            return checkpointStorageAccess;
+        }
+
+        try {
+            FsMergingCheckpointStorageAccess mergingCheckpointStorageAccess =
+                    ((FsCheckpointStorageAccess) checkpointStorageAccess)
+                            
.toMergingStorageAccess(fileMergingSnapshotManager, environment);

Review Comment:
   I'm not sure whether it is better to make `toMergingStorageAccess` an 
interface of `CheckpointStorageAccess` and remove the logic of type checking 
here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/NonFileMergingSnapshotManager.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.checkpoint.filemerging;
+
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointedStateScope;
+
+import java.io.IOException;
+
+/** A dummy FileMergingSnapshotManager which does not perform file merging. */
+public class NonFileMergingSnapshotManager implements 
FileMergingSnapshotManager {

Review Comment:
   Seems this class is just a place holder. I checked variables and instances 
and it does not have to be `@Nonnull` for instances of 
`FileMergingSnapshotManager` everywhere. I suggest returning `null` in 
`TaskStateManager.getFileMergingSnapshotManager()` to indicate not doing 
file-merging, and remove this class. WDYT?



-- 
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]

Reply via email to