mxm commented on code in PR #13341: URL: https://github.com/apache/iceberg/pull/13341#discussion_r2154123122
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java: ########## @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.io.IOException; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.flink.sink.SinkUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** + * Dynamic version of the IcebergSink which supports: + * + * <ul> + * <li>Writing to any number of tables (No more 1:1 sink/topic relationship). + * <li>Creating and updating tables based on the user-supplied routing. + * <li>Updating the schema and partition spec of tables based on the user-supplied specification. + * </ul> + */ +@Experimental +public class DynamicIcebergSink + implements Sink<DynamicRecordInternal>, + SupportsPreWriteTopology<DynamicRecordInternal>, + SupportsCommitter<DynamicCommittable>, + SupportsPreCommitTopology<DynamicWriteResult, DynamicCommittable>, + SupportsPostCommitTopology<DynamicCommittable> { + + private final CatalogLoader catalogLoader; + private final Map<String, String> snapshotProperties; + private final String uidPrefix; + private final String sinkId; + private final Map<String, String> writeProperties; + private final transient FlinkWriteConf flinkWriteConf; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final boolean overwriteMode; + private final int workerPoolSize; + private final int cacheMaximumSize; + + DynamicIcebergSink( + CatalogLoader catalogLoader, + Map<String, String> snapshotProperties, + String uidPrefix, + Map<String, String> writeProperties, + FlinkWriteConf flinkWriteConf, + int cacheMaximumSize) { + this.catalogLoader = catalogLoader; + this.snapshotProperties = snapshotProperties; + this.uidPrefix = uidPrefix; + this.writeProperties = writeProperties; + this.flinkWriteConf = flinkWriteConf; + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.overwriteMode = flinkWriteConf.overwriteMode(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + this.cacheMaximumSize = cacheMaximumSize; + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter<DynamicRecordInternal> createWriter(InitContext context) throws IOException { + return new DynamicWriter( + catalogLoader.loadCatalog(), + dataFileFormat, + targetDataFileSize, + writeProperties, + cacheMaximumSize, + new DynamicWriterMetrics(context.metricGroup()), + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); + } + + @Override + public Committer<DynamicCommittable> createCommitter(CommitterInitContext context) { Review Comment: `CommitterInitContext` is used here as opposed to `InitContext` in Flink 2.0. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org