rkhachatryan commented on a change in pull request #16341:
URL: https://github.com/apache/flink/pull/16341#discussion_r664586527
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java
##########
@@ -28,16 +34,63 @@
/** 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 -> {
+ String identifier = factory.getIdentifier().toLowerCase();
+ StateChangelogStorageFactory prev =
+ STATE_CHANGELOG_STORAGE_FACTORIES.get(identifier);
+ if (prev == null) {
+ STATE_CHANGELOG_STORAGE_FACTORIES.put(identifier,
factory);
+ } else {
+ LOG.warn(
+ "StateChangelogStorageLoader found duplicated
factory,"
+ + " using {} instead of {} for name
{}.",
+ prev.getClass().getName(),
+ factory.getClass().getName(),
+ identifier);
+ }
+ });
+ LOG.info(
+ "StateChangelogStorageLoader initialized with shortcut names
{{}}.",
+ String.join(",", STATE_CHANGELOG_STORAGE_FACTORIES.keySet()));
+ }
+
+ public static StateChangelogStorage<?> load(Configuration configuration) {
+ final String identifier =
+ configuration
+
.getString(CheckpointingOptions.STATE_CHANGE_LOG_STORAGE)
+ .toLowerCase();
+
+ StateChangelogStorageFactory factory =
STATE_CHANGELOG_STORAGE_FACTORIES.get(identifier);
+ if (factory == null) {
+ LOG.warn("Cannot find a factory for changelog storage with name
'{}'.", identifier);
+ return null;
Review comment:
1. If the storage isn't needed then `STATE_CHANGE_LOG_STORAGE` is empty,
right? Why don't we return `Optional.empty` in that case? (`null` isn't very
obvious, and should be marked `Nullable` if we keep it)
2. If an invalid identifier is provided, why don't we signal it explicitly
(by throwing an exception for example)? I think returning `null` (or empty
Optional) can hide misconfiguration issues.
##########
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:
(please see the comment above about returning `null` first)
If we stick with nulls then we'll be trying to load the storage each time,
right? (even if it's not needed)
##########
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:
(please see the comment above about misconfiguration first)
I think if we allow nullable storage then we'll get a warning about it twice
(here and in `StateChangelogStorageLoader`), plus an exception from backend or
writer.
--
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]