Skip to content

Commit

Permalink
[FLINK-29157][docs] Clarify the contract between CompletedCheckpointS…
Browse files Browse the repository at this point in the history
…tore and SharedStateRegistry
  • Loading branch information
fredia authored and klion26 committed Oct 26, 2022
1 parent f8c6a66 commit 63767c5
Showing 1 changed file with 13 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -39,6 +40,16 @@ public interface CompletedCheckpointStore {
* <p>Only a bounded number of checkpoints is kept. When exceeding the maximum number of
* retained checkpoints, the oldest one will be discarded.
*
* <p>After <a href="https://issues.apache.org/jira/browse/FLINK-24611">FLINK-24611</a>, {@link
* SharedStateRegistry#unregisterUnusedState} should be called here to subsume unused state.
* <font color="#FF0000"><strong>Note</strong></font>, 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.
*
* <p>After <a href="https://issues.apache.org/jira/browse/FLINK-25872">FLINK-25872</a>, {@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.
*/
Expand Down Expand Up @@ -81,7 +92,8 @@ default long getLatestCheckpointId() {
* Shuts down the store.
*
* <p>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
Expand Down

0 comments on commit 63767c5

Please sign in to comment.