diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 2256d1e874ce..be2a8db03097 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -406,7 +406,8 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); int writerParallelism = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * 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 java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + *

{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * 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 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; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + *

In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * 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 java.io.IOException; +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.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.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: + * + *

    + *
  • There is a single {@link IcebergCommittable} for every checkpoint + *
  • There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + *
  • There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + *
+ */ +class IcebergCommitter implements Committer { + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + public static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private final String branch; + private final Map snapshotProperties; + private final boolean replacePartitions; + private IcebergFilesCommitterMetrics committerMetrics; + private Table table; + private final TableLoader tableLoader; + private int maxContinuousEmptyCommits; + private ExecutorService workerPool; + private int continuousEmptyCheckpoints = 0; + + IcebergCommitter( + TableLoader tableLoader, + String branch, + Map snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.tableLoader = tableLoader; + 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-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + NavigableMap> commitRequestMap = Maps.newTreeMap(); + for (CommitRequest request : commitRequests) { + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + long maxCommittedCheckpointId = + SinkUtil.getMaxCommittedCheckpointId(table, last.jobId(), last.operatorId(), branch); + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + NavigableMap> uncommitted = + commitRequestMap.tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests(uncommitted, last.jobId(), last.operatorId()); + } + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link + * org.apache.iceberg.flink.sink.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 commitRequestMap The checkpointId to {@link CommitRequest} 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} + * @throws IOException On commit failure + */ + private void commitPendingRequests( + NavigableMap> commitRequestMap, + String newFlinkJobId, + String operatorId) + throws IOException { + long checkpointId = commitRequestMap.lastKey(); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry> e : commitRequestMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue().getCommittable().manifest())) { + pendingResults.put(e.getKey(), EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue().getCommittable().manifest()); + 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); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void logCommitSummary(CommitSummary summary, String description) { + LOG.info( + "Preparing for commit: {} on table: {} branch: {} with summary: {}.", + description, + table, + branch, + summary); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + 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); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId); + } + continuousEmptyCheckpoints = 0; + } else { + long checkpointId = pendingResults.lastKey(); + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + 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); + } + String description = "dynamic partition overwrite"; + + logCommitSummary(summary, description); + commitOperation(dynamicOverwrite, description, newFlinkJobId, operatorId, checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + String description = "append"; + logCommitSummary(summary, description); + // fail all commits as really its only one + commitOperation(appendFiles, description, newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + + String description = "rowDelta"; + logCommitSummary(summary, description); + commitOperation(rowDelta, description, newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(SinkUtil.MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(SinkUtil.FLINK_JOB_ID, newFlinkJobId); + operation.set(SinkUtil.OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + if (committerMetrics != null) { + committerMetrics.commitDuration(durationMs); + } + } + + @Override + public void close() throws IOException { + tableLoader.close(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..622daa808897 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -44,13 +44,11 @@ import org.apache.iceberg.PartitionSpec; 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.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.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -187,7 +185,7 @@ public void initializeState(StateInitializationContext context) throws Exception // it's safe to assign the max committed checkpoint id from restored flink job to the current // flink job. this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + SinkUtil.getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); NavigableMap uncommittedDataFiles = Maps.newTreeMap(checkpointsState.get().iterator().next()) @@ -280,7 +278,7 @@ private void commitUpToCheckpoint( commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); committerMetrics.updateCommitSummary(summary); pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void commitPendingResult( @@ -303,27 +301,6 @@ private void commitPendingResult( } } - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - private void replacePartitions( NavigableMap pendingResults, CommitSummary summary, @@ -489,28 +466,4 @@ static ListStateDescriptor> buildStateDescriptor() { longComparator); return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java new file mode 100644 index 000000000000..d080169544cd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -0,0 +1,742 @@ +/* + * 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.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +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.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +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.flink.util.Preconditions; +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.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + *
    + *
  • {@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + *
  • {@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data/delete files, and + * generates the {@link org.apache.iceberg.io.WriteResult} objects for the files + *
  • {@link SupportsPreCommitTopology} which we use to place the {@link + * org.apache.iceberg.flink.sink.IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.IcebergCommittable} + *
  • {@link org.apache.iceberg.flink.sink.IcebergCommitter} which commits the incoming{@link + * org.apache.iceberg.flink.sink.IcebergCommittable}s to the Iceberg table + *
  • {@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + *
+ * + * The job graph looks like below: + * + *
{@code
+ *                            Flink sink
+ *               +-----------------------------------------------------------------------------------+
+ *               |                                                                                   |
+ * +-------+     | +----------+                               +-------------+      +---------------+ |
+ * | Map 1 | ==> | | writer 1 |                               | committer 1 | ---> | post commit 1 | |
+ * +-------+     | +----------+                               +-------------+      +---------------+ |
+ *               |             \                             /                \                      |
+ *               |              \                           /                  \                     |
+ *               |               \                         /                    \                    |
+ * +-------+     | +----------+   \ +-------------------+ /   +-------------+    \ +---------------+ |
+ * | Map 2 | ==> | | writer 2 | --->| commit aggregator |     | committer 2 |      | post commit 2 | |
+ * +-------+     | +----------+     +-------------------+     +-------------+      +---------------+ |
+ *               |                                             Commit only on                        |
+ *               |                                             committer 1                           |
+ *               +-----------------------------------------------------------------------------------+
+ * }
+ */ +@Experimental +public class IcebergSink + implements Sink, + SupportsPreWriteTopology, + SupportsCommitter, + SupportsPreCommitTopology, + SupportsPostCommitTopology { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + private final TableLoader tableLoader; + private final Map snapshotProperties; + private final String uidSuffix; + private final String sinkId; + private final Map writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + + private final Table table; + private final List equalityFieldColumns = null; + + private IcebergSink( + TableLoader tableLoader, + Table table, + Map snapshotProperties, + String uidSuffix, + Map writeProperties, + RowType flinkRowType, + SerializableSupplier
tableSupplier, + FlinkWriteConf flinkWriteConf, + List equalityFieldIds, + String branch, + boolean overwriteMode) { + this.tableLoader = tableLoader; + this.snapshotProperties = snapshotProperties; + this.uidSuffix = uidSuffix; + this.writeProperties = writeProperties; + this.flinkRowType = flinkRowType; + this.tableSupplier = tableSupplier; + this.flinkWriteConf = flinkWriteConf; + this.equalityFieldIds = equalityFieldIds; + this.branch = branch; + this.overwriteMode = overwriteMode; + this.table = table; + this.upsertMode = flinkWriteConf.upsertMode(); + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter createWriter(InitContext context) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + targetDataFileSize, + dataFileFormat, + writeProperties, + equalityFieldIds, + upsertMode); + IcebergStreamWriterMetrics metrics = + new IcebergStreamWriterMetrics(context.metricGroup(), table.name()); + return new IcebergSinkWriter( + tableSupplier.get().name(), + taskWriterFactory, + metrics, + context.getSubtaskId(), + context.getAttemptNumber()); + } + + @Override + public Committer createCommitter(CommitterInitContext context) { + IcebergFilesCommitterMetrics metrics = + new IcebergFilesCommitterMetrics(context.metricGroup(), table.name()); + return new IcebergCommitter( + tableLoader, branch, snapshotProperties, overwriteMode, workerPoolSize, sinkId, metrics); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new IcebergCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream> committables) { + // TODO Support small file compaction + } + + @Override + public DataStream addPreWriteTopology(DataStream inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream> addPreCommitTopology( + DataStream> writeResults) { + TypeInformation> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + String suffix = defaultSuffix(uidSuffix, table.name()); + String preCommitAggregatorUid = String.format("Sink pre-commit aggregator: %s", suffix); + + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + return writeResults + .global() + .transform(preCommitAggregatorUid, typeInformation, new IcebergWriteAggregator(tableLoader)) + .uid(preCommitAggregatorUid) + .setParallelism(1) + .setMaxParallelism(1) + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + .global(); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new WriteResultSerializer(); + } + + public static class Builder { + private TableLoader tableLoader; + private String uidSuffix = ""; + private Function> inputCreator = null; + private TableSchema tableSchema; + private SerializableTable table; + private final Map writeOptions = Maps.newHashMap(); + private final Map snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private List equalityFieldColumns = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forRow(DataStream input, TableSchema inputTableSchema) { + RowType rowType = (RowType) inputTableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = inputTableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return forMapperOutputType( + input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(inputTableSchema); + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidSuffix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism of map operator same as its input to keep map operator + // chaining its input, and avoid rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidSuffix != null) { + String uid = String.format("Sink pre-writer mapper: %s", newUidSuffix); + inputStream.name(uid).uid(uid); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link SerializableTable} instance is used for initializing {@link + * IcebergStreamWriter} which will write all the records into {@link DataFile}s and emit them to + * downstream operator. Providing a table would avoid so many table loading from each separate + * task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = (SerializableTable) SerializableTable.copyOf(newTable); + return this; + } + + /** + * The table loader is used for loading tables in {@link + * org.apache.iceberg.flink.sink.IcebergCommitter} lazily, we need this loader because {@link + * Table} is not serializable and could not just use the loaded table from Builder#table in the + * remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + TableLoader tableLoader() { + return tableLoader; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the IcebergSink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator). Actual operator uid will be appended + * with a suffix like "Sink Committer: $uidSuffix". + * + *

Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid. + * + *

Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newSuffix suffix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidSuffix(String newSuffix) { + this.uidSuffix = newSuffix; + return this; + } + + public Builder snapshotProperties(Map properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + IcebergSink build() { + + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader(), "Table loader shouldn't be null"); + + // Set the table if it is not yet set in the builder, so we can do the equalityId checks + SerializableTable serializableTable = checkAndGetTable(tableLoader(), table); + this.table = serializableTable; + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + SerializableSupplier

tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: + DataStreamSink rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream distributeDataStream(DataStream input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java @@ -0,0 +1,113 @@ +/* + * 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 java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..7d86baa14fc2 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -95,9 +95,9 @@ public void endInput() throws IOException { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * 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 java.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..1cb7f4dea1e8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -26,13 +26,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier
tableSupplier; private final Map props; private final String flinkJobId; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * 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 java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private 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"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * 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 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; + +class WriteResultSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java @@ -0,0 +1,62 @@ +/* + * 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.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List transformsToStreamElement(Collection elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) value; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) value; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map ROW_KIND_MAP = + static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List equalityFieldColumns, KeySelector keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List findValidSnapshots() { + List findValidSnapshots() { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable reader = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 0b0c55f51c32..a2e9be9303c3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 53b7c4c0cc91..c21c3d5cc21b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -134,7 +133,9 @@ public void testUserProvidedManifestLocation() throws IOException { String operatorId = newOperatorUniqueId(); File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..abdb2b5c79ae --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * 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.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +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.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +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.api.operators.StreamingRuntimeContext; +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.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +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.SnapshotRef; +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.TestHelpers; +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.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 { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest buildCommitRequestFor( + String myJobID, long checkpoint, Collection writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer> { + + CommittableMessageSerializer serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer> duplicate() { + return null; + } + + @Override + public CommittableMessage createInstance() { + return null; + } + + @Override + public CommittableMessage copy( + CommittableMessage from) { + return from; + } + + @Override + public CommittableMessage copy( + CommittableMessage from, CommittableMessage reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage deserialize( + CommittableMessage reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return null; + } + }; +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..67ca9d08b206 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -139,8 +139,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -1079,8 +1078,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * 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.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -0,0 +1,267 @@ +/* + * 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.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +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.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + new String( + deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java new file mode 100644 index 000000000000..93da40ab5c9a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,119 @@ +/* + * 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.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +}