reswqa commented on code in PR #22432: URL: https://github.com/apache/flink/pull/22432#discussion_r1175224675
########## 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: >if replaced chainedOutputs and nonChainedOutputs arrays, with a single array boolean[] isOutputChained That's really good advice. However, it seems that type conversions like the following cannot be completely avoided, because `CollectAndCheckIfEmitted` is placed in `RecordWriterOutput`. I'm concerned about the performance impact of this. Another approach is that we move `CollectAndCheckIfEmitted` to the interface `Output`/`Collector`, but this will touch the `public` API, or we need to introduce a special interface to handle this? ``` for (int i = 0; i < outputs.length; i++) { if (isOutputChained[i]) { outputs[i].collect(record); } else { emitted |= ((RecordWriterOutput<T>) outputs[i]).collectAndCheckIfEmitted(record); } } ``` ########## 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: >if replaced chainedOutputs and nonChainedOutputs arrays, with a single array boolean[] isOutputChained That's really good advice. However, it seems that type conversions like the following cannot be completely avoided, because `CollectAndCheckIfEmitted` is placed in `RecordWriterOutput`. I'm concerned about the performance impact of this. Another approach is that we move `CollectAndCheckIfEmitted` to the interface `Output`/`Collector`, but this will touch the `public` API, or we need to introduce a special interface to handle this? 🤔 ``` for (int i = 0; i < outputs.length; i++) { if (isOutputChained[i]) { outputs[i].collect(record); } else { emitted |= ((RecordWriterOutput<T>) outputs[i]).collectAndCheckIfEmitted(record); } } ``` -- 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