rkhachatryan commented on a change in pull request #16341: URL: https://github.com/apache/flink/pull/16341#discussion_r663722747
########## 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: Fair enough. ########## 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: Sorry for bringing this up again, but I think it's actually better to **have** this lock :) The code is correct without it, but there is an implicit contract that `initialize()` and `load()` are called from the same thread, which is a bit fragile. WDYT? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java ########## @@ -0,0 +1,157 @@ +/* + * 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 javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; + +/** This class holds the all {@link StateChangelogStorage} objects for a task executor (manager). */ +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. + */ + @GuardedBy("lock") + private final Map<JobID, StateChangelogStorage<?>> changelogStoragesByJobId; + + /** Guarding lock for changelogStoragesByJobId and closed-flag. */ + private final Object lock; + + @GuardedBy("lock") + private boolean closed; + + /** shutdown hook for this manager. */ + private final Thread shutdownHook; + + public TaskExecutorStateChangelogStoragesManager() { + this.changelogStoragesByJobId = new HashMap<>(); + this.lock = new Object(); + this.closed = false; + + // register a shutdown hook + this.shutdownHook = + ShutdownHookUtil.addShutdownHook(this::shutdown, getClass().getSimpleName(), LOG); Review comment: This registers a hook for the JVM shutdown, right? With many short-lived jobs, this cleanup can be delayed and can cause resource leak. Is it possible to close the storage when the last task (or, ideally, slot) of the corresponding job is released on this TM? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java ########## @@ -0,0 +1,157 @@ +/* + * 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 javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; + +/** This class holds the all {@link StateChangelogStorage} objects for a task executor (manager). */ +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. + */ + @GuardedBy("lock") + private final Map<JobID, StateChangelogStorage<?>> changelogStoragesByJobId; + + /** Guarding lock for changelogStoragesByJobId and closed-flag. */ + private final Object lock; + + @GuardedBy("lock") + private boolean closed; + + /** shutdown hook for this manager. */ + private final Thread shutdownHook; + + public TaskExecutorStateChangelogStoragesManager() { + this.changelogStoragesByJobId = new HashMap<>(); + this.lock = new Object(); + this.closed = false; + + // register a shutdown hook + this.shutdownHook = + ShutdownHookUtil.addShutdownHook(this::shutdown, getClass().getSimpleName(), LOG); + } + + public StateChangelogStorage<?> stateChangelogStorageForJob( + @Nonnull JobID jobId, Configuration configuration) { + synchronized (lock) { + if (closed) { Review comment: Why do we need to synhronize here? Is it because of `closed` flag? I'm asking because the task thread isn't yet created when this method is called; In `TaskExecutor.submitTask` (which calls this method), job and taskSlot tables are accessed without any synchronization. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java ########## @@ -669,13 +677,18 @@ private void stopTaskExecutorServices() throws Exception { taskInformation.getJobVertexId(), tdd.getSubtaskIndex()); + final StateChangelogStorage<?> changelogStorage = + changelogStoragesManager.stateChangelogStorageForJob( + jobId, jobInformation.getJobConfiguration()); Review comment: Just a question: do we need to merge job configuration with task configuration here? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java ########## @@ -0,0 +1,157 @@ +/* + * 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 javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; + +/** This class holds the all {@link StateChangelogStorage} objects for a task executor (manager). */ +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. + */ + @GuardedBy("lock") + private final Map<JobID, StateChangelogStorage<?>> changelogStoragesByJobId; + + /** Guarding lock for changelogStoragesByJobId and closed-flag. */ + private final Object lock; + + @GuardedBy("lock") + private boolean closed; + + /** shutdown hook for this manager. */ + private final Thread shutdownHook; + + public TaskExecutorStateChangelogStoragesManager() { + this.changelogStoragesByJobId = new HashMap<>(); + this.lock = new Object(); + this.closed = false; + + // register a shutdown hook + this.shutdownHook = + ShutdownHookUtil.addShutdownHook(this::shutdown, getClass().getSimpleName(), LOG); Review comment: Ah, I didn't notice that call. Thanks for the clarification. ########## 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 agree, I'd even throw an exception if `load` is called without `initialize`; or if `initialize` called twice. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java ########## @@ -669,13 +677,18 @@ private void stopTaskExecutorServices() throws Exception { taskInformation.getJobVertexId(), tdd.getSubtaskIndex()); + final StateChangelogStorage<?> changelogStorage = + changelogStoragesManager.stateChangelogStorageForJob( + jobId, jobInformation.getJobConfiguration()); Review comment: I think we are currently missing the API to set `STATE_CHANGE_LOG_STORAGE` from the user program (similar to `enableChangelogStateBackend()` for `ENABLE_STATE_CHANGE_LOG`). If we add it (and implement similarly) then the value will come in `taskInformation.getTaskConfiguration()`. And the value configured in yaml will come in `taskManagerConfiguration.getConfiguration()` as you pointed out. So `taskInformation.getTaskConfiguration()` should override `taskManagerConfiguration.getConfiguration()` - if the API is added. From the consistency point of view, the API should be added in this PR (something like `enableChangelogStateBackend("memory")`). ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java ########## @@ -669,13 +677,18 @@ private void stopTaskExecutorServices() throws Exception { taskInformation.getJobVertexId(), tdd.getSubtaskIndex()); + final StateChangelogStorage<?> changelogStorage = + changelogStoragesManager.stateChangelogStorageForJob( + jobId, jobInformation.getJobConfiguration()); Review comment: I think we are currently missing the API to set `STATE_CHANGE_LOG_STORAGE` from the user program (similar to `enableChangelogStateBackend()` for `ENABLE_STATE_CHANGE_LOG`). If we add it (and implement similarly) then the value will come in `taskInformation.getTaskConfiguration()`. And the value configured in yaml will come in `taskManagerConfiguration.getConfiguration()` as you pointed out. So `taskInformation.getTaskConfiguration()` should override `taskManagerConfiguration.getConfiguration()` - if the API is added. From the consistency point of view, the API should be added in this PR (something like `enableChangelogStateBackend("memory")`). But I'm also fine with adding it later. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java ########## @@ -28,16 +34,51 @@ /** 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); + + /** + * 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); } - @SuppressWarnings({"rawtypes"}) - public Iterator<StateChangelogStorage> load() { - return concat( - pluginManager.load(StateChangelogStorage.class), - ServiceLoader.load(StateChangelogStorage.class).iterator()); + public static void initialize(PluginManager pluginManager) { + 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().toLowerCase(), factory)); Review comment: Should we warn or fail if we have duplicated factories? (given that we don't allow to initialize twice) ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java ########## @@ -669,13 +677,18 @@ private void stopTaskExecutorServices() throws Exception { taskInformation.getJobVertexId(), tdd.getSubtaskIndex()); + final StateChangelogStorage<?> changelogStorage = + changelogStoragesManager.stateChangelogStorageForJob( + jobId, jobInformation.getJobConfiguration()); Review comment: Sure, I meant a separate (overloaded) method, so users would have both. Sorry for the confusion. -- 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