rkhachatryan commented on a change in pull request #16341: URL: https://github.com/apache/flink/pull/16341#discussion_r663746163
########## 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? -- 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