You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "pnowojski (via GitHub)" <gi...@apache.org> on 2023/04/24 10:10:58 UTC

[GitHub] [flink] pnowojski commented on a diff in pull request #22432: [FLINK-18808][runtime] Include side outputs in numRecordsOut metric

pnowojski commented on code in PR #22432:
URL: https://github.com/apache/flink/pull/22432#discussion_r1173917579


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CopyingBroadcastingOutputCollector.java:
##########
@@ -27,37 +29,57 @@
  */
 final class CopyingBroadcastingOutputCollector<T> extends BroadcastingOutputCollector<T> {
 
-    public CopyingBroadcastingOutputCollector(Output<StreamRecord<T>>[] outputs) {
-        super(outputs);
+    public CopyingBroadcastingOutputCollector(
+            Output<StreamRecord<T>>[] allOutputs, Counter numRecordsOutForTask) {
+        super(allOutputs, numRecordsOutForTask);
     }
 
     @Override
     public void collect(StreamRecord<T> record) {
+        boolean emitted = false;
+        for (int i = 0; i < nonChainedOutputs.length - 1; i++) {
+            RecordWriterOutput<T> output = nonChainedOutputs[i];
+            StreamRecord<T> shallowCopy = record.copy(record.getValue());
+            emitted |= output.collectAndCheckIfEmitted(shallowCopy);
+        }
 
-        for (int i = 0; i < outputs.length - 1; i++) {
-            Output<StreamRecord<T>> output = outputs[i];
+        if (chainedOutputs.length == 0 && nonChainedOutputs.length > 0) {
+            emitted |=
+                    nonChainedOutputs[nonChainedOutputs.length - 1].collectAndCheckIfEmitted(
+                            record);
+        } else if (nonChainedOutputs.length > 0) {
             StreamRecord<T> shallowCopy = record.copy(record.getValue());
-            output.collect(shallowCopy);
+            emitted |=
+                    nonChainedOutputs[nonChainedOutputs.length - 1].collectAndCheckIfEmitted(
+                            shallowCopy);
         }
 
-        if (outputs.length > 0) {
-            // don't copy for the last output
-            outputs[outputs.length - 1].collect(record);
+        if (chainedOutputs.length > 0) {
+            for (int i = 0; i < chainedOutputs.length - 1; i++) {
+                Output<StreamRecord<T>> output = chainedOutputs[i];
+                StreamRecord<T> shallowCopy = record.copy(record.getValue());
+                output.collect(shallowCopy);
+            }
+            chainedOutputs[chainedOutputs.length - 1].collect(record);
+        }
+
+        if (emitted) {
+            numRecordsOutForTask.inc();
         }
     }
 
     @Override
     public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {
-        for (int i = 0; i < outputs.length - 1; i++) {
-            Output<StreamRecord<T>> output = outputs[i];
+        for (int i = 0; i < allOutputs.length - 1; i++) {
+            Output<StreamRecord<T>> output = allOutputs[i];
 
             StreamRecord<X> shallowCopy = record.copy(record.getValue());
             output.collect(outputTag, shallowCopy);
         }
 
-        if (outputs.length > 0) {
+        if (allOutputs.length > 0) {
             // don't copy for the last output
-            outputs[outputs.length - 1].collect(outputTag, record);
+            allOutputs[allOutputs.length - 1].collect(outputTag, record);
         }
     }
 }

Review Comment:
   Shouldn't this be also changed?
   
   Doesn't it mean we are missing unit tests for the this?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java:
##########
@@ -17,52 +17,77 @@
 
 package org.apache.flink.streaming.runtime.tasks;
 
+import org.apache.flink.metrics.Counter;
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
 import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
 import org.apache.flink.util.OutputTag;
 import org.apache.flink.util.XORShiftRandom;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Random;
 
 class BroadcastingOutputCollector<T> implements WatermarkGaugeExposingOutput<StreamRecord<T>> {
 
-    protected final Output<StreamRecord<T>>[] outputs;
+    protected final Output<StreamRecord<T>>[] allOutputs;
+
+    protected final Output<StreamRecord<T>>[] chainedOutputs;
+
+    protected final RecordWriterOutput<T>[] nonChainedOutputs;
+
+    protected final Counter numRecordsOutForTask;
     private final Random random = new XORShiftRandom();
     private final WatermarkGauge watermarkGauge = new WatermarkGauge();
 
-    public BroadcastingOutputCollector(Output<StreamRecord<T>>[] outputs) {
-        this.outputs = outputs;
+    @SuppressWarnings({"unchecked"})
+    public BroadcastingOutputCollector(
+            Output<StreamRecord<T>>[] allOutputs, Counter numRecordsOutForTask) {
+        this.allOutputs = allOutputs;
+        this.numRecordsOutForTask = numRecordsOutForTask;
+
+        List<Output<StreamRecord<T>>> chainedOutputs = new ArrayList<>(4);
+        List<RecordWriterOutput<T>> nonChainedOutputs = new ArrayList<>(4);
+        for (Output<StreamRecord<T>> output : allOutputs) {
+            if (output instanceof RecordWriterOutput) {
+                nonChainedOutputs.add((RecordWriterOutput<T>) output);
+            } else {
+                chainedOutputs.add(output);
+            }
+        }
+        this.chainedOutputs = chainedOutputs.toArray(new Output[0]);
+        this.nonChainedOutputs = nonChainedOutputs.toArray(new RecordWriterOutput[0]);

Review Comment:
   Could we move this logic of splitting into separate lists up, for example to the `OperatorChain`? Here `instanceof` check might be fragile in the future, while in the `OperatorChain#createOutputCollector`  it looks like we have this information at hand when we iterate over `operatorConfig`.
   
   Actually, wouldn't it simplify the logic, especially in `CopyingBroadcastingOutputCollector`, if replaced `chainedOutputs` and `nonChainedOutputs` arrays, with a single array `boolean[] isOutputChained` (also constructed at the `OperatorChain` level? Then Instead of iterating over two arrays one after another, you could easily handle everything in the one loop:
   
   ```
   for (int i = 0; i < outputs.length; i++) {
       ...
       if (isOutputChained[i]) {
           ...
       }
   }
   ```
   



-- 
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-unsubscribe@flink.apache.org

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