stevenzwu commented on a change in pull request #856: URL: https://github.com/apache/incubator-iceberg/pull/856#discussion_r420914501
########## File path: flink/src/main/java/org/apache/iceberg/flink/connector/sink/IcebergCommitter.java ########## @@ -0,0 +1,642 @@ +/* + * 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.connector.sink; + +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.util.Preconditions; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.connector.IcebergConnectorConstant; +import org.apache.iceberg.flink.connector.model.CommitMetadata; +import org.apache.iceberg.flink.connector.model.CommitMetadataUtil; +import org.apache.iceberg.flink.connector.model.FlinkManifestFile; +import org.apache.iceberg.flink.connector.model.FlinkManifestFileUtil; +import org.apache.iceberg.flink.connector.model.GenericFlinkManifestFile; +import org.apache.iceberg.flink.connector.model.ManifestFileState; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalogs; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This operator commit data files to Iceberg table. + * <p> + * This operator should always run with parallelism of 1. + * Because Iceberg lib perform optimistic concurrency control, + * this can help reduce contention and retries + * when committing files to Iceberg table. + * <p> + * Here are some known contentions + * 1) Flink jobs running in multiple regions, + * since Iceberg metadata service and commit only happens in us-east-1. + * 2) auto tuning and auto lift services may update Iceberg table infrequently. + */ +@SuppressWarnings("checkstyle:HiddenField") +public class IcebergCommitter extends RichSinkFunction<FlinkDataFile> Review comment: yeah. OperatorCoordinator seems like the better way to go. One drawback of the old two-stage operators model is that it can invalidate an embarrassingly parallel DAG. I brought this up to Becket Qin. This is the reverse/mirroring problem of FLIP-27 (source interface redesign). glad that we can use the OperatorCoordinator for sink too. It will probably be a relative big refactoring. might be better to track it in a separate issue. ---------------------------------------------------------------- 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. 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