From 4eea332451b5daf44d406f9547a9a9d07ef77b65 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 16 May 2025 16:17:09 +0200 Subject: [PATCH 01/23] Flink: Add dynamic writer and committer This adds the dynamic version of the writer and committer for the Flink Dynamic Iceberg Sink. Conceptually, they work similar to the IcebergSink, but they support writing to multiple tables. Write results from each table are aggregated from the DynamicWriter in the DynamicWriteResultAggregator, from where they are sent to the DynamicCommitter. Broken out of #12424, depends on #13032. --- .../iceberg/flink/sink/CommitSummary.java | 50 ++- .../iceberg/flink/sink/DeltaManifests.java | 4 +- .../flink/sink/DeltaManifestsSerializer.java | 4 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 10 +- .../flink/sink/IcebergCommittable.java | 2 +- .../sink/IcebergCommittableSerializer.java | 2 +- .../sink/IcebergFilesCommitterMetrics.java | 10 +- .../sink/IcebergStreamWriterMetrics.java | 8 +- .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 27 +- .../flink/sink/WriteResultSerializer.java | 2 +- .../sink/dynamic/DynamicCommittable.java | 81 ++++ .../dynamic/DynamicCommittableSerializer.java | 74 ++++ .../flink/sink/dynamic/DynamicCommitter.java | 418 ++++++++++++++++++ .../sink/dynamic/DynamicCommitterMetrics.java | 52 +++ .../sink/dynamic/DynamicWriteResult.java | 42 ++ .../dynamic/DynamicWriteResultAggregator.java | 190 ++++++++ .../dynamic/DynamicWriteResultSerializer.java | 64 +++ .../flink/sink/dynamic/DynamicWriter.java | 219 +++++++++ .../sink/dynamic/DynamicWriterMetrics.java | 51 +++ .../flink/sink/dynamic/WriteTarget.java | 145 ++++++ 21 files changed, 1412 insertions(+), 47 deletions(-) create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 2109c91bddf7..62de3170d55c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -19,13 +19,14 @@ package org.apache.iceberg.flink.sink; import java.util.Arrays; +import java.util.List; import java.util.NavigableMap; import java.util.concurrent.atomic.AtomicLong; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.util.ScanTaskUtil; -class CommitSummary { +public class CommitSummary { private final AtomicLong dataFilesCount = new AtomicLong(); private final AtomicLong dataFilesRecordCount = new AtomicLong(); @@ -34,30 +35,35 @@ class CommitSummary { private final AtomicLong deleteFilesRecordCount = new AtomicLong(); private final AtomicLong deleteFilesByteCount = new AtomicLong(); - CommitSummary(NavigableMap pendingResults) { - pendingResults - .values() + public CommitSummary() {} + + public CommitSummary(NavigableMap pendingResults) { + pendingResults.values().forEach(this::addWriteResult); + } + + public void addAll(NavigableMap> pendingResults) { + pendingResults.values().forEach(writeResults -> writeResults.forEach(this::addWriteResult)); + } + + private void addWriteResult(WriteResult writeResult) { + dataFilesCount.addAndGet(writeResult.dataFiles().length); + Arrays.stream(writeResult.dataFiles()) + .forEach( + dataFile -> { + dataFilesRecordCount.addAndGet(dataFile.recordCount()); + dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); + }); + deleteFilesCount.addAndGet(writeResult.deleteFiles().length); + Arrays.stream(writeResult.deleteFiles()) .forEach( - writeResult -> { - dataFilesCount.addAndGet(writeResult.dataFiles().length); - Arrays.stream(writeResult.dataFiles()) - .forEach( - dataFile -> { - dataFilesRecordCount.addAndGet(dataFile.recordCount()); - dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); - }); - deleteFilesCount.addAndGet(writeResult.deleteFiles().length); - Arrays.stream(writeResult.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); - deleteFilesByteCount.addAndGet(deleteBytes); - }); + deleteFile -> { + deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); + long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); + deleteFilesByteCount.addAndGet(deleteBytes); }); } - long dataFilesCount() { + public long dataFilesCount() { return dataFilesCount.get(); } @@ -69,7 +75,7 @@ long dataFilesByteCount() { return dataFilesByteCount.get(); } - long deleteFilesCount() { + public long deleteFilesCount() { return deleteFilesCount.get(); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index 036970c06d5b..89c26692b113 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java @@ -23,7 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -class DeltaManifests { +public class DeltaManifests { private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; @@ -56,7 +56,7 @@ CharSequence[] referencedDataFiles() { return referencedDataFiles; } - List manifests() { + public List manifests() { List manifests = Lists.newArrayListWithCapacity(2); if (dataManifest != null) { manifests.add(dataManifest); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java index 92ca284b12ba..59de599a6886 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java @@ -28,12 +28,12 @@ import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -class DeltaManifestsSerializer implements SimpleVersionedSerializer { +public class DeltaManifestsSerializer implements SimpleVersionedSerializer { private static final int VERSION_1 = 1; private static final int VERSION_2 = 2; private static final byte[] EMPTY_BINARY = new byte[0]; - static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + public static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); @Override public int getVersion() { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index 9571efdc5268..d107c2739b04 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -38,7 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class FlinkManifestUtil { +public class FlinkManifestUtil { private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; @@ -66,7 +66,7 @@ static List readDataFiles( } } - static ManifestOutputFileFactory createOutputFileFactory( + public static ManifestOutputFileFactory createOutputFileFactory( Supplier tableSupplier, Map tableProps, String flinkJobId, @@ -83,7 +83,7 @@ static ManifestOutputFileFactory createOutputFileFactory( * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same * partition spec */ - static DeltaManifests writeCompletedFiles( + public static DeltaManifests writeCompletedFiles( WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) throws IOException { @@ -114,7 +114,7 @@ static DeltaManifests writeCompletedFiles( return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); } - static WriteResult readCompletedFiles( + public static WriteResult readCompletedFiles( DeltaManifests deltaManifests, FileIO io, Map specsById) throws IOException { WriteResult.Builder builder = WriteResult.builder(); @@ -135,7 +135,7 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } - static void deleteCommittedManifests( + public static void deleteCommittedManifests( Table table, List manifests, String newFlinkJobId, long checkpointId) { for (ManifestFile manifest : manifests) { try { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 408c3e9a9d5f..8b06949a5519 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -31,7 +31,7 @@ *

