stevenzwu commented on code in PR #10179: URL: https://github.com/apache/iceberg/pull/10179#discussion_r1631782354
########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java: ########## @@ -0,0 +1,54 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +public class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + public static List<StreamElement> fromOutput(Collection<Object> elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord<CommittableMessage<?>>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + public static CommittableSummary<?> toCommittableSummary(StreamElement element) { Review Comment: this method does assertion along with type extraction. not sure if it is the best structure. ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergCommitter.java: ########## @@ -0,0 +1,362 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointStoreUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +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.flink.TableLoader; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +@Internal +public class IcebergCommitter implements Committer<IcebergCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + public static final String FLINK_JOB_ID = "flink.job-id"; + public static final String OPERATOR_ID = "flink.operator-id"; + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient Table table; + private transient int maxContinuousEmptyCommits; + private transient ExecutorService workerPool; + private transient int continuousEmptyCheckpoints = 0; + + public IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + table + "-" + sinkId, workerPoolSize); Review Comment: nit: I know the old code named it this way. maybe we can rename it to more accurate `iceberg-committer-pool` ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java: ########## @@ -140,8 +141,7 @@ public void testCheckAndGetEqualityFieldIds() { DataStream<Row> dataStream = env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + Builder builder = IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); Review Comment: we can't just switch existing tests to the new sink. both sink implementations need to be tested. ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java: ########## @@ -0,0 +1,54 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +public class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + public static List<StreamElement> fromOutput(Collection<Object> elements) { Review Comment: it is hard to understand the purpose of this method by just looking at the signature. ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergCommitter.java: ########## @@ -0,0 +1,362 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointStoreUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +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.flink.TableLoader; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +@Internal +public class IcebergCommitter implements Committer<IcebergCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + public static final String FLINK_JOB_ID = "flink.job-id"; + public static final String OPERATOR_ID = "flink.operator-id"; + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient Table table; + private transient int maxContinuousEmptyCommits; + private transient ExecutorService workerPool; + private transient int continuousEmptyCheckpoints = 0; + + public IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + table + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + NavigableMap<Long, byte[]> manifestMap = Maps.newTreeMap(); + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + + if (commitRequests.isEmpty()) { + return; + } + + for (CommitRequest<IcebergCommittable> request : commitRequests) { + manifestMap.put(request.getCommittable().checkpointId(), request.getCommittable().manifest()); + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + long maxCommittedCheckpointId = getMaxCommittedCheckpointId(commitRequestMap); + + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + + // Commit the remaining + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); Review Comment: wondering if the following flow is a bit clearer * get max committed checkpoint id * mark the committed as finished * use `tailMap` for the uncommitted view * extract `NavigableMap<Long, byte[]> manifestMap` from the uncommitted view/map ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergCommitter.java: ########## @@ -0,0 +1,362 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointStoreUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +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.flink.TableLoader; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +@Internal +public class IcebergCommitter implements Committer<IcebergCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + public static final String FLINK_JOB_ID = "flink.job-id"; + public static final String OPERATOR_ID = "flink.operator-id"; + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient Table table; + private transient int maxContinuousEmptyCommits; + private transient ExecutorService workerPool; + private transient int continuousEmptyCheckpoints = 0; + + public IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + table + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + NavigableMap<Long, byte[]> manifestMap = Maps.newTreeMap(); + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + + if (commitRequests.isEmpty()) { Review Comment: nit: move this to be beginning of the method ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/WriteResultSerializer.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +public class WriteResultSerializer implements SimpleVersionedSerializer<WriteResult> { + private static final int VERSION_0 = 0; + + @Override + public int getVersion() { + return VERSION_0; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); Review Comment: why do we need to include the length? looking at the `deserialize` method, `InstantiationUtil.deserializeObject` can directly parse the `byte[] serialized` ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java: ########## @@ -0,0 +1,832 @@ +/* + * 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.flink.sink.SinkTestUtil.fromOutput; +import static org.apache.iceberg.flink.sink.SinkTestUtil.toCommittableSummary; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +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.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { Review Comment: in addition to new tests, does it cover all the existing tests in `IcebergFilesCommitter`? are there some existing test not quite applicable in v2 sink? ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergCommitter.java: ########## @@ -0,0 +1,362 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointStoreUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +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.flink.TableLoader; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +@Internal +public class IcebergCommitter implements Committer<IcebergCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + public static final String FLINK_JOB_ID = "flink.job-id"; + public static final String OPERATOR_ID = "flink.operator-id"; + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient Table table; + private transient int maxContinuousEmptyCommits; + private transient ExecutorService workerPool; + private transient int continuousEmptyCheckpoints = 0; + + public IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + table + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + NavigableMap<Long, byte[]> manifestMap = Maps.newTreeMap(); + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + + if (commitRequests.isEmpty()) { + return; + } + + for (CommitRequest<IcebergCommittable> request : commitRequests) { + manifestMap.put(request.getCommittable().checkpointId(), request.getCommittable().manifest()); + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + long maxCommittedCheckpointId = getMaxCommittedCheckpointId(commitRequestMap); + + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + + // Commit the remaining + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + + NavigableMap<Long, byte[]> uncommitted = + Maps.newTreeMap(manifestMap).tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitUpToCheckpoint(uncommitted, last.jobId(), last.operatorId(), last.checkpointId()); + } + } + + /** + * Gets the last checkpointId which is committed to this branch of the table. The commits are + * identified by the {@link IcebergCommittable} (jobId/operatorId/checkpointId). Only used by + * {@link IcebergCommittable} (SinkV2). + * + * @param requests The {@link IcebergCommittable}s ordered by checkpointId + * @return The checkpointId for the first commit (historically backward) + */ + public long getMaxCommittedCheckpointId( + NavigableMap<Long, CommitRequest<IcebergCommittable>> requests) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = CheckpointStoreUtil.INVALID_CHECKPOINT_ID; + + while (snapshot != null) { + Map<String, String> summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + String snapshotCheckpointId = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (snapshotCheckpointId != null) { + long checkpointId = Long.parseLong(snapshotCheckpointId); + CommitRequest<IcebergCommittable> request = requests.get(checkpointId); + if (request != null + && request.getCommittable().jobId().equals(snapshotFlinkJobId) + && request.getCommittable().operatorId().equals(snapshotOperatorId)) { + lastCommittedCheckpointId = checkpointId; + break; + } + } + + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link DeltaManifests} + * ordered by the checkpointId, and writing the new snapshot to the Iceberg table. The {@link + * org.apache.iceberg.SnapshotSummary} will contain the jobId, snapshotId, checkpointId so in case + * of job restart we can identify which changes are committed, and which are still waiting for the + * commit. + * + * @param deltaManifestsMap The checkpointId to {@link DeltaManifests} map of the changes to + * commit + * @param newFlinkJobId The jobId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param operatorId The operatorId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param checkpointId The checkpointId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @throws IOException On commit failure + */ + public void commitUpToCheckpoint( + NavigableMap<Long, byte[]> deltaManifestsMap, + String newFlinkJobId, + String operatorId, + long checkpointId) + throws IOException { + NavigableMap<Long, byte[]> pendingMap = deltaManifestsMap.headMap(checkpointId, true); + List<ManifestFile> manifests = Lists.newArrayList(); + NavigableMap<Long, WriteResult> pendingResults = Maps.newTreeMap(); + for (Map.Entry<Long, byte[]> e : pendingMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { + // Skip the empty flink manifest. + continue; + } + + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + pendingMap.clear(); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void commitPendingResult( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } + continuousEmptyCheckpoints = 0; + } else { + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + + commitOperation( + dynamicOverwrite, + summary, + "dynamic partition overwrite", + newFlinkJobId, + operatorId, + checkpointId); + } + + private void commitDeltaTxn( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { Review Comment: I am wondering if it is necessary to pass in the last checkpointId through multiple upstream methods. For the delete file case, committables are committed one by one and the checkpoint id is extracted from the map entry `e.getKey()`. For the other case, why don't we check the max checkpointId from the last entry of `pendingResults` in this method. ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergCommitter.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +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.flink.TableLoader; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +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.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +public class IcebergCommitter implements Committer<IcebergCommittable>, Serializable { + private static final long serialVersionUID = 1L; + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private static final CommitRequest<IcebergCommittable> EMPTY_COMMIT_REQUEST = + new CommitRequest<IcebergCommittable>() { + @Override + public IcebergCommittable getCommittable() { + return new IcebergCommittable(EMPTY_MANIFEST_DATA, "jobId", "operatorId", -1L); + } + + @Override + public int getNumberOfRetries() { + return 0; + } + + @Override + public void signalFailedWithKnownReason(Throwable t) {} + + @Override + public void signalFailedWithUnknownReason(Throwable t) {} + + @Override + public void retryLater() {} + + @Override + public void updateAndRetryLater(IcebergCommittable committable) {} + + @Override + public void signalAlreadyCommitted() {} + }; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + public static final long INITIAL_CHECKPOINT_ID = -1L; + public static final String FLINK_JOB_ID = "flink.job-id"; + public static final String OPERATOR_ID = "flink.operator-id"; + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private final int workerPoolSize; + private final String prefix; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient Table table; + private transient int maxContinuousEmptyCommits; + private transient ExecutorService workerPool; + private transient int continuousEmptyCheckpoints = 0; + + public IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String prefix, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.workerPoolSize = workerPoolSize; + this.prefix = prefix; + this.committerMetrics = committerMetrics; + + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + table + "-" + prefix, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + NavigableMap<Long, byte[]> manifestMap = Maps.newTreeMap(); + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + + if (commitRequests.isEmpty()) { + commitRequests.add(EMPTY_COMMIT_REQUEST); + } + + for (CommitRequest<IcebergCommittable> request : commitRequests) { + manifestMap.put(request.getCommittable().checkpointId(), request.getCommittable().manifest()); + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + long maxCommittedCheckpointId = getMaxCommittedCheckpointId(commitRequestMap); + + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + + // Commit the remaining + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + + NavigableMap<Long, byte[]> uncommitted = + Maps.newTreeMap(manifestMap).tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitUpToCheckpoint( + commitRequestMap, uncommitted, last.jobId(), last.operatorId(), last.checkpointId()); + } + } + + /** + * Gets the last checkpointId which is committed to this branch of the table. The commits are + * identified by the {@link IcebergCommittable} (jobId/operatorId/checkpointId). Only used by + * {@link IcebergCommittable} (SinkV2). + * + * @param requests The {@link IcebergCommittable}s ordered by checkpointId + * @return The checkpointId for the first commit (historically backward) + */ + public long getMaxCommittedCheckpointId( Review Comment: you are right. the signature is different with CommitRequest now -- 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