rkhachatryan commented on code in PR #20217: URL: https://github.com/apache/flink/pull/20217#discussion_r937909250
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { + synchronized (registeredHandles) { + checkState(open, "Attempt to register state to closed LocalStateRegistry."); + if (registeredHandles.containsKey(handle)) { + long pre = registeredHandles.get(handle); + if (checkpointID > pre) { + registeredHandles.put(handle, checkpointID); + } + } else { + registeredHandles.put(handle, checkpointID); + } + } + return handle; + } + + public void unRegister(long upTo) { + List<StreamStateHandle> handles = new ArrayList<>(); + synchronized (registeredHandles) { + Iterator<Entry<StreamStateHandle, Long>> iterator = + registeredHandles.entrySet().iterator(); + while (iterator.hasNext()) { + Entry<StreamStateHandle, Long> entry = iterator.next(); + if (entry.getValue() < upTo) { + handles.add(entry.getKey()); + iterator.remove(); + } + } + } + for (StreamStateHandle handle : handles) { + scheduleAsyncDelete(handle); + } + } + + private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) { + if (streamStateHandle != null) { + LOG.trace("Scheduled delete of state handle {}.", streamStateHandle); + Runnable discardRunner = + () -> { + try { + streamStateHandle.discardState(); + } catch (Exception exception) { + LOG.warn( + "A problem occurred during asynchronous disposal of a stream handle {}.", + streamStateHandle); + } + }; + try { + asyncDisposalExecutor.execute(discardRunner); + } catch (RejectedExecutionException ex) { + discardRunner.run(); + } + } + } + + @Override + public void close() { + synchronized (registeredHandles) { + open = false; + } + } Review Comment: As mentioned above, this method should probably discard the remaining state. If not, and if this class will stay per task (likely not); I'd remove `open` because it requires adding `syncrhonized` sections, which IMO doesn't worth enforcing the lifecycle without any cleanup. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { Review Comment: 1. Could you please state the purpose of this class in the javadoc; given that there are `ChangelogTaskLocalStateStore` and `TaskChangelogRegistry` already? I guess it's discarding of local changelog; which can't be discarded by `ChangelogTaskLocalStateStore` because they are not bound to a specific checkpoint. 1. The name `LocalStateRegistry` is quite generic. How about `LocalTaskChangelogRegistry` or `LocalChangelogRegistry`? 1. Does it have to be `public`? If so, please annotate it with `@Internal`. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { + synchronized (registeredHandles) { + checkState(open, "Attempt to register state to closed LocalStateRegistry."); + if (registeredHandles.containsKey(handle)) { + long pre = registeredHandles.get(handle); + if (checkpointID > pre) { + registeredHandles.put(handle, checkpointID); + } + } else { + registeredHandles.put(handle, checkpointID); + } + } + return handle; + } + + public void unRegister(long upTo) { Review Comment: I'd add some javadoc explaining when this method is called. I'd also rename it to reflect what it actually does. WDYT about `discardUpToCheckpoint(long checkpointID)`? ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java: ########## @@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) { } } + public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) { + File outDir = provider.selectAllocationBaseDirectory(0); + if (!outDir.exists() && !outDir.mkdirs()) { + LOG.error( + "Local state base directory does not exist and could not be created: " + + outDir); + } + return new Path(outDir.toURI().toString(), CHECKPOINT_TASK_OWNED_STATE_DIR); + } + + @Override + public void abortCheckpoint(long abortedCheckpointId) { + + LOG.debug( + "Received abort information for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.", + abortedCheckpointId, + jobID, + jobVertexID, + subtaskIndex); + + pruneCheckpoints( + snapshotCheckpointId -> snapshotCheckpointId == abortedCheckpointId, false); + + // Local store only keeps one checkpoint, discard all changelog handle in taskowned + // directory. + // Scenarios: + // cp1: m1 + // confirm cp1, do nothing + // cp2: m1, c1 + // abort cp2, delete m1, c1 + // cp3: m1, c1, c2 + // confirm cp3, do nothing + // -> if failover, restore from local cp3 will fail, because m1 does not exist, c1 may not + // exist either(depend on BatchingStateChangeUploadScheduler). + File[] fileInTaskOwned = + new File(getLocalTaskOwnedDirectory(getLocalRecoveryDirectoryProvider()).toUri()) + .listFiles(); + syncDiscardFileForCollection( + fileInTaskOwned == null ? Collections.emptyList() : Arrays.asList(fileInTaskOwned)); Review Comment: Do we really need this if there's a similar deletion in `dispose()`? I'm concerned that this will delete ALL local state, even the used one. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { + synchronized (registeredHandles) { + checkState(open, "Attempt to register state to closed LocalStateRegistry."); + if (registeredHandles.containsKey(handle)) { + long pre = registeredHandles.get(handle); + if (checkpointID > pre) { + registeredHandles.put(handle, checkpointID); + } + } else { + registeredHandles.put(handle, checkpointID); + } Review Comment: How about simplifying it with: ``` registeredHandles.put( handle, Math.max(checkpointID, registeredHandles.getOrDefault(handle, -1L))); ``` ? ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { + synchronized (registeredHandles) { + checkState(open, "Attempt to register state to closed LocalStateRegistry."); + if (registeredHandles.containsKey(handle)) { + long pre = registeredHandles.get(handle); + if (checkpointID > pre) { + registeredHandles.put(handle, checkpointID); + } + } else { + registeredHandles.put(handle, checkpointID); + } + } + return handle; Review Comment: The returned value is never used. ########## flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java: ########## @@ -346,12 +376,42 @@ private static ChangelogStateHandleStreamImpl buildHandle( tuples.add(Tuple2.of(uploadResult.getStreamStateHandle(), uploadResult.getOffset())); size += uploadResult.getSize(); } - return new ChangelogStateHandleStreamImpl( - tuples, - keyGroupRange, - size, - incrementalSize, - FsStateChangelogStorageFactory.IDENTIFIER); + ChangelogStateHandleStreamImpl jmChangelogStateHandle = + new ChangelogStateHandleStreamImpl( + tuples, + keyGroupRange, + size, + incrementalSize, + FsStateChangelogStorageFactory.IDENTIFIER); + if (localRecoveryConfig.isLocalRecoveryEnabled()) { + size = 0; + List<Tuple2<StreamStateHandle, Long>> localTuples = new ArrayList<>(); + for (UploadResult uploadResult : results.values()) { + if (uploadResult.getLocalStreamHandleStateHandle() != null) { + localTuples.add( + Tuple2.of( + uploadResult.getLocalStreamHandleStateHandle(), + uploadResult.getOffset())); Review Comment: This makes `FsStateChangelogWriter` be aware that local and remote handles use the same offset. But the decision to reuse the offset is made in `AbstractStateChangeFsUploader.uploadInternal`. So I'd introduce a separate field for local offset in `UploadTasksResult` and `UploadResult` and pass the same offset there. That would make it more clear and easier to change. WDYT? ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java: ########## @@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) { } } + public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) { + File outDir = provider.selectAllocationBaseDirectory(0); + if (!outDir.exists() && !outDir.mkdirs()) { + LOG.error( + "Local state base directory does not exist and could not be created: " + + outDir); + } + return new Path(outDir.toURI().toString(), CHECKPOINT_TASK_OWNED_STATE_DIR); + } + + @Override + public void abortCheckpoint(long abortedCheckpointId) { + + LOG.debug( + "Received abort information for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.", + abortedCheckpointId, + jobID, + jobVertexID, + subtaskIndex); + + pruneCheckpoints( + snapshotCheckpointId -> snapshotCheckpointId == abortedCheckpointId, false); + + // Local store only keeps one checkpoint, discard all changelog handle in taskowned + // directory. + // Scenarios: + // cp1: m1 + // confirm cp1, do nothing + // cp2: m1, c1 + // abort cp2, delete m1, c1 + // cp3: m1, c1, c2 + // confirm cp3, do nothing + // -> if failover, restore from local cp3 will fail, because m1 does not exist, c1 may not + // exist either(depend on BatchingStateChangeUploadScheduler). + File[] fileInTaskOwned = + new File(getLocalTaskOwnedDirectory(getLocalRecoveryDirectoryProvider()).toUri()) + .listFiles(); + syncDiscardFileForCollection( + fileInTaskOwned == null ? Collections.emptyList() : Arrays.asList(fileInTaskOwned)); Review Comment: Do we really need this if there's a similar deletion in `dispose()`? I'm concerned that this will delete ALL local state, even the used one. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; Review Comment: 1. Why don't we use `PhysicalStateHandleID` as a key, so that we don't rely on handle equality/hashCode? 2. I'd add "checkpoint" to the field name to make the contents clear, something like `handleToLastUsedCheckpointID` ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java: ########## @@ -101,6 +103,7 @@ public TaskExecutorLocalStateStoresManager( this.discardExecutor = discardExecutor; this.lock = new Object(); this.closed = false; + this.localStateRegistry = new LocalStateRegistry(discardExecutor); Review Comment: I think the lifecycle of this object (`LocalStateRegistry`) should be per job. The code here assumes it is per TM. The code in `LocalStateRegistry` probably assumes it is per backend. If per job, a good candidate to manage it would be `TaskExecutorLocalStateStoresManager`. ########## flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingStateChangeFsUploader.java: ########## @@ -0,0 +1,93 @@ +/* + * 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.changelog.fs; + +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystem.WriteMode; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.LocalRecoveryDirectoryProvider; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.flink.runtime.state.ChangelogTaskLocalStateStore.getLocalTaskOwnedDirectory; + +/** + * A StateChangeFsUploader implementation that writes the changes to remote and local. + * + * <p>The total discard logic of local dstl files is: + * Review Comment: I think `ChangelogTaskLocalStateStore` should also be mentioned here IIUC, it is invoked twice: - after 1 (`storeLocalState` from `AsyncCheckpointRunnable`) - after 2 (`confirmCheckpoint`) Could you please also clarify the responsibilities: - `LocalStateRegistry` and `ChangelogTaskLocalStateStore` - local files (managed purely by TM) - `TaskChangelogRegistry` - "remote" files (managed also by JM); ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { + synchronized (registeredHandles) { + checkState(open, "Attempt to register state to closed LocalStateRegistry."); + if (registeredHandles.containsKey(handle)) { + long pre = registeredHandles.get(handle); + if (checkpointID > pre) { + registeredHandles.put(handle, checkpointID); + } + } else { + registeredHandles.put(handle, checkpointID); + } + } + return handle; + } + + public void unRegister(long upTo) { + List<StreamStateHandle> handles = new ArrayList<>(); + synchronized (registeredHandles) { + Iterator<Entry<StreamStateHandle, Long>> iterator = + registeredHandles.entrySet().iterator(); + while (iterator.hasNext()) { + Entry<StreamStateHandle, Long> entry = iterator.next(); + if (entry.getValue() < upTo) { + handles.add(entry.getKey()); + iterator.remove(); + } Review Comment: I'm afraid this is not enough because the actual underlying file might be used by multiple state backends / tasks (`AbstractStateChangeFsUploader` is shared by multiple of them). So we either need to - track usage by backend (similar to `TaskChangelogRegistry`) - or create a separate duplicating stream per state backend - and manage separate offsets ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java: ########## @@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) { } } + public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) { + File outDir = provider.selectAllocationBaseDirectory(0); Review Comment: Why the index is always `0`, shouldn't it be rotated? Something like `random.nextInt(numDirs)` ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.RejectedExecutionException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This registry manages handles which is written for local recovery. */ +public class LocalStateRegistry implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class); + /** + * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint + * that refer to this handle. + */ + private final Map<StreamStateHandle, Long> registeredHandles; + + /** This flag indicates whether the registry is open or if close() was called. */ + private boolean open; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public LocalStateRegistry(Executor ioExecutor) { + this.registeredHandles = new HashMap<>(); + this.asyncDisposalExecutor = ioExecutor; + this.open = true; + } + + public StreamStateHandle register(StreamStateHandle handle, long checkpointID) { Review Comment: I'd add some javadoc explaining when this method is called. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java: ########## @@ -181,6 +226,16 @@ public void pruneCheckpoints(LongPredicate pruningChecker, boolean breakOnceChec @Override public CompletableFuture<Void> dispose() { deleteMaterialization(id -> true); + // delete all ChangelogStateHandle in taskowned directory. + discardExecutor.execute( + () -> + syncDiscardFileForCollection( Review Comment: Won't this clash with other jobs' state? Why don't we put this logic in `LocalStateRegistry` - that would allow to avoid clashing and guessing the locations. WDYT? -- 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]
