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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java
##########
@@ -18,26 +18,81 @@
 package org.apache.flink.runtime.state.changelog;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.plugin.PluginManager;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.ServiceLoader;
+import java.util.concurrent.locks.ReentrantLock;
 
 import static 
org.apache.flink.shaded.guava18.com.google.common.collect.Iterators.concat;
 
 /** A thin wrapper around {@link PluginManager} to load {@link 
StateChangelogStorage}. */
 @Internal
 public class StateChangelogStorageLoader {
-    private final PluginManager pluginManager;
 
-    public StateChangelogStorageLoader(PluginManager pluginManager) {
-        this.pluginManager = pluginManager;
+    private static final Logger LOG = 
LoggerFactory.getLogger(StateChangelogStorageLoader.class);
+
+    /** Object used to protect calls to specific methods. */
+    private static final ReentrantLock LOCK = new ReentrantLock(true);
+
+    /**
+     * Mapping of state changelog storage identifier to the corresponding 
storage factories,
+     * populated in {@link 
StateChangelogStorageLoader#initialize(PluginManager)}.
+     */
+    private static final HashMap<String, StateChangelogStorageFactory>
+            STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
+
+    static {
+        // Guarantee to trigger once.
+        initialize(null);
+    }
+
+    public static void initialize(PluginManager pluginManager) {
+        LOCK.lock();
+        try {
+            STATE_CHANGELOG_STORAGE_FACTORIES.clear();
+            Iterator<StateChangelogStorageFactory> iterator =
+                    pluginManager == null
+                            ? 
ServiceLoader.load(StateChangelogStorageFactory.class).iterator()
+                            : concat(
+                                    
pluginManager.load(StateChangelogStorageFactory.class),
+                                    
ServiceLoader.load(StateChangelogStorageFactory.class)
+                                            .iterator());
+            iterator.forEachRemaining(
+                    factory ->
+                            STATE_CHANGELOG_STORAGE_FACTORIES.putIfAbsent(
+                                    factory.getIdentifier(), factory));
+            LOG.info(
+                    "StateChangelogStorageLoader initialized with shortcut 
names {{}}.",
+                    String.join(",", 
STATE_CHANGELOG_STORAGE_FACTORIES.keySet()));
+        } finally {
+            LOCK.unlock();
+        }
     }
 
     @SuppressWarnings({"rawtypes"})
-    public Iterator<StateChangelogStorage> load() {
-        return concat(
-                pluginManager.load(StateChangelogStorage.class),
-                ServiceLoader.load(StateChangelogStorage.class).iterator());
+    public static StateChangelogStorage load(Configuration configuration) {
+        final String identifier =
+                
configuration.getString(CheckpointingOptions.STATE_CHANGE_LOG_STORAGE);
+        LOCK.lock();

Review comment:
       If we really want to make the code stable (although I don't think this 
problem existed currently), I think we should at least make the code logic 
clean instead of just adding a lock here.




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