stevenzwu commented on code in PR #10179: URL: https://github.com/apache/iceberg/pull/10179#discussion_r1707839978
########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java: ########## @@ -0,0 +1,811 @@ +/* + * 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; + +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +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.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.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.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.committer.IcebergCommittable; +import org.apache.iceberg.flink.sink.committer.IcebergCommittableSerializer; +import org.apache.iceberg.flink.sink.committer.IcebergCommitter; +import org.apache.iceberg.flink.sink.committer.IcebergWriteAggregator; +import org.apache.iceberg.flink.sink.committer.WriteResultSerializer; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + * <ul> + * <li>{@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + * <li>{@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data files, and + * generates a {@link IcebergCommittable} to store the {@link + * org.apache.iceberg.io.WriteResult} + * <li>{@link SupportsPreCommitTopology} which we use to to place the {@link + * IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.DeltaManifests} + * <li>{@link IcebergCommitter} which stores the incoming {@link + * org.apache.iceberg.flink.sink.DeltaManifests}s in state for recovery, and commits them to + * the Iceberg table + * <li>{@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + * </ul> + * + * The job graph looks like below: + * + * <pre>{@code + * Flink sink + * +-----------------------------------------------------------------------------------+ + * | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | Map 1 | ==> | | writer 1 | | committer 1 | ---> | post commit 1 | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | \ / \ | + * | \ / \ | + * | \ / \ | + * +-------+ | +----------+ \ +-------------------+ / +-------------+ \ +---------------+ | + * | Map 2 | ==> | | writer 2 | --->| commit aggregator | | committer 2 | | post commit 2 | | + * +-------+ | +----------+ +-------------------+ +-------------+ +---------------+ | + * | Commit only on | + * | committer 1 | + * +-----------------------------------------------------------------------------------+ + * }</pre> + */ +@Experimental +public class IcebergSink + implements Sink<RowData>, + SupportsPreWriteTopology<RowData>, + SupportsCommitter<IcebergCommittable>, + SupportsPreCommitTopology<WriteResult, IcebergCommittable>, + SupportsPostCommitTopology<IcebergCommittable> { + private final Table initTable; + private final TableLoader tableLoader; + private final Map<String, String> snapshotProperties; + private final String uidPrefix; + private final String sinkId; + private final Map<String, String> writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier<Table> tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List<Integer> equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + private final Table table; + private final List<String> equalityFieldColumns = null; + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + + private IcebergSink( + Table initTable, + TableLoader tableLoader, + Table table, + Map<String, String> snapshotProperties, + String uidPrefix, Review Comment: how do we set uid for the operators created by the v2 sink framework like writer and committer operators? -- 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