From 63767c5ed91642c67f97d9f16ff2b8955f9ae421 Mon Sep 17 00:00:00 2001 From: fredia Date: Thu, 13 Oct 2022 18:47:19 +0800 Subject: [PATCH] [FLINK-29157][docs] Clarify the contract between CompletedCheckpointStore and SharedStateRegistry --- .../checkpoint/CompletedCheckpointStore.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java index df239a645a9a9..6abba9b03c4e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +40,16 @@ public interface CompletedCheckpointStore { *

Only a bounded number of checkpoints is kept. When exceeding the maximum number of * retained checkpoints, the oldest one will be discarded. * + *

After FLINK-24611, {@link + * SharedStateRegistry#unregisterUnusedState} should be called here to subsume unused state. + * Note, the {@link CompletedCheckpoint} passed to + * {@link SharedStateRegistry#registerAllAfterRestored} or {@link + * SharedStateRegistryFactory#create} must be the same object as the input parameter, otherwise + * the state may be deleted by mistake. + * + *

After FLINK-25872, {@link + * CheckpointsCleaner#cleanSubsumedCheckpoints} should be called explicitly here. + * * @return the subsumed oldest completed checkpoint if possible, return null if no checkpoint * needs to be discarded on subsume. */ @@ -81,7 +92,8 @@ default long getLatestCheckpointId() { * Shuts down the store. * *

The job status is forwarded and used to decide whether state should actually be discarded - * or kept. + * or kept. {@link SharedStateRegistry#unregisterUnusedState} and {@link + * CheckpointsCleaner#cleanSubsumedCheckpoints} should be called here to subsume unused state. * * @param jobStatus Job state on shut down * @param checkpointsCleaner that will cleanup completed checkpoints if needed