Myasuka commented on a change in pull request #16341: URL: https://github.com/apache/flink/pull/16341#discussion_r662767236
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageFactory.java ########## @@ -0,0 +1,35 @@ +/* + * 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.changelog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; + +/** + * A factory for {@link StateChangelogStorage}. Please use {@link StateChangelogStorageLoader} to + * create {@link StateChangelogStorage}. + */ +@Internal Review comment: I think `StateChangelogStorageFactory` might not only be internal and could be extended by users. I think this class should be `PublicEvolving` just as `StateBackendFactory`. ########## 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) { Review comment: I think we could change `StateChangelogStorage` to `StateChangelogStorage<?>` to remove the `SuppressWarnings({"rawtypes"})`. ########## File path: flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java ########## @@ -114,6 +114,25 @@ + "if no value set (job or cluster), the change log will not be " + "enabled."); + /** Which storage to use to store state changelog. */ + @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS) + public static final ConfigOption<String> STATE_CHANGE_LOG_STORAGE = + ConfigOptions.key("state.backend.changelog.storage") + .stringType() + .defaultValue("memory") Review comment: We can change this to `InMemoryStateChangelogStorageFactory.getIdentifier()` ########## 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: I wonder shall we must have the lock here? ########## 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(); + try { + StateChangelogStorageFactory factory = + STATE_CHANGELOG_STORAGE_FACTORIES.get(identifier); Review comment: We'd better to make processing the identifier without case sensitivity. -- 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