hililiwei commented on code in PR #4904: URL: https://github.com/apache/iceberg/pull/4904#discussion_r937738811
########## flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/committer/FilesCommitter.java: ########## @@ -0,0 +1,314 @@ +/* + * 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.committer; + +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.Snapshot; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +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.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FilesCommitter implements Committer<FilesCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final long INITIAL_CHECKPOINT_ID = -1L; + private static final Logger LOG = LoggerFactory.getLogger(FilesCommitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final int MAX_RECOMMIT_TIMES = 3; + + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-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 Map<String, Long> maxCommittedCheckpointIdForJob = Maps.newHashMap(); + + private transient ExecutorService workerPool; + + public 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 dataFilesNumStored = 0; + int deleteFilesNumStored = 0; + + Collection<CommitRequest<FilesCommittable>> store = Lists.newArrayList(); + String jobId = null; + long maxCommittableCheckpointId = INITIAL_CHECKPOINT_ID; + + for (CommitRequest<FilesCommittable> request : requests) { + WriteResult committable = request.getCommittable().committable(); + Long committableCheckpointId = request.getCommittable().checkpointId(); + jobId = request.getCommittable().jobID(); + if (request.getNumberOfRetries() > MAX_RECOMMIT_TIMES) { + String message = + String.format( + "Failed to commit transaction %s after retrying %d times", + request.getCommittable(), MAX_RECOMMIT_TIMES); + request.signalFailedWithUnknownReason(new RuntimeException(message)); + } else if (request.getNumberOfRetries() > 0) { + commitResult( + committable.dataFiles().length, + committable.deleteFiles().length, + Lists.newArrayList(request)); + } else if (committableCheckpointId + > maxCommittedCheckpointIdForJob.getOrDefault(jobId, Long.MAX_VALUE) + || getMaxCommittedCheckpointId(table, jobId) == -1) { Review Comment: Maybe I didn't quite catch what you mean. But one thing, I think, is that the table can be changed by other job between the time we get the `maxCommitterCheckpointId` and the commit. For a flink job, the JobId is unique, parallel jobs do not use the same jobid for writing. When multiple Flink tasks are written in parallel, Iceberg itself ensures the transactional for commit actions. When it commits a conflict, it tries again. For `committableCheckpointId> maxCommittedCheckpointIdForJob.getOrDefault(jobId, Long.MAX_VALUE) || getMaxCommittedCheckpointId(table, jobId) == -1`: In theory, `maxCommittedCheckpointIdForJob` it will contain only at most two key, one is the job id of the current job, one is the `pre-job-id` of the pre-job. No other task will have the same pre-job-id at the same time. So, we think that if `maxCommittedCheckpointIdForJob` does not contain the jobid of this request , and also no corresponding records in the table, then, that means The jobid has never written data to the table. At this point, whether we are restore from the pre-job or it is the first time of this job, will accumulate the data and commit them all at once later. -- 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]
