wangyang0918 commented on a change in pull request #13871:
URL: https://github.com/apache/flink/pull/13871#discussion_r518547824
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStore.java
##########
@@ -118,33 +112,22 @@ public boolean requiresExternalizedCheckpoints() {
}
/**
- * Gets the latest checkpoint from ZooKeeper and removes all others.
- *
- * <p><strong>Important</strong>: Even if there are more than one
checkpoint in ZooKeeper,
- * this will only recover the latest and discard the others. Otherwise,
there is no guarantee
- * that the history of checkpoints is consistent.
+ * Recover all the valid checkpoints from state handle store. All the
successfully recovered checkpoints will
+ * be added to {@link #completedCheckpoints} sorted by checkpoint id.
*/
@Override
public void recover() throws Exception {
- LOG.info("Recovering checkpoints from ZooKeeper.");
+ LOG.info("Recovering checkpoints from {}.",
checkpointStateHandleStore);
// Get all there is first
- List<Tuple2<RetrievableStateHandle<CompletedCheckpoint>,
String>> initialCheckpoints;
- while (true) {
- try {
- initialCheckpoints =
checkpointsInZooKeeper.getAll();
- break;
- }
- catch (ConcurrentModificationException e) {
- LOG.warn("Concurrent modification while reading
from ZooKeeper. Retrying.");
- }
- }
+ final List<Tuple2<RetrievableStateHandle<CompletedCheckpoint>,
String>> initialCheckpoints =
+ checkpointStateHandleStore.getAll();
Review comment:
1. IIUC, we will never throw a `ConcurrentModificationException` in
`StateHandleStore#getAllAndLock`. So this retry does not take any effect.
2. For `ZooKeeperStateHandleStore` implementation, it could retry internally
to avoid concurrent modifications.
3. For `KubernetesStateHandleStore` implementation, we do not need retry
since we could get all the data in one time.
----------------------------------------------------------------
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:
[email protected]