rkhachatryan commented on a change in pull request #8693: URL: https://github.com/apache/flink/pull/8693#discussion_r427135277
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnables.java ########## @@ -0,0 +1,85 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.IOUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Track the {@link AsyncCheckpointRunnable}s in {@link SubtaskCheckpointCoordinator}. + */ +@Internal +public class AsyncCheckpointRunnables implements Closeable { + private final Map<Long, AsyncCheckpointRunnable> checkpoints; + /** Lock that guards state of this registry. **/ + private final Object lock; + + public AsyncCheckpointRunnables() { Review comment: nit: inline constructor? ########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java ########## @@ -179,6 +179,13 @@ public void notifyCheckpointComplete(long completedCheckpointId) { } } + @Override + public void notifyCheckpointAborted(long abortedCheckpointId) throws Exception { + synchronized (materializedSstFiles) { + materializedSstFiles.keySet().removeIf(checkpointId -> checkpointId == abortedCheckpointId); Review comment: `materializedSstFiles.remove(abortedCheckpointId);` ? ########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java ########## @@ -464,6 +464,11 @@ public void notifyCheckpointComplete(long completedCheckpointId) throws Exceptio } } + @Override + public void notifyCheckpointAborted(long checkpointId) { + // nothing to do + } + Review comment: I think this is one of these places which SHOULD have an implementation - notify snapshotStrategies (I don't think they can be null despite the checks in `notifyCheckpointComplete`). ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnables.java ########## @@ -0,0 +1,85 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.IOUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Track the {@link AsyncCheckpointRunnable}s in {@link SubtaskCheckpointCoordinator}. + */ +@Internal +public class AsyncCheckpointRunnables implements Closeable { Review comment: I doubt whether we need this class, given that: 1. the only user is `SubtaskCheckpointCoordinatorImpl` 2. it's lifecycle is bound to one of `SubtaskCheckpointCoordinatorImpl` 3. the only field is a map (after removing the lock) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnables.java ########## @@ -0,0 +1,85 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.IOUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Track the {@link AsyncCheckpointRunnable}s in {@link SubtaskCheckpointCoordinator}. + */ +@Internal +public class AsyncCheckpointRunnables implements Closeable { + private final Map<Long, AsyncCheckpointRunnable> checkpoints; + /** Lock that guards state of this registry. **/ + private final Object lock; + + public AsyncCheckpointRunnables() { + this.checkpoints = new HashMap<>(); + this.lock = new Object(); + } + + public void registerAsyncCheckpointRunnable(long checkpointId, AsyncCheckpointRunnable asyncCheckpointRunnable) { + synchronized (lock) { + checkpoints.put(checkpointId, asyncCheckpointRunnable); Review comment: 1. Can we add a check that we don't already have this `checkpointId`? 2. Ideally, we should also check that this registry was not closed yet ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnables.java ########## @@ -0,0 +1,85 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.IOUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Track the {@link AsyncCheckpointRunnable}s in {@link SubtaskCheckpointCoordinator}. + */ +@Internal +public class AsyncCheckpointRunnables implements Closeable { + private final Map<Long, AsyncCheckpointRunnable> checkpoints; + /** Lock that guards state of this registry. **/ + private final Object lock; Review comment: Isn't `ConcurrentMap` is enough here? (so we can remove `lock` and probably improve performance a bit). ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -142,6 +187,13 @@ public void checkpointState( // We generally try to emit the checkpoint barrier as soon as possible to not affect downstream // checkpoint alignments + // Step (0): Record the last triggered checkpointId. + lastCheckpointId = metadata.getCheckpointId(); Review comment: `lastCheckpointId = Math.max(lastCheckpointId, metadata.getCheckpointId());` ? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -384,4 +478,12 @@ private static OperatorSnapshotFutures checkpointStreamOperator( throw ex; } } + + private static class NotifiedCheckpointAbortedException extends Exception { Review comment: Couldn't find any usages (remove?). ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -328,6 +421,7 @@ private OperatorSnapshotFutures buildOperatorSnapshotFutures( @Override public void close() throws IOException { + asyncCheckpointRunnables.close(); Review comment: `try/catch` (or guava Closer or some other way to close both no matter what?) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -126,6 +162,15 @@ public ChannelStateWriter getChannelStateWriter() { return channelStateWriter; } + @VisibleForTesting + int getAbortedCheckpointSize() { + return abortedCheckpointIds.size(); + } + + private boolean checkpointAlreadyAborted(long checkpointId) { + return abortedCheckpointIds.remove(checkpointId); Review comment: This is `O(n)`. I [suggested](https://github.com/apache/flink/pull/8693#discussion_r425881776) to use `LinkedHashMap`. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -126,6 +162,15 @@ public ChannelStateWriter getChannelStateWriter() { return channelStateWriter; } + @VisibleForTesting + int getAbortedCheckpointSize() { + return abortedCheckpointIds.size(); + } + + private boolean checkpointAlreadyAborted(long checkpointId) { Review comment: ~~nit~~ ambiguous name (not `nit` because I indeed suggested a wrong name )) `getAndClearAbortedStatus`? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org