hililiwei commented on code in PR #4904: URL: https://github.com/apache/iceberg/pull/4904#discussion_r990934356
########## flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java: ########## @@ -0,0 +1,696 @@ +/* + * 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.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.StatefulSink; +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.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; +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.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.FilesCommittable; +import org.apache.iceberg.flink.sink.committer.FilesCommittableSerializer; +import org.apache.iceberg.flink.sink.committer.FilesCommitter; +import org.apache.iceberg.flink.sink.writer.StreamWriter; +import org.apache.iceberg.flink.sink.writer.StreamWriterState; +import org.apache.iceberg.flink.sink.writer.StreamWriterStateSerializer; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. It includes + * writers that can write data to files in parallel and route commit info globally to one Committer. + * Post commit topology will take of compacting the already written files and updating the file log + * after the compaction. + * + * <pre>{@code + * Flink sink + * +------------------------------------------------------------------+ + * | | + * +-------+ | +-------------+ +---------------+ | + * | Map 1 | ==> | | writer 1 | =| | committer 1 | | + * +-------+ | +-------------+ +---------------+ | + * | \ | + * | \ | + * | \ | + * +-------+ | +-------------+ \ +---------------+ +---------------+ | + * | Map 2 | ==> | | writer 2 | =| --- >| committer 2 | ---> | post commit | | + * +-------+ | +-------------+ +---------------+ +---------------+ | + * | | + * +------------------------------------------------------------------+ + * }</pre> + */ +public class IcebergSink + implements StatefulSink<RowData, StreamWriterState>, + WithPreWriteTopology<RowData>, + WithPreCommitTopology<RowData, FilesCommittable>, + WithPostCommitTopology<RowData, FilesCommittable> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + + private final TableLoader tableLoader; + private final Table table; + private final boolean overwrite; + private final boolean upsertMode; + private final DistributionMode distributionMode; + private final List<String> equalityFieldColumns; + private final List<Integer> equalityFieldIds; + private final FileFormat dataFileFormat; + private final int workerPoolSize; + private final long targetDataFileSize; + private final String uidPrefix; + private final Map<String, String> snapshotProperties; + private final RowType flinkRowType; + private Committer<FilesCommittable> committer; + + private IcebergSink( + TableLoader tableLoader, + @Nullable Table table, + List<Integer> equalityFieldIds, + List<String> equalityFieldColumns, + @Nullable String uidPrefix, + Map<String, String> snapshotProperties, + boolean upsertMode, + boolean overwrite, + TableSchema tableSchema, + DistributionMode distributionMode, + int workerPoolSize, + long targetDataFileSize, + FileFormat dataFileFormat) { + Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); + Preconditions.checkNotNull(table, "Table shouldn't be null"); + + this.tableLoader = tableLoader; + this.table = table; + this.equalityFieldIds = equalityFieldIds; + this.equalityFieldColumns = equalityFieldColumns; + this.uidPrefix = uidPrefix == null ? "" : uidPrefix; + this.snapshotProperties = snapshotProperties; + this.flinkRowType = toFlinkRowType(table.schema(), tableSchema); + this.upsertMode = upsertMode; + this.overwrite = overwrite; + this.distributionMode = distributionMode; + this.workerPoolSize = workerPoolSize; + this.targetDataFileSize = targetDataFileSize; + this.dataFileFormat = dataFileFormat; + } + + @Override + public DataStream<RowData> addPreWriteTopology(DataStream<RowData> inputDataStream) { + return distributeDataStream( + inputDataStream, + equalityFieldIds, + table.spec(), + table.schema(), + flinkRowType, + distributionMode, + equalityFieldColumns); + } + + @Override + public StreamWriter createWriter(InitContext context) { + return restoreWriter(context, null); + } + + @Override + public StreamWriter restoreWriter( + InitContext context, Collection<StreamWriterState> recoveredState) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkRowType, + targetDataFileSize, + dataFileFormat, + equalityFieldIds, + upsertMode); + StreamWriter streamWriter = + new StreamWriter( + table.name(), + taskWriterFactory, + context.getSubtaskId(), + context.getNumberOfParallelSubtasks()); + if (recoveredState == null) { + return streamWriter; + } + + return streamWriter.restoreWriter(recoveredState); + } + + @Override + public SimpleVersionedSerializer<StreamWriterState> getWriterStateSerializer() { + return new StreamWriterStateSerializer(); + } + + @Override + public DataStream<CommittableMessage<FilesCommittable>> addPreCommitTopology( + DataStream<CommittableMessage<FilesCommittable>> writeResults) { + return writeResults + .map( + new RichMapFunction< + CommittableMessage<FilesCommittable>, CommittableMessage<FilesCommittable>>() { + @Override + public CommittableMessage<FilesCommittable> map( + CommittableMessage<FilesCommittable> message) { + if (message instanceof CommittableWithLineage) { + CommittableWithLineage<FilesCommittable> committableWithLineage = + (CommittableWithLineage<FilesCommittable>) message; + FilesCommittable committable = committableWithLineage.getCommittable(); + committable.checkpointId(committableWithLineage.getCheckpointId().orElse(-1)); + committable.subtaskId(committableWithLineage.getSubtaskId()); + committable.jobID(getRuntimeContext().getJobId().toString()); + } + return message; + } + }) + .uid(uidPrefix + "-pre-commit-topology") + .global(); Review Comment: > The lack of GlobalCommitter in the v2 sink interface makes it not a good fit for Iceberg sink. I added a few more comment in the writer class: https://github.com/apache/iceberg/pull/4904/files#r951000248. > > I think we need to address this high-level design question first with the Flink community before we can really adopt the v2 sink interface. So far, we haven't heard back anything on the email thread. We will have to figure out who can we work with from the Flink community. > > cc @hililiwei @pvary @kbendick @rdblue As discussed in the mail thread, the ability to customize Commiter parallelism is what we need. This capability does not seem to be available until at least >flink 1.17, and for now, there are no obvious blocking issues that require us to use the new sink right away. So it looks like we might be able to wait until flink provides this capability before going back and optimizing it. But maybe one thing we need to consider is that the auto-compacting of small files is a good thing for the user. Maybe we can consider using it the current way (using global), because it doesn't have a problem with functionality, just commiters that won't do anything. Adaptation comes after flink provides the ability to customize Commiter parallelism. -- 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