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


##########
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:
   Good catch! 👍  
   
   I somehow lost the changes here when migrating from the original code. You 
are right, I found that the previous test did not cover `ObjectReuse`, I will 
modify tests to cover this case. 



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

Reply via email to