Zakelly commented on a change in pull request #16341:
URL: https://github.com/apache/flink/pull/16341#discussion_r665011824



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorage;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorageLoader;
+import org.apache.flink.util.ShutdownHookUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This class holds the all {@link StateChangelogStorage} objects for a task 
executor (manager). No
+ * thread-safe.
+ */
+public class TaskExecutorStateChangelogStoragesManager {
+
+    /** Logger for this class. */
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(TaskExecutorStateChangelogStoragesManager.class);
+
+    /**
+     * This map holds all state changelog storages for tasks running on the 
task manager / executor
+     * that own the instance of this. Maps from job id to all the subtask's 
state changelog
+     * storages.
+     */
+    private final Map<JobID, StateChangelogStorage<?>> 
changelogStoragesByJobId;
+
+    private boolean closed;
+
+    /** shutdown hook for this manager. */
+    private final Thread shutdownHook;
+
+    public TaskExecutorStateChangelogStoragesManager() {
+        this.changelogStoragesByJobId = new HashMap<>();
+        this.closed = false;
+
+        // register a shutdown hook
+        this.shutdownHook =
+                ShutdownHookUtil.addShutdownHook(this::shutdown, 
getClass().getSimpleName(), LOG);
+    }
+
+    public StateChangelogStorage<?> stateChangelogStorageForJob(
+            @Nonnull JobID jobId, Configuration configuration) {
+        if (closed) {
+            throw new IllegalStateException(
+                    "TaskExecutorStateChangelogStoragesManager is already 
closed and cannot "
+                            + "register a new StateChangelogStorage.");
+        }
+
+        StateChangelogStorage<?> stateChangelogStorage = 
changelogStoragesByJobId.get(jobId);
+
+        if (stateChangelogStorage == null) {
+            stateChangelogStorage = 
StateChangelogStorageLoader.load(configuration);

Review comment:
       Yeah. I think it's better to change the value type for map 
```changelogStoragesByJobId``` as ```Reference<StateChangelogStorage<?>>```, 
then we could record the previous loaded ```null``` value. WDYT?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorage;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorageLoader;
+import org.apache.flink.util.ShutdownHookUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This class holds the all {@link StateChangelogStorage} objects for a task 
executor (manager). No
+ * thread-safe.
+ */
+public class TaskExecutorStateChangelogStoragesManager {
+
+    /** Logger for this class. */
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(TaskExecutorStateChangelogStoragesManager.class);
+
+    /**
+     * This map holds all state changelog storages for tasks running on the 
task manager / executor
+     * that own the instance of this. Maps from job id to all the subtask's 
state changelog
+     * storages.
+     */
+    private final Map<JobID, StateChangelogStorage<?>> 
changelogStoragesByJobId;
+
+    private boolean closed;
+
+    /** shutdown hook for this manager. */
+    private final Thread shutdownHook;
+
+    public TaskExecutorStateChangelogStoragesManager() {
+        this.changelogStoragesByJobId = new HashMap<>();
+        this.closed = false;
+
+        // register a shutdown hook
+        this.shutdownHook =
+                ShutdownHookUtil.addShutdownHook(this::shutdown, 
getClass().getSimpleName(), LOG);
+    }
+
+    public StateChangelogStorage<?> stateChangelogStorageForJob(
+            @Nonnull JobID jobId, Configuration configuration) {
+        if (closed) {
+            throw new IllegalStateException(
+                    "TaskExecutorStateChangelogStoragesManager is already 
closed and cannot "
+                            + "register a new StateChangelogStorage.");
+        }
+
+        StateChangelogStorage<?> stateChangelogStorage = 
changelogStoragesByJobId.get(jobId);
+
+        if (stateChangelogStorage == null) {
+            stateChangelogStorage = 
StateChangelogStorageLoader.load(configuration);
+
+            changelogStoragesByJobId.put(jobId, stateChangelogStorage);
+
+            if (stateChangelogStorage != null) {
+                LOG.debug(
+                        "Registered new state changelog storage for job {} : 
{}.",
+                        jobId,
+                        stateChangelogStorage);
+            } else {
+                LOG.info(
+                        "Try to registered new state changelog storage for job 
{},"
+                                + "but result is null.",
+                        jobId);

Review comment:
       I think it's OK for some redundant log when error occurs. 😄 




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to