Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -333,6 +333,12 @@ void maybePruneState(Object key, long sequenceNum) throws Exception {
* via {@link #snapshotLastCompletedSequenceNumbers}. After pruning, the entry for that
* checkpoint is removed. No-op when durable execution is disabled.
*
* <p><b>Invariant:</b> the {@code checkpointIdToSeqNums.remove} below and the {@code put} in
* {@link #snapshotLastCompletedSequenceNumbers} MUST share the same {@code actionStateStore !=
* null} guard. Dropping the guard on either side breaks the symmetry and reintroduces the
* unbounded-map leak tracked by <a href="https://github.com/apache/flink-agents/issues/645">
* issue #645</a>.
*
* @param checkpointId the id of the completed checkpoint.
*/
void notifyCheckpointComplete(long checkpointId) {
Expand Down Expand Up @@ -365,6 +371,12 @@ void snapshotRecoveryMarker() throws Exception {
* strictly up to the sequence number that was committed by that checkpoint. No-op when durable
* execution is disabled.
*
* <p><b>Invariant:</b> the {@code checkpointIdToSeqNums.put} below and the {@code remove} in
* {@link #notifyCheckpointComplete(long)} MUST share the same {@code actionStateStore != null}
* guard. Dropping the guard on either side breaks the symmetry and reintroduces the
* unbounded-map leak tracked by <a href="https://github.com/apache/flink-agents/issues/645">
* issue #645</a>.
*
* @param keyedStateBackend the keyed state backend to scan.
* @param checkpointId the id of the checkpoint being snapshotted.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;

/** Contract tests for {@link DurableExecutionManager}. */
Expand All @@ -69,6 +70,30 @@ void noStoreModeMakesAllMaybeOperationsNoOp() throws Exception {
dem.close();
}

@Test
@SuppressWarnings("unchecked")
void noStoreModeSnapshotAndNotifyKeepCheckpointMapEmpty() throws Exception {
DurableExecutionManager dem = new DurableExecutionManager(null);
KeyedStateBackend<Object> backend = mock(KeyedStateBackend.class);

// Cycle 1: snapshot + notify with null store. The snapshot-side guard must short-circuit
// before any backend access, and the cleanup-side guard must leave the map untouched.
dem.snapshotLastCompletedSequenceNumbers(backend, 1L);
assertThat(dem.getCheckpointIdToSeqNums()).isEmpty();
verifyNoInteractions(backend);
dem.notifyCheckpointComplete(1L);
assertThat(dem.getCheckpointIdToSeqNums()).isEmpty();

// Cycle 2: confirm the invariant holds across multiple checkpoints.
dem.snapshotLastCompletedSequenceNumbers(backend, 2L);
assertThat(dem.getCheckpointIdToSeqNums()).isEmpty();
verifyNoInteractions(backend);
dem.notifyCheckpointComplete(2L);
assertThat(dem.getCheckpointIdToSeqNums()).isEmpty();

dem.close();
}

@Test
void withInjectedStorePersistsTaskResult() throws Exception {
InMemoryActionStateStore store = new InMemoryActionStateStore(false);
Expand Down
Loading