mxm commented on a change in pull request #13571:
URL: https://github.com/apache/beam/pull/13571#discussion_r545000289



##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -992,7 +997,23 @@ public void onProcessingTime(InternalTimer<ByteBuffer, 
TimerData> timer) {
 
   // allow overriding this in ExecutableStageDoFnOperator to set the key 
context
   protected void fireTimerInternal(ByteBuffer key, TimerData timerData) {
+    long oldHold = keyCoder != null ? keyedStateInternals.minWatermarkHoldMs() 
: -1L;
     fireTimer(timerData);
+    emitWatermarkIfHoldChanged(oldHold);
+  }
+
+  void emitWatermarkIfHoldChanged(long currentWatermarkHold) {
+    if (keyCoder != null) {
+      long newWatermarkHold = keyedStateInternals.minWatermarkHoldMs();
+      if (newWatermarkHold > currentWatermarkHold) {
+        try {
+          processInputWatermark(false);

Review comment:
       This logic should be removed. All logic which deals with watermark 
emission should be handled through `processWatermark`.

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -739,9 +741,12 @@ public final void processWatermark1(Watermark mark) throws 
Exception {
     }
 
     currentInputWatermark = mark.getTimestamp();

Review comment:
       We need the following to generalize watermark emission to be able to 
call this method from other places. 
   
   ```
   if (mark.getTimestamp > currentInputWatermark) {
     currentInputWatermark = mark.getTimestamp();
   }

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -739,9 +741,12 @@ public final void processWatermark1(Watermark mark) throws 
Exception {
     }
 
     currentInputWatermark = mark.getTimestamp();
+    processInputWatermark(true);
+  }
 
+  private void processInputWatermark(boolean advanceInputWatermark) throws 
Exception {

Review comment:
       Please remove if we don't want to scatter the logic about watermark 
advancement.

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -784,7 +789,7 @@ private long computeOutputWatermark(long 
inputWatermarkHold) {
     return potentialOutputWatermark;
   }
 
-  private void maybeEmitWatermark(long watermark) {
+  void maybeEmitWatermark(long watermark) {

Review comment:
       We remove the `private` modifier?

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -641,8 +641,10 @@ protected final void setBundleFinishedCallback(Runnable 
callback) {
   @Override
   public final void processElement(StreamRecord<WindowedValue<InputT>> 
streamRecord) {
     checkInvokeStartBundle();
+    long oldHold = keyCoder != null ? keyedStateInternals.minWatermarkHoldMs() 
: -1L;
     doFnRunner.processElement(streamRecord.getValue());
     checkInvokeFinishBundleByCount();
+    emitWatermarkIfHoldChanged(oldHold);

Review comment:
       Is this required on every element? I'd rather trigger this only if we 
set / remove a hold.




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to