mxm commented on code in PR #15433:
URL: https://github.com/apache/iceberg/pull/15433#discussion_r2851572372
##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java:
##########
@@ -411,28 +447,38 @@ public DataStreamSink<DynamicRecordInternal> append() {
.name(operatorName("generator"))
.returns(type);
- DataStreamSink<DynamicRecordInternal> rowDataDataStreamSink =
- converted
- .getSideOutput(
- new OutputTag<>(
- DynamicRecordProcessor.DYNAMIC_TABLE_UPDATE_STREAM,
- new DynamicRecordInternalType(catalogLoader, true,
cacheMaximumSize)))
- .keyBy((KeySelector<DynamicRecordInternal, String>)
DynamicRecordInternal::tableName)
- .map(
- new DynamicTableUpdateOperator(
- catalogLoader,
- cacheMaximumSize,
- cacheRefreshMs,
- inputSchemasPerTableCacheMaximumSize,
- tableCreator,
- caseSensitive,
- dropUnusedColumns))
- .uid(prefixIfNotNull(uidPrefix, "-updater"))
- .name(operatorName("Updater"))
- .returns(type)
- .union(converted)
- .sinkTo(sink)
- .uid(prefixIfNotNull(uidPrefix, "-sink"));
+ DataStreamSink<DynamicRecordInternal> rowDataDataStreamSink;
+ if (passthroughRecords) {
+ if (!immediateUpdate) {
+ throw new UnsupportedOperationException(
+ "Immediate update must be enabled to pass through records");
+ }
+ rowDataDataStreamSink =
converted.sinkTo(sink).uid(prefixIfNotNull(uidPrefix, "-sink"));
+ } else {
Review Comment:
This will ignore `DistributionMode` and partitioning in `DynamicRecord`. I
saw that you listed this in the docs, but I'm not sure we should diverge too
much from the normal mode of operation. I think what we can do, is to add a new
chained side output with an extra DynamicWriter for this quick path.
It may be worth adding a new DistributionMode. Currently NONE does a
round-robin, which is slightly confusing, we could rename it to ROUND_ROBIN and
use NONE for this direct path.
--
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]