mxm commented on code in PR #20485: URL: https://github.com/apache/flink/pull/20485#discussion_r960476385
########## 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); + } Review Comment: I thought we might expose DataInputStatus of the SourceOperator somewhere. Semantically, one split being paused should yield the same status as multiple splits being all paused. -- 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