sqd commented on code in PR #15433:
URL: https://github.com/apache/iceberg/pull/15433#discussion_r3087697553
##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java:
##########
@@ -357,43 +416,79 @@ private String operatorName(String suffix) {
return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
}
- private DynamicIcebergSink build() {
+ private DynamicIcebergSink build(
+ SingleOutputStreamOperator<DynamicRecordInternal> converted,
+ DynamicRecordInternalType sideOutputType) {
Preconditions.checkArgument(
generator != null, "Please use withGenerator() to convert the input
DataStream.");
Preconditions.checkNotNull(catalogLoader, "Catalog loader shouldn't be
null");
- uidPrefix = Optional.ofNullable(uidPrefix).orElse("");
-
Configuration flinkConfig =
readableConfig instanceof Configuration
? (Configuration) readableConfig
: Configuration.fromMap(readableConfig.toMap());
- return instantiateSink(writeOptions, flinkConfig);
+ // Forward writer: chained with generator via forward edge, no data
shuffle
+ ForwardWriterSink forwardWriterSink =
+ new ForwardWriterSink(catalogLoader, writeOptions, flinkConfig,
cacheMaximumSize);
+ TypeInformation<CommittableMessage<DynamicWriteResult>>
writeResultTypeInfo =
+ CommittableMessageTypeInfo.of(DynamicWriteResultSerializer::new);
+
+ DataStream<CommittableMessage<DynamicWriteResult>> forwardWriteResults =
+ converted
+ .getSideOutput(
+ new
OutputTag<>(DynamicRecordProcessor.DYNAMIC_FORWARD_STREAM, sideOutputType))
+ .transform(
+ operatorName("Forward-Writer"),
+ writeResultTypeInfo,
+ new SinkWriterOperatorFactory<>(forwardWriterSink))
+ .uid(prefixIfNotNull(uidPrefix, "-forward-writer"));
Review Comment:
Now using `containsOnly`, and added the uid for the forward writer
--
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]