AlexYinHan commented on code in PR #22890:
URL: https://github.com/apache/flink/pull/22890#discussion_r1255575331
##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageWorkerView.java:
##########
@@ -75,4 +77,14 @@ CheckpointStreamFactory resolveCheckpointStorageLocation(
* @return A toolset for additional operations for state owned by tasks.
*/
CheckpointStateToolset createTaskOwnedCheckpointStateToolset();
+
+ /**
+ * Return {@link
org.apache.flink.runtime.state.filesystem.FsMergingCheckpointStorageAccess} if
+ * file merging is enabled. Otherwise, return itself.
+ */
+ default CheckpointStorageWorkerView toFileMergingStorage(
Review Comment:
It might be a little weird to have the interface moved here in
```CheckpointStorageWorkerView```. The interface does not make much sense other
than in ```FsCheckpointStorageAccess``` (like in
```MemoryBackendCheckpointStorageAccess```).
Maybe we can at least make it clearer in the comments here: 'Return xxx if
file merging is enabled and supported by this ```CheckpointStorageWorkerView```'
##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageAccess.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager;
+import
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** An implementation of file merging checkpoint storage to file systems. */
+public class FsMergingCheckpointStorageAccess extends
FsCheckpointStorageAccess {
+
+ /** FileMergingSnapshotManager manage files and meta information for
checkpoint. */
Review Comment:
manage -> manages
checkpoint -> checkpoints
##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorFileMergingManager.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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.
Review Comment:
I guess 'that own the instance of this' is not necessary in this comment. It
makes the sentence a bit convoluted.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorFileMergingManager.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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).
Review Comment:
'the all' -> 'all the'
##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageAccess.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager;
+import
org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** An implementation of file merging checkpoint storage to file systems. */
+public class FsMergingCheckpointStorageAccess extends
FsCheckpointStorageAccess {
+
+ /** FileMergingSnapshotManager manage files and meta information for
checkpoint. */
+ private final FileMergingSnapshotManager fileMergingSnapshotManager;
+
+ /** The identity of subtask. */
+ private final FileMergingSnapshotManager.SubtaskKey subtaskKey;
+
+ public FsMergingCheckpointStorageAccess(
+ FileSystem fs,
+ Path checkpointBaseDirectory,
+ Path sharedStateDirectory,
+ Path taskOwnedStateDirectory,
+ @Nullable Path defaultSavepointDirectory,
+ JobID jobId,
+ int fileSizeThreshold,
+ int writeBufferSize,
+ FileMergingSnapshotManager fileMergingSnapshotManager,
+ Environment environment)
+ throws IOException {
+ super(
+ fs,
+ checkpointBaseDirectory,
+ defaultSavepointDirectory,
+ jobId,
+ fileSizeThreshold,
+ writeBufferSize);
+ this.fileMergingSnapshotManager = fileMergingSnapshotManager;
+ this.subtaskKey =
+ new SubtaskKey(
+
OperatorID.fromJobVertexID(environment.getJobVertexId()),
+ environment.getTaskInfo());
+ initSegmentSnapshotManager(
Review Comment:
I guess the function name could be a little misleading here.
'SegmentSnapshotManager' should be 'FileMergingSnapshotManager' and what the
function does is not all about the initialization of the SnapshotManager.
Maybe we can simply put these two lines of code at the end of the
constructor, instead of wrapping them in a private function.
--
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]