hililiwei commented on code in PR #4904: URL: https://github.com/apache/iceberg/pull/4904#discussion_r927797880
########## flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/v2/FilesCommitter.java: ########## @@ -0,0 +1,224 @@ +/* + * 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.v2; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class FilesCommitter implements Committer<FilesCommittable>, Serializable { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FilesCommitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + + // TableLoader to load iceberg table lazily. + private final TableLoader tableLoader; + private final boolean replacePartitions; + private final Map<String, String> snapshotProperties; + + // It will have an unique identifier for one job. + private final Table table; + private transient int continuousEmptyCheckpoints; + + private transient ExecutorService workerPool; + + FilesCommitter(TableLoader tableLoader, boolean replacePartitions, Map<String, String> snapshotProperties, + int workerPoolSize) { + this.tableLoader = tableLoader; + this.replacePartitions = replacePartitions; + this.snapshotProperties = snapshotProperties; + + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + Thread.currentThread().getName(), workerPoolSize); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + } + + @Override + public void commit(Collection<CommitRequest<FilesCommittable>> requests) + throws IOException, InterruptedException { + int dataFilesNumRestored = 0; + int deleteFilesNumRestored = 0; + + int dataFilesNumStored = 0; + int deleteFilesNumStored = 0; + + Collection<CommitRequest<FilesCommittable>> restored = Lists.newArrayList(); + Collection<CommitRequest<FilesCommittable>> store = Lists.newArrayList(); + + for (CommitRequest<FilesCommittable> request : requests) { + WriteResult committable = request.getCommittable().committable(); + if (request.getNumberOfRetries() > 0) { + restored.add(request); + dataFilesNumRestored = dataFilesNumRestored + committable.dataFiles().length; + deleteFilesNumRestored = deleteFilesNumRestored + committable.deleteFiles().length; + } else { + store.add(request); + dataFilesNumStored = dataFilesNumStored + committable.dataFiles().length; + deleteFilesNumStored = deleteFilesNumStored + committable.deleteFiles().length; + } + } + + if (restored.size() > 0) { + commitResult(dataFilesNumRestored, deleteFilesNumRestored, restored); Review Comment: Sorry for the late reply. I've been thinking about what you said. The goal should be to avoid writing duplicate data. So our best bet was to come up with a mechanism that would allow us to submit the data exactly one way. I will test the failed commit case again soon. Thank you. -- 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]
