Skip to content

Commit

Permalink
[hotfix] Refactor SourceOperatorSlitWatermarkAlignmentTest and suppor…
Browse files Browse the repository at this point in the history
…t pausing splits in MockSourceReader
  • Loading branch information
pnowojski committed Aug 17, 2024
1 parent 99bf496 commit 308f05a
Show file tree
Hide file tree
Showing 2 changed files with 48 additions and 42 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,15 @@
import javax.annotation.concurrent.GuardedBy;

import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

/** A mock {@link SourceReader} for unit tests. */
public class MockSourceReader implements SourceReader<Integer, MockSourceSplit> {
private final Set<String> pausedSplits = new HashSet<>();
private final List<MockSourceSplit> assignedSplits = new ArrayList<>();
private final List<SourceEvent> receivedSourceEvents = new ArrayList<>();
private final List<Long> completedCheckpoints = new ArrayList<>();
Expand Down Expand Up @@ -100,11 +104,16 @@ public InputStatus pollNext(ReaderOutput<Integer> sourceOutput) throws Exception
|| waitingForSplitsBehaviour == WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS;
currentSplitIndex = 0;
// Find first splits with available records.
while (currentSplitIndex < assignedSplits.size()
&& !assignedSplits.get(currentSplitIndex).isAvailable()) {
finished &= assignedSplits.get(currentSplitIndex).isFinished();
for (MockSourceSplit assignedSplit : assignedSplits) {
finished &= assignedSplit.isFinished();
if (!pausedSplits.contains(assignedSplit.splitId())) {
if (assignedSplit.isAvailable()) {
break;
}
}
currentSplitIndex++;
}

// Read from the split with available record.
if (currentSplitIndex < assignedSplits.size()) {
if (idle) {
Expand Down Expand Up @@ -151,6 +160,12 @@ public void addSplits(List<MockSourceSplit> splits) {
markAvailable();
}

public void pauseOrResumeSplits(
Collection<String> splitsToPause, Collection<String> splitsToResume) {
pausedSplits.removeAll(splitsToResume);
pausedSplits.addAll(splitsToPause);
}

@Override
public void notifyNoMoreSplits() {
splitsAssignmentState = SplitsAssignmentState.NO_MORE_SPLITS;
Expand Down Expand Up @@ -224,4 +239,8 @@ public List<Long> getCompletedCheckpoints() {
public List<Long> getAbortedCheckpoints() {
return abortedCheckpoints;
}

public Set<String> getPausedSplits() {
return pausedSplits;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ Licensed to the Apache Software Foundation (ASF) under one
import org.apache.flink.api.common.eventtime.WatermarkOutput;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.connector.source.mocks.MockSourceReader;
import org.apache.flink.api.connector.source.mocks.MockSourceReader.WaitingForSplits;
import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer;
import org.apache.flink.configuration.Configuration;
Expand All @@ -42,46 +43,28 @@ Licensed to the Apache Software Foundation (ASF) under one
import org.apache.flink.streaming.util.MockOutput;
import org.apache.flink.streaming.util.MockStreamConfig;

import org.assertj.core.api.Condition;
import org.junit.jupiter.api.Test;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.Optional;
import java.util.function.Predicate;

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

/** Unit test for split alignment in {@link SourceOperator}. */
class SourceOperatorSplitWatermarkAlignmentTest {
private static final WatermarkGenerator<Integer> WATERMARK_GENERATOR =
new WatermarkGenerator<Integer>() {

private long maxWatermark = Long.MIN_VALUE;

@Override
public void onEvent(Integer event, long eventTimestamp, WatermarkOutput output) {
if (eventTimestamp > maxWatermark) {
this.maxWatermark = eventTimestamp;
output.emitWatermark(new Watermark(maxWatermark));
}
}

@Override
public void onPeriodicEmit(WatermarkOutput output) {
output.emitWatermark(new Watermark(maxWatermark));
}
};
class SourceOperatorSplitWatermarkAlignmentTest {

@Test
void testSplitWatermarkAlignment() throws Exception {

final SplitAligningSourceReader sourceReader = new SplitAligningSourceReader();
MockSourceReader sourceReader = new MockSourceReader(WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS, false, true);
SourceOperator<Integer, MockSourceSplit> operator =
new TestingSourceOperator<>(
sourceReader,
WatermarkStrategy.forGenerator(ctx -> WATERMARK_GENERATOR)
WatermarkStrategy.forGenerator(ctx -> new TestWatermarkGenerator())
.withTimestampAssigner((r, l) -> r)
.withWatermarkAlignment("group-1", Duration.ofMillis(1)),
new TestProcessingTimeService(),
Expand All @@ -97,8 +80,8 @@ void testSplitWatermarkAlignment() throws Exception {
operator.initializeState(new StreamTaskStateInitializerImpl(env, new MemoryStateBackend()));

operator.open();
final MockSourceSplit split1 = new MockSourceSplit(0, 0, 10);
final MockSourceSplit split2 = new MockSourceSplit(1, 10, 20);
MockSourceSplit split1 = new MockSourceSplit(0, 0, 10);
MockSourceSplit split2 = new MockSourceSplit(1, 10, 20);
split1.addRecord(5);
split1.addRecord(11);
split2.addRecord(3);
Expand All @@ -107,25 +90,25 @@ void testSplitWatermarkAlignment() throws Exception {
operator.handleOperatorEvent(
new AddSplitEvent<>(
Arrays.asList(split1, split2), new MockSourceSplitSerializer()));
final CollectingDataOutput<Integer> dataOutput = new CollectingDataOutput<>();
CollectingDataOutput<Integer> dataOutput = new CollectingDataOutput<>();

operator.emitNext(dataOutput); // split 1 emits 5

operator.handleOperatorEvent(
new WatermarkAlignmentEvent(4)); // pause by coordinator message
assertThat(sourceReader.pausedSplits).containsExactly("0");
assertThat(sourceReader.getPausedSplits()).containsExactly("0");

operator.handleOperatorEvent(new WatermarkAlignmentEvent(5));
assertThat(sourceReader.pausedSplits).isEmpty();
assertThat(sourceReader.getPausedSplits()).isEmpty();

operator.emitNext(dataOutput); // split 1 emits 11
operator.emitNext(dataOutput); // split 2 emits 3

assertThat(sourceReader.pausedSplits).containsExactly("0");
assertThat(sourceReader.getPausedSplits()).containsExactly("0");

operator.emitNext(dataOutput); // split 2 emits 6

assertThat(sourceReader.pausedSplits).containsExactly("0", "1");
assertThat(sourceReader.getPausedSplits()).containsExactly("0", "1");
}

private Environment getTestingEnvironment() {
Expand All @@ -139,17 +122,21 @@ private Environment getTestingEnvironment() {
new TestTaskStateManager());
}

private static class SplitAligningSourceReader extends MockSourceReader {
Set<String> pausedSplits = new HashSet<>();
private static class TestWatermarkGenerator implements WatermarkGenerator<Integer> {

public SplitAligningSourceReader() {
super(WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS, false, true);
private long maxWatermark = Long.MIN_VALUE;

@Override
public void onEvent(Integer event, long eventTimestamp, WatermarkOutput output) {
if (eventTimestamp > maxWatermark) {
this.maxWatermark = eventTimestamp;
output.emitWatermark(new Watermark(maxWatermark));
}
}

public void pauseOrResumeSplits(
Collection<String> splitsToPause, Collection<String> splitsToResume) {
pausedSplits.removeAll(splitsToResume);
pausedSplits.addAll(splitsToPause);
@Override
public void onPeriodicEmit(WatermarkOutput output) {
output.emitWatermark(new Watermark(maxWatermark));
}
}
}

0 comments on commit 308f05a

Please sign in to comment.