Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Flink] finalize checkpoint marks in the new Flink source implementation #30849

Closed
wants to merge 1 commit into from
Closed
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 @@ -21,6 +21,8 @@
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -75,6 +77,11 @@ public class FlinkUnboundedSourceReader<T>
private int currentReaderIndex;
private volatile boolean shouldEmitWatermark;

/** Pending checkpoints which have not been acknowledged yet. */
private transient LinkedHashMap<Long, List<UnboundedSource.CheckpointMark>> pendingCheckpoints;
/** Keep a maximum of 32 checkpoints for {@code CheckpointMark.finalizeCheckpoint()}. */
private static final int MAX_NUMBER_PENDING_CHECKPOINTS = 32;

public FlinkUnboundedSourceReader(
String stepName,
SourceReaderContext context,
Expand All @@ -84,6 +91,7 @@ public FlinkUnboundedSourceReader(
this.readers = new ArrayList<>();
this.dataAvailableFutureRef = new AtomicReference<>(DUMMY_FUTURE);
this.currentReaderIndex = 0;
pendingCheckpoints = new LinkedHashMap<>();
}

@VisibleForTesting
Expand All @@ -97,6 +105,7 @@ protected FlinkUnboundedSourceReader(
this.readers = new ArrayList<>();
this.dataAvailableFutureRef = new AtomicReference<>(DUMMY_FUTURE);
this.currentReaderIndex = 0;
pendingCheckpoints = new LinkedHashMap<>();
}

@Override
Expand Down Expand Up @@ -217,6 +226,50 @@ protected Source.Reader<T> createReader(@Nonnull FlinkSourceSplit<T> sourceSplit
return createUnboundedSourceReader(beamSource, sourceSplit.getSplitState());
}

@Override
public List<FlinkSourceSplit<T>> snapshotState(long checkpointId) {

List<UnboundedSource.CheckpointMark> checkpointMarks = new ArrayList<>(allReaders().size());
allReaders()
.forEach(
(splitId, readerAndOutput) -> {
UnboundedSource.UnboundedReader<T> reader = asUnbounded(readerAndOutput.reader);
checkpointMarks.add(reader.getCheckpointMark());
});

// cleanup old pending checkpoints and add new checkpoint
int diff = pendingCheckpoints.size() - MAX_NUMBER_PENDING_CHECKPOINTS;
if (diff >= 0) {
for (Iterator<Long> iterator = pendingCheckpoints.keySet().iterator(); diff >= 0; diff--) {
iterator.next();
iterator.remove();
}
}
pendingCheckpoints.put(checkpointId, checkpointMarks);
return super.snapshotState(checkpointId);
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {

List<UnboundedSource.CheckpointMark> checkpointMarks = pendingCheckpoints.get(checkpointId);
if (checkpointMarks != null) {

// remove old checkpoints including the current one
Iterator<Long> iterator = pendingCheckpoints.keySet().iterator();
long currentId;
do {
currentId = iterator.next();
iterator.remove();
} while (currentId != checkpointId);

// confirm all marks
for (UnboundedSource.CheckpointMark mark : checkpointMarks) {
mark.finalizeCheckpoint();
}
}
}

// -------------- private helper methods ----------------

private void emitRecord(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -282,6 +282,31 @@ public void testPendingBytesMetric() throws Exception {
}
}

@Test
public void testCheckMarksFinalized() throws Exception {

final int numSplits = 2;
final int numRecordsPerSplit = 10;

List<FlinkSourceSplit<KV<Integer, Integer>>> splits =
createSplits(numSplits, numRecordsPerSplit, 0);
RecordsValidatingOutput validatingOutput = new RecordsValidatingOutput(splits);
// Create a reader, take a snapshot.
try (SourceReader<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
FlinkSourceSplit<KV<Integer, Integer>>>
reader = createReader()) {
List<Integer> finalizeTracker = new ArrayList<>();
TestCountingSource.setFinalizeTracker(finalizeTracker);
pollAndValidate(reader, splits, validatingOutput, numSplits * numRecordsPerSplit / 2);
assertTrue(finalizeTracker.isEmpty());
reader.snapshotState(0L);
// notifyCheckpointComplete is normally called by the SourceOperator
reader.notifyCheckpointComplete(0L);
assertFalse(finalizeTracker.isEmpty());
}
}

// --------------- private helper classes -----------------
/** A source whose advance() method only returns true occasionally. */
private static class DummySource extends TestCountingSource {
Expand Down
Loading