{@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 { +public class IcebergCommittable implements Serializable { private final byte[] manifest; private final String jobId; private final String operatorId; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java index e2b388a83c75..1d83c211e001 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -30,7 +30,7 @@ * *

In both cases only the respective part is serialized. */ -class IcebergCommittableSerializer implements SimpleVersionedSerializer { +public class IcebergCommittableSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; @Override diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 5b28c4acb1c5..86af9c5154f7 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -24,7 +24,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.flink.util.ElapsedTimeGauge; -class IcebergFilesCommitterMetrics { +public class IcebergFilesCommitterMetrics { private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); private final AtomicLong lastCommitDurationMs = new AtomicLong(); private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; @@ -35,7 +35,7 @@ class IcebergFilesCommitterMetrics { private final Counter committedDeleteFilesRecordCount; private final Counter committedDeleteFilesByteCount; - IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { + public IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { MetricGroup committerMetrics = metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); @@ -52,16 +52,16 @@ class IcebergFilesCommitterMetrics { this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); } - void checkpointDuration(long checkpointDurationMs) { + public void checkpointDuration(long checkpointDurationMs) { lastCheckpointDurationMs.set(checkpointDurationMs); } - void commitDuration(long commitDurationMs) { + public void commitDuration(long commitDurationMs) { lastCommitDurationMs.set(commitDurationMs); } /** This is called upon a successful commit. */ - void updateCommitSummary(CommitSummary stats) { + public void updateCommitSummary(CommitSummary stats) { elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); committedDataFilesCount.inc(stats.dataFilesCount()); committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ab458ad2e7cb..04ea868c0a38 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -28,7 +28,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.util.ScanTaskUtil; -class IcebergStreamWriterMetrics { +public class IcebergStreamWriterMetrics { // 1,024 reservoir size should cost about 8KB, which is quite small. // It should also produce good accuracy for histogram distribution (like percentiles). private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; @@ -40,7 +40,7 @@ class IcebergStreamWriterMetrics { private final Histogram dataFilesSizeHistogram; private final Histogram deleteFilesSizeHistogram; - IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { + public IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { MetricGroup writerMetrics = metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); @@ -63,7 +63,7 @@ class IcebergStreamWriterMetrics { new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); } - void updateFlushResult(WriteResult result) { + public void updateFlushResult(WriteResult result) { flushedDataFiles.inc(result.dataFiles().length); flushedDeleteFiles.inc(result.deleteFiles().length); flushedReferencedDataFiles.inc(result.referencedDataFiles().length); @@ -84,7 +84,7 @@ void updateFlushResult(WriteResult result) { }); } - void flushDuration(long flushDurationMs) { + public void flushDuration(long flushDurationMs) { lastFlushDurationMs.set(flushDurationMs); } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 30517cd38216..81434ad171fb 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -30,7 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; -class ManifestOutputFileFactory { +public class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; @@ -70,7 +70,7 @@ private String generatePath(long checkpointId) { fileCount.incrementAndGet())); } - OutputFile create(long checkpointId) { + public OutputFile create(long checkpointId) { String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 67422a1afeb1..8dc8d38869bc 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -80,6 +80,28 @@ public RowDataTaskWriterFactory( Map writeProperties, List equalityFieldIds, boolean upsert) { + this( + tableSupplier, + flinkSchema, + targetFileSizeBytes, + format, + writeProperties, + equalityFieldIds, + upsert, + tableSupplier.get().schema(), + tableSupplier.get().spec()); + } + + public RowDataTaskWriterFactory( + SerializableSupplier

tableSupplier, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert, + Schema schema, + PartitionSpec spec) { this.tableSupplier = tableSupplier; Table table; @@ -90,9 +112,9 @@ public RowDataTaskWriterFactory( table = tableSupplier.get(); } - this.schema = table.schema(); + this.schema = schema; this.flinkSchema = flinkSchema; - this.spec = table.spec(); + this.spec = spec; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; this.equalityFieldIds = equalityFieldIds; @@ -148,6 +170,7 @@ public void initialize(int taskId, int attemptId) { OutputFileFactory.builderFor(table, taskId, attemptId) .format(format) .ioSupplier(() -> tableSupplier.get().io()) + .defaultSpec(spec) .build(); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java index 5a44373cccaa..34868458b0e0 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -26,7 +26,7 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.iceberg.io.WriteResult; -class WriteResultSerializer implements SimpleVersionedSerializer { +public class WriteResultSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; @Override diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java new file mode 100644 index 000000000000..a0abbb72701d --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -0,0 +1,81 @@ +/* + * 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.dynamic; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.sink.IcebergCommittableSerializer; +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 help 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. + */ +@Internal +class DynamicCommittable implements Serializable { + + private final WriteTarget key; + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + DynamicCommittable( + WriteTarget key, byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.key = key; + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + WriteTarget key() { + return key; + } + + 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("key", key) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java new file mode 100644 index 000000000000..3ec6db579c05 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -0,0 +1,74 @@ +/* + * 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.dynamic; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.iceberg.flink.sink.IcebergCommittable; + +/** + * 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. + */ +@Internal +class DynamicCommittableSerializer implements SimpleVersionedSerializer { + + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(DynamicCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + committable.key().serializeTo(view); + 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 DynamicCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + WriteTarget key = WriteTarget.deserializeFrom(view); + 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 DynamicCommittable(key, manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java new file mode 100644 index 000000000000..cf087ed1459e --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -0,0 +1,418 @@ +/* + * 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.dynamic; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergCommittable; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + *

    + *
  • 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 + *
+ */ +@Internal +class DynamicCommitter implements Committer { + + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final Logger LOG = LoggerFactory.getLogger(DynamicCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + private final Map snapshotProperties; + private final boolean replacePartitions; + private final DynamicCommitterMetrics committerMetrics; + private final Catalog catalog; + private final Map maxContinuousEmptyCommitsMap; + private final Map continuousEmptyCheckpointsMap; + private final ExecutorService workerPool; + + DynamicCommitter( + Catalog catalog, + Map snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + DynamicCommitterMetrics committerMetrics) { + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.catalog = catalog; + this.maxContinuousEmptyCommitsMap = Maps.newHashMap(); + this.continuousEmptyCheckpointsMap = Maps.newHashMap(); + + this.workerPool = ThreadPools.newWorkerPool("iceberg-committer-pool-" + sinkId, workerPoolSize); + } + + @Override + public void commit(Collection> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + Map>>> commitRequestMap = + Maps.newHashMap(); + for (CommitRequest request : commitRequests) { + NavigableMap>> committables = + commitRequestMap.computeIfAbsent( + new TableKey(request.getCommittable()), unused -> Maps.newTreeMap()); + committables + .computeIfAbsent(request.getCommittable().checkpointId(), unused -> Lists.newArrayList()) + .add(request); + } + + for (Map.Entry>>> entry : + commitRequestMap.entrySet()) { + Table table = catalog.loadTable(TableIdentifier.parse(entry.getKey().tableName())); + DynamicCommittable last = entry.getValue().lastEntry().getValue().get(0).getCommittable(); + long maxCommittedCheckpointId = + getMaxCommittedCheckpointId( + table, last.jobId(), last.operatorId(), entry.getKey().branch()); + // Mark the already committed FilesCommittable(s) as finished + entry + .getValue() + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); + NavigableMap>> uncommitted = + entry.getValue().tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests( + table, entry.getKey().branch(), uncommitted, last.jobId(), last.operatorId()); + } + } + } + + private 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; + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link DeltaManifests} + * ordered by the checkpointId, and writing the new snapshot to the Iceberg table. The {@link + * org.apache.iceberg.SnapshotSummary} will contain the jobId, snapshotId, checkpointId so in case + * of job restart we can identify which changes are committed, and which are still waiting for the + * commit. + * + * @param 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( + Table table, + String branch, + 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()) { + for (CommitRequest committable : e.getValue()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, committable.getCommittable().manifest())) { + pendingResults + .computeIfAbsent(e.getKey(), unused -> Lists.newArrayList()) + .add(EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, committable.getCommittable().manifest()); + pendingResults + .computeIfAbsent(e.getKey(), unused -> Lists.newArrayList()) + .add(FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + } + } + + CommitSummary summary = new CommitSummary(); + summary.addAll(pendingResults); + commitPendingResult(table, branch, pendingResults, summary, newFlinkJobId, operatorId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(table.name(), summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void commitPendingResult( + Table table, + String branch, + NavigableMap> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + TableKey key = new TableKey(table.name(), branch); + int continuousEmptyCheckpoints = + continuousEmptyCheckpointsMap.computeIfAbsent(key, unused -> 0); + int maxContinuousEmptyCommits = + maxContinuousEmptyCommitsMap.computeIfAbsent( + key, + unused -> { + int result = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + result > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + return result; + }); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(table, branch, pendingResults, summary, newFlinkJobId, operatorId); + } else { + commitDeltaTxn(table, branch, 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); + } + + continuousEmptyCheckpointsMap.put(key, continuousEmptyCheckpoints); + } + + private void replacePartitions( + Table table, + String branch, + NavigableMap> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + 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. + for (WriteResult result : e.getValue()) { + ReplacePartitions dynamicOverwrite = + table.newReplacePartitions().scanManifestsWith(workerPool); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + commitOperation( + table, + branch, + dynamicOverwrite, + summary, + "dynamic partition overwrite", + newFlinkJobId, + operatorId, + e.getKey()); + } + } + } + + private void commitDeltaTxn( + Table table, + String branch, + NavigableMap> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + 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. + for (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); + commitOperation( + table, branch, rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + Table table, + String branch, + SnapshotUpdate operation, + CommitSummary summary, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + LOG.info( + "Committing {} for checkpoint {} to table {} branch {} with summary: {}", + description, + checkpointId, + table.name(), + branch, + summary); + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(FLINK_JOB_ID, newFlinkJobId); + operation.set(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(table.name(), durationMs); + } + } + + @Override + public void close() throws IOException { + // do nothing + } + + private static class TableKey implements Serializable { + private String tableName; + private String branch; + + TableKey(String tableName, String branch) { + this.tableName = tableName; + this.branch = branch; + } + + TableKey(DynamicCommittable committable) { + this.tableName = committable.key().tableName(); + this.branch = committable.key().branch(); + } + + String tableName() { + return tableName; + } + + String branch() { + return branch; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + TableKey that = (TableKey) other; + return tableName.equals(that.tableName) && branch.equals(that.branch); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, branch); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", tableName) + .add("branch", branch) + .toString(); + } + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java new file mode 100644 index 000000000000..c7f6225eb2db --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java @@ -0,0 +1,52 @@ +/* + * 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.dynamic; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.flink.sink.CommitSummary; +import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +public class DynamicCommitterMetrics { + + private final Map metrics; + private final MetricGroup mainMetricsGroup; + + public DynamicCommitterMetrics(MetricGroup mainMetricsGroup) { + this.mainMetricsGroup = mainMetricsGroup; + this.metrics = Maps.newHashMap(); + } + + public void commitDuration(String fullTableName, long commitDurationMs) { + committerMetrics(fullTableName).commitDuration(commitDurationMs); + } + + /** This is called upon a successful commit. */ + public void updateCommitSummary(String fullTableName, CommitSummary stats) { + committerMetrics(fullTableName).updateCommitSummary(stats); + } + + private IcebergFilesCommitterMetrics committerMetrics(String fullTableName) { + return metrics.computeIfAbsent( + fullTableName, tableName -> new IcebergFilesCommitterMetrics(mainMetricsGroup, tableName)); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java new file mode 100644 index 000000000000..379c2aff3a78 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java @@ -0,0 +1,42 @@ +/* + * 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.dynamic; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.io.WriteResult; + +@Internal +class DynamicWriteResult { + + private final WriteTarget key; + private final WriteResult writeResult; + + DynamicWriteResult(WriteTarget key, WriteResult writeResult) { + this.key = key; + this.writeResult = writeResult; + } + + WriteTarget key() { + return key; + } + + WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java new file mode 100644 index 000000000000..566cfb9963b7 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -0,0 +1,190 @@ +/* + * 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.dynamic; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.io.IOException; +import java.time.Duration; +import java.util.Collection; +import java.util.Map; +import org.apache.flink.annotation.Internal; +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.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.sink.DeltaManifests; +import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; +import org.apache.iceberg.flink.sink.FlinkManifestUtil; +import org.apache.iceberg.flink.sink.IcebergCommittable; +import org.apache.iceberg.flink.sink.ManifestOutputFileFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +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 DeltaManifests}, jobId, + * operatorId, checkpointId) + */ +@Internal +class DynamicWriteResultAggregator + extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(DynamicWriteResultAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private static final Duration CACHE_EXPIRATION_DURATION = Duration.ofMinutes(1); + + private final CatalogLoader catalogLoader; + private transient Map> results; + private transient Cache> specs; + private transient Cache outputFileFactories; + private transient String flinkJobId; + private transient String operatorId; + private transient int subTaskId; + private transient int attemptId; + private transient Catalog catalog; + + DynamicWriteResultAggregator(CatalogLoader catalogLoader) { + this.catalogLoader = catalogLoader; + } + + @Override + public void open() throws Exception { + this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + this.operatorId = getOperatorID().toString(); + this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.results = Maps.newHashMap(); + this.specs = + Caffeine.newBuilder().expireAfterWrite(CACHE_EXPIRATION_DURATION).softValues().build(); + this.outputFileFactories = + Caffeine.newBuilder().expireAfterWrite(CACHE_EXPIRATION_DURATION).softValues().build(); + this.catalog = catalogLoader.loadCatalog(); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + // TODO emit something if there is no incoming result for a given table for a given checkpoint + Collection> committables = + Sets.newHashSetWithExpectedSize(results.size()); + int count = 0; + for (Map.Entry> entries : results.entrySet()) { + committables.add( + new CommittableWithLineage<>( + new DynamicCommittable( + entries.getKey(), + writeToManifest(entries.getKey(), entries.getValue(), checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId), + checkpointId, + count)); + ++count; + } + + output.collect( + new StreamRecord<>( + new CommittableSummary<>(subTaskId, count, checkpointId, count, count, 0))); + committables.forEach( + c -> + output.collect( + new StreamRecord<>( + new CommittableWithLineage<>(c.getCommittable(), checkpointId, subTaskId)))); + LOG.info("Emitted {} commit message to downstream committer operator", count); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + private byte[] writeToManifest( + WriteTarget key, Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult.Builder builder = WriteResult.builder(); + writeResults.forEach(w -> builder.add(w.writeResult())); + WriteResult result = builder.build(); + + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, + () -> outputFileFactory(key.tableName()).create(checkpointId), + spec(key.tableName(), key.specId())); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + DynamicWriteResult result = + ((CommittableWithLineage) element.getValue()).getCommittable(); + WriteTarget key = result.key(); + results.computeIfAbsent(key, unused -> Sets.newHashSet()).add(result); + } + } + + private ManifestOutputFileFactory outputFileFactory(String tableName) { + return outputFileFactories.get( + tableName, + unused -> { + Table table = catalog.loadTable(TableIdentifier.parse(tableName)); + specs.put(tableName, table.specs()); + return FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + }); + } + + private PartitionSpec spec(String tableName, int specId) { + Map knownSpecs = specs.getIfPresent(tableName); + if (knownSpecs != null) { + PartitionSpec spec = knownSpecs.get(specId); + if (spec != null) { + return spec; + } + } + + Table table = catalog.loadTable(TableIdentifier.parse(tableName)); + return table.specs().get(specId); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java new file mode 100644 index 000000000000..6635990ca18b --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java @@ -0,0 +1,64 @@ +/* + * 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.dynamic; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.iceberg.flink.sink.WriteResultSerializer; +import org.apache.iceberg.io.WriteResult; + +@Internal +class DynamicWriteResultSerializer implements SimpleVersionedSerializer { + + private static final int VERSION = 1; + private static final WriteResultSerializer WRITE_RESULT_SERIALIZER = new WriteResultSerializer(); + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(DynamicWriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + writeResult.key().serializeTo(view); + byte[] result = WRITE_RESULT_SERIALIZER.serialize(writeResult.writeResult()); + view.write(result); + return out.toByteArray(); + } + + @Override + public DynamicWriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + WriteTarget key = WriteTarget.deserializeFrom(view); + byte[] resultBuf = new byte[view.available()]; + view.read(resultBuf); + WriteResult writeResult = WRITE_RESULT_SERIALIZER.deserialize(version, resultBuf); + return new DynamicWriteResult(key, writeResult); + } + + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java new file mode 100644 index 000000000000..b8e83b1140be --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -0,0 +1,219 @@ +/* + * 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.dynamic; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +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.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +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.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * 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. + */ +@Internal +class DynamicWriter implements CommittingSinkWriter { + + private static final Logger LOG = LoggerFactory.getLogger(DynamicWriter.class); + + private final Cache taskWriterFactories; + private final Map> writers; + private final DynamicWriterMetrics metrics; + private final int subTaskId; + private final int attemptId; + private final Catalog catalog; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final Map commonWriteProperties; + + DynamicWriter( + Catalog catalog, + FileFormat dataFileFormat, + long targetDataFileSize, + Map commonWriteProperties, + int cacheMaximumSize, + DynamicWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.catalog = catalog; + this.dataFileFormat = dataFileFormat; + this.targetDataFileSize = targetDataFileSize; + this.commonWriteProperties = commonWriteProperties; + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + this.taskWriterFactories = Caffeine.newBuilder().maximumSize(cacheMaximumSize).build(); + this.writers = Maps.newHashMap(); + + LOG.debug("DynamicIcebergSinkWriter created for subtask {} attemptId {}", subTaskId, attemptId); + } + + @Override + public void write(DynamicRecordInternal element, Context context) + throws IOException, InterruptedException { + writers + .computeIfAbsent( + new WriteTarget( + element.tableName(), + element.branch(), + element.schema().schemaId(), + element.spec().specId(), + element.upsertMode(), + element.equalityFields()), + writerKey -> { + RowDataTaskWriterFactory taskWriterFactory = + taskWriterFactories.get( + writerKey, + factoryKey -> { + Table table = + catalog.loadTable(TableIdentifier.parse(factoryKey.tableName())); + + // TODO: Handle precedence correctly for the write properties coming from + // the sink conf and from the table defaults + Map tableWriteProperties = + Maps.newHashMap(commonWriteProperties); + tableWriteProperties.putAll(table.properties()); + + List equalityFieldIds = + getEqualityFields(table, element.equalityFields()); + if (element.upsertMode()) { + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data."); + 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, + equalityFieldIds); + } + } + } + + return new RowDataTaskWriterFactory( + () -> table, + FlinkSchemaUtil.convert(element.schema()), + targetDataFileSize, + dataFileFormat, + tableWriteProperties, + equalityFieldIds, + element.upsertMode(), + element.schema(), + element.spec()); + }); + + taskWriterFactory.initialize(subTaskId, attemptId); + return taskWriterFactory.create(); + }) + .write(element.rowData()); + } + + @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 { + for (TaskWriter writer : writers.values()) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("subtaskId", subTaskId) + .add("attemptId", attemptId) + .add("dataFileFormat", dataFileFormat) + .add("targetDataFileSize", targetDataFileSize) + .add("writeProperties", commonWriteProperties) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + List result = Lists.newArrayList(); + for (Map.Entry> entry : writers.entrySet()) { + long startNano = System.nanoTime(); + WriteResult writeResult = entry.getValue().complete(); + WriteTarget writeTarget = entry.getKey(); + metrics.updateFlushResult(writeTarget.tableName(), writeResult); + metrics.flushDuration( + writeTarget.tableName(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer for table {} subtask {} attempt {} flushed {} data files and {} delete files", + writeTarget.tableName(), + subTaskId, + attemptId, + writeResult.dataFiles().length, + writeResult.deleteFiles().length); + + result.add(new DynamicWriteResult(writeTarget, writeResult)); + } + + writers.clear(); + + return result; + } + + private static List getEqualityFields(Table table, List equalityFieldIds) { + if (equalityFieldIds != null && !equalityFieldIds.isEmpty()) { + return equalityFieldIds; + } + Set identifierFieldIds = table.schema().identifierFieldIds(); + if (identifierFieldIds != null && !identifierFieldIds.isEmpty()) { + return Lists.newArrayList(identifierFieldIds); + } + return Collections.emptyList(); + } + + @VisibleForTesting + DynamicWriterMetrics getMetrics() { + return metrics; + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java new file mode 100644 index 000000000000..404ffaabdd71 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -0,0 +1,51 @@ +/* + * 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.dynamic; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.flink.sink.IcebergStreamWriterMetrics; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +public class DynamicWriterMetrics { + + private final Map metrics; + private final MetricGroup mainMetricsGroup; + + public DynamicWriterMetrics(MetricGroup mainMetricsGroup) { + this.mainMetricsGroup = mainMetricsGroup; + this.metrics = Maps.newHashMap(); + } + + public void updateFlushResult(String fullTableName, WriteResult result) { + writerMetrics(fullTableName).updateFlushResult(result); + } + + public void flushDuration(String fullTableName, long flushDurationMs) { + writerMetrics(fullTableName).flushDuration(flushDurationMs); + } + + IcebergStreamWriterMetrics writerMetrics(String fullTableName) { + return metrics.computeIfAbsent( + fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java new file mode 100644 index 000000000000..dcb4ec316339 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -0,0 +1,145 @@ +/* + * 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.dynamic; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +class WriteTarget implements Serializable { + + private final String tableName; + private final String branch; + private final Integer schemaId; + private final Integer specId; + private final boolean upsertMode; + private final List equalityFields; + + WriteTarget( + String tableName, + String branch, + Integer schemaId, + Integer specId, + boolean upsertMode, + List equalityFields) { + this.tableName = tableName; + this.branch = branch != null ? branch : "main"; + this.schemaId = schemaId; + this.specId = specId; + this.upsertMode = upsertMode; + this.equalityFields = equalityFields; + } + + String tableName() { + return tableName; + } + + String branch() { + return branch; + } + + Integer schemaId() { + return schemaId; + } + + Integer specId() { + return specId; + } + + boolean upsertMode() { + return upsertMode; + } + + List equalityFields() { + return equalityFields; + } + + void serializeTo(DataOutputView view) throws IOException { + view.writeUTF(tableName); + view.writeUTF(branch); + view.writeInt(schemaId); + view.writeInt(specId); + view.writeBoolean(upsertMode); + view.writeInt(equalityFields.size()); + for (Integer equalityField : equalityFields) { + view.writeInt(equalityField); + } + } + + static WriteTarget deserializeFrom(DataInputView view) throws IOException { + return new WriteTarget( + view.readUTF(), + view.readUTF(), + view.readInt(), + view.readInt(), + view.readBoolean(), + readList(view)); + } + + private static List readList(DataInputView view) throws IOException { + int numFields = view.readInt(); + List equalityFields = Lists.newArrayList(); + for (int i = 0; i < numFields; i++) { + equalityFields.add(view.readInt()); + } + return equalityFields; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + WriteTarget that = (WriteTarget) other; + return Objects.equals(tableName, that.tableName) + && Objects.equals(branch, that.branch) + && Objects.equals(schemaId, that.schemaId) + && Objects.equals(specId, that.specId) + && upsertMode == that.upsertMode + && Objects.equals(equalityFields, that.equalityFields); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, branch, schemaId, specId, upsertMode, equalityFields); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", tableName) + .add("branch", branch) + .add("schemaId", schemaId) + .add("specId", specId) + .add("upsertMode", upsertMode) + .toString(); + } +} From b36534fe63b5ced5e89f79e68b6664bbf40e4112 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 15:55:58 +0200 Subject: [PATCH 02/23] Add previously missing test cases --- .../sink/dynamic/DynamicCommittable.java | 18 +++ .../TestDynamicCommittableSerializer.java | 45 ++++++++ .../flink/sink/dynamic/TestDynamicWriter.java | 107 ++++++++++++++++++ 3 files changed, 170 insertions(+) create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index a0abbb72701d..b9d3a57ef1ab 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -19,6 +19,8 @@ package org.apache.iceberg.flink.sink.dynamic; import java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; import org.apache.flink.annotation.Internal; import org.apache.iceberg.flink.sink.IcebergCommittableSerializer; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -69,6 +71,22 @@ Long checkpointId() { return checkpointId; } + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) return false; + DynamicCommittable that = (DynamicCommittable) o; + return checkpointId == that.checkpointId + && Objects.equals(key, that.key) + && Objects.deepEquals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + return Objects.hash(key, Arrays.hashCode(manifest), jobId, operatorId, checkpointId); + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java new file mode 100644 index 000000000000..a032b96e2abd --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java @@ -0,0 +1,45 @@ +/* + * 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.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; + +class TestDynamicCommittableSerializer { + + @Test + void testRoundtrip() throws IOException { + DynamicCommittable committable = + new DynamicCommittable( + new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new byte[] {3, 4}, + JobID.generate().toHexString(), + new OperatorID().toHexString(), + 5); + + DynamicCommittableSerializer serializer = new DynamicCommittableSerializer(); + assertThat(serializer.deserialize(serializer.getVersion(), serializer.serialize(committable))) + .isEqualTo(committable); + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java new file mode 100644 index 000000000000..7f30fa87e379 --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -0,0 +1,107 @@ +/* + * 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.dynamic; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.File; +import java.net.URI; +import java.util.Collection; +import java.util.Map; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; +import org.apache.iceberg.io.WriteResult; +import org.junit.jupiter.api.Test; + +class TestDynamicWriter extends TestFlinkIcebergSinkBase { + + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("myTable"); + + @Test + void testDynamicWriter() throws Exception { + runWriterTest(); + } + + private static DynamicWriter runWriterTest() throws Exception { + return runWriterTest(Map.of()); + } + + private static DynamicWriter runWriterTest(Map writeProperties) throws Exception { + Catalog catalog = CATALOG_EXTENSION.catalog(); + Table table = catalog.createTable(TABLE_IDENTIFIER, SimpleDataUtil.SCHEMA); + + DynamicWriter dynamicWriter = + new DynamicWriter( + catalog, + FileFormat.PARQUET, + 1024L, + writeProperties, + 100, + new DynamicWriterMetrics(new UnregisteredMetricsGroup()), + 0, + 0); + + DynamicRecordInternal record = new DynamicRecordInternal(); + record.setTableName(TABLE_IDENTIFIER.name()); + record.setSchema(table.schema()); + record.setSpec(table.spec()); + record.setRowData(SimpleDataUtil.createRowData(1, "test")); + + assertThat(getNumDataFiles(table)).isEqualTo(0); + + dynamicWriter.write(record, null); + Collection writeResults1 = dynamicWriter.prepareCommit(); + + assertThat(getNumDataFiles(table)).isEqualTo(1); + + assertThat(writeResults1.size()).isEqualTo(1); + WriteResult wr1 = writeResults1.iterator().next().writeResult(); + assertThat(wr1.dataFiles().length).isEqualTo(1); + assertThat(wr1.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); + assertThat(wr1.deleteFiles()).isEmpty(); + + dynamicWriter.write(record, null); + Collection writeResults2 = dynamicWriter.prepareCommit(); + + assertThat(getNumDataFiles(table)).isEqualTo(2); + + assertThat(writeResults2.size()).isEqualTo(1); + WriteResult wr2 = writeResults2.iterator().next().writeResult(); + assertThat(wr2.dataFiles().length).isEqualTo(1); + assertThat(wr2.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); + assertThat(wr2.deleteFiles()).isEmpty(); + + dynamicWriter.close(); + + return dynamicWriter; + } + + private static int getNumDataFiles(Table table) { + File dataDir = new File(URI.create(table.location()).getPath(), "data"); + if (dataDir.exists()) { + return dataDir.listFiles((dir, name) -> !name.startsWith(".")).length; + } + return 0; + } +} From e4eb33d043141f82f0b9456c33eaa5857f757b48 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 16:14:09 +0200 Subject: [PATCH 03/23] annotations and newlines --- .../main/java/org/apache/iceberg/flink/sink/CommitSummary.java | 2 ++ .../main/java/org/apache/iceberg/flink/sink/DeltaManifests.java | 2 ++ .../java/org/apache/iceberg/flink/sink/IcebergCommittable.java | 2 ++ .../apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java | 2 ++ .../apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java | 2 ++ .../apache/iceberg/flink/sink/ManifestOutputFileFactory.java | 2 ++ .../org/apache/iceberg/flink/sink/WriteResultSerializer.java | 2 ++ .../org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java | 2 ++ 8 files changed, 16 insertions(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 62de3170d55c..1b786e46452f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -22,10 +22,12 @@ import java.util.List; import java.util.NavigableMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.util.ScanTaskUtil; +@Internal public class CommitSummary { private final AtomicLong dataFilesCount = new AtomicLong(); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index 89c26692b113..92c50165c0f5 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java @@ -19,10 +19,12 @@ package org.apache.iceberg.flink.sink; import java.util.List; +import org.apache.flink.annotation.Internal; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +@Internal public class DeltaManifests { private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 8b06949a5519..060c4005def4 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Arrays; import java.util.Objects; +import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** @@ -31,6 +32,7 @@ *

{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer * and the Aggregator operator and between the Aggregator and the Committer as well. */ +@Internal public class IcebergCommittable implements Serializable { private final byte[] manifest; private final String jobId; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 86af9c5154f7..ce81ef11f13c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -20,10 +20,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.flink.util.ElapsedTimeGauge; +@Internal public class IcebergFilesCommitterMetrics { private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); private final AtomicLong lastCommitDurationMs = new AtomicLong(); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index 04ea868c0a38..6359459360fe 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -21,6 +21,7 @@ import com.codahale.metrics.SlidingWindowReservoir; import java.util.Arrays; import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Histogram; @@ -28,6 +29,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.util.ScanTaskUtil; +@Internal public class IcebergStreamWriterMetrics { // 1,024 reservoir size should cost about 8KB, which is quite small. // It should also produce good accuracy for histogram distribution (like percentiles). diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 81434ad171fb..6ba87bea30c2 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; +import org.apache.flink.annotation.Internal; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.Table; @@ -30,6 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; +@Internal public class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java index 34868458b0e0..40a3ce0cb846 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -20,12 +20,14 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.apache.flink.annotation.Internal; 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; +@Internal public class WriteResultSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index cf087ed1459e..a596556d27e4 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -168,6 +168,7 @@ private static long getMaxCommittedCheckpointId( break; } } + Long parentSnapshotId = snapshot.parentId(); snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; } @@ -253,6 +254,7 @@ private void commitPendingResult( } else { commitDeltaTxn(table, branch, pendingResults, summary, newFlinkJobId, operatorId); } + continuousEmptyCheckpoints = 0; } else { long checkpointId = pendingResults.lastKey(); From 65ab8157b99f210741a77cb9053fc02fa2a1ccec Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 16:19:14 +0200 Subject: [PATCH 04/23] another new line --- .../iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java index 3ec6db579c05..6609a7e42aee 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -69,6 +69,7 @@ public DynamicCommittable deserialize(int version, byte[] serialized) throws IOE view.read(manifestBuf); return new DynamicCommittable(key, manifestBuf, jobId, operatorId, checkpointId); } + throw new IOException("Unrecognized version or corrupt state: " + version); } } From 0d58f5bc8ce607fd0542261fba9b5e0a9083e6c7 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 16:22:05 +0200 Subject: [PATCH 05/23] Add another test case --- .../TestDynamicCommittableSerializer.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java index a032b96e2abd..f4109a6476aa 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.dynamic; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import org.apache.flink.api.common.JobID; @@ -42,4 +43,20 @@ void testRoundtrip() throws IOException { assertThat(serializer.deserialize(serializer.getVersion(), serializer.serialize(committable))) .isEqualTo(committable); } + + @Test + void testUnsupportedVersion() throws IOException { + DynamicCommittable committable = + new DynamicCommittable( + new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new byte[] {3, 4}, + JobID.generate().toHexString(), + new OperatorID().toHexString(), + 5); + + DynamicCommittableSerializer serializer = new DynamicCommittableSerializer(); + assertThatThrownBy(() -> serializer.deserialize(-1, serializer.serialize(committable))) + .hasMessage("Unrecognized version or corrupt state: -1") + .isInstanceOf(IOException.class); + } } From 31eada5fc01c9f0383d6fe1fca23ce08abb52af7 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 16:43:37 +0200 Subject: [PATCH 06/23] checkstyle --- .../iceberg/flink/sink/dynamic/DynamicCommittable.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index b9d3a57ef1ab..b52a283a426b 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -73,7 +73,10 @@ Long checkpointId() { @Override public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) return false; + if (o == null || getClass() != o.getClass()) { + return false; + } + DynamicCommittable that = (DynamicCommittable) o; return checkpointId == that.checkpointId && Objects.equals(key, that.key) From 27bfa52dc3f9d9c85b0c915091e36ca1f9f46a70 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 May 2025 17:03:32 +0200 Subject: [PATCH 07/23] Missing @Internal annotation --- .../org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java index 59de599a6886..6ad41bacf337 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java @@ -23,11 +23,13 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +@Internal public class DeltaManifestsSerializer implements SimpleVersionedSerializer { private static final int VERSION_1 = 1; private static final int VERSION_2 = 2; From 80b95c9f565346adfe94f73b57c04ec83db099a9 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 21 May 2025 17:48:54 +0200 Subject: [PATCH 08/23] PR comments --- .../iceberg/flink/sink/IcebergCommittable.java | 4 +--- .../flink/sink/dynamic/DynamicCommittable.java | 2 +- .../dynamic/DynamicCommittableSerializer.java | 8 +++----- .../flink/sink/dynamic/DynamicCommitter.java | 16 +++++++++------- .../dynamic/DynamicWriteResultAggregator.java | 3 +-- .../flink/sink/dynamic/DynamicWriter.java | 7 +++---- 6 files changed, 18 insertions(+), 22 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 060c4005def4..408c3e9a9d5f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -21,7 +21,6 @@ import java.io.Serializable; import java.util.Arrays; import java.util.Objects; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** @@ -32,8 +31,7 @@ *

{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer * and the Aggregator operator and between the Aggregator and the Committer as well. */ -@Internal -public class IcebergCommittable implements Serializable { +class IcebergCommittable implements Serializable { private final byte[] manifest; private final String jobId; private final String operatorId; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index b52a283a426b..e8b5ed1fd331 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -67,7 +67,7 @@ String operatorId() { return operatorId; } - Long checkpointId() { + long checkpointId() { return checkpointId; } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java index 6609a7e42aee..3284d6bd7251 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -24,13 +24,11 @@ import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.iceberg.flink.sink.IcebergCommittable; /** - * 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. + * This serializer is used for serializing the {@link DynamicCommittable} objects between the {@link + * DynamicWriter} and the {@link DynamicWriteResultAggregator} operator and for sending it down to + * the {@link DynamicCommitter}. */ @Internal class DynamicCommittableSerializer implements SimpleVersionedSerializer { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index a596556d27e4..ea567b647602 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -35,6 +35,7 @@ import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; @@ -43,7 +44,6 @@ import org.apache.iceberg.flink.sink.DeltaManifests; import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; import org.apache.iceberg.flink.sink.FlinkManifestUtil; -import org.apache.iceberg.flink.sink.IcebergCommittable; 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; @@ -60,7 +60,7 @@ * commits. The implementation builds on the following assumptions: * *

    - *
  • There is a single {@link IcebergCommittable} for every checkpoint + *
  • There is a single {@link DynamicCommittable} for every table / 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 @@ -118,6 +118,9 @@ public void commit(Collection> commitRequests) return; } + // For every table and every checkpoint, we store the list of to-be-committed + // DynamicCommittable. + // There may be DynamicCommittable from previous checkpoints which have not been committed yet. Map>>> commitRequestMap = Maps.newHashMap(); for (CommitRequest request : commitRequests) { @@ -179,13 +182,12 @@ private static long getMaxCommittedCheckpointId( /** * Commits the data to the Iceberg table by reading the file data from the {@link DeltaManifests} * ordered by the checkpointId, and writing the new snapshot to the Iceberg table. The {@link - * org.apache.iceberg.SnapshotSummary} will contain the jobId, snapshotId, checkpointId so in case - * of job restart we can identify which changes are committed, and which are still waiting for the - * commit. + * 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} + * @param newFlinkJobId The jobId to store in the {@link SnapshotSummary} + * @param operatorId The operatorId to store in the {@link SnapshotSummary} * @throws IOException On commit failure */ private void commitPendingRequests( diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index 566cfb9963b7..b3c976357867 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.sink.DeltaManifests; import org.apache.iceberg.flink.sink.DeltaManifestsSerializer; import org.apache.iceberg.flink.sink.FlinkManifestUtil; -import org.apache.iceberg.flink.sink.IcebergCommittable; import org.apache.iceberg.flink.sink.ManifestOutputFileFactory; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -50,7 +49,7 @@ /** * Operator which aggregates the individual {@link WriteResult} objects to a single {@link - * IcebergCommittable} per checkpoint (storing the serialized {@link DeltaManifests}, jobId, + * DynamicCommittable} per checkpoint (storing the serialized {@link DeltaManifests}, jobId, * operatorId, checkpointId) */ @Internal diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index b8e83b1140be..e47ca69e2180 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -38,7 +38,6 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; @@ -50,9 +49,9 @@ import org.slf4j.LoggerFactory; /** - * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link - * 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. + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the + * DynamicIcebergSink. Writes out the data to the final place, and emits {@link DynamicWriteResult} + * for every unique {@link WriteTarget} at checkpoint time. */ @Internal class DynamicWriter implements CommittingSinkWriter { From b847ffad5d942e0345f3041be449e27837cb5e64 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 22 May 2025 11:47:22 +0200 Subject: [PATCH 09/23] Comment update --- .../org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java | 2 +- .../flink/sink/dynamic/DynamicWriteResultAggregator.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index ea567b647602..3e051dc5d549 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -60,7 +60,7 @@ * commits. The implementation builds on the following assumptions: * *
      - *
    • There is a single {@link DynamicCommittable} for every table / checkpoint + *
    • There is a single {@link DynamicCommittable} for every table / branch / 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 diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index b3c976357867..e2a4222c0c30 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -96,7 +96,6 @@ public void finish() throws IOException { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { - // TODO emit something if there is no incoming result for a given table for a given checkpoint Collection> committables = Sets.newHashSetWithExpectedSize(results.size()); int count = 0; From 63f4f4477bad7bc133b772e809f6657151fb9e74 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 2 Jun 2025 12:55:20 +0200 Subject: [PATCH 10/23] Improve DynamicWriter test --- .../sink/IcebergStreamWriterMetrics.java | 11 ++ .../flink/sink/dynamic/TestDynamicWriter.java | 152 +++++++++++++----- 2 files changed, 125 insertions(+), 38 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index 6359459360fe..ccc07cb369fb 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.concurrent.atomic.AtomicLong; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Histogram; @@ -89,4 +90,14 @@ public void updateFlushResult(WriteResult result) { public void flushDuration(long flushDurationMs) { lastFlushDurationMs.set(flushDurationMs); } + + @VisibleForTesting + public Counter getFlushedDataFiles() { + return flushedDataFiles; + } + + @VisibleForTesting + public Counter getFlushedDeleteFiles() { + return flushedDeleteFiles; + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 7f30fa87e379..0a723c9d5700 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import java.io.File; @@ -32,69 +33,144 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Test; class TestDynamicWriter extends TestFlinkIcebergSinkBase { - private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("myTable"); + private static final TableIdentifier TABLE1 = TableIdentifier.of("myTable1"); + private static final TableIdentifier TABLE2 = TableIdentifier.of("myTable2"); @Test void testDynamicWriter() throws Exception { - runWriterTest(); + Catalog catalog = CATALOG_EXTENSION.catalog(); + Table table1 = catalog.createTable(TABLE1, SimpleDataUtil.SCHEMA); + Table table2 = catalog.createTable(TABLE2, SimpleDataUtil.SCHEMA); + + DynamicWriter dynamicWriter = createDynamicWriter(catalog); + + DynamicRecordInternal record1 = getDynamicRecordInternal(table1); + DynamicRecordInternal record2 = getDynamicRecordInternal(table2); + + assertThat(getNumDataFiles(table1)).isEqualTo(0); + + dynamicWriter.write(record1, null); + dynamicWriter.write(record2, null); + Collection writeResults = dynamicWriter.prepareCommit(); + + assertThat(writeResults.size()).isEqualTo(2); + assertThat(getNumDataFiles(table1)).isEqualTo(1); + assertThat( + dynamicWriter + .getMetrics() + .writerMetrics(TABLE1.name()) + .getFlushedDataFiles() + .getCount()) + .isEqualTo(1); + assertThat( + dynamicWriter + .getMetrics() + .writerMetrics(TABLE2.name()) + .getFlushedDataFiles() + .getCount()) + .isEqualTo(1); + + WriteResult wr1 = writeResults.iterator().next().writeResult(); + assertThat(wr1.dataFiles().length).isEqualTo(1); + assertThat(wr1.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); + assertThat(wr1.deleteFiles()).isEmpty(); + + dynamicWriter.write(record1, null); + dynamicWriter.write(record2, null); + writeResults = dynamicWriter.prepareCommit(); + + assertThat(writeResults.size()).isEqualTo(2); + assertThat(getNumDataFiles(table1)).isEqualTo(2); + assertThat( + dynamicWriter + .getMetrics() + .writerMetrics(TABLE1.name()) + .getFlushedDataFiles() + .getCount()) + .isEqualTo(2); + assertThat( + dynamicWriter + .getMetrics() + .writerMetrics(TABLE2.name()) + .getFlushedDataFiles() + .getCount()) + .isEqualTo(2); + + WriteResult wr2 = writeResults.iterator().next().writeResult(); + assertThat(wr2.dataFiles().length).isEqualTo(1); + assertThat(wr2.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); + assertThat(wr2.deleteFiles()).isEmpty(); + + dynamicWriter.close(); } - private static DynamicWriter runWriterTest() throws Exception { - return runWriterTest(Map.of()); + @Test + void testDynamicWriterUpsert() throws Exception { + Catalog catalog = CATALOG_EXTENSION.catalog(); + DynamicWriter dyamicWriter = createDynamicWriter(catalog); + Table table1 = CATALOG_EXTENSION.catalog().createTable(TABLE1, SimpleDataUtil.SCHEMA); + + DynamicRecordInternal record = getDynamicRecordInternal(table1); + record.setUpsertMode(true); + record.setEqualityFieldIds(Lists.newArrayList(1)); + + dyamicWriter.write(record, null); + dyamicWriter.prepareCommit(); + + assertThat( + dyamicWriter + .getMetrics() + .writerMetrics(TABLE1.name()) + .getFlushedDeleteFiles() + .getCount()) + .isEqualTo(1); + assertThat( + dyamicWriter.getMetrics().writerMetrics(TABLE1.name()).getFlushedDataFiles().getCount()) + .isEqualTo(1); } - private static DynamicWriter runWriterTest(Map writeProperties) throws Exception { + @Test + void testDynamicWriterUpsertNoEqualityFields() { Catalog catalog = CATALOG_EXTENSION.catalog(); - Table table = catalog.createTable(TABLE_IDENTIFIER, SimpleDataUtil.SCHEMA); + DynamicWriter dyamicWriter = createDynamicWriter(catalog); + Table table1 = CATALOG_EXTENSION.catalog().createTable(TABLE1, SimpleDataUtil.SCHEMA); + + DynamicRecordInternal record = getDynamicRecordInternal(table1); + record.setUpsertMode(true); + assertThatThrownBy(() -> dyamicWriter.write(record, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data."); + } + + private static @NotNull DynamicWriter createDynamicWriter(Catalog catalog) { DynamicWriter dynamicWriter = new DynamicWriter( catalog, FileFormat.PARQUET, 1024L, - writeProperties, + Map.of(), 100, new DynamicWriterMetrics(new UnregisteredMetricsGroup()), 0, 0); + return dynamicWriter; + } + private static @NotNull DynamicRecordInternal getDynamicRecordInternal(Table table1) { DynamicRecordInternal record = new DynamicRecordInternal(); - record.setTableName(TABLE_IDENTIFIER.name()); - record.setSchema(table.schema()); - record.setSpec(table.spec()); + record.setTableName(TableIdentifier.parse(table1.name()).name()); + record.setSchema(table1.schema()); + record.setSpec(table1.spec()); record.setRowData(SimpleDataUtil.createRowData(1, "test")); - - assertThat(getNumDataFiles(table)).isEqualTo(0); - - dynamicWriter.write(record, null); - Collection writeResults1 = dynamicWriter.prepareCommit(); - - assertThat(getNumDataFiles(table)).isEqualTo(1); - - assertThat(writeResults1.size()).isEqualTo(1); - WriteResult wr1 = writeResults1.iterator().next().writeResult(); - assertThat(wr1.dataFiles().length).isEqualTo(1); - assertThat(wr1.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); - assertThat(wr1.deleteFiles()).isEmpty(); - - dynamicWriter.write(record, null); - Collection writeResults2 = dynamicWriter.prepareCommit(); - - assertThat(getNumDataFiles(table)).isEqualTo(2); - - assertThat(writeResults2.size()).isEqualTo(1); - WriteResult wr2 = writeResults2.iterator().next().writeResult(); - assertThat(wr2.dataFiles().length).isEqualTo(1); - assertThat(wr2.dataFiles()[0].format()).isEqualTo(FileFormat.PARQUET); - assertThat(wr2.deleteFiles()).isEmpty(); - - dynamicWriter.close(); - - return dynamicWriter; + return record; } private static int getNumDataFiles(Table table) { From 1ed25c71d8d6f40466f7a6956595fa0d19ff9304 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 2 Jun 2025 14:26:54 +0200 Subject: [PATCH 11/23] Add test for DynamicCommitter --- .../sink/IcebergStreamWriterMetrics.java | 3 - .../dynamic/DynamicWriteResultAggregator.java | 2 +- .../sink/dynamic/TestDynamicCommitter.java | 220 ++++++++++++++++++ 3 files changed, 221 insertions(+), 4 deletions(-) create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ccc07cb369fb..434f3969577f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.concurrent.atomic.AtomicLong; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Histogram; @@ -91,12 +90,10 @@ public void flushDuration(long flushDurationMs) { lastFlushDurationMs.set(flushDurationMs); } - @VisibleForTesting public Counter getFlushedDataFiles() { return flushedDataFiles; } - @VisibleForTesting public Counter getFlushedDeleteFiles() { return flushedDeleteFiles; } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index e2a4222c0c30..e7fcd2f5a615 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -129,7 +129,7 @@ public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { * Write all the completed data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest( + byte[] writeToManifest( WriteTarget key, Collection writeResults, long checkpointId) throws IOException { if (writeResults.isEmpty()) { diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java new file mode 100644 index 000000000000..69391319d45a --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -0,0 +1,220 @@ +/* + * 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.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.connector.sink2.Committer.CommitRequest; +import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +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.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +class TestDynamicCommitter { + + static final String DB = "db"; + static final String TABLE1 = "table"; + static final String TABLE2 = "table2"; + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DB, TABLE1); + + Catalog catalog; + + private static final DataFile DATA_FILE = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + 42L, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, ByteBuffer.allocate(1)), // lower bounds + ImmutableMap.of(1, ByteBuffer.allocate(1)) // upper bounds + )) + .build(); + + @BeforeEach + void before() { + catalog = CATALOG_EXTENSION.catalog(); + Schema schema1 = new Schema(42); + Schema schema2 = new Schema(43); + catalog.createTable(TableIdentifier.of(TABLE1), schema1); + catalog.createTable(TableIdentifier.of(TABLE2), schema2); + } + + @Test + void testCommit() throws Exception { + Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table1.snapshots()).isEmpty(); + Table table2 = catalog.loadTable(TableIdentifier.of(TABLE2)); + assertThat(table2.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter dynamicCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + WriteTarget writeTarget1 = + new WriteTarget(TABLE1, "branch", 42, 0, true, Lists.newArrayList(1, 2)); + WriteTarget writeTarget2 = + new WriteTarget(TABLE1, "branch2", 43, 0, true, Lists.newArrayList(1, 2)); + WriteTarget writeTarget3 = + new WriteTarget(TABLE2, "branch2", 43, 0, true, Lists.newArrayList(1, 2)); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + byte[] deltaManifest1 = + aggregator.writeToManifest( + writeTarget1, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget1, WriteResult.builder().addDataFiles(DATA_FILE).build())), + 0); + byte[] deltaManifest2 = + aggregator.writeToManifest( + writeTarget2, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget2, WriteResult.builder().addDataFiles(DATA_FILE).build())), + 0); + byte[] deltaManifest3 = + aggregator.writeToManifest( + writeTarget3, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget3, WriteResult.builder().addDataFiles(DATA_FILE).build())), + 0); + + final String jobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int checkpointId = 10; + + CommitRequest commitRequest1 = + new MockCommitRequest<>( + new DynamicCommittable(writeTarget1, deltaManifest1, jobId, operatorId, checkpointId)); + + CommitRequest commitRequest2 = + new MockCommitRequest<>( + new DynamicCommittable(writeTarget2, deltaManifest2, jobId, operatorId, checkpointId)); + + CommitRequest commitRequest3 = + new MockCommitRequest<>( + new DynamicCommittable(writeTarget3, deltaManifest3, jobId, operatorId, checkpointId)); + + dynamicCommitter.commit(Lists.newArrayList(commitRequest1, commitRequest2, commitRequest3)); + + table1.refresh(); + assertThat(table1.snapshots()).hasSize(2); + Snapshot first = Iterables.getFirst(table1.snapshots(), null); + assertThat(first.summary()) + .containsAllEntriesOf( + (Map) + ImmutableMap.builder() + .put("added-data-files", "1") + .put("added-records", "42") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", "" + checkpointId) + .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "0") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "0") + .put("total-records", "42") + .build()); + Snapshot second = Iterables.get(table1.snapshots(), 1, null); + assertThat(second.summary()) + .containsAllEntriesOf( + (Map) + ImmutableMap.builder() + .put("added-data-files", "1") + .put("added-records", "42") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", "" + checkpointId) + .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "0") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "0") + .put("total-records", "42") + .build()); + + table2.refresh(); + assertThat(table2.snapshots()).hasSize(1); + Snapshot third = Iterables.getFirst(table2.snapshots(), null); + assertThat(third.summary()) + .containsAllEntriesOf( + (Map) + ImmutableMap.builder() + .put("added-data-files", "1") + .put("added-records", "42") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", "" + checkpointId) + .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "0") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "0") + .put("total-records", "42") + .build()); + } +} From d2bf814470a7866a29519b9e6bb18ad4f27dabed Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 2 Jun 2025 14:28:07 +0200 Subject: [PATCH 12/23] Add test for DynamicWriteResultSerializer --- .../TestDynamicWriteResultSerializer.java | 82 +++++++++++++++++++ 1 file changed, 82 insertions(+) create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java new file mode 100644 index 000000000000..a44827f3cfbf --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java @@ -0,0 +1,82 @@ +/* + * 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.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.io.WriteResult; +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.Test; + +public class TestDynamicWriteResultSerializer { + + private static final DataFile DATA_FILE = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + 42L, + null, + ImmutableMap.of(1, 5L), + ImmutableMap.of(1, 0L), + null, + ImmutableMap.of(1, ByteBuffer.allocate(1)), + ImmutableMap.of(1, ByteBuffer.allocate(1)))) + .build(); + + @Test + void testRoundtrip() throws IOException { + DynamicWriteResult dynamicWriteResult = + new DynamicWriteResult( + new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + WriteResult.builder().addDataFiles(DATA_FILE).build()); + + DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); + DynamicWriteResult copy = + serializer.deserialize(serializer.getVersion(), serializer.serialize(dynamicWriteResult)); + + assertThat(copy.writeResult().dataFiles()).hasSize(1); + DataFile dataFile = copy.writeResult().dataFiles()[0]; + // DataFile doesn't implement equals, but we can still do basic checks + assertThat(dataFile.path()).isEqualTo("/path/to/data-1.parquet"); + assertThat(dataFile.recordCount()).isEqualTo(42L); + } + + @Test + void testUnsupportedVersion() throws IOException { + DynamicWriteResult dynamicWriteResult = + new DynamicWriteResult( + new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + WriteResult.builder().addDataFiles(DATA_FILE).build()); + + DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); + assertThatThrownBy(() -> serializer.deserialize(-1, serializer.serialize(dynamicWriteResult))) + .hasMessage("Unrecognized version or corrupt state: -1") + .isInstanceOf(IOException.class); + } +} From 96fbc11f1d5077af18665ff91efacbbdbb4d0e9c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 2 Jun 2025 17:05:16 +0200 Subject: [PATCH 13/23] Add test for DynamicWriteResultAggregator --- .../sink/dynamic/DynamicCommittable.java | 4 + .../TestDynamicWriteResultAggregator.java | 82 +++++++++++++++++++ 2 files changed, 86 insertions(+) create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index e8b5ed1fd331..f73e53d00ac5 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -99,4 +99,8 @@ public String toString() { .add("operatorId", operatorId) .toString(); } + + public WriteTarget writeTarget() { + return key; + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java new file mode 100644 index 000000000000..6c229beb71b7 --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java @@ -0,0 +1,82 @@ +/* + * 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.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestDynamicWriteResultAggregator { + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); + + @Test + void testAggregator() throws Exception { + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table"), new Schema()); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = new OneInputStreamOperatorTestHarness<>(aggregator)) { + testHarness.open(); + + WriteTarget writeTarget1 = + new WriteTarget("table", "branch", 42, 0, true, Lists.newArrayList()); + DynamicWriteResult dynamicWriteResult1 = + new DynamicWriteResult(writeTarget1, WriteResult.builder().build()); + WriteTarget writeTarget2 = + new WriteTarget("table", "branch", 42, 0, true, Lists.newArrayList(1, 2)); + DynamicWriteResult dynamicWriteResult2 = + new DynamicWriteResult(writeTarget2, WriteResult.builder().build()); + + CommittableWithLineage committable1 = + new CommittableWithLineage<>(dynamicWriteResult1, 0, 0); + StreamRecord> record1 = + new StreamRecord<>(committable1); + testHarness.processElement(record1); + CommittableWithLineage committable2 = + new CommittableWithLineage<>(dynamicWriteResult2, 0, 0); + StreamRecord> record2 = + new StreamRecord<>(committable2); + testHarness.processElement(record2); + + assertThat(testHarness.getOutput()).isEmpty(); + + testHarness.prepareSnapshotPreBarrier(1L); + // Contains a CommittableSummary + DynamicCommittable + assertThat(testHarness.getRecordOutput()).hasSize(3); + + testHarness.prepareSnapshotPreBarrier(2L); + // Only contains a CommittableSummary + assertThat(testHarness.getRecordOutput()).hasSize(4); + } + } +} From 720a5224b2c38033f4382a2598a16643a5a9cd8d Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 3 Jun 2025 14:10:45 +0200 Subject: [PATCH 14/23] JavaDoc --- .../iceberg/flink/sink/dynamic/DynamicCommittable.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index f73e53d00ac5..53a08970a8ae 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -22,13 +22,14 @@ import java.util.Arrays; import java.util.Objects; import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.sink.DeltaManifests; import org.apache.iceberg.flink.sink.IcebergCommittableSerializer; 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 help identifying the specific commit + * serialized {@link DeltaManifests} file - which contains the commit data, and the jobId, + * operatorId, checkpointId triplet to identify 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. From d0352b00b7cd91ec799ad95af8177674f992f896 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 3 Jun 2025 14:18:13 +0200 Subject: [PATCH 15/23] JavaDoc --- .../iceberg/flink/sink/dynamic/DynamicCommittable.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index 53a08970a8ae..99906c0c871e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -23,7 +23,6 @@ import java.util.Objects; import org.apache.flink.annotation.Internal; import org.apache.iceberg.flink.sink.DeltaManifests; -import org.apache.iceberg.flink.sink.IcebergCommittableSerializer; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** @@ -31,8 +30,8 @@ * serialized {@link DeltaManifests} file - which contains the commit data, and the jobId, * operatorId, checkpointId triplet to identify 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. + *

      {@link DynamicCommittableSerializer} is used to serialize {@link DynamicCommittable} between + * the {@link DynamicWriter} and the {@link DynamicWriteResultAggregator}. */ @Internal class DynamicCommittable implements Serializable { From 55cb66b8b774b2fbd0aacfd6021ddf629392f9ee Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 10:31:50 +0200 Subject: [PATCH 16/23] VisibleForTesting --- .../flink/sink/dynamic/DynamicWriteResultAggregator.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index e7fcd2f5a615..474eecc9e829 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Map; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; 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; @@ -129,6 +130,7 @@ public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { * Write all the completed data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ + @VisibleForTesting byte[] writeToManifest( WriteTarget key, Collection writeResults, long checkpointId) throws IOException { From 5e242e0c716580e8d2a8ac5c167da6a12e457fe6 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 10:32:00 +0200 Subject: [PATCH 17/23] newline --- .../java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index dcb4ec316339..5fdda0aeec27 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -105,6 +105,7 @@ private static List readList(DataInputView view) throws IOException { for (int i = 0; i < numFields; i++) { equalityFields.add(view.readInt()); } + return equalityFields; } From 12c82b378dc1b9338f951f2bf83cdfb8b39f8682 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 10:32:07 +0200 Subject: [PATCH 18/23] debug log on writer factory creation --- .../org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index e47ca69e2180..371352f28d03 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -133,6 +133,7 @@ public void write(DynamicRecordInternal element, Context context) } } + LOG.debug("Creating new writer factory for table '{}'", table.name()); return new RowDataTaskWriterFactory( () -> table, FlinkSchemaUtil.convert(element.schema()), From 0620f419f0563a1bc160a65759108ded9b0bc2b9 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 10:32:16 +0200 Subject: [PATCH 19/23] use different write targets --- .../flink/sink/dynamic/TestDynamicWriteResultAggregator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java index 6c229beb71b7..250b233a5ed4 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java @@ -40,6 +40,7 @@ public class TestDynamicWriteResultAggregator { @Test void testAggregator() throws Exception { CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table"), new Schema()); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table2"), new Schema()); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); @@ -53,7 +54,7 @@ void testAggregator() throws Exception { DynamicWriteResult dynamicWriteResult1 = new DynamicWriteResult(writeTarget1, WriteResult.builder().build()); WriteTarget writeTarget2 = - new WriteTarget("table", "branch", 42, 0, true, Lists.newArrayList(1, 2)); + new WriteTarget("table2", "branch", 42, 0, true, Lists.newArrayList(1, 2)); DynamicWriteResult dynamicWriteResult2 = new DynamicWriteResult(writeTarget2, WriteResult.builder().build()); From 6428cfa10467d854a4af287b32cae62a12f49ac8 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 10:54:35 +0200 Subject: [PATCH 20/23] Add already-committed test --- .../sink/dynamic/TestDynamicCommitter.java | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 69391319d45a..2f742fbe34d6 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -217,4 +217,81 @@ void testCommit() throws Exception { .put("total-records", "42") .build()); } + + @Test + void testAlreadyCommitted() throws Exception { + Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table1.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter dynamicCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + WriteTarget writeTarget = + new WriteTarget(TABLE1, "branch", 42, 0, false, Lists.newArrayList(1, 2)); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + byte[] deltaManifest = + aggregator.writeToManifest( + writeTarget, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), + 0); + + final String jobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int checkpointId = 10; + + CommitRequest commitRequest = + new MockCommitRequest<>( + new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + + dynamicCommitter.commit(Lists.newArrayList(commitRequest)); + + CommitRequest oldCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + writeTarget, deltaManifest, jobId, operatorId, checkpointId - 1)); + + // Old commits requests shouldn't affect the result + dynamicCommitter.commit(Lists.newArrayList(oldCommitRequest)); + + table1.refresh(); + assertThat(table1.snapshots()).hasSize(1); + Snapshot first = Iterables.getFirst(table1.snapshots(), null); + assertThat(first.summary()) + .containsAllEntriesOf( + (Map) + ImmutableMap.builder() + .put("added-data-files", "1") + .put("added-records", "42") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", "" + checkpointId) + .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "0") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "0") + .put("total-records", "42") + .build()); + } + } From e8b1c98c637f0e0de9ee8ca80e2a322a77fe867b Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 11:06:46 +0200 Subject: [PATCH 21/23] fixup --- .../flink/sink/dynamic/TestDynamicCommitter.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 2f742fbe34d6..70de888a7a00 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -246,17 +246,17 @@ void testAlreadyCommitted() throws Exception { new OneInputStreamOperatorTestHarness(aggregator); aggregatorHarness.open(); + final String jobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int checkpointId = 10; + byte[] deltaManifest = aggregator.writeToManifest( writeTarget, Lists.newArrayList( new DynamicWriteResult( writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - 0); - - final String jobId = JobID.generate().toHexString(); - final String operatorId = new OperatorID().toHexString(); - final int checkpointId = 10; + checkpointId); CommitRequest commitRequest = new MockCommitRequest<>( From 000e7dfcf2c97e7a879d542ea0448fca24ec7001 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Wed, 4 Jun 2025 11:06:56 +0200 Subject: [PATCH 22/23] Replace partitions test --- .../sink/dynamic/TestDynamicCommitter.java | 84 +++++++++++++++++++ 1 file changed, 84 insertions(+) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 70de888a7a00..d9129d6eacf6 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -294,4 +294,88 @@ void testAlreadyCommitted() throws Exception { .build()); } + @Test + void testReplacePartitions() throws Exception { + Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table1.snapshots()).isEmpty(); + + // Overwrite mode is active + boolean overwriteMode = true; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter dynamicCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + WriteTarget writeTarget = + new WriteTarget(TABLE1, "branch", 42, 0, false, Lists.newArrayList(1, 2)); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + final String jobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int checkpointId = 10; + + byte[] deltaManifest = + aggregator.writeToManifest( + writeTarget, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), + checkpointId); + + CommitRequest commitRequest = + new MockCommitRequest<>( + new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + + dynamicCommitter.commit(Lists.newArrayList(commitRequest)); + + byte[] overwriteManifest = + aggregator.writeToManifest( + writeTarget, + Lists.newArrayList( + new DynamicWriteResult( + writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), + checkpointId + 1); + + CommitRequest overwriteRequest = + new MockCommitRequest<>( + new DynamicCommittable( + writeTarget, overwriteManifest, jobId, operatorId, checkpointId + 1)); + + dynamicCommitter.commit(Lists.newArrayList(overwriteRequest)); + + table1.refresh(); + assertThat(table1.snapshots()).hasSize(2); + Snapshot latestSnapshot = Iterables.getLast(table1.snapshots()); + assertThat(latestSnapshot.summary()) + .containsAllEntriesOf( + (Map) + ImmutableMap.builder() + .put("replace-partitions", "true") + .put("added-data-files", "1") + .put("added-records", "42") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", String.valueOf(checkpointId + 1)) + .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "0") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "0") + .put("total-records", "42") + .build()); + } } From d97b88f4bff84381d84035e131fb784881a67c4a Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 4 Jun 2025 13:42:44 +0200 Subject: [PATCH 23/23] Correct use of @Internal and make public classes package-private --- .../iceberg/flink/sink/dynamic/DynamicCommittable.java | 2 -- .../flink/sink/dynamic/DynamicCommittableSerializer.java | 2 -- .../iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java | 6 ++---- .../iceberg/flink/sink/dynamic/DynamicWriteResult.java | 2 -- .../flink/sink/dynamic/DynamicWriteResultAggregator.java | 2 -- .../flink/sink/dynamic/DynamicWriteResultSerializer.java | 2 -- .../apache/iceberg/flink/sink/dynamic/DynamicWriter.java | 2 -- .../iceberg/flink/sink/dynamic/DynamicWriterMetrics.java | 6 ++---- .../org/apache/iceberg/flink/sink/dynamic/WriteTarget.java | 2 -- .../sink/dynamic/TestDynamicWriteResultAggregator.java | 2 +- .../sink/dynamic/TestDynamicWriteResultSerializer.java | 2 +- .../flink/sink/dynamic/TestTableSerializerCache.java | 2 +- 12 files changed, 7 insertions(+), 25 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index 99906c0c871e..33edefe71eb0 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -21,7 +21,6 @@ import java.io.Serializable; import java.util.Arrays; import java.util.Objects; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.flink.sink.DeltaManifests; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -33,7 +32,6 @@ *

      {@link DynamicCommittableSerializer} is used to serialize {@link DynamicCommittable} between * the {@link DynamicWriter} and the {@link DynamicWriteResultAggregator}. */ -@Internal class DynamicCommittable implements Serializable { private final WriteTarget key; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java index 3284d6bd7251..4aadcf1f3620 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -20,7 +20,6 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; @@ -30,7 +29,6 @@ * DynamicWriter} and the {@link DynamicWriteResultAggregator} operator and for sending it down to * the {@link DynamicCommitter}. */ -@Internal class DynamicCommittableSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java index c7f6225eb2db..d34feea75285 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java @@ -19,19 +19,17 @@ package org.apache.iceberg.flink.sink.dynamic; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.flink.sink.CommitSummary; import org.apache.iceberg.flink.sink.IcebergFilesCommitterMetrics; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -@Internal -public class DynamicCommitterMetrics { +class DynamicCommitterMetrics { private final Map metrics; private final MetricGroup mainMetricsGroup; - public DynamicCommitterMetrics(MetricGroup mainMetricsGroup) { + DynamicCommitterMetrics(MetricGroup mainMetricsGroup) { this.mainMetricsGroup = mainMetricsGroup; this.metrics = Maps.newHashMap(); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java index 379c2aff3a78..85806f932ad5 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java @@ -18,10 +18,8 @@ */ package org.apache.iceberg.flink.sink.dynamic; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.io.WriteResult; -@Internal class DynamicWriteResult { private final WriteTarget key; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index 474eecc9e829..58ba183dfcd4 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -24,7 +24,6 @@ import java.time.Duration; import java.util.Collection; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.SimpleVersionedSerialization; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; @@ -53,7 +52,6 @@ * DynamicCommittable} per checkpoint (storing the serialized {@link DeltaManifests}, jobId, * operatorId, checkpointId) */ -@Internal class DynamicWriteResultAggregator extends AbstractStreamOperator> implements OneInputStreamOperator< diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java index 6635990ca18b..cf5f423fd7ff 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java @@ -20,14 +20,12 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.iceberg.flink.sink.WriteResultSerializer; import org.apache.iceberg.io.WriteResult; -@Internal class DynamicWriteResultSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index 371352f28d03..3851dbf95603 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.CommittingSinkWriter; import org.apache.flink.api.connector.sink2.SinkWriter; @@ -53,7 +52,6 @@ * DynamicIcebergSink. Writes out the data to the final place, and emits {@link DynamicWriteResult} * for every unique {@link WriteTarget} at checkpoint time. */ -@Internal class DynamicWriter implements CommittingSinkWriter { private static final Logger LOG = LoggerFactory.getLogger(DynamicWriter.class); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java index 404ffaabdd71..2e1f82df9d2d 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -19,19 +19,17 @@ package org.apache.iceberg.flink.sink.dynamic; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.flink.sink.IcebergStreamWriterMetrics; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -@Internal -public class DynamicWriterMetrics { +class DynamicWriterMetrics { private final Map metrics; private final MetricGroup mainMetricsGroup; - public DynamicWriterMetrics(MetricGroup mainMetricsGroup) { + DynamicWriterMetrics(MetricGroup mainMetricsGroup) { this.mainMetricsGroup = mainMetricsGroup; this.metrics = Maps.newHashMap(); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index 5fdda0aeec27..0a43404d13ad 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -22,13 +22,11 @@ import java.io.Serializable; import java.util.List; import java.util.Objects; -import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -@Internal class WriteTarget implements Serializable { private final String tableName; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java index 250b233a5ed4..137b87bb171d 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -public class TestDynamicWriteResultAggregator { +class TestDynamicWriteResultAggregator { @RegisterExtension static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java index a44827f3cfbf..bede5d42b9f4 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java @@ -32,7 +32,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.Test; -public class TestDynamicWriteResultSerializer { +class TestDynamicWriteResultSerializer { private static final DataFile DATA_FILE = DataFiles.builder(PartitionSpec.unpartitioned()) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java index 720a0a582e94..70a512e7c7da 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java @@ -38,7 +38,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -public class TestTableSerializerCache { +class TestTableSerializerCache { @RegisterExtension static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table");