mas-chen commented on code in PR #20485:
URL: https://github.com/apache/flink/pull/20485#discussion_r944686033
##########
docs/layouts/shortcodes/generated/pipeline_configuration.html:
##########
@@ -128,5 +128,11 @@
<td>Boolean</td>
<td>Whether name of vertex includes topological index or not. When
it is true, the name will have a prefix of index of the vertex, like
'[vertex-0]Source: source'. It is false by default</td>
</tr>
+ <tr>
+
<td><h5>pipeline.watermark-alignment.allow-unaligned-source-splits</h5></td>
Review Comment:
I think `allow-aligned-source-splits` is more intuitive. The emphasis is on
alignment, rather than unalignment like in checkpoints.
##########
flink-connectors/flink-connector-base/pom.xml:
##########
@@ -17,8 +17,8 @@
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
Review Comment:
nit: revert unintended change?
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java:
##########
@@ -559,14 +577,67 @@ private void createOutputForSplits(List<SplitT>
newSplits) {
private void updateMaxDesiredWatermark(WatermarkAlignmentEvent event) {
currentMaxDesiredWatermark = event.getMaxWatermark();
+ checkSplitWatermarkAlignment();
sourceMetricGroup.updateMaxDesiredWatermark(currentMaxDesiredWatermark);
}
- private void onWatermarkEmitted(long emittedWatermark) {
- lastEmittedWatermark = emittedWatermark;
+ @Override
+ public void updateCurrentEffectiveWatermark(long watermark) {
+ lastEmittedWatermark = watermark;
checkWatermarkAlignment();
}
+ @Override
+ public void updateCurrentSplitWatermark(String splitId, long watermark) {
+ splitCurrentWatermarks.put(splitId, watermark);
+ if (currentMaxDesiredWatermark < watermark &&
!currentlyPausedSplits.contains(splitId)) {
+ pauseOrResumeSplits(Collections.singletonList(splitId),
Collections.emptyList());
Review Comment:
Should this instead call `checkSplitWatermarkAlignment` to capture the case
where there is only 1 split?
##########
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherPauseResumeSplitReaderTest.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.base.source.reader.fetcher;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.mocks.MockSourceReader;
+import org.apache.flink.connector.base.source.reader.mocks.MockSplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import
org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.connector.testutils.source.reader.TestingReaderContext;
+import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput;
+import org.apache.flink.core.io.InputStatus;
+import org.apache.flink.core.testutils.CommonTestUtils;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Tests {@link SplitFetcher} integration to pause or resume {@link
SplitReader} based on {@link
+ * SourceReader} output.
+ */
+public class SplitFetcherPauseResumeSplitReaderTest {
+
+ /**
+ * Tests if pause or resume shows expected behavior which requires
creation and execution of
+ * {@link SplitFetcher} tasks.
+ *
+ * @throws Exception on error.
+ */
+ @ParameterizedTest(name = "Individual reader per split: {0}")
+ @ValueSource(booleans = {false, true})
+ public void testPauseResumeSplitReaders(boolean individualReader) throws
Exception {
+ final AtomicInteger numSplitReaders = new AtomicInteger();
+ final MockSplitReader.Builder readerBuilder =
+ SteppingSourceReaderTestHarness.createSplitReaderBuilder();
+ final SteppingSourceReaderTestHarness testHarness =
+ new SteppingSourceReaderTestHarness(
+ () -> {
+ numSplitReaders.getAndIncrement();
+ return readerBuilder.build();
+ },
+ new Configuration());
+
+ if (individualReader) {
+ testHarness.addPrefilledSplitsIndividualReader(2, 5);
+ assertThat(numSplitReaders.get()).isEqualTo(2);
+ } else {
+ testHarness.addPrefilledSplitsSingleReader(2, 5);
+ assertThat(numSplitReaders.get()).isEqualTo(1);
+ }
+
+ TestingReaderOutput output = new TestingReaderOutput<>();
+ testHarness.runUntilRecordsEmitted(output, 10, 2);
+ Set<Integer> recordSet = new HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet).containsExactlyInAnyOrder(new Integer[] {0, 1});
+
+ testHarness.pauseOrResumeSplits(Collections.singleton("0"),
Collections.emptyList());
+
+ testHarness.runUntilRecordsEmitted(output, 10, 5);
+ Set<Integer> recordSet2 = new HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet2).containsExactlyInAnyOrder(new Integer[] {0, 1,
3, 5, 7});
+
+ testHarness.pauseOrResumeSplits(Collections.emptyList(),
Collections.singleton("0"));
+
+ testHarness.runUntilAllRecordsEmitted(output, 10);
+ Set<Integer> recordSet3 = new HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet3)
+ .containsExactlyInAnyOrder(new Integer[] {0, 1, 2, 3, 4, 5, 6,
7, 8, 9});
+ }
+
+ /**
+ * Tests if pause or resume shows expected behavior in case of {@link
SplitReader} that does not
+ * support split or resume for scenarios (1) allowed and (2) not allowed.
+ *
+ * @throws Exception on error.
+ */
+ @ParameterizedTest(name = "Allow unaligned source splits: {0}")
+ @ValueSource(booleans = {true, false})
+ public void testPauseResumeUnsupported(boolean allowUnalignedSourceSplits)
throws Exception {
+ final AtomicInteger numSplitReaders = new AtomicInteger();
+ final Configuration configuration = new Configuration();
+ configuration.setBoolean(
+ "pipeline.watermark-alignment.allow-unaligned-source-splits",
+ allowUnalignedSourceSplits);
+ final MockSplitReader.Builder readerBuilder =
+ SteppingSourceReaderTestHarness.createSplitReaderBuilder();
+
+ final SteppingSourceReaderTestHarness testHarness =
+ new SteppingSourceReaderTestHarness(
+ () -> {
+ if (numSplitReaders.getAndIncrement() == 0) {
+ return
MockSplitReaderUnsupportedPause.cloneBuilder(readerBuilder)
+ .build();
+ } else {
+ return readerBuilder.build();
+ }
+ },
+ configuration);
+
+ testHarness.addPrefilledSplitsIndividualReader(2, 5);
+ assertThat(numSplitReaders.get()).isEqualTo(2);
+
+ TestingReaderOutput output = new TestingReaderOutput<>();
+ testHarness.runUntilRecordsEmitted(output, 10, 2);
+ Set<Integer> recordSet = new HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet).containsExactlyInAnyOrder(new Integer[] {0, 1});
+
+ testHarness.pauseOrResumeSplits(Collections.singleton("1"),
Collections.emptyList());
+
+ testHarness.runUntilRecordsEmitted(output, 10, 5);
+ Set<Integer> recordSet2 = new HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet2).containsExactlyInAnyOrder(new Integer[] {0, 1,
2, 4, 6});
+
+ testHarness.pauseOrResumeSplits(Collections.singleton("0"),
Collections.singleton("1"));
+
+ if (allowUnalignedSourceSplits) {
+ testHarness.runUntilAllRecordsEmitted(output, 10);
+ Set<Integer> recordSet3 = new
HashSet<>(output.getEmittedRecords());
+ assertThat(recordSet3)
+ .containsExactlyInAnyOrder(new Integer[] {0, 1, 2, 3, 4,
5, 6, 7, 8, 9});
+ } else {
+ assertThatThrownBy(() ->
testHarness.runUntilAllRecordsEmitted(output, 10))
+ .isInstanceOf(RuntimeException.class)
+ .hasCauseInstanceOf(UnsupportedOperationException.class);
+ }
+ }
+
+ private static class MockSteppingSplitFetcherManager<E, SplitT extends
SourceSplit>
+ extends SingleThreadFetcherManager<E, SplitT> {
+
+ public MockSteppingSplitFetcherManager(
+ FutureCompletingBlockingQueue<RecordsWithSplitIds<E>>
elementsQueue,
+ Supplier<SplitReader<E, SplitT>> splitReaderSupplier,
+ Configuration configuration) {
+ super(elementsQueue, splitReaderSupplier, configuration);
+ }
+
+ @Override
+ public void addSplits(List<SplitT> splitsToAdd) {
+ SplitFetcher<E, SplitT> fetcher = createSplitFetcher();
+ fetcher.addSplits(splitsToAdd);
+ }
+
+ public void runEachOnce() {
+ for (SplitFetcher<E, SplitT> fetcher : fetchers.values()) {
+ fetcher.runOnce();
+ }
+ }
+ }
+
+ private static class MockSplitReaderUnsupportedPause extends
MockSplitReader {
+ public MockSplitReaderUnsupportedPause(
+ int numRecordsPerSplitPerFetch,
+ boolean separatedFinishedRecord,
+ boolean blockingFetch) {
+ super(numRecordsPerSplitPerFetch, separatedFinishedRecord,
blockingFetch);
+ }
+
+ @Override
+ public void pauseOrResumeSplits(
+ Collection<MockSourceSplit> splitsToPause,
+ Collection<MockSourceSplit> splitsToResume) {
+ throw new UnsupportedOperationException();
+ }
+
+ public static class Builder extends MockSplitReader.Builder {
+ public Builder(MockSplitReader.Builder other) {
+ super(other);
+ }
+
+ @Override
+ public MockSplitReader build() {
+ return new MockSplitReaderUnsupportedPause(
+ numRecordsPerSplitPerFetch, separatedFinishedRecord,
blockingFetch);
+ }
+ }
+
+ public static Builder cloneBuilder(MockSplitReader.Builder other) {
+ return new MockSplitReaderUnsupportedPause.Builder(other);
+ }
+ }
+
+ private static class SteppingSourceReaderTestHarness {
+ private final MockSteppingSplitFetcherManager<int[], MockSourceSplit>
fetcherManager;
+ private final MockSourceReader sourceReader;
+
+ public SteppingSourceReaderTestHarness(
+ Supplier<SplitReader<int[], MockSourceSplit>>
splitReaderSupplier,
+ Configuration configuration) {
+ FutureCompletingBlockingQueue<RecordsWithSplitIds<int[]>> queue =
+ new FutureCompletingBlockingQueue<>(10);
+ this.fetcherManager =
+ new MockSteppingSplitFetcherManager<>(
+ queue, splitReaderSupplier, configuration);
+ this.sourceReader =
+ new MockSourceReader(
+ queue, fetcherManager, configuration, new
TestingReaderContext());
+ }
+
+ private static List<MockSourceSplit> createPrefilledSplits(int
numSplits, int numRecords) {
+ final List<MockSourceSplit> splits = new ArrayList<>(numSplits);
+ for (int splitId = 0; splitId < numSplits; splitId++) {
+ MockSourceSplit split = new MockSourceSplit(splitId, 0,
numRecords);
+ for (int i = 0; i < numRecords; i++) {
+ split.addRecord(i * numSplits + splitId);
+ }
+ splits.add(split);
+ }
+ return splits;
+ }
+
+ public void addPrefilledSplitsSingleReader(int numSplits, int
numRecords) {
+ sourceReader.addSplits(createPrefilledSplits(numSplits,
numRecords));
+ sourceReader.notifyNoMoreSplits();
Review Comment:
Is this a required assumption? We should also exercise the case where splits
are dynamically discovered (e.g. Kafka source topic partition discovery).
##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/PauseOrResumeSplitsTask.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.base.source.reader.fetcher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Changes the paused splits of a n{@link SplitReader}. The task is used by
default in {@link
+ * SplitFetcherManager} and assumes that a {@link SplitFetcher} has multiple
splits. For {@code
+ * SplitFetchers} with single splits, it's instead recommended to subclass
{@link
+ * SplitFetcherManager} and pause the whole {@code SplitFetcher}.
+ *
+ * @param <SplitT> the type of the split
+ */
+@Internal
+class PauseOrResumeSplitsTask<SplitT extends SourceSplit> implements
SplitFetcherTask {
+ private static final Logger LOG =
LoggerFactory.getLogger(PauseOrResumeSplitsTask.class);
+ private final SplitReader<?, SplitT> splitReader;
+ private final Collection<SplitT> splitsToPause;
+ private final Collection<SplitT> splitsToResume;
+ private final boolean allowUnalignedSourceSplits;
+
+ PauseOrResumeSplitsTask(
+ SplitReader<?, SplitT> splitReader,
+ Collection<SplitT> splitsToPause,
+ Collection<SplitT> splitsToResume,
+ boolean allowUnalignedSourceSplits) {
+ this.splitReader = checkNotNull(splitReader);
+ this.splitsToPause = checkNotNull(splitsToPause);
+ this.splitsToResume = checkNotNull(splitsToResume);
+ this.allowUnalignedSourceSplits = allowUnalignedSourceSplits;
+ }
+
+ @Override
+ public boolean run() throws IOException {
+ try {
+ splitReader.pauseOrResumeSplits(splitsToPause, splitsToResume);
+ } catch (UnsupportedOperationException e) {
+ if (!allowUnalignedSourceSplits) {
+ throw new UnsupportedOperationException("", e);
Review Comment:
Can we rethrow exception without unnecessary wrapping? `throw e`
##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorSplitWatermarkAlignmentTest.java:
##########
@@ -0,0 +1,144 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements. See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership. The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.flink.streaming.api.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.eventtime.WatermarkGenerator;
+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.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
+import
org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway;
+import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
+import org.apache.flink.runtime.source.event.AddSplitEvent;
+import org.apache.flink.runtime.source.event.WatermarkAlignmentEvent;
+import org.apache.flink.runtime.state.TestTaskStateManager;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.operators.source.CollectingDataOutput;
+import org.apache.flink.streaming.api.operators.source.TestingSourceOperator;
+import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.streaming.util.MockOutput;
+import org.apache.flink.streaming.util.MockStreamConfig;
+
+import org.junit.Test;
Review Comment:
nit: use junit5
##########
flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReader.java:
##########
@@ -132,4 +139,28 @@ default void handleSourceEvents(SourceEvent sourceEvent) {}
*/
@Override
default void notifyCheckpointComplete(long checkpointId) throws Exception
{}
+
+ /**
+ * Pauses or resumes reading of individual source splits.
+ *
+ * <p>Note that no other methods can be called in parallel, so it's fine
to non-atomically
Review Comment:
nit: This reads unclearly. What about: ", so updating subscriptions can be
atomically updated".
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java:
##########
@@ -559,14 +577,67 @@ private void createOutputForSplits(List<SplitT>
newSplits) {
private void updateMaxDesiredWatermark(WatermarkAlignmentEvent event) {
currentMaxDesiredWatermark = event.getMaxWatermark();
+ checkSplitWatermarkAlignment();
sourceMetricGroup.updateMaxDesiredWatermark(currentMaxDesiredWatermark);
}
- private void onWatermarkEmitted(long emittedWatermark) {
- lastEmittedWatermark = emittedWatermark;
+ @Override
+ public void updateCurrentEffectiveWatermark(long watermark) {
+ lastEmittedWatermark = watermark;
checkWatermarkAlignment();
}
+ @Override
+ public void updateCurrentSplitWatermark(String splitId, long watermark) {
+ splitCurrentWatermarks.put(splitId, watermark);
+ if (currentMaxDesiredWatermark < watermark &&
!currentlyPausedSplits.contains(splitId)) {
+ pauseOrResumeSplits(Collections.singletonList(splitId),
Collections.emptyList());
+ currentlyPausedSplits.add(splitId);
+ }
+ }
+
+ /**
+ * Finds the splits that are beyond the current max watermark and pauses
them. At the same time,
+ * splits that have been paused and where the global watermark caught up
are resumed.
+ *
+ * <p>Note: This takes effect only if there are multiple splits, otherwise
it does nothing.
+ */
+ private void checkSplitWatermarkAlignment() {
+ if (numSplits <= 1) {
+ return; // If there is only a single split, we do not pause the
split but the source.
+ }
+ Collection<String> splitsToPause = new ArrayList<>();
+ Collection<String> splitsToResume = new ArrayList<>();
+ splitCurrentWatermarks.forEach(
+ (splitId, splitWatermark) -> {
+ if (splitWatermark > currentMaxDesiredWatermark) {
+ splitsToPause.add(splitId);
+ } else if (currentlyPausedSplits.contains(splitId)) {
+ splitsToResume.add(splitId);
+ }
+ });
+ splitsToPause.removeAll(currentlyPausedSplits);
+ if (!splitsToPause.isEmpty() || !splitsToResume.isEmpty()) {
+ pauseOrResumeSplits(splitsToPause, splitsToResume);
+ currentlyPausedSplits.addAll(splitsToPause);
+ splitsToResume.forEach(currentlyPausedSplits::remove);
+ }
+ }
+
+ private void pauseOrResumeSplits(
+ Collection<String> splitsToPause, Collection<String>
splitsToResume) {
+ if (!allowUnalignedSourceSplits ||
sourceReaderSupportsPauseOrResumeSplits) {
+ try {
+ sourceReader.pauseOrResumeSplits(splitsToPause,
splitsToResume);
+ } catch (UnsupportedOperationException e) {
+ sourceReaderSupportsPauseOrResumeSplits = false;
+ if (!allowUnalignedSourceSplits) {
+ throw new UnsupportedOperationException("", e);
Review Comment:
ditto on wrapping exception
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]