Skip to content

Commit f33b821

Browse files
[FLINK-39388][tests] Fix flaky DataGeneratorSourceITCase#testGatedRateLimiter (#27883)
Co-authored-by: Yuepeng Pan <panyuepeng@apache.org>
1 parent 565dbbc commit f33b821

1 file changed

Lines changed: 28 additions & 4 deletions

File tree

flink-connectors/flink-connector-datagen-test/src/test/java/org/apache/flink/connector/datagen/source/DataGeneratorSourceITCase.java

Lines changed: 28 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.flink.api.common.ExecutionConfig;
2222
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
2323
import org.apache.flink.api.common.functions.FlatMapFunction;
24+
import org.apache.flink.api.common.state.CheckpointListener;
2425
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
2526
import org.apache.flink.api.common.typeinfo.TypeInformation;
2627
import org.apache.flink.api.common.typeinfo.Types;
@@ -254,21 +255,44 @@ public TypeInformation getTypeInformation() {
254255
}
255256
}
256257

258+
/**
259+
* A filter that only passes through elements received before the first checkpoint completes.
260+
*
261+
* <p>The filter stops collecting elements in {@link #notifyCheckpointComplete(long)} rather
262+
* than in {@link #snapshotState(FunctionSnapshotContext)}, to avoid a race condition where the
263+
* checkpoint barrier arrives at this operator before all upstream elements (emitted in the same
264+
* checkpoint cycle) have been processed. Using {@code notifyCheckpointComplete} ensures that
265+
* the checkpoint has fully propagated through the pipeline before we stop collecting.
266+
*/
257267
private static class FirstCheckpointFilter
258-
implements FlatMapFunction<Long, Long>, CheckpointedFunction {
268+
implements FlatMapFunction<Long, Long>, CheckpointedFunction, CheckpointListener {
259269

260-
private volatile boolean firstCheckpoint = true;
270+
private volatile boolean firstCheckpointCompleted = false;
271+
private long firstCheckpointId = Long.MIN_VALUE;
261272

262273
@Override
263274
public void flatMap(Long value, Collector<Long> out) throws Exception {
264-
if (firstCheckpoint) {
275+
if (!firstCheckpointCompleted) {
265276
out.collect(value);
266277
}
267278
}
268279

269280
@Override
270281
public void snapshotState(FunctionSnapshotContext context) throws Exception {
271-
firstCheckpoint = false;
282+
// Record the ID of the first checkpoint so we can stop collecting when it completes.
283+
if (firstCheckpointId == Long.MIN_VALUE) {
284+
firstCheckpointId = context.getCheckpointId();
285+
}
286+
}
287+
288+
@Override
289+
public void notifyCheckpointComplete(long checkpointId) throws Exception {
290+
// Stop collecting elements once the first checkpoint has completed.
291+
if (!firstCheckpointCompleted
292+
&& checkpointId >= firstCheckpointId
293+
&& firstCheckpointId != Long.MIN_VALUE) {
294+
firstCheckpointCompleted = true;
295+
}
272296
}
273297

274298
@Override

0 commit comments

Comments
 (0)