AHeise commented on a change in pull request #15771:
URL: https://github.com/apache/flink/pull/15771#discussion_r629978340



##########
File path: 
flink-core/src/main/java/org/apache/flink/api/common/eventtime/CombinedWatermark.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.api.common.eventtime;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link CombinedWatermark} combines the watermark (and idleness) updates 
of multiple
+ * partitions/shards/splits into one combined watermark.
+ */
+@Internal
+public final class CombinedWatermark extends 
AbstractList<CombinedWatermark.PartialWatermark> {
+
+    /** List of all watermark outputs, for efficient access. */
+    private final List<PartialWatermark> partialWatermarks = new ArrayList<>();
+
+    /** The combined watermark over the per-output watermarks. */
+    private long combinedWatermark = Long.MIN_VALUE;
+
+    private boolean idle = false;
+
+    public long getCombinedWatermark() {
+        return combinedWatermark;
+    }
+
+    public boolean isIdle() {
+        return idle;
+    }
+
+    @Override
+    public PartialWatermark get(int index) {
+        return partialWatermarks.get(index);
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        return partialWatermarks.remove(o);
+    }
+
+    @Override
+    public PartialWatermark set(int index, PartialWatermark element) {
+        return partialWatermarks.set(index, element);
+    }
+
+    @Override
+    public void add(int index, PartialWatermark element) {
+        partialWatermarks.add(index, element);
+    }
+
+    @Override
+    public int size() {
+        return partialWatermarks.size();
+    }
+
+    /**
+     * Checks whether we need to update the combined watermark.
+     *
+     * <p><b>NOTE:</b>It can update {@link #isIdle()} status.
+     *
+     * @return true, if the combined watermark changed
+     */
+    public boolean updateCombinedWatermark() {
+        long minimumOverAllOutputs = Long.MAX_VALUE;
+
+        boolean hasOutputs = false;
+        boolean allIdle = true;
+        for (PartialWatermark partialWatermark : partialWatermarks) {
+            if (!partialWatermark.isIdle()) {
+                minimumOverAllOutputs =
+                        Math.min(minimumOverAllOutputs, 
partialWatermark.getWatermark());
+                allIdle = false;
+            }
+            hasOutputs = true;
+        }
+
+        // if we don't have any outputs minimumOverAllOutputs is not valid, 
it's still
+        // at its initial Long.MAX_VALUE state and we must not emit that
+        this.idle = allIdle;
+        if (!hasOutputs || allIdle) {
+            return false;
+        }
+
+        if (minimumOverAllOutputs > combinedWatermark) {
+            combinedWatermark = minimumOverAllOutputs;
+            return true;
+        }
+
+        return false;
+    }
+
+    /** Per-output watermark state. */
+    public static class PartialWatermark {

Review comment:
       I think you'd have an easier time with an immutable type. Then you don't 
need a public `updateCombinedWatermark` but just update on `set`.
   But it's probably a bigger change, so it's okay to ignore.

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/common/eventtime/CombinedWatermark.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.api.common.eventtime;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link CombinedWatermark} combines the watermark (and idleness) updates 
of multiple
+ * partitions/shards/splits into one combined watermark.
+ */
+@Internal
+public final class CombinedWatermark extends 
AbstractList<CombinedWatermark.PartialWatermark> {

Review comment:
       Idea: rename `XWatermark` to `XWatermarkStatus` to indicate that it also 
contains idle state? 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
##########
@@ -628,22 +632,42 @@ public void processWatermark(Watermark mark) throws 
Exception {
         output.emitWatermark(mark);
     }
 
-    public void processWatermark1(Watermark mark) throws Exception {
-        input1Watermark = mark.getTimestamp();
-        long newMin = Math.min(input1Watermark, input2Watermark);
-        if (newMin > combinedWatermark) {
-            combinedWatermark = newMin;
-            processWatermark(new Watermark(combinedWatermark));
+    private void processWatermark(Watermark mark, int index) throws Exception {
+        combinedWatermark.get(index).setWatermark(mark.getTimestamp());

Review comment:
       Maybe you shouldn't expose `PartialWatermark` at all and just have two 
setters (status + watermark) per input index. This setter can do `update` 
implicitly and return if there was an update or not.
   
   ```
   if (combinedWatermark.updateWatermark(index, mark.getTimestamp())) {
     processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
   }
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
##########
@@ -243,6 +240,13 @@ public void setup(
         stateKeySelector2 = config.getStatePartitioner(1, 
getUserCodeClassloader());
     }
 
+    private CombinedWatermark setupCombinedWatermark() {
+        CombinedWatermark combinedWatermark = new CombinedWatermark();
+        combinedWatermark.add(new CombinedWatermark.PartialWatermark());
+        combinedWatermark.add(new CombinedWatermark.PartialWatermark());

Review comment:
       shouldn't that depend on the number of inputs? Can't we create that in 
ctor?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
##########
@@ -628,22 +632,42 @@ public void processWatermark(Watermark mark) throws 
Exception {
         output.emitWatermark(mark);
     }
 
-    public void processWatermark1(Watermark mark) throws Exception {
-        input1Watermark = mark.getTimestamp();
-        long newMin = Math.min(input1Watermark, input2Watermark);
-        if (newMin > combinedWatermark) {
-            combinedWatermark = newMin;
-            processWatermark(new Watermark(combinedWatermark));
+    private void processWatermark(Watermark mark, int index) throws Exception {
+        combinedWatermark.get(index).setWatermark(mark.getTimestamp());
+        if (combinedWatermark.updateCombinedWatermark()) {
+            processWatermark(new 
Watermark(combinedWatermark.getCombinedWatermark()));
         }
     }
 
+    public void processWatermark1(Watermark mark) throws Exception {
+        processWatermark(mark, 0);
+    }
+
     public void processWatermark2(Watermark mark) throws Exception {
-        input2Watermark = mark.getTimestamp();
-        long newMin = Math.min(input1Watermark, input2Watermark);
-        if (newMin > combinedWatermark) {
-            combinedWatermark = newMin;
-            processWatermark(new Watermark(combinedWatermark));
+        processWatermark(mark, 1);
+    }
+
+    public final void emitStreamStatus(StreamStatus streamStatus) throws 
Exception {
+        output.emitStreamStatus(streamStatus);
+    }
+
+    private void emitStreamStatus(StreamStatus streamStatus, int index) {
+        boolean wasIdle = combinedWatermark.isIdle();
+        combinedWatermark.get(index).setIdle(streamStatus.isIdle());
+        if (combinedWatermark.updateCombinedWatermark()) {
+            output.emitWatermark(new 
Watermark(combinedWatermark.getCombinedWatermark()));
         }
+        if (wasIdle != combinedWatermark.isIdle()) {
+            output.emitStreamStatus(streamStatus);
+        }

Review comment:
       ```
   if (combinedWatermark.updateStatus(index, streamStatus.isIdle())) {
     output.emitStreamStatus(streamStatus);
   }
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
##########
@@ -48,20 +47,19 @@
 
     private SerializationDelegate<StreamElement> serializationDelegate;
 
-    private final StreamStatusProvider streamStatusProvider;
-
     private final boolean supportsUnalignedCheckpoints;
 
     private final OutputTag outputTag;
 
     private final WatermarkGauge watermarkGauge = new WatermarkGauge();
 
+    private StreamStatus announcedStatus = StreamStatus.ACTIVE;

Review comment:
       Is active the default throughout whole Flink?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
##########
@@ -149,15 +142,16 @@ public void close() {}
 
         private long lastRecordTime;
 
+        private volatile boolean idle = false;
+
         private AutomaticWatermarkContext(
                 final Output<StreamRecord<T>> output,
                 final long watermarkInterval,
                 final ProcessingTimeService timeService,
                 final Object checkpointLock,
-                final StreamStatusMaintainer streamStatusMaintainer,
                 final long idleTimeout) {
 
-            super(timeService, checkpointLock, streamStatusMaintainer, 
idleTimeout);
+            super(timeService, checkpointLock, idleTimeout);

Review comment:
       Removing stream status maintainer feels like an independent change. Can 
we move that into a separate commit? It makes reviewing harder.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java
##########
@@ -146,6 +141,7 @@ public 
AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int nu
         executionConfig = parameters.getContainingTask().getExecutionConfig();
         userCodeClassLoader = 
parameters.getContainingTask().getUserCodeClassLoader();
         cancelables = parameters.getContainingTask().getCancelables();
+        this.combinedWatermark = setupCombinedWatermark(numberOfInputs);

Review comment:
       ```
   combinedWatermark = new CombinedWatermark(numberOfInputs);
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
##########
@@ -101,32 +97,51 @@ public void collect(StreamRecord<OUT> record) {
     }
 
     private <X> void pushToRecordWriter(StreamRecord<X> record) {
-        serializationDelegate.setInstance(record);
+        if (announcedStatus.isIdle()) {

Review comment:
       I think we need to add a comment here to explain why this can happen.

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/common/eventtime/CombinedWatermark.java
##########
@@ -0,0 +1,148 @@
+/*

Review comment:
       This commit is not a hotfix.

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/common/eventtime/CombinedWatermark.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.api.common.eventtime;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link CombinedWatermark} combines the watermark (and idleness) updates 
of multiple
+ * partitions/shards/splits into one combined watermark.
+ */
+@Internal
+public final class CombinedWatermark extends 
AbstractList<CombinedWatermark.PartialWatermark> {

Review comment:
       Not too sure if extending `AbstractList` is the best course in action. 
Do you even need all the methods? What happens when you just remove the parent 
class?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java
##########
@@ -137,7 +132,7 @@ public 
AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int nu
             output = countingOutput;
         }
 
-        latencyStats =
+        this.latencyStats =

Review comment:
       unnecessary change




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