pvary commented on code in PR #15433:
URL: https://github.com/apache/iceberg/pull/15433#discussion_r2964444746
##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java:
##########
@@ -430,16 +438,29 @@ public DataStreamSink<DynamicRecordInternal> append() {
.uid(prefixIfNotNull(uidPrefix, "-updater"))
.name(operatorName("Updater"))
.returns(type)
- .union(converted)
- .sinkTo(sink)
- .uid(prefixIfNotNull(uidPrefix, "-sink"));
+ .union(converted);
+
+ DataStreamSink<DynamicRecordInternal> shuffleSinkResult =
+ shuffleInput.sinkTo(shuffleSink).uid(prefixIfNotNull(uidPrefix,
"-shuffle-sink"));
FlinkWriteConf flinkWriteConf = new FlinkWriteConf(writeOptions,
readableConfig);
if (flinkWriteConf.writeParallelism() != null) {
-
rowDataDataStreamSink.setParallelism(flinkWriteConf.writeParallelism());
+ shuffleSinkResult.setParallelism(flinkWriteConf.writeParallelism());
}
- return rowDataDataStreamSink;
+ // Forward sink: handles forward records (distributionMode == null,
chainable)
+ DynamicIcebergSink forwardSink = build(true);
+ OutputTag<DynamicRecordInternal> forwardTag =
+ new OutputTag<>(
+ DynamicRecordProcessor.DYNAMIC_FORWARD_STREAM,
+ new DynamicRecordInternalType(catalogLoader, true,
cacheMaximumSize));
+
+ converted
+ .getSideOutput(forwardTag)
+ .sinkTo(forwardSink)
+ .uid(prefixIfNotNull(uidPrefix, "-forward-sink"));
+
+ return shuffleSinkResult;
Review Comment:
Do I understand correctly that we create 2 instances of DynamicSinks here?
So basically the 2 sinks doesn't share the
writers/pre-commit-topology/committers etc?
Also, do I understand correctly that the resulting stream only contains the
output of the shuffling sink?
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]