pnowojski commented on code in PR #22432:
URL: https://github.com/apache/flink/pull/22432#discussion_r1189477827
##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java:
##########
@@ -996,6 +1002,88 @@ public void testCanEmitBatchOfRecords() throws Exception {
}
}
+ @Test
+ public void testTaskSideOutputStatistics() throws Exception {
+ TaskMetricGroup taskMetricGroup =
+ UnregisteredMetricGroups.createUnregisteredTaskMetricGroup();
+
+ ResultPartitionWriter[] partitionWriters = new
ResultPartitionWriter[3];
+ for (int i = 0; i < partitionWriters.length; ++i) {
+ partitionWriters[i] =
+ new RecordOrEventCollectingResultPartitionWriter<>(
+ new ArrayDeque<>(),
+ new StreamElementSerializer<>(
+
BasicTypeInfo.INT_TYPE_INFO.createSerializer(
+ new ExecutionConfig())));
+ partitionWriters[i].setup();
+ }
+
+ try (StreamTaskMailboxTestHarness<Integer> testHarness =
+ new StreamTaskMailboxTestHarnessBuilder<>(
+ OneInputStreamTask::new,
BasicTypeInfo.INT_TYPE_INFO)
+ .addInput(BasicTypeInfo.INT_TYPE_INFO)
+ .addAdditionalOutput(partitionWriters)
+ .setupOperatorChain(new OperatorID(), new
OddEvenOperator())
+
.chain(BasicTypeInfo.INT_TYPE_INFO.createSerializer(new ExecutionConfig()))
+ .setOperatorFactory(SimpleOperatorFactory.of(new
OddEvenOperator()))
+ .addNonChainedOutputsCount(
+ new OutputTag<>("odd",
BasicTypeInfo.INT_TYPE_INFO), 2)
+ .addNonChainedOutputsCount(1)
+ .build()
+
.chain(BasicTypeInfo.INT_TYPE_INFO.createSerializer(new ExecutionConfig()))
+ .setOperatorFactory(SimpleOperatorFactory.of(new
DuplicatingOperator()))
+ .addNonChainedOutputsCount(1)
+ .build()
+ .finish()
+ .setTaskMetricGroup(taskMetricGroup)
+ .build()) {
+ Counter numRecordsInCounter =
+
taskMetricGroup.getIOMetricGroup().getNumRecordsInCounter();
+ Counter numRecordsOutCounter =
+
taskMetricGroup.getIOMetricGroup().getNumRecordsOutCounter();
+
+ final int numEvenRecords = 5;
+ final int numOddRecords = 3;
+
+ for (int x = 0; x < numEvenRecords; x++) {
+ testHarness.processElement(new StreamRecord<>(2 * x));
+ }
+
+ for (int x = 0; x < numOddRecords; x++) {
+ testHarness.processElement(new StreamRecord<>(2 * x + 1));
+ }
+ assertEquals(numOddRecords + numEvenRecords,
numRecordsInCounter.getCount());
+ assertEquals(
+ numOddRecords
+ + (numOddRecords + numEvenRecords)
+ + (numOddRecords + numEvenRecords) * 2,
+ numRecordsOutCounter.getCount());
Review Comment:
Thanks for the explanation. I think I understand it. One thing is not clear
to me.
> The first operator does not have NonChainedOutput
So what is going on with the first operator 🤔? That first operator is trying
to emit records with tags, but tags are only supported in non-chained outputs.
First operator doesn't have any, and as you pointed out, `ChainedOutput` is
ignoring records with tags. So the `if/else` branch in the first operator's
`processElement` is effectively a dead code?
If the above is correct, is this construct officially supported behaviour of
Flink? Can user create such pipelines? If not, I would either remove the first
operator, or replace it with just a no-op/pass-through operator that just
forwards input records.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]