Skip to content

Commit

Permalink
[FLINK-36751] Fix PausableRelativeClock does not pause when the sourc…
Browse files Browse the repository at this point in the history
…e only has one split
  • Loading branch information
haishui126 authored and rkhachatryan committed Nov 25, 2024
1 parent 2546d60 commit 98045aa
Show file tree
Hide file tree
Showing 2 changed files with 48 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,6 @@ public class SourceOperator<OUT, SplitT extends SourceSplit> extends AbstractStr

private final List<SplitT> splitsToInitializeOutput = new ArrayList<>();

private int numSplits;
private final Map<String, Long> splitCurrentWatermarks = new HashMap<>();
private final Set<String> currentlyPausedSplits = new HashSet<>();

Expand Down Expand Up @@ -615,7 +614,6 @@ public void handleOperatorEvent(OperatorEvent event) {
private void handleAddSplitsEvent(AddSplitEvent<SplitT> event) {
try {
List<SplitT> newSplits = event.splits(splitSerializer);
numSplits += newSplits.size();
if (operatingMode == OperatingMode.OUTPUT_NOT_INITIALIZED) {
// For splits arrived before the main output is initialized, store them into the
// pending list. Outputs of these splits will be created once the main output is
Expand Down Expand Up @@ -656,9 +654,7 @@ public void updateCurrentEffectiveWatermark(long watermark) {
@Override
public void updateCurrentSplitWatermark(String splitId, long watermark) {
splitCurrentWatermarks.put(splitId, watermark);
if (numSplits > 1
&& watermark > currentMaxDesiredWatermark
&& !currentlyPausedSplits.contains(splitId)) {
if (watermark > currentMaxDesiredWatermark && !currentlyPausedSplits.contains(splitId)) {
pauseOrResumeSplits(Collections.singletonList(splitId), Collections.emptyList());
currentlyPausedSplits.add(splitId);
}
Expand All @@ -676,11 +672,6 @@ public void splitFinished(String splitId) {
* <p>Note: This takes effect only if there are multiple splits, otherwise it does nothing.
*/
private void checkSplitWatermarkAlignment() {
if (numSplits <= 1) {
// A single split can't overtake any other splits assigned to this operator instance.
// It is sufficient for the source to stop processing.
return;
}
Collection<String> splitsToPause = new ArrayList<>();
Collection<String> splitsToResume = new ArrayList<>();
splitCurrentWatermarks.forEach(
Expand Down Expand Up @@ -717,12 +708,14 @@ private void checkWatermarkAlignment() {
if (shouldWaitForAlignment()) {
operatingMode = OperatingMode.WAITING_FOR_ALIGNMENT;
waitingForAlignmentFuture = new CompletableFuture<>();
mainInputActivityClock.pause();
}
} else if (operatingMode == OperatingMode.WAITING_FOR_ALIGNMENT) {
checkState(!waitingForAlignmentFuture.isDone());
if (!shouldWaitForAlignment()) {
operatingMode = OperatingMode.READING;
waitingForAlignmentFuture.complete(null);
mainInputActivityClock.unPause();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,10 +48,13 @@ Licensed to the Apache Software Foundation (ASF) under one

import org.assertj.core.api.Condition;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;

import static org.assertj.core.api.Assertions.assertThat;

Expand Down Expand Up @@ -172,8 +175,49 @@ void testBackpressureAndIdleness() throws Exception {
assertThat(dataOutput.getEvents()).doNotHave(new AnyWatermark());
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
void testSingleSplitWatermarkAlignmentAndIdleness(boolean usePerSplitOutputs) throws Exception {
long idleTimeout = 100;
MockSourceReader sourceReader =
new MockSourceReader(
WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS, false, usePerSplitOutputs);
TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
processingTimeService.setCurrentTime(1);
SourceOperator<Integer, MockSourceSplit> operator =
createAndOpenSourceOperatorWithIdleness(
sourceReader, processingTimeService, idleTimeout);

MockSourceSplit split0 = new MockSourceSplit(0, 0, 10);
int maxAllowedWatermark = 4;
int maxEmittedWatermark = maxAllowedWatermark + 1;
// enough records should emit from split0 to make the mainSplit or perSplit is idle,
// then split0 gets blocked and record (maxEmittedWatermark + 100) is never emitted from
// split0
split0.addRecord(1)
.addRecord(1)
.addRecord(1)
.addRecord(1)
.addRecord(maxEmittedWatermark)
.addRecord(maxEmittedWatermark + 100);

operator.handleOperatorEvent(
new AddSplitEvent<>(
Collections.singletonList(split0), new MockSourceSplitSerializer()));
CollectingDataOutput<Integer> dataOutput = new CollectingDataOutput<>();

operator.handleOperatorEvent(
new WatermarkAlignmentEvent(maxAllowedWatermark)); // blocks split0

for (int i = 0; i < 10; i++) {
operator.emitNext(dataOutput);
processingTimeService.advance(idleTimeout);
}
assertThat(dataOutput.getEvents()).doesNotContain(WatermarkStatus.IDLE);
}

@Test
void testSplitWatermarkAlignmentAndIdleness() throws Exception {
void testMultiSplitWatermarkAlignmentAndIdleness() throws Exception {
long idleTimeout = 100;
MockSourceReader sourceReader =
new MockSourceReader(WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS, false, true);
Expand Down

0 comments on commit 98045aa

Please sign in to comment.