Guosmilesmile commented on code in PR #15279:
URL: https://github.com/apache/iceberg/pull/15279#discussion_r2793445740
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java:
##########
@@ -184,4 +226,55 @@ public String asSummaryString() {
public void applyOverwrite(boolean newOverwrite) {
this.overwrite = newOverwrite;
}
+
+ private DataStreamSink<?> createDynamicIcebergSink(DataStream<RowData>
dataStream) {
+ TableCreator tableCreator = createTableCreator();
+ DynamicRecordGenerator<RowData> generator =
+ createDynamicRecordGenerator(dynamicRecordGeneratorImpl);
+
+ DynamicIcebergSink.Builder<RowData> builder =
+ DynamicIcebergSink.forInput(dataStream)
+ .generator(generator)
+ .catalogLoader(catalogLoader)
+ .setAll(writeProps)
+ .tableCreator(tableCreator)
+ .flinkConf(readableConfig);
+
+ return builder.append();
+ }
+
+ private TableCreator createTableCreator() {
+ final Map<String, String> tableProperties =
+ org.apache.iceberg.util.PropertyUtil.propertiesWithPrefix(writeProps,
"table.props.");
Review Comment:
If I’m not mistaken, if we want to set the table property
`write.parquet.row-group-size-bytes`, do we need to specify it here as
`table.props.write.parquet.row-group-size-bytes`? I think this should be
documented and we should add a corresponding test case.
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java:
##########
@@ -131,6 +140,45 @@ public String factoryIdentifier() {
return FACTORY_IDENTIFIER;
}
+ private IcebergTableSink getIcebergTableSinkWithDynamicSinkProps(
+ Context context, Configuration flinkConf, Map<String, String>
writeProps) {
+ String dynamicRecordGeneratorImpl =
+ flinkConf.get(FlinkCreateTableOptions.DYNAMIC_RECORD_GENERATOR_IMPL);
+ Preconditions.checkNotNull(
+ dynamicRecordGeneratorImpl,
+ "%s must be specified when use-dynamic-iceberg-sink is true",
+ FlinkCreateTableOptions.DYNAMIC_RECORD_GENERATOR_IMPL.key());
+
+ CatalogLoader catalogLoader;
+ if (catalog != null) {
+ catalogLoader = catalog.getCatalogLoader();
+ } else {
+ FlinkCatalog flinkCatalog =
+ createCatalogLoader(writeProps,
flinkConf.get(FlinkCreateTableOptions.CATALOG_NAME));
+ catalogLoader = flinkCatalog.getCatalogLoader();
+ }
+ ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable();
Review Comment:
nit: new line.
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java:
##########
@@ -184,4 +226,55 @@ public String asSummaryString() {
public void applyOverwrite(boolean newOverwrite) {
this.overwrite = newOverwrite;
}
+
+ private DataStreamSink<?> createDynamicIcebergSink(DataStream<RowData>
dataStream) {
+ TableCreator tableCreator = createTableCreator();
+ DynamicRecordGenerator<RowData> generator =
+ createDynamicRecordGenerator(dynamicRecordGeneratorImpl);
+
+ DynamicIcebergSink.Builder<RowData> builder =
+ DynamicIcebergSink.forInput(dataStream)
+ .generator(generator)
+ .catalogLoader(catalogLoader)
+ .setAll(writeProps)
+ .tableCreator(tableCreator)
+ .flinkConf(readableConfig);
+
+ return builder.append();
+ }
+
+ private TableCreator createTableCreator() {
+ final Map<String, String> tableProperties =
+ org.apache.iceberg.util.PropertyUtil.propertiesWithPrefix(writeProps,
"table.props.");
Review Comment:
Do we need to use the package name here?
--
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]