mxm commented on code in PR #20485:
URL: https://github.com/apache/flink/pull/20485#discussion_r951456255


##########
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:
   +1 for no wrapping.



##########
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:
   +1 for no wrapping.
   



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to