Zakelly commented on code in PR #24895:
URL: https://github.com/apache/flink/pull/24895#discussion_r1630562624


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MailboxWatermarkProcessor.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A helper class to let operators emit watermarks incrementally from mailbox. 
Instead of emitting
+ * all the watermarks at once in a single {@code processWatermark} call, if a 
mail in mailbox is
+ * present, the process of firing timers is interrupted and a continuation to 
finish it off later is
+ * scheduled via a mailbox mail.
+ *
+ * <p>Note that interrupting firing timers can change order of some 
invocations. It is possible that
+ * between firing timers, some records might be processed.
+ */
+@Internal
+public class MailboxWatermarkProcessor<OUT> {
+    protected static final Logger LOG = 
LoggerFactory.getLogger(MailboxWatermarkProcessor.class);
+
+    private final Output<StreamRecord<OUT>> output;
+    private final MailboxExecutor mailboxExecutor;
+    private final InternalTimeServiceManager<?> internalTimeServiceManager;
+    /**
+     * Flag to indicate whether a progress watermark is scheduled in the 
mailbox. This is used to
+     * avoid duplicate scheduling in case we have multiple watermarks to 
process.
+     */
+    private boolean progressWatermarkScheduled = false;
+
+    private Watermark maxInputWatermark = Watermark.UNINITIALIZED;
+
+    public MailboxWatermarkProcessor(
+            Output<StreamRecord<OUT>> output,
+            MailboxExecutor mailboxExecutor,
+            InternalTimeServiceManager<?> internalTimeServiceManager) {
+        this.output = checkNotNull(output);
+        this.mailboxExecutor = checkNotNull(mailboxExecutor);
+        this.internalTimeServiceManager = 
checkNotNull(internalTimeServiceManager);
+    }
+
+    public void emitWatermarkInsideMailbox(Watermark mark) throws Exception {
+        maxInputWatermark =
+                new Watermark(Math.max(maxInputWatermark.getTimestamp(), 
mark.getTimestamp()));

Review Comment:
   Will this 'swallow' some watermarks? Would this be a problem?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java:
##########
@@ -728,6 +736,16 @@ public <K> TypeSerializer<K> 
getStateKeySerializer(ClassLoader cl) {
         }
     }
 
+    /**
+     * Returns whether the state key serializer is set in the config. This 
signals that the operator
+     * is stateful.
+     *
+     * @return true if state key serializer is defined
+     */
+    public boolean hasStateKeySerializer() {

Review Comment:
   Seems this is unused?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java:
##########
@@ -307,18 +307,36 @@ void onProcessingTime(long time) throws Exception {
     }
 
     public void advanceWatermark(long time) throws Exception {
-        currentWatermark = time;
+        Preconditions.checkState(
+                tryAdvanceWatermark(
+                        time,
+                        () -> {
+                            // Never stop advancing.
+                            return false;
+                        }));
+    }
 
+    /**
+     * @return true if following watermarks can be processed immediately. 
False if the firing timers
+     *     should be interrupted as soon as possible.
+     */
+    public boolean tryAdvanceWatermark(
+            long time, InternalTimeServiceManager.ShouldStopAdvancingFn 
shouldStopAdvancingFn)
+            throws Exception {
+        currentWatermark = time;
         InternalTimer<K, N> timer;
-
         while ((timer = eventTimeTimersQueue.peek()) != null
                 && timer.getTimestamp() <= time
                 && !cancellationContext.isCancelled()) {
             keyContext.setCurrentKey(timer.getKey());
             eventTimeTimersQueue.poll();
             triggerTarget.onEventTime(timer);
             taskIOMetricGroup.getNumFiredTimers().inc();
+            if (shouldStopAdvancingFn.shouldStopAdvancing()) {
+                return false;
+            }

Review Comment:
   How about moving this to the while condition as the last part to check, also 
let a variable take the result of `shouldStopAdvancingFn` and return it in 
bottom. Thus when the last timer get processed, it will return true.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarness.java:
##########
@@ -136,6 +136,17 @@ public boolean processSingleStep() throws Exception {
         return false;
     }
 
+    public boolean processSingleMailboxLoop() throws Exception {
+        if (streamTask.mailboxProcessor.isMailboxLoopRunning()) {
+            return streamTask.runSingleMailboxLoop();
+        }
+        return false;
+    }
+
+    public MailboxExecutor getMainMailboxExecutor() {
+        return streamTask.mailboxProcessor.getMainMailboxExecutor();
+    }
+

Review Comment:
   Seems unused? also the `StreamTask#runSingleMailboxLoop` and 
`MailboxProcessor#runSingleMailboxLoop`



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java:
##########
@@ -86,7 +86,11 @@ Tuple2<OP, Optional<ProcessingTimeService>> createOperator(
                                 processingTimeService != null
                                         ? () -> processingTimeService
                                         : processingTimeServiceFactory,
-                                operatorEventDispatcher));
+                                operatorEventDispatcher,
+                                mailboxExecutor));
+        if (op instanceof YieldingOperator) {
+            ((YieldingOperator<?>) op).setMailboxExecutor(mailboxExecutor);
+        }

Review Comment:
   This is confusing. Do you mean the factory is not `YeildingOperatorFactory` 
so we have to directly set the mailbox to operator here? I'd suggest we use the 
`YeildingOperatorFactory` to do so.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java:
##########
@@ -40,6 +40,15 @@
  */
 @Internal
 public interface InternalTimeServiceManager<K> {
+
+    /** Signals whether the watermark should continue advancing. */
+    @Internal
+    @FunctionalInterface
+    interface ShouldStopAdvancingFn {
+
+        boolean shouldStopAdvancing();

Review Comment:
   I suggest a short method name, since the interface name have expressed this 
well.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java:
##########
@@ -103,4 +103,9 @@ public boolean tryYield() {
             return false;
         }
     }
+
+    @Override
+    public boolean shouldInterrupt() {
+        return mailbox.hasMail();

Review Comment:
   So any mail will interrupt the timer processing? I thought only 
checkpoint-related mail will do that...



-- 
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]

Reply via email to