pnowojski commented on code in PR #22432: URL: https://github.com/apache/flink/pull/22432#discussion_r1180210194
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputWithRecordsCountCheck.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** This is a wrapper for outputs to check whether its record output needs to be counted. */ +@Internal +public interface OutputWithRecordsCountCheck<OUT> { Review Comment: I think this should extend from `Output`, otherwise it can lead to strange issues like ```public class X implements Output<Foo>, OutputWithRecordsCountCheck<Bar>``` ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputWithRecordsCountCheck.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** This is a wrapper for outputs to check whether its record output needs to be counted. */ Review Comment: ``` This is a wrapper for outputs to check whether the collected record has been emitted to a downstream subtask or to a chained operator ``` ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java: ########## @@ -774,6 +801,14 @@ private <T> WatermarkGaugeExposingOutput<StreamRecord<T>> createOutputCollector( return result; } + private static Counter createStreamCounter(StreamTask<?, ?> containingTask) { Review Comment: rename to `createNumRecordsOutCounter`? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java: ########## @@ -743,22 +747,45 @@ private <T> WatermarkGaugeExposingOutput<StreamRecord<T>> createOutputCollector( if (allOutputs.size() == 1) { result = allOutputs.get(0); + // only if this is a single RecordWriterOutput, reuse its numRecordOut for task. + if (result instanceof RecordWriterOutput) { + TaskIOMetricGroup taskIOMetricGroup = + containingTask.getEnvironment().getMetricGroup().getIOMetricGroup(); + Counter counter = new SimpleCounter(); + ((RecordWriterOutput<T>) result).setNumRecordsOut(counter); + taskIOMetricGroup.reuseRecordsOutputCounter(counter); Review Comment: reuse `createNumRecordsOutCounter`? ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java: ########## @@ -262,4 +289,143 @@ public StreamConfigChainer<OWNER> name(String name) { public void setBufferTimeout(int bufferTimeout) { this.bufferTimeout = bufferTimeout; } + + /** Helper class to build operator node. */ + public static class StreamConfigEdgeChainer<OWNER, IN, OUT> { Review Comment: nit: for the future, it would be easier to review, if you added this builder class in a separate commit, preceding your changes. Then in your bug fix you could have only added `outEdgesInOrder` and `setTailNonChainedOutputs ` fields to the builder, which would make the bug fix commit smaller and easier to review. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java: ########## @@ -31,12 +32,19 @@ class BroadcastingOutputCollector<T> implements WatermarkGaugeExposingOutput<StreamRecord<T>> { + protected final OutputWithRecordsCountCheck<StreamRecord<T>>[] outputWithRecordsCountChecks; protected final Output<StreamRecord<T>>[] outputs; private final Random random = new XORShiftRandom(); private final WatermarkGauge watermarkGauge = new WatermarkGauge(); + protected final Counter numRecordsOutForTask; - public BroadcastingOutputCollector(Output<StreamRecord<T>>[] outputs) { + public BroadcastingOutputCollector( + OutputWithRecordsCountCheck<StreamRecord<T>>[] outputWithRecordsCountChecks, + Output<StreamRecord<T>>[] outputs, + Counter numRecordsOutForTask) { this.outputs = outputs; + this.outputWithRecordsCountChecks = outputWithRecordsCountChecks; Review Comment: That could be solved by making `OutputWithRecordsCountCheck` extend from `Output`. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputWithRecordsCountCheck.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** This is a wrapper for outputs to check whether its record output needs to be counted. */ +@Internal +public interface OutputWithRecordsCountCheck<OUT> { Review Comment: Maybe let's rename this to `OutputWithChainingCheck`? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputWithRecordsCountCheck.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** This is a wrapper for outputs to check whether its record output needs to be counted. */ +@Internal +public interface OutputWithRecordsCountCheck<OUT> { + /** + * Collect a record and check if it needs to be counted. + * + * @param record The record to collect. + */ + boolean collectAndCheckIfCountNeeded(OUT record); + + /** + * Collect a record to the side output identified by the given {@link OutputTag} and check if it + * needs to be counted. + * + * @param record The record to collect. + * @param outputTag Identification of side outputs. + */ + <X> boolean collectAndCheckIfCountNeeded(OutputTag<X> outputTag, StreamRecord<X> record); Review Comment: I would drop those java docs, as they are repeating the class level java doc. `@params` are not adding much information as well. However I would explain what does the return value mean. ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java: ########## @@ -77,6 +86,7 @@ private void head(OperatorID headOperatorID) { headConfig.setChainIndex(chainIndex); } + @Deprecated Review Comment: nit: could you add a java-doc pointing to what should be used instead of those deprecate methods? -- 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