From 473177620e2d3f7efa5972e7634efb3902ae22b9 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Thu, 27 Feb 2025 16:10:09 +0100 Subject: [PATCH 1/6] Dynamic Sink Iceberg core changes --- .../org/apache/iceberg/UnboundPartitionSpec.java | 12 ++++++------ .../apache/iceberg/expressions/NamedReference.java | 2 +- .../apache/iceberg/expressions/UnboundTransform.java | 2 +- .../main/java/org/apache/iceberg/SchemaUpdate.java | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java b/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java index cc8526f9072c..b7630110d772 100644 --- a/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java @@ -71,11 +71,11 @@ private PartitionSpec.Builder copyToBuilder(Schema schema) { return builder; } - static Builder builder() { + public static Builder builder() { return new Builder(); } - static class Builder { + public static class Builder { private final List fields; private int specId = 0; @@ -83,12 +83,12 @@ private Builder() { this.fields = Lists.newArrayList(); } - Builder withSpecId(int newSpecId) { + public Builder withSpecId(int newSpecId) { this.specId = newSpecId; return this; } - Builder addField(String transformAsString, int sourceId, int partitionId, String name) { + public Builder addField(String transformAsString, int sourceId, int partitionId, String name) { fields.add(new UnboundPartitionField(transformAsString, sourceId, partitionId, name)); return this; } @@ -98,12 +98,12 @@ Builder addField(String transformAsString, int sourceId, String name) { return this; } - UnboundPartitionSpec build() { + public UnboundPartitionSpec build() { return new UnboundPartitionSpec(specId, fields); } } - static class UnboundPartitionField { + public static class UnboundPartitionField { private final Transform transform; private final int sourceId; private final Integer partitionId; diff --git a/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java b/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java index cc5ba3ceaf4f..a05370f347dc 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java @@ -26,7 +26,7 @@ public class NamedReference implements UnboundTerm, Reference { private final String name; - NamedReference(String name) { + public NamedReference(String name) { Preconditions.checkNotNull(name, "Name cannot be null"); this.name = name; } diff --git a/api/src/main/java/org/apache/iceberg/expressions/UnboundTransform.java b/api/src/main/java/org/apache/iceberg/expressions/UnboundTransform.java index cae84733c8d5..05f52cfff31e 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/UnboundTransform.java +++ b/api/src/main/java/org/apache/iceberg/expressions/UnboundTransform.java @@ -26,7 +26,7 @@ public class UnboundTransform implements UnboundTerm, Term { private final NamedReference ref; private final Transform transform; - UnboundTransform(NamedReference ref, Transform transform) { + public UnboundTransform(NamedReference ref, Transform transform) { this.ref = ref; this.transform = transform; } diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index db02a0e96e10..7acbbdfd22d6 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory; /** Schema evolution API implementation. */ -class SchemaUpdate implements UpdateSchema { +public class SchemaUpdate implements UpdateSchema { private static final Logger LOG = LoggerFactory.getLogger(SchemaUpdate.class); private static final int TABLE_ROOT_ID = -1; @@ -71,7 +71,7 @@ class SchemaUpdate implements UpdateSchema { } /** For testing only. */ - SchemaUpdate(Schema schema, int lastColumnId) { + public SchemaUpdate(Schema schema, int lastColumnId) { this(null, null, schema, lastColumnId); } From 5a16da0ed6060b453e1e5c5c9a1ef33a67c4e6dc Mon Sep 17 00:00:00 2001 From: Max Michels Date: Thu, 27 Feb 2025 16:16:48 +0100 Subject: [PATCH 2/6] Flink 1.19: Dynamic Iceberg Sink Co-authored-by: Peter Vary --- flink/v1.19/build.gradle | 3 + ...RecordSerializerDeserializerBenchmark.java | 137 ++++ .../apache/iceberg/flink/FlinkConfParser.java | 7 + .../apache/iceberg/flink/FlinkWriteConf.java | 4 + .../flink/sink/BucketPartitionerUtil.java | 2 +- .../iceberg/flink/sink/CommitSummary.java | 50 +- .../iceberg/flink/sink/DeltaManifests.java | 4 +- .../flink/sink/DeltaManifestsSerializer.java | 4 +- .../flink/sink/EqualityFieldKeySelector.java | 10 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 10 +- .../flink/sink/IcebergCommittable.java | 2 +- .../sink/IcebergCommittableSerializer.java | 2 +- .../iceberg/flink/sink/IcebergCommitter.java | 2 +- .../flink/sink/IcebergFilesCommitter.java | 2 +- .../sink/IcebergFilesCommitterMetrics.java | 10 +- .../iceberg/flink/sink/IcebergSink.java | 2 +- .../sink/IcebergStreamWriterMetrics.java | 8 +- .../flink/sink/IcebergWriteAggregator.java | 3 +- .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../flink/sink/NonThrowingKeySelector.java | 28 + .../flink/sink/PartitionKeySelector.java | 11 +- .../flink/sink/RowDataTaskWriterFactory.java | 27 +- .../flink/sink/WriteResultSerializer.java | 2 +- .../sink/dynamic/CompareSchemasVisitor.java | 254 ++++++++ .../sink/dynamic/DynamicCommittable.java | 80 +++ .../dynamic/DynamicCommittableSerializer.java | 73 +++ .../flink/sink/dynamic/DynamicCommitter.java | 417 ++++++++++++ .../sink/dynamic/DynamicCommitterMetrics.java | 51 ++ .../sink/dynamic/DynamicIcebergSink.java | 439 +++++++++++++ .../sink/dynamic/DynamicKeySelector.java | 384 +++++++++++ .../flink/sink/dynamic/DynamicRecord.java | 128 ++++ .../sink/dynamic/DynamicRecordConverter.java | 31 + .../sink/dynamic/DynamicRecordInternal.java | 164 +++++ .../DynamicRecordInternalSerializer.java | 299 +++++++++ .../dynamic/DynamicRecordInternalType.java | 102 +++ .../sink/dynamic/DynamicRecordProcessor.java | 196 ++++++ .../dynamic/DynamicTableUpdateOperator.java | 71 ++ .../sink/dynamic/DynamicWriteResult.java | 41 ++ .../dynamic/DynamicWriteResultAggregator.java | 190 ++++++ .../dynamic/DynamicWriteResultSerializer.java | 63 ++ .../flink/sink/dynamic/DynamicWriter.java | 218 +++++++ .../sink/dynamic/DynamicWriterMetrics.java | 50 ++ .../sink/dynamic/EvolveSchemaVisitor.java | 184 ++++++ .../sink/dynamic/PartitionSpecAdjustment.java | 45 ++ .../sink/dynamic/PartitionSpecEvolver.java | 119 ++++ .../flink/sink/dynamic/RowDataEvolver.java | 169 +++++ .../sink/dynamic/RowDataSerializerCache.java | 102 +++ .../flink/sink/dynamic/TableDataCache.java | 257 ++++++++ .../flink/sink/dynamic/TableUpdater.java | 208 ++++++ .../flink/sink/dynamic/WriteTarget.java | 144 +++++ .../apache/iceberg/flink/SimpleDataUtil.java | 22 +- .../org/apache/iceberg/flink/TestHelpers.java | 2 +- .../flink/sink/TestFlinkIcebergSinkBase.java | 5 +- ...namicRecordInternalSerializerTestBase.java | 75 +++ ...rdInternalSerializerWriteSchemaIdTest.java | 41 ++ ...cordInternalSerializerWriteSchemaTest.java | 34 + .../dynamic/TestCompareSchemasVisitor.java | 209 ++++++ .../sink/dynamic/TestDynamicIcebergSink.java | 591 +++++++++++++++++ .../dynamic/TestDynamicIcebergSinkPerf.java | 226 +++++++ .../flink/sink/dynamic/TestDynamicWriter.java | 106 +++ .../sink/dynamic/TestEvolveSchemaVisitor.java | 607 ++++++++++++++++++ .../dynamic/TestPartitionSpecAdjustment.java | 57 ++ .../sink/dynamic/TestRowDataEvolver.java | 248 +++++++ .../sink/dynamic/TestTableDataCache.java | 85 +++ .../flink/sink/dynamic/TestTableUpdater.java | 90 +++ 65 files changed, 7144 insertions(+), 67 deletions(-) create mode 100644 flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 599ba085e4c4..7723d5fc6712 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -68,6 +68,9 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation libs.datasketches + // for caching in DynamicSink + implementation libs.caffeine + testImplementation libs.flink119.connector.test.utils testImplementation libs.flink119.core testImplementation libs.flink119.runtime diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java new file mode 100644 index 000000000000..2d102449b5b3 --- /dev/null +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java @@ -0,0 +1,137 @@ +/* + * 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.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class DynamicRecordSerializerDeserializerBenchmark { + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private List rows = Lists.newArrayListWithExpectedSize(SAMPLE_SIZE); + private DynamicRecordInternalType type; + + public static void main(String[] args) throws RunnerException { + Options options = + new OptionsBuilder() + .include(DynamicRecordSerializerDeserializerBenchmark.class.getSimpleName()) + .build(); + new Runner(options).run(); + } + + @Setup + public void setupBenchmark() throws IOException { + List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); + this.rows = + records.stream() + .map( + r -> + new DynamicRecordInternal( + "t", + "main", + SCHEMA, + PartitionSpec.unpartitioned(), + 1, + RowDataConverter.convert(SCHEMA, r), + false, + List.of())) + .collect(Collectors.toList()); + + File warehouse = Files.createTempFile("perf-bench", null).toFile(); + CatalogLoader catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getPath())); + this.type = new DynamicRecordInternalType(catalogLoader, true, 100); + } + + @Benchmark + @Threads(1) + public void testSerialize(Blackhole blackhole) throws IOException { + TypeSerializer serializer = + type.createSerializer((SerializerConfig) null); + DataOutputSerializer outputView = new DataOutputSerializer(1024); + for (int i = 0; i < SAMPLE_SIZE; ++i) { + serializer.serialize(rows.get(i), outputView); + } + } + + @Benchmark + @Threads(1) + public void testSerializeAndDeserialize(Blackhole blackhole) throws IOException { + TypeSerializer serializer = + type.createSerializer((SerializerConfig) null); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + for (int i = 0; i < SAMPLE_SIZE; ++i) { + serializer.serialize(rows.get(i), outputView); + serializer.deserialize(new DataInputDeserializer(outputView.getSharedBuffer())); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index d5eea6706b39..b9b15e4d730e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -27,6 +27,7 @@ import org.apache.flink.util.TimeUtils; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; class FlinkConfParser { @@ -41,6 +42,12 @@ class FlinkConfParser { this.readableConfig = readableConfig; } + FlinkConfParser(Map options, ReadableConfig readableConfig) { + this.tableProperties = ImmutableMap.of(); + this.options = options; + this.readableConfig = readableConfig; + } + public BooleanConfParser booleanConf() { return new BooleanConfParser(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index a31902d49a8b..d72c05e2ef1f 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -55,6 +55,10 @@ public FlinkWriteConf( this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); } + public FlinkWriteConf(Map writeOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(writeOptions, readableConfig); + } + public boolean overwriteMode() { return confParser .booleanConf() diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index c33207728d3e..22dd40a61b9d 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -26,7 +26,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.PartitionSpecVisitor; -final class BucketPartitionerUtil { +public final class BucketPartitionerUtil { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = "Invalid number of buckets: %s (must be 1)"; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 2109c91bddf7..62de3170d55c 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index 036970c06d5b..89c26692b113 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java index 92ca284b12ba..59de599a6886 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 18b269d6c3e9..5b971415a9d9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.sink; import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.Schema; @@ -30,10 +29,10 @@ import org.apache.iceberg.util.StructProjection; /** - * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record - * will be emitted to same writer in order. + * Create a {@link NonThrowingKeySelector} to shuffle by equality fields, to ensure same equality + * fields record will be emitted to same writer in order. */ -class EqualityFieldKeySelector implements KeySelector { +public class EqualityFieldKeySelector implements NonThrowingKeySelector { private final Schema schema; private final RowType flinkSchema; @@ -43,7 +42,8 @@ class EqualityFieldKeySelector implements KeySelector { private transient StructProjection structProjection; private transient StructLikeWrapper structLikeWrapper; - EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { + public EqualityFieldKeySelector( + Schema schema, RowType flinkSchema, List equalityFieldIds) { this.schema = schema; this.flinkSchema = flinkSchema; this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index 9571efdc5268..d107c2739b04 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 408c3e9a9d5f..8b06949a5519 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java index e2b388a83c75..1d83c211e001 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java index 2245b36f15a7..d7c35aa4f708 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -57,7 +57,7 @@ * same jobId-operatorId-checkpointId triplet * */ -class IcebergCommitter implements Committer { +public class IcebergCommitter implements Committer { private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; public static final WriteResult EMPTY_WRITE_RESULT = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 609deb621f43..9a50b7a900c5 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -76,7 +76,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // the max committed one to iceberg table, for avoiding committing the same data files twice. This // id will be attached to iceberg's meta when committing the iceberg transaction. private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + public static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; // TableLoader to load iceberg table lazily. private final TableLoader tableLoader; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 5b28c4acb1c5..86af9c5154f7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index d080169544cd..c88de85a25ed 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -571,7 +571,7 @@ private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table return (SerializableTable) SerializableTable.copyOf(table); } - private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + public static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { if (requestedSchema != null) { // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing // iceberg schema. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ab458ad2e7cb..04ea868c0a38 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java index 794ade577976..2b3c53b0e31a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -40,7 +40,8 @@ * IcebergCommittable} per checkpoint (storing the serialized {@link * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) */ -class IcebergWriteAggregator extends AbstractStreamOperator> +public class IcebergWriteAggregator + extends AbstractStreamOperator> implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 30517cd38216..81434ad171fb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java new file mode 100644 index 000000000000..a9953ea8bd01 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import org.apache.flink.api.java.functions.KeySelector; + +/** A non-throwing variant of Flink's {@link KeySelector}. */ +public interface NonThrowingKeySelector extends KeySelector { + + @Override + K getKey(I value); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java index df951684b446..f10dc710200d 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.PartitionKey; @@ -27,11 +26,11 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be - * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy - * for {@link FlinkSink}. + * Create a {@link NonThrowingKeySelector} to shuffle by partition key, then each partition/bucket + * will be wrote by only one task. That will reduce lots of small files in partitioned fanout write + * policy for {@link FlinkSink}. */ -class PartitionKeySelector implements KeySelector { +public class PartitionKeySelector implements NonThrowingKeySelector { private final Schema schema; private final PartitionKey partitionKey; @@ -39,7 +38,7 @@ class PartitionKeySelector implements KeySelector { private transient RowDataWrapper rowDataWrapper; - PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + public PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { this.schema = schema; this.partitionKey = new PartitionKey(spec, schema); this.flinkSchema = flinkSchema; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 67422a1afeb1..8dc8d38869bc 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java index 5a44373cccaa..34868458b0e0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java new file mode 100644 index 000000000000..99abc95d7102 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java @@ -0,0 +1,254 @@ +/* + * 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.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.schema.SchemaWithPartnerVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** Visitor class which compares two schemas and decides whether they are compatible. */ +public class CompareSchemasVisitor + extends SchemaWithPartnerVisitor { + + private final Schema tableSchema; + + private CompareSchemasVisitor(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + public static Result visit(Schema dataSchema, Schema tableSchema) { + return visit(dataSchema, tableSchema, true); + } + + public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + return visit( + dataSchema, + -1, + new CompareSchemasVisitor(tableSchema), + new PartnerIdByNameAccessors(tableSchema, caseSensitive)); + } + + @Override + public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return downstream; + } + + @Override + public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + Result result = fields.stream().reduce(Result::merge).orElse(Result.INCOMPATIBLE); + + if (result == Result.INCOMPATIBLE) { + return Result.INCOMPATIBLE; + } + + Type tableSchemaType = + tableSchemaId == -1 ? tableSchema.asStruct() : tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isStructType()) { + return Result.INCOMPATIBLE; + } + + if (struct.fields().size() != tableSchemaType.asStructType().fields().size()) { + return Result.CONVERSION_NEEDED; + } + + for (int i = 0; i < struct.fields().size(); ++i) { + if (!struct + .fields() + .get(i) + .name() + .equals(tableSchemaType.asStructType().fields().get(i).name())) { + return Result.CONVERSION_NEEDED; + } + } + + return result; + } + + @Override + public Result field(Types.NestedField field, Integer tableSchemaId, Result typeResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + if (typeResult != Result.SAME) { + return typeResult; + } + + if (tableSchema.findField(tableSchemaId).isRequired() && field.isOptional()) { + return Result.INCOMPATIBLE; + } else { + return Result.SAME; + } + } + + @Override + public Result list(Types.ListType list, Integer tableSchemaId, Result elementsResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return elementsResult; + } + + @Override + public Result map( + Types.MapType map, Integer tableSchemaId, Result keyResult, Result valueResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return keyResult.merge(valueResult); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Result primitive(Type.PrimitiveType primitive, Integer tableSchemaId) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + Type tableSchemaType = tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isPrimitiveType()) { + return Result.INCOMPATIBLE; + } + + Type.PrimitiveType tableSchemaPrimitiveType = tableSchemaType.asPrimitiveType(); + if (primitive.equals(tableSchemaPrimitiveType)) { + return Result.SAME; + } else if (primitive.equals(Types.IntegerType.get()) + && tableSchemaPrimitiveType.equals(Types.LongType.get())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.equals(Types.FloatType.get()) + && tableSchemaPrimitiveType.equals(Types.DoubleType.get())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.equals(Types.DateType.get()) + && tableSchemaPrimitiveType.equals(Types.TimestampType.withoutZone())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.typeId() == Type.TypeID.DECIMAL + && tableSchemaPrimitiveType.typeId() == Type.TypeID.DECIMAL) { + Types.DecimalType dataType = (Types.DecimalType) primitive; + Types.DecimalType tableType = (Types.DecimalType) tableSchemaPrimitiveType; + return dataType.scale() == tableType.scale() && dataType.precision() < tableType.precision() + ? Result.CONVERSION_NEEDED + : Result.INCOMPATIBLE; + } else { + return Result.INCOMPATIBLE; + } + } + + static class PartnerIdByNameAccessors implements PartnerAccessors { + private final Schema tableSchema; + private boolean caseSensitive = true; + + PartnerIdByNameAccessors(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + private PartnerIdByNameAccessors(Schema tableSchema, boolean caseSensitive) { + this(tableSchema); + this.caseSensitive = caseSensitive; + } + + @Override + public Integer fieldPartner(Integer tableSchemaFieldId, int fieldId, String name) { + Types.StructType struct; + if (tableSchemaFieldId == -1) { + struct = tableSchema.asStruct(); + } else { + struct = tableSchema.findField(tableSchemaFieldId).type().asStructType(); + } + + Types.NestedField field = + caseSensitive ? struct.field(name) : struct.caseInsensitiveField(name); + if (field != null) { + return field.fieldId(); + } + + return null; + } + + @Override + public Integer mapKeyPartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(0).fieldId(); + } + + return null; + } + + @Override + public Integer mapValuePartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(1).fieldId(); + } + + return null; + } + + @Override + public Integer listElementPartner(Integer tableSchemaListId) { + Types.NestedField listField = tableSchema.findField(tableSchemaListId); + if (listField != null) { + return listField.type().asListType().fields().get(0).fieldId(); + } + + return null; + } + } + + public enum Result { + SAME(0), + CONVERSION_NEEDED(1), + INCOMPATIBLE(2); + + private static final Map BY_ID = Maps.newHashMap(); + + static { + for (Result e : Result.values()) { + if (BY_ID.put(e.id, e) != null) { + throw new IllegalArgumentException("Duplicate id: " + e.id); + } + } + } + + private final int id; + + Result(int id) { + this.id = id; + } + + private Result merge(Result other) { + return BY_ID.get(Math.max(this.id, other.id)); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java new file mode 100644 index 000000000000..5f26d3406cd1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -0,0 +1,80 @@ +/* + * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java new file mode 100644 index 000000000000..c885e048127f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -0,0 +1,73 @@ +/* + * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java new file mode 100644 index 000000000000..205b74384710 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -0,0 +1,417 @@ +/* + * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java new file mode 100644 index 000000000000..8d815d6f6923 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.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.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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java new file mode 100644 index 000000000000..dd0442bc21ea --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java @@ -0,0 +1,439 @@ +/* + * 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.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; + +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** + * Dynamic version of the IcebergSink which supports: + * + *
    + *
  1. Writing to any number of tables (No more 1:1 sink/topic relationship). + *
  2. Creating and updating tables based on the user-supplied routing. + *
  3. Updating the schema and partition spec of tables based on the user-supplied specification. + *
+ */ +@Experimental +public class DynamicIcebergSink + implements Sink, + SupportsPreWriteTopology, + SupportsCommitter, + SupportsPreCommitTopology, + SupportsPostCommitTopology { + + private final CatalogLoader catalogLoader; + private final Map snapshotProperties; + private final String uidPrefix; + private final String sinkId; + private final Map writeProperties; + private final transient FlinkWriteConf flinkWriteConf; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final boolean overwriteMode; + private final int workerPoolSize; + + private DynamicIcebergSink( + CatalogLoader catalogLoader, + Map snapshotProperties, + String uidPrefix, + Map writeProperties, + FlinkWriteConf flinkWriteConf) { + this.catalogLoader = catalogLoader; + this.snapshotProperties = snapshotProperties; + this.uidPrefix = uidPrefix; + this.writeProperties = writeProperties; + this.flinkWriteConf = flinkWriteConf; + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.overwriteMode = flinkWriteConf.overwriteMode(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter createWriter(InitContext context) { + return new DynamicWriter( + catalogLoader.loadCatalog(), + dataFileFormat, + targetDataFileSize, + writeProperties, + new DynamicWriterMetrics(context.metricGroup()), + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); + } + + @Override + public Committer createCommitter(CommitterInitContext context) { + DynamicCommitterMetrics metrics = new DynamicCommitterMetrics(context.metricGroup()); + return new DynamicCommitter( + catalogLoader.loadCatalog(), + snapshotProperties, + overwriteMode, + workerPoolSize, + sinkId, + metrics); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new DynamicCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream> committables) {} + + @Override + public DataStream addPreWriteTopology( + DataStream inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream> addPreCommitTopology( + DataStream> writeResults) { + TypeInformation> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + return writeResults + .keyBy( + committable -> { + if (committable instanceof CommittableSummary) { + return "__summary"; + } else { + CommittableWithLineage result = + (CommittableWithLineage) committable; + return result.getCommittable().key().tableName(); + } + }) + .transform( + prefixIfNotNull(uidPrefix, sinkId + " Pre Commit"), + typeInformation, + new DynamicWriteResultAggregator(catalogLoader)) + .uid(prefixIfNotNull(uidPrefix, sinkId + "-pre-commit-topology")); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new DynamicWriteResultSerializer(); + } + + public static class Builder { + private DataStream input; + private DynamicRecordConverter converter; + private CatalogLoader catalogLoader; + private String uidPrefix = null; + private final Map writeOptions = Maps.newHashMap(); + private final Map snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private boolean immediateUpdate = false; + private int cacheMaximumSize = 100; + private long cacheRefreshMs = 1_000; + + private Builder() {} + + public Builder forInput(DataStream inputStream) { + this.input = inputStream; + return this; + } + + public Builder withConverter(DynamicRecordConverter inputConverter) { + this.converter = inputConverter; + return this; + } + + /** + * The catalog loader is used for loading tables in {@link DynamicCommitter} lazily, we need + * this loader because {@link Table} is not serializable and could not just use the loaded table + * from Builder#table in the remote task manager. + * + * @param newCatalogLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder catalogLoader(CatalogLoader newCatalogLoader) { + this.catalogLoader = newCatalogLoader; + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link DynamicIcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * Set the uid prefix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator) Actual operator uid will be appended + * with a suffix like "uidPrefix-writer". + * + *

If provided, this prefix is also applied to operator names. + * + *

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

Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newPrefix prefix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + + public Builder snapshotProperties(Map properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + public Builder immediateTableUpdate(boolean newImmediateUpdate) { + this.immediateUpdate = newImmediateUpdate; + return this; + } + + /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ + public Builder cacheMaxSize(int maxSize) { + this.cacheMaximumSize = maxSize; + return this; + } + + /** Maximum interval for cache items renewals. */ + public Builder cacheRefreshMs(long refreshMs) { + this.cacheRefreshMs = refreshMs; + return this; + } + + private String operatorName(String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + public DynamicIcebergSink build() { + + Preconditions.checkArgument( + converter != null, "Please use withConverter() to convert the input DataStream."); + Preconditions.checkNotNull(catalogLoader, "Catalog loader shouldn't be null"); + + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(writeOptions, readableConfig); + + Map writeProperties = + writeProperties(flinkWriteConf.dataFileFormat(), flinkWriteConf); + + uidPrefix = Optional.ofNullable(uidPrefix).orElse(""); + + // FlinkWriteConf properties needed to be set separately, so we do not have to serialize the + // full conf + return new DynamicIcebergSink( + catalogLoader, snapshotSummary, uidPrefix, writeProperties, flinkWriteConf); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + DynamicRecordInternalType type = + new DynamicRecordInternalType(catalogLoader, false, cacheMaximumSize); + DynamicIcebergSink sink = build(); + SingleOutputStreamOperator converted = + input + .process( + new DynamicRecordProcessor<>( + converter, catalogLoader, immediateUpdate, cacheMaximumSize, cacheRefreshMs)) + .uid(prefixIfNotNull(uidPrefix, "-converter")) + .name(operatorName("Converter")) + .returns(type); + + DataStreamSink rowDataDataStreamSink = + converted + .getSideOutput( + new OutputTag<>( + DynamicRecordProcessor.DYNAMIC_TABLE_UPDATE_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize))) + .keyBy((KeySelector) DynamicRecordInternal::tableName) + .map(new DynamicTableUpdateOperator(catalogLoader, cacheMaximumSize, cacheRefreshMs)) + .uid(prefixIfNotNull(uidPrefix, "-updater")) + .name(operatorName("Updater")) + .returns(type) + .union(converted) + .sinkTo(sink) + .uid(prefixIfNotNull(uidPrefix, "-sink")); + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + + return rowDataDataStreamSink; + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties(FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + DataStream distributeDataStream(DataStream input) { + return input.keyBy(DynamicRecordInternal::writerKey); + } + + private static String prefixIfNotNull(String uidPrefix, String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + /** + * Initialize a {@link IcebergSink.Builder} to export the data from input data stream with {@link + * RowData}s into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public static Builder forInput(DataStream input) { + return new Builder().forInput(input); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java new file mode 100644 index 000000000000..9a79be6838d2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java @@ -0,0 +1,384 @@ +/* + * 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.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.EqualityFieldKeySelector; +import org.apache.iceberg.flink.sink.NonThrowingKeySelector; +import org.apache.iceberg.flink.sink.PartitionKeySelector; +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.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class DynamicKeySelector implements NonThrowingKeySelector { + private static final Logger LOG = LoggerFactory.getLogger(DynamicKeySelector.class); + + private final int maxWriteParallelism; + private final Cache> keySelectorCache; + + DynamicKeySelector(int maxCacheSize, int maxWriteParallelism) { + this.maxWriteParallelism = maxWriteParallelism; + this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); + } + + @Override + public Integer getKey(Input input) { + SelectorKey cacheKey = new SelectorKey(input); + return keySelectorCache + .get( + cacheKey, + k -> + getKeySelector( + input.tableName, + input.schema, + input.spec, + input.mode, + input.equalityFields, + input.writeParallelism)) + .getKey(input.rowData); + } + + public NonThrowingKeySelector getKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + DistributionMode mode, + List equalityFields, + int writeParallelism) { + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFields.isEmpty()) { + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + case HASH: + if (equalityFields.isEmpty()) { + if (spec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFields.contains(partitionField.name()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + schema.columns().stream() + .filter(c -> equalityFields.contains(c.name())) + .collect(Collectors.toList())); + } + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } + + case RANGE: + if (schema.identifierFieldIds().isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + default: + throw new IllegalArgumentException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + private static NonThrowingKeySelector equalityFieldKeySelector( + String tableName, + Schema schema, + List equalityFields, + int writeParallelism, + int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new EqualityFieldKeySelector( + schema, + FlinkSchemaUtil.convert(schema), + DynamicRecordProcessor.getEqualityFieldIds(equalityFields, schema)), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector partitionKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + int writeParallelism, + int maxWriteParallelism) { + NonThrowingKeySelector inner = + new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); + return new TargetLimitedKeySelector( + in -> inner.getKey(in).hashCode(), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector tableKeySelector( + String tableName, int writeParallelism, int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new RoundRobinKeySelector<>(writeParallelism), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + /** + * Generates a new key using the salt as a base, and reduces the target key range of the {@link + * #wrapped} {@link NonThrowingKeySelector} to {@link #writeParallelism}. + */ + private static class TargetLimitedKeySelector + implements NonThrowingKeySelector { + private final NonThrowingKeySelector wrapped; + private final int writeParallelism; + private final int[] distinctKeys; + + @SuppressWarnings("checkstyle:ParameterAssignment") + TargetLimitedKeySelector( + NonThrowingKeySelector wrapped, + int salt, + int writeParallelism, + int maxWriteParallelism) { + if (writeParallelism > maxWriteParallelism) { + LOG.warn( + "writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", + writeParallelism, + maxWriteParallelism, + maxWriteParallelism); + writeParallelism = maxWriteParallelism; + } + this.wrapped = wrapped; + this.writeParallelism = writeParallelism; + this.distinctKeys = new int[writeParallelism]; + + // Ensures that the generated keys are always result in unique slotId + Set targetSlots = Sets.newHashSetWithExpectedSize(writeParallelism); + int nextKey = salt; + for (int i = 0; i < writeParallelism; ++i) { + int subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); + while (targetSlots.contains(subtaskId)) { + ++nextKey; + subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); + } + + targetSlots.add(subtaskId); + distinctKeys[i] = nextKey; + ++nextKey; + } + } + + @Override + public Integer getKey(RowData value) { + return distinctKeys[Math.abs(wrapped.getKey(value).hashCode()) % writeParallelism]; + } + + private static int subtaskId(int key, int writeParallelism, int maxWriteParallelism) { + return KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxWriteParallelism, + writeParallelism, + KeyGroupRangeAssignment.computeKeyGroupForKeyHash(key, maxWriteParallelism)); + } + } + + /** + * Generates evenly distributed keys between [0..{@link #maxTarget}) range using round-robin + * algorithm. + * + * @param unused input for key generation + */ + private static class RoundRobinKeySelector implements NonThrowingKeySelector { + private final int maxTarget; + private int lastTarget = 0; + + RoundRobinKeySelector(int maxTarget) { + this.maxTarget = maxTarget; + } + + @Override + public Integer getKey(T value) { + lastTarget = (lastTarget + 1) % maxTarget; + return lastTarget; + } + } + + static class Input { + private final String tableName; + private final String branch; + private final Integer schemaId; + private final Integer specId; + private final Schema schema; + private final PartitionSpec spec; + private final DistributionMode mode; + private final int writeParallelism; + private final List equalityFields; + private final RowData rowData; + + Input( + DynamicRecord dynamicRecord, + Schema schemaOverride, + PartitionSpec specOverride, + RowData rowDataOverride) { + this( + dynamicRecord.tableIdentifier().toString(), + dynamicRecord.branch(), + schemaOverride != null ? schemaOverride.schemaId() : null, + specOverride != null ? specOverride.specId() : null, + schemaOverride, + specOverride, + dynamicRecord.mode(), + dynamicRecord.writeParallelism(), + dynamicRecord.equalityFields() != null + ? dynamicRecord.equalityFields() + : Collections.emptyList(), + rowDataOverride); + } + + private Input( + String tableName, + String branch, + Integer schemaId, + Integer specId, + Schema schema, + PartitionSpec spec, + DistributionMode mode, + int writeParallelism, + List equalityFields, + RowData rowData) { + this.tableName = tableName; + this.branch = branch; + this.schemaId = schemaId; + this.specId = specId; + this.schema = schema; + this.spec = spec; + this.mode = mode; + this.writeParallelism = writeParallelism; + this.equalityFields = equalityFields; + this.rowData = rowData; + } + } + + /** + * Cache key for the {@link NonThrowingKeySelector}. Only contains the {@link Schema} and the + * {@link PartitionSpec} if the ids are not available. + */ + private static class SelectorKey { + private final String tableName; + private final String branch; + private final Integer schemaId; + private final Integer specId; + private final Schema schema; + private final PartitionSpec spec; + private final List equalityFields; + + private SelectorKey(Input input) { + this.tableName = input.tableName; + this.branch = input.branch; + this.schemaId = input.schemaId; + this.schema = schemaId == null ? input.schema : null; + this.specId = input.specId; + this.spec = specId == null ? input.spec : null; + this.equalityFields = input.equalityFields; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + SelectorKey that = (SelectorKey) other; + return Objects.equals(tableName, that.tableName) + && Objects.equals(branch, that.branch) + && Objects.equals(schemaId, that.schemaId) + && Objects.equals(specId, that.specId) + && Objects.equals(schema, that.schema) + && Objects.equals(spec, that.spec) + && Objects.equals(equalityFields, that.equalityFields); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, branch, schemaId, specId, schema, spec, equalityFields); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", tableName) + .add("branch", branch) + .add("schemaId", schemaId) + .add("specId", specId) + .add("schema", schema) + .add("spec", spec) + .add("eqalityFields", equalityFields) + .toString(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java new file mode 100644 index 000000000000..193d9bfc7f66 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -0,0 +1,128 @@ +/* + * 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.List; +import javax.annotation.Nullable; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; + +public class DynamicRecord { + private TableIdentifier tableIdentifier; + private String branch; + private Schema schema; + private PartitionSpec spec; + private RowData rowData; + private DistributionMode mode; + private int writeParallelism; + private boolean upsertMode; + @Nullable private List equalityFields; + + public DynamicRecord( + TableIdentifier tableIdentifier, + String branch, + Schema schema, + PartitionSpec spec, + RowData rowData, + DistributionMode mode, + int writeParallelism) { + this.tableIdentifier = tableIdentifier; + this.branch = branch; + this.schema = schema; + this.spec = spec; + this.rowData = rowData; + this.mode = mode; + this.writeParallelism = writeParallelism; + } + + public TableIdentifier tableIdentifier() { + return tableIdentifier; + } + + public void setTableIdentifier(TableIdentifier tableIdentifier) { + this.tableIdentifier = tableIdentifier; + } + + public String branch() { + return branch; + } + + public void setBranch(String branch) { + this.branch = branch; + } + + public Schema schema() { + return schema; + } + + public void setSchema(Schema schema) { + this.schema = schema; + } + + public PartitionSpec spec() { + return spec; + } + + public void setSpec(PartitionSpec spec) { + this.spec = spec; + } + + public RowData rowData() { + return rowData; + } + + public void setRowData(RowData rowData) { + this.rowData = rowData; + } + + public DistributionMode mode() { + return mode; + } + + public void setMode(DistributionMode mode) { + this.mode = mode; + } + + public int writeParallelism() { + return writeParallelism; + } + + public void writeParallelism(int parallelism) { + this.writeParallelism = parallelism; + } + + public boolean upsertMode() { + return upsertMode; + } + + public void setUpsertMode(boolean upsertMode) { + this.upsertMode = upsertMode; + } + + public List equalityFields() { + return equalityFields; + } + + public void setEqualityFields(List equalityFields) { + this.equalityFields = equalityFields; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java new file mode 100644 index 000000000000..612b94ffb0cf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java @@ -0,0 +1,31 @@ +/* + * 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.api.common.functions.OpenContext; +import org.apache.flink.util.Collector; + +/** Conversion method to return input type into a DynamicRecord */ +public interface DynamicRecordConverter extends Serializable { + default void open(OpenContext openContext) throws Exception {} + + /** Takes a user-defined input type and converts it one or multiple {@link DynamicRecord}s. */ + void convert(T inputRecord, Collector out) throws Exception; +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java new file mode 100644 index 000000000000..25e61bd20e81 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java @@ -0,0 +1,164 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +@Internal +class DynamicRecordInternal { + private String tableName; + private String branch; + private Schema schema; + private PartitionSpec spec; + private int writerKey; + private RowData rowData; + private boolean upsertMode; + private List equalityFieldIds; + + DynamicRecordInternal() {} + + DynamicRecordInternal( + String tableName, + String branch, + Schema schema, + PartitionSpec spec, + int writerKey, + RowData rowData, + boolean upsertMode, + List equalityFieldsIds) { + this.tableName = tableName; + this.branch = branch; + this.schema = schema; + this.spec = spec; + this.writerKey = writerKey; + this.rowData = rowData; + this.upsertMode = upsertMode; + this.equalityFieldIds = equalityFieldsIds; + } + + public String tableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String branch() { + return branch; + } + + public void setBranch(String branch) { + this.branch = branch; + } + + public Schema schema() { + return schema; + } + + public void setSchema(Schema schema) { + this.schema = schema; + } + + public PartitionSpec spec() { + return spec; + } + + public void setSpec(PartitionSpec spec) { + this.spec = spec; + } + + public int writerKey() { + return writerKey; + } + + public void setWriterKey(int writerKey) { + this.writerKey = writerKey; + } + + public RowData rowData() { + return rowData; + } + + public void setRowData(RowData rowData) { + this.rowData = rowData; + } + + public boolean upsertMode() { + return upsertMode; + } + + public void setUpsertMode(boolean upsertMode) { + this.upsertMode = upsertMode; + } + + public List equalityFields() { + return equalityFieldIds; + } + + public void setEqualityFieldIds(List equalityFieldIds) { + this.equalityFieldIds = equalityFieldIds; + } + + @Override + public int hashCode() { + return Objects.hash( + tableName, branch, schema, spec, writerKey, rowData, upsertMode, equalityFieldIds); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + DynamicRecordInternal that = (DynamicRecordInternal) other; + boolean tableFieldsMatch = + Objects.equals(tableName, that.tableName) + && Objects.equals(branch, that.branch) + && schema.schemaId() == that.schema.schemaId() + && Objects.equals(spec, that.spec) + && writerKey == that.writerKey + && upsertMode == that.upsertMode + && Objects.equals(equalityFieldIds, that.equalityFieldIds); + if (!tableFieldsMatch) { + return false; + } + + if (rowData.getClass().equals(that.rowData.getClass())) { + return Objects.equals(rowData, that.rowData); + } else { + RowDataSerializer rowDataSerializer = new RowDataSerializer(FlinkSchemaUtil.convert(schema)); + return rowDataSerializer + .toBinaryRow(rowData) + .equals(rowDataSerializer.toBinaryRow(that.rowData)); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java new file mode 100644 index 000000000000..250532dfec17 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -0,0 +1,299 @@ +/* + * 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.util.Collections; +import java.util.List; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +class DynamicRecordInternalSerializer extends TypeSerializer { + private static final long serialVersionUID = 1L; + + private final RowDataSerializerCache serializerCache; + private final boolean writeSchemaAndSpec; + + DynamicRecordInternalSerializer( + RowDataSerializerCache serializerCache, boolean writeSchemaAndSpec) { + this.serializerCache = serializerCache; + this.writeSchemaAndSpec = writeSchemaAndSpec; + } + + @Override + public TypeSerializer duplicate() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), + writeSchemaAndSpec); + } + + @Override + public DynamicRecordInternal createInstance() { + return new DynamicRecordInternal(); + } + + @Override + public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutputView) + throws IOException { + dataOutputView.writeUTF(toSerialize.tableName()); + dataOutputView.writeUTF(toSerialize.branch()); + if (writeSchemaAndSpec) { + dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); + } else { + dataOutputView.writeInt(toSerialize.schema().schemaId()); + dataOutputView.writeInt(toSerialize.spec().specId()); + } + dataOutputView.writeInt(toSerialize.writerKey()); + final Tuple3 rowDataSerializer; + if (writeSchemaAndSpec) { + rowDataSerializer = + serializerCache.serializer( + toSerialize.tableName(), toSerialize.schema(), toSerialize.spec(), null, null); + } else { + // Check that the schema id can be resolved. Not strictly necessary for serialization. + rowDataSerializer = + serializerCache.serializer( + toSerialize.tableName(), + null, + null, + toSerialize.schema().schemaId(), + toSerialize.spec().specId()); + } + rowDataSerializer.f0.serialize(toSerialize.rowData(), dataOutputView); + dataOutputView.writeBoolean(toSerialize.upsertMode()); + dataOutputView.writeInt(toSerialize.equalityFields().size()); + for (Integer equalityField : toSerialize.equalityFields()) { + dataOutputView.writeInt(equalityField); + } + } + + @Override + public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOException { + String tableName = dataInputView.readUTF(); + String branch = dataInputView.readUTF(); + Schema schema = null; + PartitionSpec spec = null; + Integer schemaId = null; + Integer specId = null; + if (writeSchemaAndSpec) { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); + } else { + schemaId = dataInputView.readInt(); + specId = dataInputView.readInt(); + } + + int writerKey = dataInputView.readInt(); + Tuple3 rowDataSerializer = + serializerCache.serializer(tableName, schema, spec, schemaId, specId); + RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); + boolean upsertMode = dataInputView.readBoolean(); + int numEqualityFields = dataInputView.readInt(); + final List equalityFieldIds; + if (numEqualityFields > 0) { + equalityFieldIds = Lists.newArrayList(); + } else { + equalityFieldIds = Collections.emptyList(); + } + for (int i = 0; i < numEqualityFields; i++) { + equalityFieldIds.add(dataInputView.readInt()); + } + return new DynamicRecordInternal( + tableName, + branch, + rowDataSerializer.f1, + rowDataSerializer.f2, + writerKey, + rowData, + upsertMode, + equalityFieldIds); + } + + @Override + public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputView dataInputView) + throws IOException { + String tableName = dataInputView.readUTF(); + reuse.setTableName(tableName); + String branch = dataInputView.readUTF(); + reuse.setBranch(branch); + + Schema schema = null; + PartitionSpec spec = null; + Integer schemaId = null; + Integer specId = null; + if (writeSchemaAndSpec) { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); + reuse.setSchema(schema); + reuse.setSpec(spec); + } else { + schemaId = dataInputView.readInt(); + specId = dataInputView.readInt(); + } + + int writerKey = dataInputView.readInt(); + reuse.setWriterKey(writerKey); + Tuple3 rowDataSerializer = + serializerCache.serializer(tableName, schema, spec, schemaId, specId); + RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); + boolean upsertMode = dataInputView.readBoolean(); + int numEqualityFields = dataInputView.readInt(); + final List equalityFieldIds; + if (numEqualityFields > 0) { + equalityFieldIds = Lists.newArrayList(); + } else { + equalityFieldIds = Collections.emptyList(); + } + for (int i = 0; i < numEqualityFields; i++) { + equalityFieldIds.add(dataInputView.readInt()); + } + return new DynamicRecordInternal( + tableName, + branch, + rowDataSerializer.f1, + rowDataSerializer.f2, + writerKey, + rowData, + upsertMode, + equalityFieldIds); + } + + @Override + public DynamicRecordInternal copy(DynamicRecordInternal from) { + return new DynamicRecordInternal( + from.tableName(), + from.branch(), + from.schema(), + from.spec(), + from.writerKey(), + from.rowData(), + from.upsertMode(), + from.equalityFields()); + } + + @Override + public DynamicRecordInternal copy(DynamicRecordInternal from, DynamicRecordInternal reuse) { + reuse.setTableName(from.tableName()); + reuse.setBranch(from.branch()); + reuse.setSchema(from.schema()); + reuse.setSpec(from.spec()); + reuse.setWriterKey(from.writerKey()); + reuse.setRowData(from.rowData()); + reuse.setUpsertMode(from.upsertMode()); + reuse.setEqualityFieldIds(from.equalityFields()); + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj instanceof DynamicRecordInternalSerializer) { + DynamicRecordInternalSerializer other = (DynamicRecordInternalSerializer) obj; + return writeSchemaAndSpec == other.writeSchemaAndSpec; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(writeSchemaAndSpec); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); + } + + public static class DynamicRecordInternalTypeSerializerSnapshot + implements TypeSerializerSnapshot { + + private boolean writeSchemaAndSpec; + + // Zero args constructor is required to instantiate this class on restore + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DynamicRecordInternalTypeSerializerSnapshot() {} + + DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { + this.writeSchemaAndSpec = writeSchemaAndSpec; + } + + @Override + public int getCurrentVersion() { + return 0; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + out.writeBoolean(writeSchemaAndSpec); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + this.writeSchemaAndSpec = in.readBoolean(); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + @Override + public TypeSerializer restoreSerializer() { + // Note: We pass in a null serializer cache which would create issues if we tried to use this + // restored serializer, but since we are using {@code + // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be + // used. A new one will be created via {@code DynamicRecordInternalType}. + return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java new file mode 100644 index 000000000000..8b22ca1a57cb --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java @@ -0,0 +1,102 @@ +/* + * 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.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.flink.CatalogLoader; + +@Internal +class DynamicRecordInternalType extends TypeInformation { + private final CatalogLoader catalogLoader; + private final boolean writeSchemaAndSpec; + private final int cacheSize; + + DynamicRecordInternalType( + CatalogLoader catalogLoader, boolean writeSchemaAndSpec, int cacheSize) { + this.catalogLoader = catalogLoader; + this.writeSchemaAndSpec = writeSchemaAndSpec; + this.cacheSize = cacheSize; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return DynamicRecordInternal.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(SerializerConfig serializerConfig) { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(catalogLoader, cacheSize), writeSchemaAndSpec); + } + + @Override + @Deprecated + public TypeSerializer createSerializer(ExecutionConfig config) { + return this.createSerializer(config.getSerializerConfig()); + } + + @Override + public String toString() { + return getClass().getName(); + } + + @Override + public boolean equals(Object o) { + return canEqual(o); + } + + @Override + public int hashCode() { + return getClass().getName().hashCode(); + } + + @Override + public boolean canEqual(Object o) { + return o instanceof DynamicRecordInternalType; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java new file mode 100644 index 000000000000..86536e24b08e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -0,0 +1,196 @@ +/* + * 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.Collections; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; + +@Internal +class DynamicRecordProcessor extends ProcessFunction + implements Collector { + static final String DYNAMIC_TABLE_UPDATE_STREAM = "dynamic-table-update-stream"; + + private final DynamicRecordConverter converter; + private final CatalogLoader catalogLoader; + private final boolean immediateUpdate; + private final int cacheMaximumSize; + private final long cacheRefreshMs; + + private transient TableDataCache tableCache; + private transient DynamicKeySelector selector; + private transient TableUpdater updater; + private transient OutputTag updateStream; + private transient Collector collector; + private transient Context context; + + DynamicRecordProcessor( + DynamicRecordConverter converter, + CatalogLoader catalogLoader, + boolean immediateUpdate, + int cacheMaximumSize, + long cacheRefreshMs) { + this.converter = converter; + this.catalogLoader = catalogLoader; + this.immediateUpdate = immediateUpdate; + this.cacheMaximumSize = cacheMaximumSize; + this.cacheRefreshMs = cacheRefreshMs; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + Catalog catalog = catalogLoader.loadCatalog(); + this.tableCache = new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs); + this.selector = + new DynamicKeySelector( + cacheMaximumSize, getRuntimeContext().getTaskInfo().getMaxNumberOfParallelSubtasks()); + if (immediateUpdate) { + updater = new TableUpdater(tableCache, catalog); + } + + updateStream = + new OutputTag<>( + DYNAMIC_TABLE_UPDATE_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize)) {}; + + converter.open(openContext); + } + + @Override + public void processElement(T element, Context ctx, Collector out) + throws Exception { + this.context = ctx; + this.collector = out; + converter.convert(element, this); + } + + @Override + public void collect(DynamicRecord data) { + boolean exists = tableCache.exists(data.tableIdentifier()).f0; + String foundBranch = exists ? tableCache.branch(data.tableIdentifier(), data.branch()) : null; + + Tuple2 foundSchema = + exists + ? tableCache.schema(data.tableIdentifier(), data.schema()) + : TableDataCache.NOT_FOUND; + + adjustPartitionSpecToTableSchema(foundSchema.f0, data); + PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; + + if (!exists + || foundBranch == null + || foundSpec == null + || foundSchema.f1 == CompareSchemasVisitor.Result.INCOMPATIBLE) { + if (immediateUpdate) { + Tuple3 newData = + updater.update(data.tableIdentifier(), data.branch(), data.schema(), data.spec()); + emit(collector, data, newData.f0, newData.f1, newData.f2); + } else { + int writerKey = + selector.getKey( + new DynamicKeySelector.Input( + data, + foundSchema.f0 != null ? foundSchema.f0 : data.schema(), + foundSpec != null ? foundSpec : data.spec(), + data.rowData())); + context.output( + updateStream, + new DynamicRecordInternal( + data.tableIdentifier().toString(), + data.branch(), + data.schema(), + data.spec(), + writerKey, + data.rowData(), + data.upsertMode(), + getEqualityFieldIds(data.equalityFields(), data.schema()))); + } + } else { + emit(collector, data, foundSchema.f0, foundSchema.f1, foundSpec); + } + } + + private static void adjustPartitionSpecToTableSchema(Schema schema, DynamicRecord data) { + if (schema != null) { + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(schema, data.spec()); + data.setSpec(adjustedSpec); + } + } + + private void emit( + Collector out, + DynamicRecord data, + Schema schema, + CompareSchemasVisitor.Result result, + PartitionSpec spec) { + RowData rowData = + result == CompareSchemasVisitor.Result.SAME + ? data.rowData() + : RowDataEvolver.convert(data.rowData(), data.schema(), schema); + int writerKey = selector.getKey(new DynamicKeySelector.Input(data, schema, spec, rowData)); + String tableName = data.tableIdentifier().toString(); + out.collect( + new DynamicRecordInternal( + tableName, + data.branch(), + schema, + spec, + writerKey, + rowData, + data.upsertMode(), + getEqualityFieldIds(data.equalityFields(), schema))); + } + + static List getEqualityFieldIds(List equalityFields, Schema schema) { + if (equalityFields == null || equalityFields.isEmpty()) { + if (!schema.identifierFieldIds().isEmpty()) { + return Lists.newArrayList(schema.identifierFieldIds()); + } else { + return Collections.emptyList(); + } + } + List equalityFieldIds = Lists.newArrayList(equalityFields.size()); + for (String equalityField : equalityFields) { + Types.NestedField field = schema.findField(equalityField); + Preconditions.checkNotNull( + field, "Equality field %s does not exist in schema", equalityField); + equalityFieldIds.add(field.fieldId()); + } + return equalityFieldIds; + } + + @Override + public void close() {} +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java new file mode 100644 index 000000000000..17ee853104b6 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -0,0 +1,71 @@ +/* + * 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.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; + +@Internal +class DynamicTableUpdateOperator + extends RichMapFunction { + private final CatalogLoader catalogLoader; + private final int cacheMaximumSize; + private final long cacheRefreshMs; + private transient TableUpdater updater; + + DynamicTableUpdateOperator( + CatalogLoader catalogLoader, int cacheMaximumSize, long cacheRefreshMs) { + this.catalogLoader = catalogLoader; + this.cacheMaximumSize = cacheMaximumSize; + this.cacheRefreshMs = cacheRefreshMs; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + Catalog catalog = catalogLoader.loadCatalog(); + this.updater = + new TableUpdater(new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs), catalog); + } + + @Override + public DynamicRecordInternal map(DynamicRecordInternal data) throws Exception { + Tuple3 newData = + updater.update( + TableIdentifier.parse(data.tableName()), data.branch(), data.schema(), data.spec()); + + data.setSchema(newData.f0); + data.setSpec(newData.f2); + + if (newData.f1 == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { + RowData newRowData = RowDataEvolver.convert(data.rowData(), data.schema(), newData.f0); + data.setRowData(newRowData); + } + + return data; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java new file mode 100644 index 000000000000..f7f6af86e39a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java @@ -0,0 +1,41 @@ +/* + * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java new file mode 100644 index 000000000000..566cfb9963b7 --- /dev/null +++ b/flink/v1.19/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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java new file mode 100644 index 000000000000..4432abd19199 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java new file mode 100644 index 000000000000..bc54898bd644 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -0,0 +1,218 @@ +/* + * 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 static final int CACHE_MAXIMUM_SIZE = 100; + + 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, + 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(CACHE_MAXIMUM_SIZE).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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java new file mode 100644 index 000000000000..7112fc40f310 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -0,0 +1,50 @@ +/* + * 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); + } + + private IcebergStreamWriterMetrics writerMetrics(String fullTableName) { + return metrics.computeIfAbsent( + fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java new file mode 100644 index 000000000000..6b020a5bda73 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java @@ -0,0 +1,184 @@ +/* + * 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.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.schema.SchemaWithPartnerVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * Visitor class that accumulates the set of changes needed to evolve an existing schema into the + * target schema. Changes are applied to an {@link UpdateSchema} operation. + */ +public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { + + private final UpdateSchema api; + private final Schema existingSchema; + private final Schema targetSchema; + + private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + this.api = api; + this.existingSchema = existingSchema; + this.targetSchema = targetSchema; + } + + /** + * Adds changes needed to produce the target schema to an {@link UpdateSchema} operation. + * + *

Changes are accumulated to evolve the existingSchema into a targetSchema. + * + * @param api an UpdateSchema for adding changes + * @param existingSchema an existing schema + * @param targetSchema a new schema to compare with the existing + */ + public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + visit( + targetSchema, + -1, + new EvolveSchemaVisitor(api, existingSchema, targetSchema), + new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); + } + + @Override + public Boolean struct(Types.StructType struct, Integer partnerId, List existingFields) { + if (partnerId == null) { + return true; + } + + // Add, update and order fields in the struct + Types.StructType partnerStruct = findFieldType(partnerId).asStructType(); + String after = null; + for (Types.NestedField targetField : struct.fields()) { + Types.NestedField nestedField = partnerStruct.field(targetField.name()); + final String columnName; + if (nestedField != null) { + updateColumn(nestedField, targetField); + columnName = this.existingSchema.findColumnName(nestedField.fieldId()); + } else { + addColumn(partnerId, targetField); + columnName = this.targetSchema.findColumnName(targetField.fieldId()); + } + + setPosition(columnName, after); + after = columnName; + } + + // Ensure that unused fields are made optional + for (Types.NestedField existingField : partnerStruct.fields()) { + if (struct.field(existingField.name()) == null) { + if (existingField.isRequired()) { + this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); + } + } + } + + return false; + } + + @Override + public Boolean field(Types.NestedField field, Integer partnerId, Boolean isFieldMissing) { + return partnerId == null; + } + + @Override + public Boolean list(Types.ListType list, Integer partnerId, Boolean isElementMissing) { + if (partnerId == null) { + return true; + } + + Preconditions.checkState( + !isElementMissing, "Error traversing schemas: element is missing, but list is present"); + + Types.ListType partnerList = findFieldType(partnerId).asListType(); + updateColumn(partnerList.fields().get(0), list.fields().get(0)); + + return false; + } + + @Override + public Boolean map( + Types.MapType map, Integer partnerId, Boolean isKeyMissing, Boolean isValueMissing) { + if (partnerId == null) { + return true; + } + + Preconditions.checkState( + !isKeyMissing, "Error traversing schemas: key is missing, but map is present"); + Preconditions.checkState( + !isValueMissing, "Error traversing schemas: value is missing, but map is present"); + + Types.MapType partnerMap = findFieldType(partnerId).asMapType(); + updateColumn(partnerMap.fields().get(0), map.fields().get(0)); + updateColumn(partnerMap.fields().get(1), map.fields().get(1)); + + return false; + } + + @Override + public Boolean primitive(Type.PrimitiveType primitive, Integer partnerId) { + return partnerId == null; + } + + private Type findFieldType(int fieldId) { + if (fieldId == -1) { + return existingSchema.asStruct(); + } else { + return existingSchema.findField(fieldId).type(); + } + } + + private void addColumn(int parentId, Types.NestedField field) { + String parentName = targetSchema.findColumnName(parentId); + api.addColumn(parentName, field.name(), field.type(), field.doc()); + } + + private void updateColumn(Types.NestedField existingField, Types.NestedField targetField) { + String existingColumnName = this.existingSchema.findColumnName(existingField.fieldId()); + + boolean needsOptionalUpdate = targetField.isOptional() && existingField.isRequired(); + boolean needsTypeUpdate = + targetField.type().isPrimitiveType() && !targetField.type().equals(existingField.type()); + boolean needsDocUpdate = + targetField.doc() != null && !targetField.doc().equals(existingField.doc()); + + if (needsOptionalUpdate) { + api.makeColumnOptional(existingColumnName); + } + + if (needsTypeUpdate) { + api.updateColumn(existingColumnName, targetField.type().asPrimitiveType()); + } + + if (needsDocUpdate) { + api.updateColumnDoc(existingColumnName, targetField.doc()); + } + } + + private void setPosition(String columnName, String after) { + if (after == null) { + this.api.moveFirst(columnName); + } else { + this.api.moveAfter(columnName, after); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java new file mode 100644 index 000000000000..12fa3f9e7f3b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.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 org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.UnboundPartitionSpec; + +public class PartitionSpecAdjustment { + + private PartitionSpecAdjustment() {} + + public static PartitionSpec adjustPartitionSpecToTableSchema( + Schema tableSchema, PartitionSpec userSpec) { + if (userSpec.isUnpartitioned()) { + return userSpec; + } + UnboundPartitionSpec.Builder builder = + UnboundPartitionSpec.builder().withSpecId(userSpec.specId()); + for (PartitionField field : userSpec.fields()) { + String sourceFieldName = userSpec.schema().idToName().get(field.sourceId()); + int adjustedSourceId = tableSchema.findField(sourceFieldName).fieldId(); + builder.addField( + field.transform().toString(), adjustedSourceId, field.fieldId(), field.name()); + } + return builder.build().bind(tableSchema); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java new file mode 100644 index 000000000000..e3746b82ce9f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.List; +import java.util.Objects; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.Term; +import org.apache.iceberg.expressions.UnboundTransform; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Checks compatibility of PartitionSpecs and evolves one into the other. */ +public class PartitionSpecEvolver { + + private PartitionSpecEvolver() {} + + /** + * Checks whether two PartitionSpecs are compatible with each other. Less strict than {@code + * PartitionSpec#compatible} in the sense that it tolerates differently named partition fields, as + * long as their transforms and source ids match. + */ + public static boolean checkCompatibility(PartitionSpec first, PartitionSpec second) { + if (first.equals(second)) { + return true; + } + + if (first.fields().size() != second.fields().size()) { + return false; + } + + for (int i = 0; i < first.fields().size(); i++) { + PartitionField firstField = first.fields().get(i); + PartitionField secondField = second.fields().get(i); + if (firstField.sourceId() != secondField.sourceId() + || !firstField.transform().toString().equals(secondField.transform().toString())) { + return false; + } + } + + return true; + } + + public static PartitionSpecEvolverResult evolve( + PartitionSpec currentSpec, PartitionSpec targetSpec, Schema schema) { + if (currentSpec.compatibleWith(targetSpec)) { + return new PartitionSpecEvolverResult(); + } + + PartitionSpecEvolverResult result = new PartitionSpecEvolverResult(); + + int maxNumFields = Math.max(currentSpec.fields().size(), targetSpec.fields().size()); + for (int i = 0; i < maxNumFields; i++) { + PartitionField currentField = Iterables.get(currentSpec.fields(), i, null); + PartitionField targetField = Iterables.get(targetSpec.fields(), i, null); + + if (!Objects.equals(currentField, targetField)) { + if (currentField != null) { + result.remove(toTerm(currentField, schema)); + } + + if (targetField != null) { + result.add(toTerm(targetField, schema)); + } + } + } + + return result; + } + + public static class PartitionSpecEvolverResult { + private final List termsToAdd = Lists.newArrayList(); + private final List termsToRemove = Lists.newArrayList(); + + public void add(Term term) { + termsToAdd.add(term); + } + + public void remove(Term term) { + termsToRemove.add(term); + } + + public List termsToAdd() { + return termsToAdd; + } + + public List termsToRemove() { + return termsToRemove; + } + + public boolean isEmpty() { + return termsToAdd.isEmpty() && termsToRemove.isEmpty(); + } + } + + private static Term toTerm(PartitionField field, Schema schema) { + String sourceName = schema.findField(field.sourceId()).name(); + return new UnboundTransform<>(new NamedReference<>(sourceName), field.transform()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java new file mode 100644 index 000000000000..f5f7cd2c4c3d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java @@ -0,0 +1,169 @@ +/* + * 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.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class RowDataEvolver { + private RowDataEvolver() {} + + public static RowData convert(RowData sourceData, Schema sourceSchema, Schema targetSchema) { + return convertStruct( + sourceData, FlinkSchemaUtil.convert(sourceSchema), FlinkSchemaUtil.convert(targetSchema)); + } + + private static Object convert(Object object, LogicalType sourceType, LogicalType targetType) { + if (object == null) { + return null; + } + + switch (targetType.getTypeRoot()) { + case BOOLEAN: + case INTEGER: + case FLOAT: + case VARCHAR: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case BINARY: + case VARBINARY: + return object; + case DOUBLE: + if (object instanceof Float) { + return ((Float) object).doubleValue(); + } else { + return object; + } + case BIGINT: + if (object instanceof Integer) { + return ((Integer) object).longValue(); + } else { + return object; + } + case DECIMAL: + DecimalType toDecimalType = (DecimalType) targetType; + DecimalData decimalData = (DecimalData) object; + if (((DecimalType) sourceType).getPrecision() == toDecimalType.getPrecision()) { + return object; + } else { + return DecimalData.fromBigDecimal( + decimalData.toBigDecimal(), toDecimalType.getPrecision(), toDecimalType.getScale()); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + if (object instanceof Integer) { + LocalDateTime dateTime = + LocalDateTime.of(LocalDate.ofEpochDay((Integer) object), LocalTime.MIN); + return TimestampData.fromLocalDateTime(dateTime); + } else { + return object; + } + case ROW: + return convertStruct((RowData) object, (RowType) sourceType, (RowType) targetType); + case ARRAY: + return convertArray((ArrayData) object, (ArrayType) sourceType, (ArrayType) targetType); + case MAP: + return convertMap((MapData) object, (MapType) sourceType, (MapType) targetType); + default: + throw new UnsupportedOperationException("Not a supported type: " + targetType); + } + } + + private static RowData convertStruct(RowData sourceData, RowType sourceType, RowType targetType) { + GenericRowData targetData = new GenericRowData(targetType.getFields().size()); + List targetFields = targetType.getFields(); + for (int i = 0; i < targetFields.size(); i++) { + RowType.RowField targetField = targetFields.get(i); + + int sourceFieldId = sourceType.getFieldIndex(targetField.getName()); + if (sourceFieldId == -1) { + if (targetField.getType().isNullable()) { + targetData.setField(i, null); + } else { + throw new IllegalArgumentException( + String.format( + "Field %s in target schema %s is non-nullable but does not exist in source schema.", + i + 1, targetType)); + } + } else { + RowData.FieldGetter getter = + RowData.createFieldGetter(sourceType.getTypeAt(sourceFieldId), sourceFieldId); + targetData.setField( + i, + convert( + getter.getFieldOrNull(sourceData), + sourceType.getFields().get(sourceFieldId).getType(), + targetField.getType())); + } + } + + return targetData; + } + + private static ArrayData convertArray( + ArrayData sourceData, ArrayType sourceType, ArrayType targetType) { + LogicalType fromElementType = sourceType.getElementType(); + LogicalType toElementType = targetType.getElementType(); + ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(fromElementType); + Object[] convertedArray = new Object[sourceData.size()]; + for (int i = 0; i < convertedArray.length; i++) { + convertedArray[i] = + convert(elementGetter.getElementOrNull(sourceData, i), fromElementType, toElementType); + } + return new GenericArrayData(convertedArray); + } + + private static MapData convertMap(MapData sourceData, MapType sourceType, MapType targetType) { + LogicalType fromMapKeyType = sourceType.getKeyType(); + LogicalType fromMapValueType = sourceType.getValueType(); + LogicalType toMapKeyType = targetType.getKeyType(); + LogicalType toMapValueType = targetType.getValueType(); + ArrayData keyArray = sourceData.keyArray(); + ArrayData valueArray = sourceData.valueArray(); + ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(fromMapKeyType); + ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(fromMapValueType); + Map convertedMap = Maps.newLinkedHashMap(); + for (int i = 0; i < keyArray.size(); ++i) { + convertedMap.put( + convert(keyGetter.getElementOrNull(keyArray, i), fromMapKeyType, toMapKeyType), + convert(valueGetter.getElementOrNull(valueArray, i), fromMapValueType, toMapValueType)); + } + + return new GenericMapData(convertedMap); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java new file mode 100644 index 000000000000..68269a32ee6e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java @@ -0,0 +1,102 @@ +/* + * 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.Serializable; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class RowDataSerializerCache implements Serializable { + private final CatalogLoader catalogLoader; + private final int maximumSize; + private transient Cache serializers; + + RowDataSerializerCache(CatalogLoader catalogLoader, int maximumSize) { + this.catalogLoader = catalogLoader; + this.maximumSize = maximumSize; + } + + Tuple3 serializer( + String tableName, + Schema unknownSchema, + PartitionSpec unknownSpec, + Integer schemaId, + Integer specId) { + if (serializers == null) { + // We need to initialize the cache at the first time + this.serializers = Caffeine.newBuilder().maximumSize(maximumSize).build(); + } + + SerializerInfo info = serializers.get(tableName, SerializerInfo::new); + Schema schema = unknownSchema != null ? unknownSchema : info.schemas.get(schemaId); + PartitionSpec spec = unknownSpec != null ? unknownSpec : info.specs.get(specId); + + if (schema == null || spec == null) { + info.update(); + schema = info.schemas.get(schemaId); + spec = info.specs.get(specId); + } + + RowDataSerializer serializer = + info.serializers.computeIfAbsent( + schema, s -> new RowDataSerializer(FlinkSchemaUtil.convert(s))); + + return Tuple3.of(serializer, schema, spec); + } + + CatalogLoader catalogLoader() { + return catalogLoader; + } + + int maximumSize() { + return maximumSize; + } + + private class SerializerInfo { + private final String tableName; + private final Map serializers; + private Map schemas; + private Map specs; + + SerializerInfo(String tableName) { + this.tableName = tableName; + this.serializers = Maps.newHashMapWithExpectedSize(2); + this.schemas = Maps.newHashMapWithExpectedSize(0); + this.specs = Maps.newHashMapWithExpectedSize(0); + } + + private void update() { + Table table = catalogLoader.loadCatalog().loadTable(TableIdentifier.parse(tableName)); + schemas = table.schemas(); + specs = table.specs(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java new file mode 100644 index 000000000000..a86a4b878405 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java @@ -0,0 +1,257 @@ +/* + * 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.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TableDataCache { + private static final Logger LOG = LoggerFactory.getLogger(TableDataCache.class); + private static final int MAX_SIZE = 10; + private static final Tuple2 EXISTS = Tuple2.of(true, null); + private static final Tuple2 NOT_EXISTS = Tuple2.of(false, null); + static final Tuple2 NOT_FOUND = + Tuple2.of(null, CompareSchemasVisitor.Result.INCOMPATIBLE); + + private final Catalog catalog; + private final long refreshMs; + private final Cache cache; + + TableDataCache(Catalog catalog, int maximumSize, long refreshMs) { + this.catalog = catalog; + this.refreshMs = refreshMs; + this.cache = Caffeine.newBuilder().maximumSize(maximumSize).build(); + } + + Tuple2 exists(TableIdentifier identifier) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { + return EXISTS; + } else if (needsRefresh(cached, true)) { + return refreshTable(identifier); + } else { + return NOT_EXISTS; + } + } + + String branch(TableIdentifier identifier, String branch) { + return branch(identifier, branch, true); + } + + Tuple2 schema(TableIdentifier identifier, Schema input) { + return schema(identifier, input, true); + } + + PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { + return spec(identifier, spec, true); + } + + void update(TableIdentifier identifier, Table table) { + cache.put( + identifier, + new CacheItem(true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); + } + + private String branch(TableIdentifier identifier, String branch, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && cached.tableExists && cached.branches.contains(branch)) { + return branch; + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return branch(identifier, branch, false); + } else { + return null; + } + } + + private Tuple2 schema( + TableIdentifier identifier, Schema input, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + Schema compatible = null; + if (cached != null && cached.tableExists) { + // This only works if the {@link Schema#equals(Object)} returns true for the old schema + // and a new schema. Performance is paramount as this code is on the hot path. Every other + // way for comparing 2 schemas were performing worse than the + // {@link CompareByNameVisitor#visit(Schema, Schema, boolean)}, so caching was useless. + Tuple2 lastResult = + cached.schema.lastResults.get(input); + if (lastResult != null) { + return lastResult; + } + + for (Map.Entry tableSchema : cached.schema.schemas.entrySet()) { + CompareSchemasVisitor.Result result = + CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); + if (result == CompareSchemasVisitor.Result.SAME) { + Tuple2 newResult = + Tuple2.of(tableSchema.getValue(), CompareSchemasVisitor.Result.SAME); + cached.schema.update(input, newResult); + return newResult; + } else if (compatible == null && result == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { + compatible = tableSchema.getValue(); + } + } + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return schema(identifier, input, false); + } else if (compatible != null) { + Tuple2 newResult = + Tuple2.of(compatible, CompareSchemasVisitor.Result.CONVERSION_NEEDED); + cached.schema.update(input, newResult); + return newResult; + } else if (cached != null && cached.tableExists) { + cached.schema.update(input, NOT_FOUND); + return NOT_FOUND; + } else { + return NOT_FOUND; + } + } + + private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && cached.tableExists) { + for (PartitionSpec tableSpec : cached.specs.values()) { + if (PartitionSpecEvolver.checkCompatibility(tableSpec, spec)) { + return tableSpec; + } + } + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return spec(identifier, spec, false); + } else { + return null; + } + } + + private Tuple2 refreshTable(TableIdentifier identifier) { + try { + Table table = catalog.loadTable(identifier); + cache.put( + identifier, + new CacheItem( + true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); + return EXISTS; + } catch (NoSuchTableException e) { + LOG.debug("Table doesn't exist {}", identifier, e); + cache.put(identifier, new CacheItem(false, null, null, null)); + return Tuple2.of(false, e); + } + } + + private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { + return allowRefresh + && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); + } + + public void invalidate(TableIdentifier identifier) { + cache.invalidate(identifier); + } + + /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ + static class CacheItem { + private final long created = System.currentTimeMillis(); + + private final boolean tableExists; + private final Set branches; + private final SchemaInfo schema; + private final Map specs; + + private CacheItem( + boolean tableExists, + Set branches, + SchemaInfo schema, + Map specs) { + this.tableExists = tableExists; + this.branches = branches; + this.schema = schema; + this.specs = specs; + } + + @VisibleForTesting + SchemaInfo getSchemaInfo() { + return schema; + } + } + + /** + * Stores precalculated results for {@link CompareSchemasVisitor#visit(Schema, Schema, boolean)} + * in the cache. + */ + static class SchemaInfo { + private final Map schemas; + private final Map> lastResults; + + private SchemaInfo(Map schemas) { + this.schemas = schemas; + this.lastResults = new LimitedLinkedHashMap<>(); + } + + private void update( + Schema newLastSchema, Tuple2 newLastResult) { + lastResults.put(newLastSchema, newLastResult); + } + + @VisibleForTesting + Tuple2 getLastResult(Schema schema) { + return lastResults.get(schema); + } + } + + @SuppressWarnings("checkstyle:IllegalType") + private static class LimitedLinkedHashMap extends LinkedHashMap { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + boolean remove = size() > MAX_SIZE; + if (remove) { + LOG.warn( + "Performance degraded as records with different schema is generated for the same table. " + + "Likely the DynamicRecord.schema is not reused. " + + "Reuse the same instance if the record schema is the same to improve performance"); + } + + return remove; + } + } + + @VisibleForTesting + Cache getInternalCache() { + return cache; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java new file mode 100644 index 000000000000..7be1d247f5d1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -0,0 +1,208 @@ +/* + * 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.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TableUpdater { + private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); + private final TableDataCache cache; + private final Catalog catalog; + + TableUpdater(TableDataCache cache, Catalog catalog) { + this.cache = cache; + this.catalog = catalog; + } + + /** + * Creates or updates a table to make sure that the given branch, schema, spec exists. + * + * @return a {@link Tuple3} of the new {@link Schema}, the status of the schema compared to the + * requested one, and the new {@link PartitionSpec#specId()}. + */ + Tuple3 update( + TableIdentifier tableIdentifier, String branch, Schema schema, PartitionSpec spec) { + findOrCreateTable(tableIdentifier, schema, spec); + findOrCreateBranch(tableIdentifier, branch); + Tuple2 newSchema = + findOrCreateSchema(tableIdentifier, schema); + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(newSchema.f0, spec); + PartitionSpec newSpec = findOrCreateSpec(tableIdentifier, adjustedSpec, newSchema.f0); + return Tuple3.of(newSchema.f0, newSchema.f1, newSpec); + } + + private void findOrCreateTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { + Tuple2 exists = cache.exists(identifier); + if (Boolean.FALSE.equals(exists.f0)) { + if (exists.f1 instanceof NoSuchNamespaceException) { + SupportsNamespaces catalogWithNameSpace = (SupportsNamespaces) catalog; + LOG.info("Namespace {} not found during table search. Creating namespace", identifier); + try { + catalogWithNameSpace.createNamespace(identifier.namespace()); + } catch (AlreadyExistsException e) { + LOG.debug("Namespace {} created concurrently", identifier.namespace(), e); + } + + createTable(identifier, schema, spec); + } else { + LOG.info("Table {} not found during table search. Creating table.", identifier); + createTable(identifier, schema, spec); + } + } + } + + private void createTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { + try { + Table table = catalog.createTable(identifier, schema, spec); + cache.update(identifier, table); + } catch (AlreadyExistsException e) { + LOG.info("Table {} created concurrently. Skipping creation.", identifier, e); + } + } + + private void findOrCreateBranch(TableIdentifier identifier, String branch) { + String fromCache = cache.branch(identifier, branch); + if (fromCache == null) { + try { + // TODO: Which snapshot should be used to create the branch? + catalog.loadTable(identifier).manageSnapshots().createBranch(branch).commit(); + LOG.info("Branch {} for {} created", branch, identifier); + } catch (Exception e) { + LOG.info( + "Failed to create branch {} for {}. Maybe created concurrently?", + branch, + identifier, + e); + } + } + } + + private Tuple2 findOrCreateSchema( + TableIdentifier identifier, Schema schema) { + Tuple2 fromCache = cache.schema(identifier, schema); + if (fromCache.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { + return fromCache; + } else { + Table table = catalog.loadTable(identifier); + Schema tableSchema = table.schema(); + CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); + switch (result) { + case SAME: + case CONVERSION_NEEDED: + cache.update(identifier, table); + return Tuple2.of(tableSchema, result); + case INCOMPATIBLE: + LOG.info( + "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); + UpdateSchema updateApi = table.updateSchema(); + EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); + + try { + updateApi.commit(); + cache.invalidate(identifier); + Tuple2 comparisonAfterMigration = + cache.schema(identifier, schema); + Schema newSchema = comparisonAfterMigration.f0; + LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); + return comparisonAfterMigration; + } catch (CommitFailedException e) { + LOG.info( + "Schema update failed for {} from {} to {}", identifier, tableSchema, schema, e); + Tuple2 newSchema = + cache.schema(identifier, schema); + if (newSchema.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { + LOG.info("Table {} schema updated concurrently to {}", identifier, schema); + return newSchema; + } else { + throw e; + } + } + default: + throw new IllegalArgumentException("Unknown comparison result"); + } + } + } + + private PartitionSpec findOrCreateSpec( + TableIdentifier identifier, PartitionSpec targetSpec, Schema newSchema) { + PartitionSpec currentSpec = cache.spec(identifier, targetSpec); + if (currentSpec != null) { + return currentSpec; + } + + Table table = catalog.loadTable(identifier); + currentSpec = table.spec(); + + PartitionSpecEvolver.PartitionSpecEvolverResult result = + PartitionSpecEvolver.evolve(currentSpec, targetSpec, newSchema); + if (result.isEmpty()) { + LOG.info("Returning equivalent existing spec {} for {}", currentSpec, targetSpec); + return currentSpec; + } + + LOG.info( + "Spec for table {} has been altered. Updating from {} to {}", + identifier, + currentSpec, + targetSpec); + UpdatePartitionSpec updater = table.updateSpec(); + result.termsToRemove().forEach(updater::removeField); + result.termsToAdd().forEach(updater::addField); + + try { + updater.commit(); + } catch (CommitFailedException e) { + LOG.info( + "Partition spec update failed for {} from {} to {}", + identifier, + currentSpec, + targetSpec, + e); + PartitionSpec newSpec = cache.spec(identifier, targetSpec); + Schema maybeUpdatedSchema = cache.schema(identifier, newSchema).f0; + result = PartitionSpecEvolver.evolve(targetSpec, newSpec, maybeUpdatedSchema); + if (result.isEmpty()) { + LOG.info("Table {} partition spec updated concurrently to {}", identifier, newSpec); + return newSpec; + } else { + throw e; + } + } + + cache.invalidate(identifier); + return cache.spec(identifier, targetSpec); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java new file mode 100644 index 000000000000..43e94c2274a4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -0,0 +1,144 @@ +/* + * 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(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 1767f774922a..ffa1db71d652 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -82,12 +82,19 @@ private SimpleDataUtil() {} Types.NestedField.optional(1, "id", Types.IntegerType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); + public static final Schema SCHEMA2 = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get())); + public static final TableSchema FLINK_SCHEMA = TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); public static final Record RECORD = GenericRecord.create(SCHEMA); + public static final Record RECORD2 = GenericRecord.create(SCHEMA2); public static Table createTable( String path, Map properties, boolean partitioned) { @@ -107,6 +114,14 @@ public static Record createRecord(Integer id, String data) { return record; } + public static Record createRecord(Integer id, String data, String extra) { + Record record = RECORD2.copy(); + record.setField("id", id); + record.setField("data", data); + record.setField("extra", extra); + return record; + } + public static RowData createRowData(Integer id, String data) { return GenericRowData.of(id, StringData.fromString(data)); } @@ -224,7 +239,12 @@ private static List convertToRecords(List rows) { for (RowData row : rows) { Integer id = row.isNullAt(0) ? null : row.getInt(0); String data = row.isNullAt(1) ? null : row.getString(1).toString(); - records.add(createRecord(id, data)); + if (row.getArity() == 2) { + records.add(createRecord(id, data)); + } else { + String extra = row.isNullAt(2) ? null : row.getString(2).toString(); + records.add(createRecord(id, data, extra)); + } } return records; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 6f8fc518a8f9..90dcfe500417 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -131,7 +131,7 @@ public static List convertRowDataToRow(List rowDataList, RowType r .collect(Collectors.toList()); } - private static List convertRecordToRow(List expectedRecords, Schema schema) { + public static List convertRecordToRow(List expectedRecords, Schema schema) { List expected = Lists.newArrayList(); @SuppressWarnings("unchecked") DataStructureConverter converter = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index 9ce36cc1e8d0..29db9f6158c8 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.TestFixtures.DATABASE; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -64,8 +65,8 @@ public class TestFlinkIcebergSinkBase { protected Table table; protected StreamExecutionEnvironment env; - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(Collections.singletonList(rows)); } protected List createRows(String prefix) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java new file mode 100644 index 000000000000..0c5794f18c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -0,0 +1,75 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test base for DynamicRecordInternalSerializer which allows to instantiate different serializer + * version, e.g. with writing the schema itself or just the schema id. + */ +abstract class DynamicRecordInternalSerializerTestBase + extends SerializerTestBase { + + static final String TABLE = "myTable"; + static final String BRANCH = "myBranch"; + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", TABLE); + + static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "number", Types.FloatType.get())); + + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); + + @Override + protected DynamicRecordInternal[] getTestData() { + GenericRowData rowData = new GenericRowData(3); + rowData.setField(0, 123L); + rowData.setField(1, StringData.fromString("test")); + rowData.setField(2, 1.23f); + + return new DynamicRecordInternal[] { + new DynamicRecordInternal(TABLE, BRANCH, SCHEMA, SPEC, 42, rowData, true, List.of(1, 3)) + }; + } + + @Override + protected Class getTypeClass() { + return DynamicRecordInternal.class; + } + + @Override + protected int getLength() { + return -1; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java new file mode 100644 index 000000000000..0bf454edc2e9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java @@ -0,0 +1,41 @@ +/* + * 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.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.BeforeEach; + +/** Test writing only the schema id on serialization. */ +class DynamicRecordInternalSerializerWriteSchemaIdTest + extends DynamicRecordInternalSerializerTestBase { + + final boolean writeSchemaAndSpec = false; + + @BeforeEach + void before() { + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.parse(TABLE), SCHEMA, SPEC); + } + + @Override + protected TypeSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java new file mode 100644 index 000000000000..c51e54b4b804 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java @@ -0,0 +1,34 @@ +/* + * 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.api.common.typeutils.TypeSerializer; + +/** Test the full schema */ +class DynamicRecordInternalSerializerWriteSchemaTest + extends DynamicRecordInternalSerializerTestBase { + + final boolean writeSchemaAndSpec = true; + + @Override + protected TypeSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java new file mode 100644 index 000000000000..6edebaecce2e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java @@ -0,0 +1,209 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.junit.jupiter.api.Test; + +class TestCompareSchemasVisitor { + + @Test + void testSchema() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get(), "comment"), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get(), "comment"), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testSchemaDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testSchemaDifferent() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())), + new Schema( + optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testSchemaWithMoreColumns() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())), + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); + } + + @Test + void testDifferentType() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testCompatibleType() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); + } + + @Test + void testWithRequiredChange() { + Schema dataSchema = + new Schema(optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); + Schema tableSchema = + new Schema(required(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema)) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + assertThat(CompareSchemasVisitor.visit(tableSchema, dataSchema)) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testStructDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))), + new Schema( + optional(0, "id", IntegerType.get()), + optional( + 1, "struct1", StructType.of(optional(2, "extra", IntegerType.get())))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testStructChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "struct1", StructType.of(optional(2, "extra", LongType.get())))), + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testMapDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))), + new Schema( + optional(0, "id", IntegerType.get()), + optional( + 1, "map1", MapType.ofOptional(2, 3, IntegerType.get(), StringType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testMapChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, LongType.get(), StringType.get()))), + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testListDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))), + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "list1", ListType.ofOptional(2, IntegerType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testListChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "list1", ListType.ofOptional(2, LongType.get()))), + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java new file mode 100644 index 000000000000..e6069656bd65 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -0,0 +1,591 @@ +/* + * 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.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestDynamicIcebergSink extends TestFlinkIcebergSinkBase { + + private static long seed; + + @BeforeEach + void before() { + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(2); + seed = 0; + } + + private static class DynamicIcebergDataImpl implements Serializable { + Row rowProvided; + Row rowExpected; + Schema schemaProvided; + Schema schemaExpected; + String tableName; + String branch; + PartitionSpec partitionSpec; + boolean upsertMode; + List equalityFields; + + private DynamicIcebergDataImpl( + Schema schemaProvided, String tableName, String branch, PartitionSpec partitionSpec) { + this( + schemaProvided, + schemaProvided, + tableName, + branch, + partitionSpec, + false, + Collections.emptyList(), + false); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + Schema schemaExpected, + String tableName, + String branch, + PartitionSpec partitionSpec) { + this( + schemaProvided, + schemaExpected, + tableName, + branch, + partitionSpec, + false, + Collections.emptyList(), + false); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + String tableName, + String branch, + PartitionSpec partitionSpec, + boolean upsertMode, + List equalityFields, + boolean isDuplicate) { + this( + schemaProvided, + schemaProvided, + tableName, + branch, + partitionSpec, + upsertMode, + equalityFields, + isDuplicate); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + Schema schemaExpected, + String tableName, + String branch, + PartitionSpec partitionSpec, + boolean upsertMode, + List equalityFields, + boolean isDuplicate) { + this.rowProvided = randomRow(schemaProvided, isDuplicate ? seed : ++seed); + this.rowExpected = isDuplicate ? null : rowProvided; + this.schemaProvided = schemaProvided; + this.schemaExpected = schemaExpected; + this.tableName = tableName; + this.branch = branch; + this.partitionSpec = partitionSpec; + this.upsertMode = upsertMode; + this.equalityFields = equalityFields; + } + } + + private static class Converter implements DynamicRecordConverter { + + @Override + public void convert(DynamicIcebergDataImpl row, Collector out) { + TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, row.tableName); + String branch = row.branch; + Schema schema = row.schemaProvided; + PartitionSpec spec = row.partitionSpec; + DynamicRecord dynamicRecord = + new DynamicRecord( + tableIdentifier, + branch, + schema, + spec, + converter(schema).toInternal(row.rowProvided), + spec.isPartitioned() ? DistributionMode.HASH : DistributionMode.NONE, + 10); + dynamicRecord.setUpsertMode(row.upsertMode); + dynamicRecord.setEqualityFields(row.equalityFields); + out.collect(dynamicRecord); + } + } + + private static DataFormatConverters.RowConverter converter(Schema schema) { + RowType rowType = FlinkSchemaUtil.convert(schema); + TableSchema tableSchema = FlinkSchemaUtil.toSchema(rowType); + return new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); + } + + @Test + void testWrite() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testWritePartitioned() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); + + runTest(rows); + } + + @Test + void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { + Schema schema = + new Schema( + // Use zero-based schema field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + Schema schema2 = + new Schema( + // Use zero-based schema field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get()), + Types.NestedField.optional(2, "extra", Types.StringType.get())); + PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("extra", 23).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), + new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); + + runTest(rows); + } + + @Test + void testSchemaEvolutionFieldOrderChanges() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + Schema expectedSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + Schema schema2 = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "extra", Types.StringType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + Schema expectedSchema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); + + for (DynamicIcebergDataImpl row : rows) { + if (row.schemaExpected == expectedSchema) { + // We manually adjust the expected Row to match the second expected schema + row.rowExpected = Row.of(row.rowProvided.getField(0), null, row.rowProvided.getField(1)); + } + } + + runTest(rows); + } + + @Test + void testMultipleTables() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testMultipleTablesPartitioned() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); + + runTest(rows); + } + + @Test + void testSchemaEvolutionAddField() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows, this.env, 1); + } + + @Test + void testRowEvolutionNullMissingOptionalField() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows, this.env, 1); + } + + @Test + void testSchemaEvolutionNonBackwardsCompatible() throws Exception { + Schema backwardsIncompatibleSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + // Required column is missing in this schema + Schema erroringSchema = + new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + backwardsIncompatibleSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); + + try { + runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); + Assert.fail(); + } catch (JobExecutionException e) { + assertThat( + ExceptionUtils.findThrowable( + e, + t -> + t.getMessage() + .contains( + "Field 2 in target schema ROW<`id` INT NOT NULL, `data` STRING NOT NULL> is non-nullable but does not exist in source schema."))) + .isNotEmpty(); + } + } + + @Test + void testPartitionSpecEvolution() throws Exception { + PartitionSpec spec1 = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 5).identity("data").build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); + + runTest(rows); + } + + @Test + void testMultipleBranches() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testWriteDynamicRowData() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testUpsert() throws Exception { + List rows = + Lists.newArrayList( + // Insert one rows + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + false), + // Remaining rows are duplicates + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true)); + + executeDynamicSink(rows, env, true, 1); + + try (CloseableIterable iterable = + IcebergGenerics.read( + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.of("default", "t1"))) + .build()) { + List records = Lists.newArrayList(); + for (Record record : iterable) { + records.add(record); + } + + assertThat(records.size()).isEqualTo(1); + Record actual = records.get(0); + DynamicIcebergDataImpl input = rows.get(0); + assertThat(actual.get(0)).isEqualTo(input.rowProvided.getField(0)); + assertThat(actual.get(1)).isEqualTo(input.rowProvided.getField(1)); + // There is an additional _pos field which gets added + } + } + + private void runTest(List dynamicData) throws Exception { + runTest(dynamicData, this.env, 2); + } + + private void runTest( + List dynamicData, StreamExecutionEnvironment env, int parallelism) + throws Exception { + runTest(dynamicData, env, true, parallelism); + runTest(dynamicData, env, false, parallelism); + } + + private void runTest( + List dynamicData, + StreamExecutionEnvironment env, + boolean immediateUpdate, + int parallelism) + throws Exception { + executeDynamicSink(dynamicData, env, immediateUpdate, parallelism); + verifyResults(dynamicData); + } + + private void executeDynamicSink( + List dynamicData, + StreamExecutionEnvironment env, + boolean immediateUpdate, + int parallelism) + throws Exception { + DataStream dataStream = + env.addSource(createBoundedSource(dynamicData), TypeInformation.of(new TypeHint<>() {})); + env.setParallelism(parallelism); + + DynamicIcebergSink.forInput(dataStream) + .withConverter(new Converter()) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .writeParallelism(parallelism) + .immediateTableUpdate(immediateUpdate) + .append(); + + // Write the data + env.execute("Test Iceberg DataStream"); + } + + private void verifyResults(List dynamicData) throws IOException { + // Calculate the expected result + Map, List> expectedData = Maps.newHashMap(); + Map expectedSchema = Maps.newHashMap(); + dynamicData.forEach( + r -> { + Schema oldSchema = expectedSchema.get(r.tableName); + if (oldSchema == null || oldSchema.columns().size() < r.schemaProvided.columns().size()) { + expectedSchema.put(r.tableName, r.schemaExpected); + } + }); + + dynamicData.forEach( + r -> { + List data = + expectedData.computeIfAbsent( + Tuple2.of(r.tableName, r.branch), unused -> Lists.newArrayList()); + data.addAll( + convertToRowData(expectedSchema.get(r.tableName), ImmutableList.of(r.rowExpected))); + }); + + // Check the expected result + int count = dynamicData.size(); + for (Map.Entry, List> e : expectedData.entrySet()) { + SimpleDataUtil.assertTableRows( + CATALOG_EXTENSION + .catalogLoader() + .loadCatalog() + .loadTable(TableIdentifier.of(DATABASE, e.getKey().f0)), + e.getValue(), + e.getKey().f1); + count -= e.getValue().size(); + } + + // Found every record + assertThat(count).isZero(); + } + + private List convertToRowData(Schema schema, List rows) { + DataFormatConverters.RowConverter converter = converter(schema); + return rows.stream() + .map( + r -> { + Row updateRow = r; + // We need conversion to generate the missing columns + if (r.getArity() != schema.columns().size()) { + updateRow = new Row(schema.columns().size()); + for (int i = 0; i < r.getArity(); ++i) { + updateRow.setField(i, r.getField(i)); + } + } + return converter.toInternal(updateRow); + }) + .collect(Collectors.toList()); + } + + private static Row randomRow(Schema schema, long seedOverride) { + return TestHelpers.convertRecordToRow( + RandomGenericData.generate(schema, 1, seedOverride), schema) + .get(0); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java new file mode 100644 index 000000000000..b7c145da2b77 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -0,0 +1,226 @@ +/* + * 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.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE; +import static org.apache.iceberg.flink.sink.dynamic.DynamicCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; + +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.util.Collector; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +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.TableIdentifier; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Disabled("Please enable manually for performance testing.") +class TestDynamicIcebergSinkPerf { + private static final Logger LOG = LoggerFactory.getLogger(TestDynamicIcebergSinkPerf.class); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TABLE); + + private static final int SAMPLE_SIZE = 50_000; + private static final int RECORD_SIZE = 5_000_000; + private static final int TABLE_NUM = 3; + private static final int PARALLELISM = 2; + private static final int WRITE_PARALLELISM = 2; + private static final TableIdentifier[] IDENTIFIERS = new TableIdentifier[TABLE_NUM]; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + private static final List RANGE = + IntStream.range(0, RECORD_SIZE).boxed().collect(Collectors.toList()); + + private static List rows; + private StreamExecutionEnvironment env; + + @BeforeEach + void before() { + for (int i = 0; i < TABLE_NUM; ++i) { + // So the table name hash difference is bigger than 1 + IDENTIFIERS[i] = TableIdentifier.of(DATABASE, TABLE + "_" + (i * 13)); + + Table table = + CATALOG_EXTENSION + .catalog() + .createTable( + IDENTIFIERS[i], + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); + + table.manageSnapshots().createBranch("main").commit(); + } + + List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); + rows = Lists.newArrayListWithCapacity(records.size()); + for (int i = 0; i < records.size(); ++i) { + rows.add( + new DynamicRecord( + IDENTIFIERS[i % TABLE_NUM], + "main", + SCHEMA, + PartitionSpec.unpartitioned(), + RowDataConverter.convert(SCHEMA, records.get(i)), + DistributionMode.NONE, + WRITE_PARALLELISM)); + } + + Configuration configuration = MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + configuration.setString("rest.flamegraph.enabled", "true"); + env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration) + .enableCheckpointing(100) + .setParallelism(PARALLELISM) + .setMaxParallelism(PARALLELISM); + env.getConfig().enableObjectReuse(); + } + + @AfterEach + void after() { + for (TableIdentifier identifier : IDENTIFIERS) { + CATALOG_EXTENSION.catalog().dropTable(identifier); + } + } + + private static class IdBasedConverter implements DynamicRecordConverter { + + @Override + public void convert(Integer id, Collector out) { + out.collect(rows.get(id % SAMPLE_SIZE)); + } + } + + @Test + void testDynamicSink() throws Exception { + // So we make sure that the writer threads are the same for the 2 tests + env.setMaxParallelism(PARALLELISM * TABLE_NUM * 2); + env.setParallelism(PARALLELISM * TABLE_NUM * 2); + runTest( + s -> { + DynamicIcebergSink.forInput(s) + .withConverter(new IdBasedConverter()) + .immediateTableUpdate(true) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .append(); + }); + } + + @Test + void testIcebergSink() throws Exception { + runTest( + s -> { + for (int i = 0; i < IDENTIFIERS.length; ++i) { + TableLoader tableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), IDENTIFIERS[i]); + final int finalInt = i; + IcebergSink.forRowData( + s.flatMap( + (FlatMapFunction) + (input, collector) -> { + if (input % TABLE_NUM == finalInt) { + collector.collect(rows.get(input % SAMPLE_SIZE).rowData()); + } + }) + .returns(InternalTypeInfo.of(FlinkSchemaUtil.convert(SCHEMA))) + .rebalance()) + .tableLoader(tableLoader) + .uidSuffix("Uid" + i) + .writeParallelism(WRITE_PARALLELISM) + .append(); + } + }); + } + + private void runTest(Consumer> sink) throws Exception { + DataStream dataStream = + env.addSource( + new BoundedTestSource<>( + ImmutableList.of( + RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE), + true), + TypeInformation.of(Integer.class)); + + sink.accept(dataStream); + + long before = System.currentTimeMillis(); + env.execute(); + + for (TableIdentifier identifier : IDENTIFIERS) { + Table table = CATALOG_EXTENSION.catalog().loadTable(identifier); + for (Snapshot snapshot : table.snapshots()) { + long records = 0; + for (DataFile dataFile : snapshot.addedDataFiles(table.io())) { + records += dataFile.recordCount(); + } + + LOG.info( + "TEST RESULT: For table {} snapshot {} written {} records in {} ms", + identifier, + snapshot.snapshotId(), + records, + snapshot.timestampMillis() - before); + before = snapshot.timestampMillis(); + } + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java new file mode 100644 index 000000000000..ad96cf9747c4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -0,0 +1,106 @@ +/* + * 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, + 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; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java new file mode 100644 index 000000000000..05501b1e6e43 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java @@ -0,0 +1,607 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.of; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaUpdate; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.UUIDType; +import org.junit.Assert; +import org.junit.Test; + +public class TestEvolveSchemaVisitor { + + private static List primitiveTypes() { + return Lists.newArrayList( + StringType.get(), + TimeType.get(), + Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + UUIDType.get(), + Types.DateType.get(), + Types.BooleanType.get(), + Types.BinaryType.get(), + DoubleType.get(), + IntegerType.get(), + Types.FixedType.ofLength(10), + DecimalType.of(10, 2), + LongType.get(), + FloatType.get()); + } + + private static Types.NestedField[] primitiveFields( + Integer initialValue, List primitiveTypes) { + return primitiveFields(initialValue, primitiveTypes, true); + } + + private static Types.NestedField[] primitiveFields( + Integer initialValue, List primitiveTypes, boolean optional) { + AtomicInteger atomicInteger = new AtomicInteger(initialValue); + return primitiveTypes.stream() + .map( + type -> + of( + atomicInteger.incrementAndGet(), + optional, + type.toString(), + Types.fromPrimitiveString(type.toString()))) + .toArray(Types.NestedField[]::new); + } + + @Test + public void testAddTopLevelPrimitives() { + Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testMakeTopLevelPrimitivesOptional() { + Schema existingSchema = new Schema(primitiveFields(0, primitiveTypes(), false)); + Assert.assertTrue(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)); + + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); + Schema newSchema = updateApi.apply(); + Assert.assertEquals(14, newSchema.asStruct().fields().size()); + Assert.assertTrue(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)); + } + + @Test + public void testIdentifyFieldsByName() { + Schema existingSchema = + new Schema(Types.NestedField.optional(42, "myField", Types.LongType.get())); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + Schema newSchema = + new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); + EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); + Assert.assertTrue(updateApi.apply().sameSchema(existingSchema)); + } + + @Test + public void testChangeOrderTopLevelPrimitives() { + Schema existingSchema = + new Schema( + Arrays.asList(optional(1, "a", StringType.get()), optional(2, "b", StringType.get()))); + Schema targetSchema = + new Schema( + Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddTopLevelListOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testMakeTopLevelListOfPrimitivesOptional() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema existingSchema = + new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); + Schema targetSchema = new Schema(); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Schema expectedSchema = + new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); + Assert.assertEquals(expectedSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddTopLevelMapOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddTopLevelStructOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); + Assert.assertEquals(currentSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddNestedPrimitive() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); + Schema targetSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testMakeNestedPrimitiveOptional() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = + new Schema( + optional(1, "aStruct", StructType.of(required(2, "primitive", primitiveType)))); + Schema targetSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddNestedPrimitives() { + Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); + Schema targetSchema = + new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedLists() { + Schema targetSchema = + new Schema( + optional( + 1, + "aList", + ListType.ofOptional( + 2, + ListType.ofOptional( + 3, + ListType.ofOptional( + 4, + ListType.ofOptional( + 5, + ListType.ofOptional( + 6, + ListType.ofOptional( + 7, + ListType.ofOptional( + 8, + ListType.ofOptional( + 9, + ListType.ofOptional( + 10, DecimalType.of(11, 20)))))))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedStruct() { + Schema targetSchema = + new Schema( + optional( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "struct3", + StructType.of( + optional( + 4, + "struct4", + StructType.of( + optional( + 5, + "struct5", + StructType.of( + optional( + 6, + "struct6", + StructType.of( + optional( + 7, + "aString", + StringType.get())))))))))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedMaps() { + Schema targetSchema = + new Schema( + optional( + 1, + "struct", + MapType.ofOptional( + 2, + 3, + StringType.get(), + MapType.ofOptional( + 4, + 5, + StringType.get(), + MapType.ofOptional( + 6, + 7, + StringType.get(), + MapType.ofOptional( + 8, + 9, + StringType.get(), + MapType.ofOptional( + 10, + 11, + StringType.get(), + MapType.ofOptional( + 12, 13, StringType.get(), StringType.get())))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testDetectInvalidTopLevelList() { + Schema currentSchema = + new Schema(optional(1, "aList", ListType.ofOptional(2, StringType.get()))); + Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, LongType.get()))); + Assert.assertThrows( + "Cannot change column type: aList.element: string -> long", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 2), currentSchema, targetSchema)); + } + + @Test + public void testDetectInvalidTopLevelMapValue() { + + Schema currentSchema = + new Schema( + optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), LongType.get()))); + + Assert.assertThrows( + "Cannot change column type: aMap.value: string -> long", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + public void testDetectInvalidTopLevelMapKey() { + Schema currentSchema = + new Schema( + optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, UUIDType.get(), StringType.get()))); + Assert.assertThrows( + "Cannot change column type: aMap.key: string -> uuid", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + // int 32-bit signed integers -> Can promote to long + public void testTypePromoteIntegerToLong() { + Schema currentSchema = new Schema(required(1, "aCol", IntegerType.get())); + Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Schema applied = updateApi.apply(); + Assert.assertEquals(1, applied.asStruct().fields().size()); + Assert.assertEquals(LongType.get(), applied.asStruct().fields().get(0).type()); + } + + @Test + // float 32-bit IEEE 754 floating point -> Can promote to double + public void testTypePromoteFloatToDouble() { + Schema currentSchema = new Schema(required(1, "aCol", FloatType.get())); + Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Schema applied = updateApi.apply(); + Assert.assertEquals(1, applied.asStruct().fields().size()); + Assert.assertEquals(DoubleType.get(), applied.asStruct().fields().get(0).type()); + } + + @Test + public void testInvalidTypePromoteDoubleToFloat() { + Schema currentSchema = new Schema(required(1, "aCol", DoubleType.get())); + Schema targetSchema = new Schema(required(1, "aCol", FloatType.get())); + Assert.assertThrows( + "Cannot change column type: aCol: double -> float", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + // decimal(P,S) Fixed-point decimal; precision P, scale S -> Scale is fixed [1], precision must be + // 38 or less + public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { + Schema currentSchema = new Schema(required(1, "aCol", DecimalType.of(20, 1))); + Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddPrimitiveToNestedStruct() { + Schema existingSchema = + new Schema( + required( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "list", + ListType.ofOptional( + 4, + StructType.of(optional(5, "number", IntegerType.get()))))))))); + + Schema targetSchema = + new Schema( + required( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "list", + ListType.ofOptional( + 4, + StructType.of( + optional(5, "number", LongType.get()), + optional(6, "time", TimeType.get()))))))))); + + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 5); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testReplaceListWithPrimitive() { + Schema currentSchema = + new Schema(optional(1, "aColumn", ListType.ofOptional(2, StringType.get()))); + Schema targetSchema = new Schema(optional(1, "aColumn", StringType.get())); + Assert.assertThrows( + "Cannot change column type: aColumn: list -> string", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + public void addNewTopLevelStruct() { + Schema currentSchema = + new Schema( + optional( + 1, + "map1", + MapType.ofOptional( + 2, + 3, + StringType.get(), + ListType.ofOptional( + 4, StructType.of(optional(5, "string1", StringType.get())))))); + + Schema targetSchema = + new Schema( + optional( + 1, + "map1", + MapType.ofOptional( + 2, + 3, + StringType.get(), + ListType.ofOptional( + 4, StructType.of(optional(5, "string1", StringType.get()))))), + optional( + 6, + "struct1", + StructType.of( + optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 5); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAppendNestedStruct() { + Schema currentSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional( + 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); + + Schema targetSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), + optional( + 5, + "repeat", + StructType.of( + optional( + 6, + "s1", + StructType.of( + optional( + 7, + "s2", + StructType.of( + optional( + 8, + "s3", + StructType.of( + optional( + 9, + "s4", + StringType.get())))))))))))))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 4); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testMakeNestedStructOptional() { + Schema currentSchema = getNestedSchemaWithOptionalModifier(false); + Schema targetSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional( + 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 9); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals( + updateApi.apply().asStruct(), getNestedSchemaWithOptionalModifier(true).asStruct()); + } + + private static Schema getNestedSchemaWithOptionalModifier(boolean nestedIsOptional) { + return new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), + of( + 5, + nestedIsOptional, + "repeat", + StructType.of( + optional( + 6, + "s1", + StructType.of( + optional( + 7, + "s2", + StructType.of( + optional( + 8, + "s3", + StructType.of( + optional( + 9, "s4", StringType.get())))))))))))))); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java new file mode 100644 index 000000000000..b7cd95e00146 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java @@ -0,0 +1,57 @@ +/* + * 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 org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionSpecAdjustment { + + @Test + void testPartitionSpecSourceIdRemappingBasedOnFieldNames() { + Schema specSchema = + new Schema( + // Use zero-based field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required( + 1, + "data", + Types.StructType.of(Types.NestedField.required(2, "str", Types.StringType.get())))); + + PartitionSpec spec = PartitionSpec.builderFor(specSchema).bucket("id", 10).build(); + + Schema tableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required( + 2, + "data", + Types.StructType.of(Types.NestedField.required(3, "str", Types.StringType.get())))); + + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(tableSchema, spec); + + assertThat(adjustedSpec) + .isEqualTo(PartitionSpec.builderFor(tableSchema).bucket("id", 10).build()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java new file mode 100644 index 000000000000..2a2b4c385730 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java @@ -0,0 +1,248 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; +import org.junit.Assert; +import org.junit.jupiter.api.Test; + +class TestRowDataEvolver { + @Test + void testPrimitiveTypes() { + DataGenerator generator = new DataGenerators.Primitives(); + assertThat( + RowDataEvolver.convert( + generator.generateFlinkRowData(), + generator.icebergSchema(), + generator.icebergSchema())) + .isEqualTo(generator.generateFlinkRowData()); + } + + @Test + void testAddColumn() { + assertThat( + RowDataEvolver.convert( + SimpleDataUtil.createRowData(1, "a"), + SimpleDataUtil.SCHEMA, + SimpleDataUtil.SCHEMA2)) + .isEqualTo(GenericRowData.of(1, StringData.fromString("a"), null)); + } + + @Test + void testAddRequiredColumn() { + Schema currentSchema = new Schema(Types.NestedField.optional(1, "id", Types.IntegerType.get())); + Schema targetSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get())); + + Assert.assertThrows( + IllegalArgumentException.class, + () -> RowDataEvolver.convert(GenericRowData.of(42), currentSchema, targetSchema)); + } + + @Test + void testIntToLong() { + Schema schemaWithLong = + new Schema( + Types.NestedField.optional(2, "id", Types.LongType.get()), + Types.NestedField.optional(4, "data", Types.StringType.get())); + + assertThat( + RowDataEvolver.convert( + SimpleDataUtil.createRowData(1, "a"), SimpleDataUtil.SCHEMA, schemaWithLong)) + .isEqualTo(GenericRowData.of(1L, StringData.fromString("a"))); + } + + @Test + void testFloatToDouble() { + Schema schemaWithFloat = + new Schema(Types.NestedField.optional(1, "float2double", Types.FloatType.get())); + Schema schemaWithDouble = + new Schema(Types.NestedField.optional(2, "float2double", Types.DoubleType.get())); + + assertThat(RowDataEvolver.convert(GenericRowData.of(1.5f), schemaWithFloat, schemaWithDouble)) + .isEqualTo(GenericRowData.of(1.5d)); + } + + @Test + void testDateToTimestamp() { + Schema schemaWithFloat = + new Schema(Types.NestedField.optional(1, "date2timestamp", Types.DateType.get())); + Schema schemaWithDouble = + new Schema( + Types.NestedField.optional(2, "date2timestamp", Types.TimestampType.withoutZone())); + + DateTime time = new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); + int days = + Days.daysBetween(new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC), time).getDays(); + + assertThat(RowDataEvolver.convert(GenericRowData.of(days), schemaWithFloat, schemaWithDouble)) + .isEqualTo(GenericRowData.of(TimestampData.fromEpochMillis(time.getMillis()))); + } + + @Test + void testIncreasePrecision() { + Schema before = + new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2))); + Schema after = + new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(10, 2))); + + assertThat( + RowDataEvolver.convert( + GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 9, 2)), + before, + after)) + .isEqualTo(GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 10, 2))); + } + + @Test + void testStructAddOptionalFields() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.required( + 10, + structField.name(), + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + optional(103, "optional", Types.StringType.get()), + required(102, "name", Types.StringType.get())))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of(1, null, StringData.fromString("Jane"))); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } + + @Test + void testStructAddRequiredFieldsWithOptionalRoot() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, + "newFieldOptionalField", + Types.StructType.of( + Types.NestedField.optional( + structField.fieldId(), + structField.name(), + Types.StructType.of( + optional(101, "id", Types.IntegerType.get()), + // Required columns which leads to nulling the entire struct + required(103, "required", Types.StringType.get()), + required(102, "name", Types.StringType.get())))))); + + RowData expectedData = GenericRowData.of(StringData.fromString("row_id_value"), null); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(expectedData); + } + + @Test + void testStructAddRequiredFields() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.required( + 10, + structField.name(), + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required(103, "required", Types.StringType.get()), + required(102, "name", Types.StringType.get())))); + + Assert.assertThrows( + IllegalArgumentException.class, + () -> RowDataEvolver.convert(oldData, oldSchema, newSchema)); + } + + @Test + void testMap() { + DataGenerator generator = new DataGenerators.MapOfPrimitives(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField mapField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, + mapField.name(), + Types.MapType.ofRequired(101, 102, Types.StringType.get(), Types.LongType.get()))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), 1L, StringData.fromString("Joe"), 2L))); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } + + @Test + void testArray() { + DataGenerator generator = new DataGenerators.ArrayOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField arrayField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, arrayField.name(), Types.ListType.ofOptional(101, Types.LongType.get()))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(new Long[] {1L, 2L, 3L})); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java new file mode 100644 index 000000000000..c0ca74eba240 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java @@ -0,0 +1,85 @@ +/* + * 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.commons.lang3.SerializationUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestTableDataCache extends TestFlinkIcebergSinkBase { + + @Test + void testCaching() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + + Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); + assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) + .isEqualTo(schema1); + + assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2)) + .isEqualTo(TableDataCache.NOT_FOUND); + + schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) + .isEqualTo(schema1); + } + + @Test + void testCacheInvalidationAfterSchemaChange() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); + + catalog.dropTable(tableIdentifier); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()); + + Schema schema2 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f0; + assertThat(schema2.sameSchema(SimpleDataUtil.SCHEMA2)).isTrue(); + } + + @Test + void testCachingDisabled() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 0, Long.MAX_VALUE); + + // Cleanup routine doesn't run after every write + cache.getInternalCache().cleanUp(); + assertThat(cache.getInternalCache().estimatedSize()).isEqualTo(0); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java new file mode 100644 index 000000000000..8dbf630490b0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestTableUpdater extends TestFlinkIcebergSinkBase { + + @Test + void testInvalidateOldCacheEntryOnUpdate() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + Schema updated = + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) + .f0; + assertThat(updated.sameSchema(SimpleDataUtil.SCHEMA2)); + assertThat( + cache + .schema(tableIdentifier, SimpleDataUtil.SCHEMA2) + .f0 + .sameSchema(SimpleDataUtil.SCHEMA2)) + .isTrue(); + } + + @Test + void testLastResultInvalidation() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + // Initialize cache + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); + + // Update table behind the scenes + catalog.dropTable(tableIdentifier); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); + + // Cache still stores the old information + assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f1) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + + assertThat( + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) + .f1) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + + // Last result cache should be cleared + assertThat( + cache + .getInternalCache() + .getIfPresent(tableIdentifier) + .getSchemaInfo() + .getLastResult(SimpleDataUtil.SCHEMA2)) + .isNull(); + } +} From 11c908216956a66c95985b5df04e81d83bf47609 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Thu, 27 Feb 2025 16:18:56 +0100 Subject: [PATCH 3/6] Flink 1.20: Dynamic Iceberg Sink Co-authored-by: Peter Vary --- flink/v1.20/build.gradle | 3 + ...RecordSerializerDeserializerBenchmark.java | 137 ++++ .../apache/iceberg/flink/FlinkConfParser.java | 7 + .../apache/iceberg/flink/FlinkWriteConf.java | 4 + .../flink/sink/BucketPartitionerUtil.java | 2 +- .../iceberg/flink/sink/CommitSummary.java | 50 +- .../iceberg/flink/sink/DeltaManifests.java | 4 +- .../flink/sink/DeltaManifestsSerializer.java | 4 +- .../flink/sink/EqualityFieldKeySelector.java | 10 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 10 +- .../flink/sink/IcebergCommittable.java | 2 +- .../sink/IcebergCommittableSerializer.java | 2 +- .../iceberg/flink/sink/IcebergCommitter.java | 2 +- .../flink/sink/IcebergFilesCommitter.java | 2 +- .../sink/IcebergFilesCommitterMetrics.java | 10 +- .../iceberg/flink/sink/IcebergSink.java | 2 +- .../sink/IcebergStreamWriterMetrics.java | 8 +- .../flink/sink/IcebergWriteAggregator.java | 3 +- .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../flink/sink/NonThrowingKeySelector.java | 28 + .../flink/sink/PartitionKeySelector.java | 11 +- .../flink/sink/RowDataTaskWriterFactory.java | 27 +- .../flink/sink/WriteResultSerializer.java | 2 +- .../sink/dynamic/CompareSchemasVisitor.java | 254 ++++++++ .../sink/dynamic/DynamicCommittable.java | 80 +++ .../dynamic/DynamicCommittableSerializer.java | 73 +++ .../flink/sink/dynamic/DynamicCommitter.java | 417 ++++++++++++ .../sink/dynamic/DynamicCommitterMetrics.java | 51 ++ .../sink/dynamic/DynamicIcebergSink.java | 439 +++++++++++++ .../sink/dynamic/DynamicKeySelector.java | 384 +++++++++++ .../flink/sink/dynamic/DynamicRecord.java | 128 ++++ .../sink/dynamic/DynamicRecordConverter.java | 31 + .../sink/dynamic/DynamicRecordInternal.java | 164 +++++ .../DynamicRecordInternalSerializer.java | 299 +++++++++ .../dynamic/DynamicRecordInternalType.java | 102 +++ .../sink/dynamic/DynamicRecordProcessor.java | 196 ++++++ .../dynamic/DynamicTableUpdateOperator.java | 71 ++ .../sink/dynamic/DynamicWriteResult.java | 41 ++ .../dynamic/DynamicWriteResultAggregator.java | 190 ++++++ .../dynamic/DynamicWriteResultSerializer.java | 63 ++ .../flink/sink/dynamic/DynamicWriter.java | 218 +++++++ .../sink/dynamic/DynamicWriterMetrics.java | 50 ++ .../sink/dynamic/EvolveSchemaVisitor.java | 184 ++++++ .../sink/dynamic/PartitionSpecAdjustment.java | 45 ++ .../sink/dynamic/PartitionSpecEvolver.java | 119 ++++ .../flink/sink/dynamic/RowDataEvolver.java | 169 +++++ .../sink/dynamic/RowDataSerializerCache.java | 102 +++ .../flink/sink/dynamic/TableDataCache.java | 257 ++++++++ .../flink/sink/dynamic/TableUpdater.java | 208 ++++++ .../flink/sink/dynamic/WriteTarget.java | 144 +++++ .../apache/iceberg/flink/SimpleDataUtil.java | 22 +- .../org/apache/iceberg/flink/TestHelpers.java | 2 +- .../flink/sink/TestFlinkIcebergSinkBase.java | 5 +- ...namicRecordInternalSerializerTestBase.java | 76 +++ ...rdInternalSerializerWriteSchemaIdTest.java | 41 ++ ...cordInternalSerializerWriteSchemaTest.java | 34 + .../dynamic/TestCompareSchemasVisitor.java | 209 ++++++ .../sink/dynamic/TestDynamicIcebergSink.java | 591 +++++++++++++++++ .../dynamic/TestDynamicIcebergSinkPerf.java | 226 +++++++ .../flink/sink/dynamic/TestDynamicWriter.java | 106 +++ .../sink/dynamic/TestEvolveSchemaVisitor.java | 607 ++++++++++++++++++ .../dynamic/TestPartitionSpecAdjustment.java | 57 ++ .../sink/dynamic/TestRowDataEvolver.java | 248 +++++++ .../sink/dynamic/TestTableDataCache.java | 85 +++ .../flink/sink/dynamic/TestTableUpdater.java | 90 +++ 65 files changed, 7145 insertions(+), 67 deletions(-) create mode 100644 flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java diff --git a/flink/v1.20/build.gradle b/flink/v1.20/build.gradle index 3e308d22b021..63b2d353263e 100644 --- a/flink/v1.20/build.gradle +++ b/flink/v1.20/build.gradle @@ -68,6 +68,9 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation libs.datasketches + // for caching in DynamicSink + implementation libs.caffeine + testImplementation libs.flink120.connector.test.utils testImplementation libs.flink120.core testImplementation libs.flink120.runtime diff --git a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java new file mode 100644 index 000000000000..2d102449b5b3 --- /dev/null +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java @@ -0,0 +1,137 @@ +/* + * 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.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class DynamicRecordSerializerDeserializerBenchmark { + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private List rows = Lists.newArrayListWithExpectedSize(SAMPLE_SIZE); + private DynamicRecordInternalType type; + + public static void main(String[] args) throws RunnerException { + Options options = + new OptionsBuilder() + .include(DynamicRecordSerializerDeserializerBenchmark.class.getSimpleName()) + .build(); + new Runner(options).run(); + } + + @Setup + public void setupBenchmark() throws IOException { + List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); + this.rows = + records.stream() + .map( + r -> + new DynamicRecordInternal( + "t", + "main", + SCHEMA, + PartitionSpec.unpartitioned(), + 1, + RowDataConverter.convert(SCHEMA, r), + false, + List.of())) + .collect(Collectors.toList()); + + File warehouse = Files.createTempFile("perf-bench", null).toFile(); + CatalogLoader catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getPath())); + this.type = new DynamicRecordInternalType(catalogLoader, true, 100); + } + + @Benchmark + @Threads(1) + public void testSerialize(Blackhole blackhole) throws IOException { + TypeSerializer serializer = + type.createSerializer((SerializerConfig) null); + DataOutputSerializer outputView = new DataOutputSerializer(1024); + for (int i = 0; i < SAMPLE_SIZE; ++i) { + serializer.serialize(rows.get(i), outputView); + } + } + + @Benchmark + @Threads(1) + public void testSerializeAndDeserialize(Blackhole blackhole) throws IOException { + TypeSerializer serializer = + type.createSerializer((SerializerConfig) null); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + for (int i = 0; i < SAMPLE_SIZE; ++i) { + serializer.serialize(rows.get(i), outputView); + serializer.deserialize(new DataInputDeserializer(outputView.getSharedBuffer())); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index d5eea6706b39..b9b15e4d730e 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -27,6 +27,7 @@ import org.apache.flink.util.TimeUtils; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; class FlinkConfParser { @@ -41,6 +42,12 @@ class FlinkConfParser { this.readableConfig = readableConfig; } + FlinkConfParser(Map options, ReadableConfig readableConfig) { + this.tableProperties = ImmutableMap.of(); + this.options = options; + this.readableConfig = readableConfig; + } + public BooleanConfParser booleanConf() { return new BooleanConfParser(); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index a31902d49a8b..d72c05e2ef1f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -55,6 +55,10 @@ public FlinkWriteConf( this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); } + public FlinkWriteConf(Map writeOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(writeOptions, readableConfig); + } + public boolean overwriteMode() { return confParser .booleanConf() diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index c33207728d3e..22dd40a61b9d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -26,7 +26,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.PartitionSpecVisitor; -final class BucketPartitionerUtil { +public final class BucketPartitionerUtil { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = "Invalid number of buckets: %s (must be 1)"; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 2109c91bddf7..62de3170d55c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index 036970c06d5b..89c26692b113 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java index 92ca284b12ba..59de599a6886 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 18b269d6c3e9..5b971415a9d9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.sink; import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.Schema; @@ -30,10 +29,10 @@ import org.apache.iceberg.util.StructProjection; /** - * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record - * will be emitted to same writer in order. + * Create a {@link NonThrowingKeySelector} to shuffle by equality fields, to ensure same equality + * fields record will be emitted to same writer in order. */ -class EqualityFieldKeySelector implements KeySelector { +public class EqualityFieldKeySelector implements NonThrowingKeySelector { private final Schema schema; private final RowType flinkSchema; @@ -43,7 +42,8 @@ class EqualityFieldKeySelector implements KeySelector { private transient StructProjection structProjection; private transient StructLikeWrapper structLikeWrapper; - EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { + public EqualityFieldKeySelector( + Schema schema, RowType flinkSchema, List equalityFieldIds) { this.schema = schema; this.flinkSchema = flinkSchema; this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index 9571efdc5268..d107c2739b04 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 408c3e9a9d5f..8b06949a5519 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java index e2b388a83c75..1d83c211e001 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java index 3048b51c6cb0..eebd2694986d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -57,7 +57,7 @@ * same jobId-operatorId-checkpointId triplet * */ -class IcebergCommitter implements Committer { +public class IcebergCommitter implements Committer { private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; public static final WriteResult EMPTY_WRITE_RESULT = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b510dce28bac..6051c8b60740 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -76,7 +76,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // the max committed one to iceberg table, for avoiding committing the same data files twice. This // id will be attached to iceberg's meta when committing the iceberg transaction. private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + public static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; // TableLoader to load iceberg table lazily. private final TableLoader tableLoader; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 5b28c4acb1c5..86af9c5154f7 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index 01be4a2eef71..e4982731bc38 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -583,7 +583,7 @@ private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table return (SerializableTable) SerializableTable.copyOf(table); } - private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + public static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { if (requestedSchema != null) { // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing // iceberg schema. diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ab458ad2e7cb..04ea868c0a38 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java index 794ade577976..2b3c53b0e31a 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -40,7 +40,8 @@ * IcebergCommittable} per checkpoint (storing the serialized {@link * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) */ -class IcebergWriteAggregator extends AbstractStreamOperator> +public class IcebergWriteAggregator + extends AbstractStreamOperator> implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 30517cd38216..81434ad171fb 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java new file mode 100644 index 000000000000..a9953ea8bd01 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import org.apache.flink.api.java.functions.KeySelector; + +/** A non-throwing variant of Flink's {@link KeySelector}. */ +public interface NonThrowingKeySelector extends KeySelector { + + @Override + K getKey(I value); +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java index df951684b446..f10dc710200d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.PartitionKey; @@ -27,11 +26,11 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be - * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy - * for {@link FlinkSink}. + * Create a {@link NonThrowingKeySelector} to shuffle by partition key, then each partition/bucket + * will be wrote by only one task. That will reduce lots of small files in partitioned fanout write + * policy for {@link FlinkSink}. */ -class PartitionKeySelector implements KeySelector { +public class PartitionKeySelector implements NonThrowingKeySelector { private final Schema schema; private final PartitionKey partitionKey; @@ -39,7 +38,7 @@ class PartitionKeySelector implements KeySelector { private transient RowDataWrapper rowDataWrapper; - PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + public PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { this.schema = schema; this.partitionKey = new PartitionKey(spec, schema); this.flinkSchema = flinkSchema; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 67422a1afeb1..8dc8d38869bc 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java index 5a44373cccaa..34868458b0e0 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java new file mode 100644 index 000000000000..99abc95d7102 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java @@ -0,0 +1,254 @@ +/* + * 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.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.schema.SchemaWithPartnerVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** Visitor class which compares two schemas and decides whether they are compatible. */ +public class CompareSchemasVisitor + extends SchemaWithPartnerVisitor { + + private final Schema tableSchema; + + private CompareSchemasVisitor(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + public static Result visit(Schema dataSchema, Schema tableSchema) { + return visit(dataSchema, tableSchema, true); + } + + public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + return visit( + dataSchema, + -1, + new CompareSchemasVisitor(tableSchema), + new PartnerIdByNameAccessors(tableSchema, caseSensitive)); + } + + @Override + public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return downstream; + } + + @Override + public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + Result result = fields.stream().reduce(Result::merge).orElse(Result.INCOMPATIBLE); + + if (result == Result.INCOMPATIBLE) { + return Result.INCOMPATIBLE; + } + + Type tableSchemaType = + tableSchemaId == -1 ? tableSchema.asStruct() : tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isStructType()) { + return Result.INCOMPATIBLE; + } + + if (struct.fields().size() != tableSchemaType.asStructType().fields().size()) { + return Result.CONVERSION_NEEDED; + } + + for (int i = 0; i < struct.fields().size(); ++i) { + if (!struct + .fields() + .get(i) + .name() + .equals(tableSchemaType.asStructType().fields().get(i).name())) { + return Result.CONVERSION_NEEDED; + } + } + + return result; + } + + @Override + public Result field(Types.NestedField field, Integer tableSchemaId, Result typeResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + if (typeResult != Result.SAME) { + return typeResult; + } + + if (tableSchema.findField(tableSchemaId).isRequired() && field.isOptional()) { + return Result.INCOMPATIBLE; + } else { + return Result.SAME; + } + } + + @Override + public Result list(Types.ListType list, Integer tableSchemaId, Result elementsResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return elementsResult; + } + + @Override + public Result map( + Types.MapType map, Integer tableSchemaId, Result keyResult, Result valueResult) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + return keyResult.merge(valueResult); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Result primitive(Type.PrimitiveType primitive, Integer tableSchemaId) { + if (tableSchemaId == null) { + return Result.INCOMPATIBLE; + } + + Type tableSchemaType = tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isPrimitiveType()) { + return Result.INCOMPATIBLE; + } + + Type.PrimitiveType tableSchemaPrimitiveType = tableSchemaType.asPrimitiveType(); + if (primitive.equals(tableSchemaPrimitiveType)) { + return Result.SAME; + } else if (primitive.equals(Types.IntegerType.get()) + && tableSchemaPrimitiveType.equals(Types.LongType.get())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.equals(Types.FloatType.get()) + && tableSchemaPrimitiveType.equals(Types.DoubleType.get())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.equals(Types.DateType.get()) + && tableSchemaPrimitiveType.equals(Types.TimestampType.withoutZone())) { + return Result.CONVERSION_NEEDED; + } else if (primitive.typeId() == Type.TypeID.DECIMAL + && tableSchemaPrimitiveType.typeId() == Type.TypeID.DECIMAL) { + Types.DecimalType dataType = (Types.DecimalType) primitive; + Types.DecimalType tableType = (Types.DecimalType) tableSchemaPrimitiveType; + return dataType.scale() == tableType.scale() && dataType.precision() < tableType.precision() + ? Result.CONVERSION_NEEDED + : Result.INCOMPATIBLE; + } else { + return Result.INCOMPATIBLE; + } + } + + static class PartnerIdByNameAccessors implements PartnerAccessors { + private final Schema tableSchema; + private boolean caseSensitive = true; + + PartnerIdByNameAccessors(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + private PartnerIdByNameAccessors(Schema tableSchema, boolean caseSensitive) { + this(tableSchema); + this.caseSensitive = caseSensitive; + } + + @Override + public Integer fieldPartner(Integer tableSchemaFieldId, int fieldId, String name) { + Types.StructType struct; + if (tableSchemaFieldId == -1) { + struct = tableSchema.asStruct(); + } else { + struct = tableSchema.findField(tableSchemaFieldId).type().asStructType(); + } + + Types.NestedField field = + caseSensitive ? struct.field(name) : struct.caseInsensitiveField(name); + if (field != null) { + return field.fieldId(); + } + + return null; + } + + @Override + public Integer mapKeyPartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(0).fieldId(); + } + + return null; + } + + @Override + public Integer mapValuePartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(1).fieldId(); + } + + return null; + } + + @Override + public Integer listElementPartner(Integer tableSchemaListId) { + Types.NestedField listField = tableSchema.findField(tableSchemaListId); + if (listField != null) { + return listField.type().asListType().fields().get(0).fieldId(); + } + + return null; + } + } + + public enum Result { + SAME(0), + CONVERSION_NEEDED(1), + INCOMPATIBLE(2); + + private static final Map BY_ID = Maps.newHashMap(); + + static { + for (Result e : Result.values()) { + if (BY_ID.put(e.id, e) != null) { + throw new IllegalArgumentException("Duplicate id: " + e.id); + } + } + } + + private final int id; + + Result(int id) { + this.id = id; + } + + private Result merge(Result other) { + return BY_ID.get(Math.max(this.id, other.id)); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java new file mode 100644 index 000000000000..5f26d3406cd1 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -0,0 +1,80 @@ +/* + * 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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java new file mode 100644 index 000000000000..c885e048127f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -0,0 +1,73 @@ +/* + * 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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java new file mode 100644 index 000000000000..205b74384710 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -0,0 +1,417 @@ +/* + * 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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java new file mode 100644 index 000000000000..8d815d6f6923 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.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.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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java new file mode 100644 index 000000000000..dd0442bc21ea --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java @@ -0,0 +1,439 @@ +/* + * 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.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; + +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** + * Dynamic version of the IcebergSink which supports: + * + *
    + *
  1. Writing to any number of tables (No more 1:1 sink/topic relationship). + *
  2. Creating and updating tables based on the user-supplied routing. + *
  3. Updating the schema and partition spec of tables based on the user-supplied specification. + *
+ */ +@Experimental +public class DynamicIcebergSink + implements Sink, + SupportsPreWriteTopology, + SupportsCommitter, + SupportsPreCommitTopology, + SupportsPostCommitTopology { + + private final CatalogLoader catalogLoader; + private final Map snapshotProperties; + private final String uidPrefix; + private final String sinkId; + private final Map writeProperties; + private final transient FlinkWriteConf flinkWriteConf; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final boolean overwriteMode; + private final int workerPoolSize; + + private DynamicIcebergSink( + CatalogLoader catalogLoader, + Map snapshotProperties, + String uidPrefix, + Map writeProperties, + FlinkWriteConf flinkWriteConf) { + this.catalogLoader = catalogLoader; + this.snapshotProperties = snapshotProperties; + this.uidPrefix = uidPrefix; + this.writeProperties = writeProperties; + this.flinkWriteConf = flinkWriteConf; + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.overwriteMode = flinkWriteConf.overwriteMode(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter createWriter(InitContext context) { + return new DynamicWriter( + catalogLoader.loadCatalog(), + dataFileFormat, + targetDataFileSize, + writeProperties, + new DynamicWriterMetrics(context.metricGroup()), + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); + } + + @Override + public Committer createCommitter(CommitterInitContext context) { + DynamicCommitterMetrics metrics = new DynamicCommitterMetrics(context.metricGroup()); + return new DynamicCommitter( + catalogLoader.loadCatalog(), + snapshotProperties, + overwriteMode, + workerPoolSize, + sinkId, + metrics); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new DynamicCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream> committables) {} + + @Override + public DataStream addPreWriteTopology( + DataStream inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream> addPreCommitTopology( + DataStream> writeResults) { + TypeInformation> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + return writeResults + .keyBy( + committable -> { + if (committable instanceof CommittableSummary) { + return "__summary"; + } else { + CommittableWithLineage result = + (CommittableWithLineage) committable; + return result.getCommittable().key().tableName(); + } + }) + .transform( + prefixIfNotNull(uidPrefix, sinkId + " Pre Commit"), + typeInformation, + new DynamicWriteResultAggregator(catalogLoader)) + .uid(prefixIfNotNull(uidPrefix, sinkId + "-pre-commit-topology")); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new DynamicWriteResultSerializer(); + } + + public static class Builder { + private DataStream input; + private DynamicRecordConverter converter; + private CatalogLoader catalogLoader; + private String uidPrefix = null; + private final Map writeOptions = Maps.newHashMap(); + private final Map snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private boolean immediateUpdate = false; + private int cacheMaximumSize = 100; + private long cacheRefreshMs = 1_000; + + private Builder() {} + + public Builder forInput(DataStream inputStream) { + this.input = inputStream; + return this; + } + + public Builder withConverter(DynamicRecordConverter inputConverter) { + this.converter = inputConverter; + return this; + } + + /** + * The catalog loader is used for loading tables in {@link DynamicCommitter} lazily, we need + * this loader because {@link Table} is not serializable and could not just use the loaded table + * from Builder#table in the remote task manager. + * + * @param newCatalogLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder catalogLoader(CatalogLoader newCatalogLoader) { + this.catalogLoader = newCatalogLoader; + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link DynamicIcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * Set the uid prefix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator) Actual operator uid will be appended + * with a suffix like "uidPrefix-writer". + * + *

If provided, this prefix is also applied to operator names. + * + *

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

Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newPrefix prefix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + + public Builder snapshotProperties(Map properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + public Builder immediateTableUpdate(boolean newImmediateUpdate) { + this.immediateUpdate = newImmediateUpdate; + return this; + } + + /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ + public Builder cacheMaxSize(int maxSize) { + this.cacheMaximumSize = maxSize; + return this; + } + + /** Maximum interval for cache items renewals. */ + public Builder cacheRefreshMs(long refreshMs) { + this.cacheRefreshMs = refreshMs; + return this; + } + + private String operatorName(String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + public DynamicIcebergSink build() { + + Preconditions.checkArgument( + converter != null, "Please use withConverter() to convert the input DataStream."); + Preconditions.checkNotNull(catalogLoader, "Catalog loader shouldn't be null"); + + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(writeOptions, readableConfig); + + Map writeProperties = + writeProperties(flinkWriteConf.dataFileFormat(), flinkWriteConf); + + uidPrefix = Optional.ofNullable(uidPrefix).orElse(""); + + // FlinkWriteConf properties needed to be set separately, so we do not have to serialize the + // full conf + return new DynamicIcebergSink( + catalogLoader, snapshotSummary, uidPrefix, writeProperties, flinkWriteConf); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + DynamicRecordInternalType type = + new DynamicRecordInternalType(catalogLoader, false, cacheMaximumSize); + DynamicIcebergSink sink = build(); + SingleOutputStreamOperator converted = + input + .process( + new DynamicRecordProcessor<>( + converter, catalogLoader, immediateUpdate, cacheMaximumSize, cacheRefreshMs)) + .uid(prefixIfNotNull(uidPrefix, "-converter")) + .name(operatorName("Converter")) + .returns(type); + + DataStreamSink rowDataDataStreamSink = + converted + .getSideOutput( + new OutputTag<>( + DynamicRecordProcessor.DYNAMIC_TABLE_UPDATE_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize))) + .keyBy((KeySelector) DynamicRecordInternal::tableName) + .map(new DynamicTableUpdateOperator(catalogLoader, cacheMaximumSize, cacheRefreshMs)) + .uid(prefixIfNotNull(uidPrefix, "-updater")) + .name(operatorName("Updater")) + .returns(type) + .union(converted) + .sinkTo(sink) + .uid(prefixIfNotNull(uidPrefix, "-sink")); + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + + return rowDataDataStreamSink; + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties(FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + DataStream distributeDataStream(DataStream input) { + return input.keyBy(DynamicRecordInternal::writerKey); + } + + private static String prefixIfNotNull(String uidPrefix, String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + /** + * Initialize a {@link IcebergSink.Builder} to export the data from input data stream with {@link + * RowData}s into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public static Builder forInput(DataStream input) { + return new Builder().forInput(input); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java new file mode 100644 index 000000000000..9a79be6838d2 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java @@ -0,0 +1,384 @@ +/* + * 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.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.EqualityFieldKeySelector; +import org.apache.iceberg.flink.sink.NonThrowingKeySelector; +import org.apache.iceberg.flink.sink.PartitionKeySelector; +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.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class DynamicKeySelector implements NonThrowingKeySelector { + private static final Logger LOG = LoggerFactory.getLogger(DynamicKeySelector.class); + + private final int maxWriteParallelism; + private final Cache> keySelectorCache; + + DynamicKeySelector(int maxCacheSize, int maxWriteParallelism) { + this.maxWriteParallelism = maxWriteParallelism; + this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); + } + + @Override + public Integer getKey(Input input) { + SelectorKey cacheKey = new SelectorKey(input); + return keySelectorCache + .get( + cacheKey, + k -> + getKeySelector( + input.tableName, + input.schema, + input.spec, + input.mode, + input.equalityFields, + input.writeParallelism)) + .getKey(input.rowData); + } + + public NonThrowingKeySelector getKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + DistributionMode mode, + List equalityFields, + int writeParallelism) { + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFields.isEmpty()) { + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + case HASH: + if (equalityFields.isEmpty()) { + if (spec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFields.contains(partitionField.name()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + schema.columns().stream() + .filter(c -> equalityFields.contains(c.name())) + .collect(Collectors.toList())); + } + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } + + case RANGE: + if (schema.identifierFieldIds().isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + default: + throw new IllegalArgumentException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + private static NonThrowingKeySelector equalityFieldKeySelector( + String tableName, + Schema schema, + List equalityFields, + int writeParallelism, + int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new EqualityFieldKeySelector( + schema, + FlinkSchemaUtil.convert(schema), + DynamicRecordProcessor.getEqualityFieldIds(equalityFields, schema)), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector partitionKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + int writeParallelism, + int maxWriteParallelism) { + NonThrowingKeySelector inner = + new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); + return new TargetLimitedKeySelector( + in -> inner.getKey(in).hashCode(), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector tableKeySelector( + String tableName, int writeParallelism, int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new RoundRobinKeySelector<>(writeParallelism), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + /** + * Generates a new key using the salt as a base, and reduces the target key range of the {@link + * #wrapped} {@link NonThrowingKeySelector} to {@link #writeParallelism}. + */ + private static class TargetLimitedKeySelector + implements NonThrowingKeySelector { + private final NonThrowingKeySelector wrapped; + private final int writeParallelism; + private final int[] distinctKeys; + + @SuppressWarnings("checkstyle:ParameterAssignment") + TargetLimitedKeySelector( + NonThrowingKeySelector wrapped, + int salt, + int writeParallelism, + int maxWriteParallelism) { + if (writeParallelism > maxWriteParallelism) { + LOG.warn( + "writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", + writeParallelism, + maxWriteParallelism, + maxWriteParallelism); + writeParallelism = maxWriteParallelism; + } + this.wrapped = wrapped; + this.writeParallelism = writeParallelism; + this.distinctKeys = new int[writeParallelism]; + + // Ensures that the generated keys are always result in unique slotId + Set targetSlots = Sets.newHashSetWithExpectedSize(writeParallelism); + int nextKey = salt; + for (int i = 0; i < writeParallelism; ++i) { + int subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); + while (targetSlots.contains(subtaskId)) { + ++nextKey; + subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); + } + + targetSlots.add(subtaskId); + distinctKeys[i] = nextKey; + ++nextKey; + } + } + + @Override + public Integer getKey(RowData value) { + return distinctKeys[Math.abs(wrapped.getKey(value).hashCode()) % writeParallelism]; + } + + private static int subtaskId(int key, int writeParallelism, int maxWriteParallelism) { + return KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxWriteParallelism, + writeParallelism, + KeyGroupRangeAssignment.computeKeyGroupForKeyHash(key, maxWriteParallelism)); + } + } + + /** + * Generates evenly distributed keys between [0..{@link #maxTarget}) range using round-robin + * algorithm. + * + * @param unused input for key generation + */ + private static class RoundRobinKeySelector implements NonThrowingKeySelector { + private final int maxTarget; + private int lastTarget = 0; + + RoundRobinKeySelector(int maxTarget) { + this.maxTarget = maxTarget; + } + + @Override + public Integer getKey(T value) { + lastTarget = (lastTarget + 1) % maxTarget; + return lastTarget; + } + } + + static class Input { + private final String tableName; + private final String branch; + private final Integer schemaId; + private final Integer specId; + private final Schema schema; + private final PartitionSpec spec; + private final DistributionMode mode; + private final int writeParallelism; + private final List equalityFields; + private final RowData rowData; + + Input( + DynamicRecord dynamicRecord, + Schema schemaOverride, + PartitionSpec specOverride, + RowData rowDataOverride) { + this( + dynamicRecord.tableIdentifier().toString(), + dynamicRecord.branch(), + schemaOverride != null ? schemaOverride.schemaId() : null, + specOverride != null ? specOverride.specId() : null, + schemaOverride, + specOverride, + dynamicRecord.mode(), + dynamicRecord.writeParallelism(), + dynamicRecord.equalityFields() != null + ? dynamicRecord.equalityFields() + : Collections.emptyList(), + rowDataOverride); + } + + private Input( + String tableName, + String branch, + Integer schemaId, + Integer specId, + Schema schema, + PartitionSpec spec, + DistributionMode mode, + int writeParallelism, + List equalityFields, + RowData rowData) { + this.tableName = tableName; + this.branch = branch; + this.schemaId = schemaId; + this.specId = specId; + this.schema = schema; + this.spec = spec; + this.mode = mode; + this.writeParallelism = writeParallelism; + this.equalityFields = equalityFields; + this.rowData = rowData; + } + } + + /** + * Cache key for the {@link NonThrowingKeySelector}. Only contains the {@link Schema} and the + * {@link PartitionSpec} if the ids are not available. + */ + private static class SelectorKey { + private final String tableName; + private final String branch; + private final Integer schemaId; + private final Integer specId; + private final Schema schema; + private final PartitionSpec spec; + private final List equalityFields; + + private SelectorKey(Input input) { + this.tableName = input.tableName; + this.branch = input.branch; + this.schemaId = input.schemaId; + this.schema = schemaId == null ? input.schema : null; + this.specId = input.specId; + this.spec = specId == null ? input.spec : null; + this.equalityFields = input.equalityFields; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + SelectorKey that = (SelectorKey) other; + return Objects.equals(tableName, that.tableName) + && Objects.equals(branch, that.branch) + && Objects.equals(schemaId, that.schemaId) + && Objects.equals(specId, that.specId) + && Objects.equals(schema, that.schema) + && Objects.equals(spec, that.spec) + && Objects.equals(equalityFields, that.equalityFields); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, branch, schemaId, specId, schema, spec, equalityFields); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", tableName) + .add("branch", branch) + .add("schemaId", schemaId) + .add("specId", specId) + .add("schema", schema) + .add("spec", spec) + .add("eqalityFields", equalityFields) + .toString(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java new file mode 100644 index 000000000000..193d9bfc7f66 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -0,0 +1,128 @@ +/* + * 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.List; +import javax.annotation.Nullable; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; + +public class DynamicRecord { + private TableIdentifier tableIdentifier; + private String branch; + private Schema schema; + private PartitionSpec spec; + private RowData rowData; + private DistributionMode mode; + private int writeParallelism; + private boolean upsertMode; + @Nullable private List equalityFields; + + public DynamicRecord( + TableIdentifier tableIdentifier, + String branch, + Schema schema, + PartitionSpec spec, + RowData rowData, + DistributionMode mode, + int writeParallelism) { + this.tableIdentifier = tableIdentifier; + this.branch = branch; + this.schema = schema; + this.spec = spec; + this.rowData = rowData; + this.mode = mode; + this.writeParallelism = writeParallelism; + } + + public TableIdentifier tableIdentifier() { + return tableIdentifier; + } + + public void setTableIdentifier(TableIdentifier tableIdentifier) { + this.tableIdentifier = tableIdentifier; + } + + public String branch() { + return branch; + } + + public void setBranch(String branch) { + this.branch = branch; + } + + public Schema schema() { + return schema; + } + + public void setSchema(Schema schema) { + this.schema = schema; + } + + public PartitionSpec spec() { + return spec; + } + + public void setSpec(PartitionSpec spec) { + this.spec = spec; + } + + public RowData rowData() { + return rowData; + } + + public void setRowData(RowData rowData) { + this.rowData = rowData; + } + + public DistributionMode mode() { + return mode; + } + + public void setMode(DistributionMode mode) { + this.mode = mode; + } + + public int writeParallelism() { + return writeParallelism; + } + + public void writeParallelism(int parallelism) { + this.writeParallelism = parallelism; + } + + public boolean upsertMode() { + return upsertMode; + } + + public void setUpsertMode(boolean upsertMode) { + this.upsertMode = upsertMode; + } + + public List equalityFields() { + return equalityFields; + } + + public void setEqualityFields(List equalityFields) { + this.equalityFields = equalityFields; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java new file mode 100644 index 000000000000..612b94ffb0cf --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java @@ -0,0 +1,31 @@ +/* + * 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.api.common.functions.OpenContext; +import org.apache.flink.util.Collector; + +/** Conversion method to return input type into a DynamicRecord */ +public interface DynamicRecordConverter extends Serializable { + default void open(OpenContext openContext) throws Exception {} + + /** Takes a user-defined input type and converts it one or multiple {@link DynamicRecord}s. */ + void convert(T inputRecord, Collector out) throws Exception; +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java new file mode 100644 index 000000000000..25e61bd20e81 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java @@ -0,0 +1,164 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +@Internal +class DynamicRecordInternal { + private String tableName; + private String branch; + private Schema schema; + private PartitionSpec spec; + private int writerKey; + private RowData rowData; + private boolean upsertMode; + private List equalityFieldIds; + + DynamicRecordInternal() {} + + DynamicRecordInternal( + String tableName, + String branch, + Schema schema, + PartitionSpec spec, + int writerKey, + RowData rowData, + boolean upsertMode, + List equalityFieldsIds) { + this.tableName = tableName; + this.branch = branch; + this.schema = schema; + this.spec = spec; + this.writerKey = writerKey; + this.rowData = rowData; + this.upsertMode = upsertMode; + this.equalityFieldIds = equalityFieldsIds; + } + + public String tableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String branch() { + return branch; + } + + public void setBranch(String branch) { + this.branch = branch; + } + + public Schema schema() { + return schema; + } + + public void setSchema(Schema schema) { + this.schema = schema; + } + + public PartitionSpec spec() { + return spec; + } + + public void setSpec(PartitionSpec spec) { + this.spec = spec; + } + + public int writerKey() { + return writerKey; + } + + public void setWriterKey(int writerKey) { + this.writerKey = writerKey; + } + + public RowData rowData() { + return rowData; + } + + public void setRowData(RowData rowData) { + this.rowData = rowData; + } + + public boolean upsertMode() { + return upsertMode; + } + + public void setUpsertMode(boolean upsertMode) { + this.upsertMode = upsertMode; + } + + public List equalityFields() { + return equalityFieldIds; + } + + public void setEqualityFieldIds(List equalityFieldIds) { + this.equalityFieldIds = equalityFieldIds; + } + + @Override + public int hashCode() { + return Objects.hash( + tableName, branch, schema, spec, writerKey, rowData, upsertMode, equalityFieldIds); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + DynamicRecordInternal that = (DynamicRecordInternal) other; + boolean tableFieldsMatch = + Objects.equals(tableName, that.tableName) + && Objects.equals(branch, that.branch) + && schema.schemaId() == that.schema.schemaId() + && Objects.equals(spec, that.spec) + && writerKey == that.writerKey + && upsertMode == that.upsertMode + && Objects.equals(equalityFieldIds, that.equalityFieldIds); + if (!tableFieldsMatch) { + return false; + } + + if (rowData.getClass().equals(that.rowData.getClass())) { + return Objects.equals(rowData, that.rowData); + } else { + RowDataSerializer rowDataSerializer = new RowDataSerializer(FlinkSchemaUtil.convert(schema)); + return rowDataSerializer + .toBinaryRow(rowData) + .equals(rowDataSerializer.toBinaryRow(that.rowData)); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java new file mode 100644 index 000000000000..250532dfec17 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -0,0 +1,299 @@ +/* + * 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.util.Collections; +import java.util.List; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +class DynamicRecordInternalSerializer extends TypeSerializer { + private static final long serialVersionUID = 1L; + + private final RowDataSerializerCache serializerCache; + private final boolean writeSchemaAndSpec; + + DynamicRecordInternalSerializer( + RowDataSerializerCache serializerCache, boolean writeSchemaAndSpec) { + this.serializerCache = serializerCache; + this.writeSchemaAndSpec = writeSchemaAndSpec; + } + + @Override + public TypeSerializer duplicate() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), + writeSchemaAndSpec); + } + + @Override + public DynamicRecordInternal createInstance() { + return new DynamicRecordInternal(); + } + + @Override + public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutputView) + throws IOException { + dataOutputView.writeUTF(toSerialize.tableName()); + dataOutputView.writeUTF(toSerialize.branch()); + if (writeSchemaAndSpec) { + dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); + } else { + dataOutputView.writeInt(toSerialize.schema().schemaId()); + dataOutputView.writeInt(toSerialize.spec().specId()); + } + dataOutputView.writeInt(toSerialize.writerKey()); + final Tuple3 rowDataSerializer; + if (writeSchemaAndSpec) { + rowDataSerializer = + serializerCache.serializer( + toSerialize.tableName(), toSerialize.schema(), toSerialize.spec(), null, null); + } else { + // Check that the schema id can be resolved. Not strictly necessary for serialization. + rowDataSerializer = + serializerCache.serializer( + toSerialize.tableName(), + null, + null, + toSerialize.schema().schemaId(), + toSerialize.spec().specId()); + } + rowDataSerializer.f0.serialize(toSerialize.rowData(), dataOutputView); + dataOutputView.writeBoolean(toSerialize.upsertMode()); + dataOutputView.writeInt(toSerialize.equalityFields().size()); + for (Integer equalityField : toSerialize.equalityFields()) { + dataOutputView.writeInt(equalityField); + } + } + + @Override + public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOException { + String tableName = dataInputView.readUTF(); + String branch = dataInputView.readUTF(); + Schema schema = null; + PartitionSpec spec = null; + Integer schemaId = null; + Integer specId = null; + if (writeSchemaAndSpec) { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); + } else { + schemaId = dataInputView.readInt(); + specId = dataInputView.readInt(); + } + + int writerKey = dataInputView.readInt(); + Tuple3 rowDataSerializer = + serializerCache.serializer(tableName, schema, spec, schemaId, specId); + RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); + boolean upsertMode = dataInputView.readBoolean(); + int numEqualityFields = dataInputView.readInt(); + final List equalityFieldIds; + if (numEqualityFields > 0) { + equalityFieldIds = Lists.newArrayList(); + } else { + equalityFieldIds = Collections.emptyList(); + } + for (int i = 0; i < numEqualityFields; i++) { + equalityFieldIds.add(dataInputView.readInt()); + } + return new DynamicRecordInternal( + tableName, + branch, + rowDataSerializer.f1, + rowDataSerializer.f2, + writerKey, + rowData, + upsertMode, + equalityFieldIds); + } + + @Override + public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputView dataInputView) + throws IOException { + String tableName = dataInputView.readUTF(); + reuse.setTableName(tableName); + String branch = dataInputView.readUTF(); + reuse.setBranch(branch); + + Schema schema = null; + PartitionSpec spec = null; + Integer schemaId = null; + Integer specId = null; + if (writeSchemaAndSpec) { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); + reuse.setSchema(schema); + reuse.setSpec(spec); + } else { + schemaId = dataInputView.readInt(); + specId = dataInputView.readInt(); + } + + int writerKey = dataInputView.readInt(); + reuse.setWriterKey(writerKey); + Tuple3 rowDataSerializer = + serializerCache.serializer(tableName, schema, spec, schemaId, specId); + RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); + boolean upsertMode = dataInputView.readBoolean(); + int numEqualityFields = dataInputView.readInt(); + final List equalityFieldIds; + if (numEqualityFields > 0) { + equalityFieldIds = Lists.newArrayList(); + } else { + equalityFieldIds = Collections.emptyList(); + } + for (int i = 0; i < numEqualityFields; i++) { + equalityFieldIds.add(dataInputView.readInt()); + } + return new DynamicRecordInternal( + tableName, + branch, + rowDataSerializer.f1, + rowDataSerializer.f2, + writerKey, + rowData, + upsertMode, + equalityFieldIds); + } + + @Override + public DynamicRecordInternal copy(DynamicRecordInternal from) { + return new DynamicRecordInternal( + from.tableName(), + from.branch(), + from.schema(), + from.spec(), + from.writerKey(), + from.rowData(), + from.upsertMode(), + from.equalityFields()); + } + + @Override + public DynamicRecordInternal copy(DynamicRecordInternal from, DynamicRecordInternal reuse) { + reuse.setTableName(from.tableName()); + reuse.setBranch(from.branch()); + reuse.setSchema(from.schema()); + reuse.setSpec(from.spec()); + reuse.setWriterKey(from.writerKey()); + reuse.setRowData(from.rowData()); + reuse.setUpsertMode(from.upsertMode()); + reuse.setEqualityFieldIds(from.equalityFields()); + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj instanceof DynamicRecordInternalSerializer) { + DynamicRecordInternalSerializer other = (DynamicRecordInternalSerializer) obj; + return writeSchemaAndSpec == other.writeSchemaAndSpec; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(writeSchemaAndSpec); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); + } + + public static class DynamicRecordInternalTypeSerializerSnapshot + implements TypeSerializerSnapshot { + + private boolean writeSchemaAndSpec; + + // Zero args constructor is required to instantiate this class on restore + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DynamicRecordInternalTypeSerializerSnapshot() {} + + DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { + this.writeSchemaAndSpec = writeSchemaAndSpec; + } + + @Override + public int getCurrentVersion() { + return 0; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + out.writeBoolean(writeSchemaAndSpec); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + this.writeSchemaAndSpec = in.readBoolean(); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + @Override + public TypeSerializer restoreSerializer() { + // Note: We pass in a null serializer cache which would create issues if we tried to use this + // restored serializer, but since we are using {@code + // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be + // used. A new one will be created via {@code DynamicRecordInternalType}. + return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java new file mode 100644 index 000000000000..8b22ca1a57cb --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java @@ -0,0 +1,102 @@ +/* + * 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.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.flink.CatalogLoader; + +@Internal +class DynamicRecordInternalType extends TypeInformation { + private final CatalogLoader catalogLoader; + private final boolean writeSchemaAndSpec; + private final int cacheSize; + + DynamicRecordInternalType( + CatalogLoader catalogLoader, boolean writeSchemaAndSpec, int cacheSize) { + this.catalogLoader = catalogLoader; + this.writeSchemaAndSpec = writeSchemaAndSpec; + this.cacheSize = cacheSize; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return DynamicRecordInternal.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(SerializerConfig serializerConfig) { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(catalogLoader, cacheSize), writeSchemaAndSpec); + } + + @Override + @Deprecated + public TypeSerializer createSerializer(ExecutionConfig config) { + return this.createSerializer(config.getSerializerConfig()); + } + + @Override + public String toString() { + return getClass().getName(); + } + + @Override + public boolean equals(Object o) { + return canEqual(o); + } + + @Override + public int hashCode() { + return getClass().getName().hashCode(); + } + + @Override + public boolean canEqual(Object o) { + return o instanceof DynamicRecordInternalType; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java new file mode 100644 index 000000000000..86536e24b08e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -0,0 +1,196 @@ +/* + * 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.Collections; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; + +@Internal +class DynamicRecordProcessor extends ProcessFunction + implements Collector { + static final String DYNAMIC_TABLE_UPDATE_STREAM = "dynamic-table-update-stream"; + + private final DynamicRecordConverter converter; + private final CatalogLoader catalogLoader; + private final boolean immediateUpdate; + private final int cacheMaximumSize; + private final long cacheRefreshMs; + + private transient TableDataCache tableCache; + private transient DynamicKeySelector selector; + private transient TableUpdater updater; + private transient OutputTag updateStream; + private transient Collector collector; + private transient Context context; + + DynamicRecordProcessor( + DynamicRecordConverter converter, + CatalogLoader catalogLoader, + boolean immediateUpdate, + int cacheMaximumSize, + long cacheRefreshMs) { + this.converter = converter; + this.catalogLoader = catalogLoader; + this.immediateUpdate = immediateUpdate; + this.cacheMaximumSize = cacheMaximumSize; + this.cacheRefreshMs = cacheRefreshMs; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + Catalog catalog = catalogLoader.loadCatalog(); + this.tableCache = new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs); + this.selector = + new DynamicKeySelector( + cacheMaximumSize, getRuntimeContext().getTaskInfo().getMaxNumberOfParallelSubtasks()); + if (immediateUpdate) { + updater = new TableUpdater(tableCache, catalog); + } + + updateStream = + new OutputTag<>( + DYNAMIC_TABLE_UPDATE_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize)) {}; + + converter.open(openContext); + } + + @Override + public void processElement(T element, Context ctx, Collector out) + throws Exception { + this.context = ctx; + this.collector = out; + converter.convert(element, this); + } + + @Override + public void collect(DynamicRecord data) { + boolean exists = tableCache.exists(data.tableIdentifier()).f0; + String foundBranch = exists ? tableCache.branch(data.tableIdentifier(), data.branch()) : null; + + Tuple2 foundSchema = + exists + ? tableCache.schema(data.tableIdentifier(), data.schema()) + : TableDataCache.NOT_FOUND; + + adjustPartitionSpecToTableSchema(foundSchema.f0, data); + PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; + + if (!exists + || foundBranch == null + || foundSpec == null + || foundSchema.f1 == CompareSchemasVisitor.Result.INCOMPATIBLE) { + if (immediateUpdate) { + Tuple3 newData = + updater.update(data.tableIdentifier(), data.branch(), data.schema(), data.spec()); + emit(collector, data, newData.f0, newData.f1, newData.f2); + } else { + int writerKey = + selector.getKey( + new DynamicKeySelector.Input( + data, + foundSchema.f0 != null ? foundSchema.f0 : data.schema(), + foundSpec != null ? foundSpec : data.spec(), + data.rowData())); + context.output( + updateStream, + new DynamicRecordInternal( + data.tableIdentifier().toString(), + data.branch(), + data.schema(), + data.spec(), + writerKey, + data.rowData(), + data.upsertMode(), + getEqualityFieldIds(data.equalityFields(), data.schema()))); + } + } else { + emit(collector, data, foundSchema.f0, foundSchema.f1, foundSpec); + } + } + + private static void adjustPartitionSpecToTableSchema(Schema schema, DynamicRecord data) { + if (schema != null) { + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(schema, data.spec()); + data.setSpec(adjustedSpec); + } + } + + private void emit( + Collector out, + DynamicRecord data, + Schema schema, + CompareSchemasVisitor.Result result, + PartitionSpec spec) { + RowData rowData = + result == CompareSchemasVisitor.Result.SAME + ? data.rowData() + : RowDataEvolver.convert(data.rowData(), data.schema(), schema); + int writerKey = selector.getKey(new DynamicKeySelector.Input(data, schema, spec, rowData)); + String tableName = data.tableIdentifier().toString(); + out.collect( + new DynamicRecordInternal( + tableName, + data.branch(), + schema, + spec, + writerKey, + rowData, + data.upsertMode(), + getEqualityFieldIds(data.equalityFields(), schema))); + } + + static List getEqualityFieldIds(List equalityFields, Schema schema) { + if (equalityFields == null || equalityFields.isEmpty()) { + if (!schema.identifierFieldIds().isEmpty()) { + return Lists.newArrayList(schema.identifierFieldIds()); + } else { + return Collections.emptyList(); + } + } + List equalityFieldIds = Lists.newArrayList(equalityFields.size()); + for (String equalityField : equalityFields) { + Types.NestedField field = schema.findField(equalityField); + Preconditions.checkNotNull( + field, "Equality field %s does not exist in schema", equalityField); + equalityFieldIds.add(field.fieldId()); + } + return equalityFieldIds; + } + + @Override + public void close() {} +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java new file mode 100644 index 000000000000..17ee853104b6 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -0,0 +1,71 @@ +/* + * 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.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; + +@Internal +class DynamicTableUpdateOperator + extends RichMapFunction { + private final CatalogLoader catalogLoader; + private final int cacheMaximumSize; + private final long cacheRefreshMs; + private transient TableUpdater updater; + + DynamicTableUpdateOperator( + CatalogLoader catalogLoader, int cacheMaximumSize, long cacheRefreshMs) { + this.catalogLoader = catalogLoader; + this.cacheMaximumSize = cacheMaximumSize; + this.cacheRefreshMs = cacheRefreshMs; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + Catalog catalog = catalogLoader.loadCatalog(); + this.updater = + new TableUpdater(new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs), catalog); + } + + @Override + public DynamicRecordInternal map(DynamicRecordInternal data) throws Exception { + Tuple3 newData = + updater.update( + TableIdentifier.parse(data.tableName()), data.branch(), data.schema(), data.spec()); + + data.setSchema(newData.f0); + data.setSpec(newData.f2); + + if (newData.f1 == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { + RowData newRowData = RowDataEvolver.convert(data.rowData(), data.schema(), newData.f0); + data.setRowData(newRowData); + } + + return data; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java new file mode 100644 index 000000000000..f7f6af86e39a --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java @@ -0,0 +1,41 @@ +/* + * 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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java new file mode 100644 index 000000000000..566cfb9963b7 --- /dev/null +++ b/flink/v1.20/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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java new file mode 100644 index 000000000000..4432abd19199 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java new file mode 100644 index 000000000000..bc54898bd644 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -0,0 +1,218 @@ +/* + * 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 static final int CACHE_MAXIMUM_SIZE = 100; + + 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, + 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(CACHE_MAXIMUM_SIZE).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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java new file mode 100644 index 000000000000..d983fb320d35 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -0,0 +1,50 @@ +/* + * 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/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java new file mode 100644 index 000000000000..6b020a5bda73 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java @@ -0,0 +1,184 @@ +/* + * 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.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.schema.SchemaWithPartnerVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * Visitor class that accumulates the set of changes needed to evolve an existing schema into the + * target schema. Changes are applied to an {@link UpdateSchema} operation. + */ +public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { + + private final UpdateSchema api; + private final Schema existingSchema; + private final Schema targetSchema; + + private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + this.api = api; + this.existingSchema = existingSchema; + this.targetSchema = targetSchema; + } + + /** + * Adds changes needed to produce the target schema to an {@link UpdateSchema} operation. + * + *

Changes are accumulated to evolve the existingSchema into a targetSchema. + * + * @param api an UpdateSchema for adding changes + * @param existingSchema an existing schema + * @param targetSchema a new schema to compare with the existing + */ + public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + visit( + targetSchema, + -1, + new EvolveSchemaVisitor(api, existingSchema, targetSchema), + new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); + } + + @Override + public Boolean struct(Types.StructType struct, Integer partnerId, List existingFields) { + if (partnerId == null) { + return true; + } + + // Add, update and order fields in the struct + Types.StructType partnerStruct = findFieldType(partnerId).asStructType(); + String after = null; + for (Types.NestedField targetField : struct.fields()) { + Types.NestedField nestedField = partnerStruct.field(targetField.name()); + final String columnName; + if (nestedField != null) { + updateColumn(nestedField, targetField); + columnName = this.existingSchema.findColumnName(nestedField.fieldId()); + } else { + addColumn(partnerId, targetField); + columnName = this.targetSchema.findColumnName(targetField.fieldId()); + } + + setPosition(columnName, after); + after = columnName; + } + + // Ensure that unused fields are made optional + for (Types.NestedField existingField : partnerStruct.fields()) { + if (struct.field(existingField.name()) == null) { + if (existingField.isRequired()) { + this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); + } + } + } + + return false; + } + + @Override + public Boolean field(Types.NestedField field, Integer partnerId, Boolean isFieldMissing) { + return partnerId == null; + } + + @Override + public Boolean list(Types.ListType list, Integer partnerId, Boolean isElementMissing) { + if (partnerId == null) { + return true; + } + + Preconditions.checkState( + !isElementMissing, "Error traversing schemas: element is missing, but list is present"); + + Types.ListType partnerList = findFieldType(partnerId).asListType(); + updateColumn(partnerList.fields().get(0), list.fields().get(0)); + + return false; + } + + @Override + public Boolean map( + Types.MapType map, Integer partnerId, Boolean isKeyMissing, Boolean isValueMissing) { + if (partnerId == null) { + return true; + } + + Preconditions.checkState( + !isKeyMissing, "Error traversing schemas: key is missing, but map is present"); + Preconditions.checkState( + !isValueMissing, "Error traversing schemas: value is missing, but map is present"); + + Types.MapType partnerMap = findFieldType(partnerId).asMapType(); + updateColumn(partnerMap.fields().get(0), map.fields().get(0)); + updateColumn(partnerMap.fields().get(1), map.fields().get(1)); + + return false; + } + + @Override + public Boolean primitive(Type.PrimitiveType primitive, Integer partnerId) { + return partnerId == null; + } + + private Type findFieldType(int fieldId) { + if (fieldId == -1) { + return existingSchema.asStruct(); + } else { + return existingSchema.findField(fieldId).type(); + } + } + + private void addColumn(int parentId, Types.NestedField field) { + String parentName = targetSchema.findColumnName(parentId); + api.addColumn(parentName, field.name(), field.type(), field.doc()); + } + + private void updateColumn(Types.NestedField existingField, Types.NestedField targetField) { + String existingColumnName = this.existingSchema.findColumnName(existingField.fieldId()); + + boolean needsOptionalUpdate = targetField.isOptional() && existingField.isRequired(); + boolean needsTypeUpdate = + targetField.type().isPrimitiveType() && !targetField.type().equals(existingField.type()); + boolean needsDocUpdate = + targetField.doc() != null && !targetField.doc().equals(existingField.doc()); + + if (needsOptionalUpdate) { + api.makeColumnOptional(existingColumnName); + } + + if (needsTypeUpdate) { + api.updateColumn(existingColumnName, targetField.type().asPrimitiveType()); + } + + if (needsDocUpdate) { + api.updateColumnDoc(existingColumnName, targetField.doc()); + } + } + + private void setPosition(String columnName, String after) { + if (after == null) { + this.api.moveFirst(columnName); + } else { + this.api.moveAfter(columnName, after); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java new file mode 100644 index 000000000000..12fa3f9e7f3b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.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 org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.UnboundPartitionSpec; + +public class PartitionSpecAdjustment { + + private PartitionSpecAdjustment() {} + + public static PartitionSpec adjustPartitionSpecToTableSchema( + Schema tableSchema, PartitionSpec userSpec) { + if (userSpec.isUnpartitioned()) { + return userSpec; + } + UnboundPartitionSpec.Builder builder = + UnboundPartitionSpec.builder().withSpecId(userSpec.specId()); + for (PartitionField field : userSpec.fields()) { + String sourceFieldName = userSpec.schema().idToName().get(field.sourceId()); + int adjustedSourceId = tableSchema.findField(sourceFieldName).fieldId(); + builder.addField( + field.transform().toString(), adjustedSourceId, field.fieldId(), field.name()); + } + return builder.build().bind(tableSchema); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java new file mode 100644 index 000000000000..e3746b82ce9f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.List; +import java.util.Objects; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.Term; +import org.apache.iceberg.expressions.UnboundTransform; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Checks compatibility of PartitionSpecs and evolves one into the other. */ +public class PartitionSpecEvolver { + + private PartitionSpecEvolver() {} + + /** + * Checks whether two PartitionSpecs are compatible with each other. Less strict than {@code + * PartitionSpec#compatible} in the sense that it tolerates differently named partition fields, as + * long as their transforms and source ids match. + */ + public static boolean checkCompatibility(PartitionSpec first, PartitionSpec second) { + if (first.equals(second)) { + return true; + } + + if (first.fields().size() != second.fields().size()) { + return false; + } + + for (int i = 0; i < first.fields().size(); i++) { + PartitionField firstField = first.fields().get(i); + PartitionField secondField = second.fields().get(i); + if (firstField.sourceId() != secondField.sourceId() + || !firstField.transform().toString().equals(secondField.transform().toString())) { + return false; + } + } + + return true; + } + + public static PartitionSpecEvolverResult evolve( + PartitionSpec currentSpec, PartitionSpec targetSpec, Schema schema) { + if (currentSpec.compatibleWith(targetSpec)) { + return new PartitionSpecEvolverResult(); + } + + PartitionSpecEvolverResult result = new PartitionSpecEvolverResult(); + + int maxNumFields = Math.max(currentSpec.fields().size(), targetSpec.fields().size()); + for (int i = 0; i < maxNumFields; i++) { + PartitionField currentField = Iterables.get(currentSpec.fields(), i, null); + PartitionField targetField = Iterables.get(targetSpec.fields(), i, null); + + if (!Objects.equals(currentField, targetField)) { + if (currentField != null) { + result.remove(toTerm(currentField, schema)); + } + + if (targetField != null) { + result.add(toTerm(targetField, schema)); + } + } + } + + return result; + } + + public static class PartitionSpecEvolverResult { + private final List termsToAdd = Lists.newArrayList(); + private final List termsToRemove = Lists.newArrayList(); + + public void add(Term term) { + termsToAdd.add(term); + } + + public void remove(Term term) { + termsToRemove.add(term); + } + + public List termsToAdd() { + return termsToAdd; + } + + public List termsToRemove() { + return termsToRemove; + } + + public boolean isEmpty() { + return termsToAdd.isEmpty() && termsToRemove.isEmpty(); + } + } + + private static Term toTerm(PartitionField field, Schema schema) { + String sourceName = schema.findField(field.sourceId()).name(); + return new UnboundTransform<>(new NamedReference<>(sourceName), field.transform()); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java new file mode 100644 index 000000000000..f5f7cd2c4c3d --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java @@ -0,0 +1,169 @@ +/* + * 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.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class RowDataEvolver { + private RowDataEvolver() {} + + public static RowData convert(RowData sourceData, Schema sourceSchema, Schema targetSchema) { + return convertStruct( + sourceData, FlinkSchemaUtil.convert(sourceSchema), FlinkSchemaUtil.convert(targetSchema)); + } + + private static Object convert(Object object, LogicalType sourceType, LogicalType targetType) { + if (object == null) { + return null; + } + + switch (targetType.getTypeRoot()) { + case BOOLEAN: + case INTEGER: + case FLOAT: + case VARCHAR: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case BINARY: + case VARBINARY: + return object; + case DOUBLE: + if (object instanceof Float) { + return ((Float) object).doubleValue(); + } else { + return object; + } + case BIGINT: + if (object instanceof Integer) { + return ((Integer) object).longValue(); + } else { + return object; + } + case DECIMAL: + DecimalType toDecimalType = (DecimalType) targetType; + DecimalData decimalData = (DecimalData) object; + if (((DecimalType) sourceType).getPrecision() == toDecimalType.getPrecision()) { + return object; + } else { + return DecimalData.fromBigDecimal( + decimalData.toBigDecimal(), toDecimalType.getPrecision(), toDecimalType.getScale()); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + if (object instanceof Integer) { + LocalDateTime dateTime = + LocalDateTime.of(LocalDate.ofEpochDay((Integer) object), LocalTime.MIN); + return TimestampData.fromLocalDateTime(dateTime); + } else { + return object; + } + case ROW: + return convertStruct((RowData) object, (RowType) sourceType, (RowType) targetType); + case ARRAY: + return convertArray((ArrayData) object, (ArrayType) sourceType, (ArrayType) targetType); + case MAP: + return convertMap((MapData) object, (MapType) sourceType, (MapType) targetType); + default: + throw new UnsupportedOperationException("Not a supported type: " + targetType); + } + } + + private static RowData convertStruct(RowData sourceData, RowType sourceType, RowType targetType) { + GenericRowData targetData = new GenericRowData(targetType.getFields().size()); + List targetFields = targetType.getFields(); + for (int i = 0; i < targetFields.size(); i++) { + RowType.RowField targetField = targetFields.get(i); + + int sourceFieldId = sourceType.getFieldIndex(targetField.getName()); + if (sourceFieldId == -1) { + if (targetField.getType().isNullable()) { + targetData.setField(i, null); + } else { + throw new IllegalArgumentException( + String.format( + "Field %s in target schema %s is non-nullable but does not exist in source schema.", + i + 1, targetType)); + } + } else { + RowData.FieldGetter getter = + RowData.createFieldGetter(sourceType.getTypeAt(sourceFieldId), sourceFieldId); + targetData.setField( + i, + convert( + getter.getFieldOrNull(sourceData), + sourceType.getFields().get(sourceFieldId).getType(), + targetField.getType())); + } + } + + return targetData; + } + + private static ArrayData convertArray( + ArrayData sourceData, ArrayType sourceType, ArrayType targetType) { + LogicalType fromElementType = sourceType.getElementType(); + LogicalType toElementType = targetType.getElementType(); + ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(fromElementType); + Object[] convertedArray = new Object[sourceData.size()]; + for (int i = 0; i < convertedArray.length; i++) { + convertedArray[i] = + convert(elementGetter.getElementOrNull(sourceData, i), fromElementType, toElementType); + } + return new GenericArrayData(convertedArray); + } + + private static MapData convertMap(MapData sourceData, MapType sourceType, MapType targetType) { + LogicalType fromMapKeyType = sourceType.getKeyType(); + LogicalType fromMapValueType = sourceType.getValueType(); + LogicalType toMapKeyType = targetType.getKeyType(); + LogicalType toMapValueType = targetType.getValueType(); + ArrayData keyArray = sourceData.keyArray(); + ArrayData valueArray = sourceData.valueArray(); + ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(fromMapKeyType); + ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(fromMapValueType); + Map convertedMap = Maps.newLinkedHashMap(); + for (int i = 0; i < keyArray.size(); ++i) { + convertedMap.put( + convert(keyGetter.getElementOrNull(keyArray, i), fromMapKeyType, toMapKeyType), + convert(valueGetter.getElementOrNull(valueArray, i), fromMapValueType, toMapValueType)); + } + + return new GenericMapData(convertedMap); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java new file mode 100644 index 000000000000..68269a32ee6e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java @@ -0,0 +1,102 @@ +/* + * 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.Serializable; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class RowDataSerializerCache implements Serializable { + private final CatalogLoader catalogLoader; + private final int maximumSize; + private transient Cache serializers; + + RowDataSerializerCache(CatalogLoader catalogLoader, int maximumSize) { + this.catalogLoader = catalogLoader; + this.maximumSize = maximumSize; + } + + Tuple3 serializer( + String tableName, + Schema unknownSchema, + PartitionSpec unknownSpec, + Integer schemaId, + Integer specId) { + if (serializers == null) { + // We need to initialize the cache at the first time + this.serializers = Caffeine.newBuilder().maximumSize(maximumSize).build(); + } + + SerializerInfo info = serializers.get(tableName, SerializerInfo::new); + Schema schema = unknownSchema != null ? unknownSchema : info.schemas.get(schemaId); + PartitionSpec spec = unknownSpec != null ? unknownSpec : info.specs.get(specId); + + if (schema == null || spec == null) { + info.update(); + schema = info.schemas.get(schemaId); + spec = info.specs.get(specId); + } + + RowDataSerializer serializer = + info.serializers.computeIfAbsent( + schema, s -> new RowDataSerializer(FlinkSchemaUtil.convert(s))); + + return Tuple3.of(serializer, schema, spec); + } + + CatalogLoader catalogLoader() { + return catalogLoader; + } + + int maximumSize() { + return maximumSize; + } + + private class SerializerInfo { + private final String tableName; + private final Map serializers; + private Map schemas; + private Map specs; + + SerializerInfo(String tableName) { + this.tableName = tableName; + this.serializers = Maps.newHashMapWithExpectedSize(2); + this.schemas = Maps.newHashMapWithExpectedSize(0); + this.specs = Maps.newHashMapWithExpectedSize(0); + } + + private void update() { + Table table = catalogLoader.loadCatalog().loadTable(TableIdentifier.parse(tableName)); + schemas = table.schemas(); + specs = table.specs(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java new file mode 100644 index 000000000000..a86a4b878405 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java @@ -0,0 +1,257 @@ +/* + * 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.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TableDataCache { + private static final Logger LOG = LoggerFactory.getLogger(TableDataCache.class); + private static final int MAX_SIZE = 10; + private static final Tuple2 EXISTS = Tuple2.of(true, null); + private static final Tuple2 NOT_EXISTS = Tuple2.of(false, null); + static final Tuple2 NOT_FOUND = + Tuple2.of(null, CompareSchemasVisitor.Result.INCOMPATIBLE); + + private final Catalog catalog; + private final long refreshMs; + private final Cache cache; + + TableDataCache(Catalog catalog, int maximumSize, long refreshMs) { + this.catalog = catalog; + this.refreshMs = refreshMs; + this.cache = Caffeine.newBuilder().maximumSize(maximumSize).build(); + } + + Tuple2 exists(TableIdentifier identifier) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { + return EXISTS; + } else if (needsRefresh(cached, true)) { + return refreshTable(identifier); + } else { + return NOT_EXISTS; + } + } + + String branch(TableIdentifier identifier, String branch) { + return branch(identifier, branch, true); + } + + Tuple2 schema(TableIdentifier identifier, Schema input) { + return schema(identifier, input, true); + } + + PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { + return spec(identifier, spec, true); + } + + void update(TableIdentifier identifier, Table table) { + cache.put( + identifier, + new CacheItem(true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); + } + + private String branch(TableIdentifier identifier, String branch, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && cached.tableExists && cached.branches.contains(branch)) { + return branch; + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return branch(identifier, branch, false); + } else { + return null; + } + } + + private Tuple2 schema( + TableIdentifier identifier, Schema input, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + Schema compatible = null; + if (cached != null && cached.tableExists) { + // This only works if the {@link Schema#equals(Object)} returns true for the old schema + // and a new schema. Performance is paramount as this code is on the hot path. Every other + // way for comparing 2 schemas were performing worse than the + // {@link CompareByNameVisitor#visit(Schema, Schema, boolean)}, so caching was useless. + Tuple2 lastResult = + cached.schema.lastResults.get(input); + if (lastResult != null) { + return lastResult; + } + + for (Map.Entry tableSchema : cached.schema.schemas.entrySet()) { + CompareSchemasVisitor.Result result = + CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); + if (result == CompareSchemasVisitor.Result.SAME) { + Tuple2 newResult = + Tuple2.of(tableSchema.getValue(), CompareSchemasVisitor.Result.SAME); + cached.schema.update(input, newResult); + return newResult; + } else if (compatible == null && result == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { + compatible = tableSchema.getValue(); + } + } + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return schema(identifier, input, false); + } else if (compatible != null) { + Tuple2 newResult = + Tuple2.of(compatible, CompareSchemasVisitor.Result.CONVERSION_NEEDED); + cached.schema.update(input, newResult); + return newResult; + } else if (cached != null && cached.tableExists) { + cached.schema.update(input, NOT_FOUND); + return NOT_FOUND; + } else { + return NOT_FOUND; + } + } + + private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boolean allowRefresh) { + CacheItem cached = cache.getIfPresent(identifier); + if (cached != null && cached.tableExists) { + for (PartitionSpec tableSpec : cached.specs.values()) { + if (PartitionSpecEvolver.checkCompatibility(tableSpec, spec)) { + return tableSpec; + } + } + } + + if (needsRefresh(cached, allowRefresh)) { + refreshTable(identifier); + return spec(identifier, spec, false); + } else { + return null; + } + } + + private Tuple2 refreshTable(TableIdentifier identifier) { + try { + Table table = catalog.loadTable(identifier); + cache.put( + identifier, + new CacheItem( + true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); + return EXISTS; + } catch (NoSuchTableException e) { + LOG.debug("Table doesn't exist {}", identifier, e); + cache.put(identifier, new CacheItem(false, null, null, null)); + return Tuple2.of(false, e); + } + } + + private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { + return allowRefresh + && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); + } + + public void invalidate(TableIdentifier identifier) { + cache.invalidate(identifier); + } + + /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ + static class CacheItem { + private final long created = System.currentTimeMillis(); + + private final boolean tableExists; + private final Set branches; + private final SchemaInfo schema; + private final Map specs; + + private CacheItem( + boolean tableExists, + Set branches, + SchemaInfo schema, + Map specs) { + this.tableExists = tableExists; + this.branches = branches; + this.schema = schema; + this.specs = specs; + } + + @VisibleForTesting + SchemaInfo getSchemaInfo() { + return schema; + } + } + + /** + * Stores precalculated results for {@link CompareSchemasVisitor#visit(Schema, Schema, boolean)} + * in the cache. + */ + static class SchemaInfo { + private final Map schemas; + private final Map> lastResults; + + private SchemaInfo(Map schemas) { + this.schemas = schemas; + this.lastResults = new LimitedLinkedHashMap<>(); + } + + private void update( + Schema newLastSchema, Tuple2 newLastResult) { + lastResults.put(newLastSchema, newLastResult); + } + + @VisibleForTesting + Tuple2 getLastResult(Schema schema) { + return lastResults.get(schema); + } + } + + @SuppressWarnings("checkstyle:IllegalType") + private static class LimitedLinkedHashMap extends LinkedHashMap { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + boolean remove = size() > MAX_SIZE; + if (remove) { + LOG.warn( + "Performance degraded as records with different schema is generated for the same table. " + + "Likely the DynamicRecord.schema is not reused. " + + "Reuse the same instance if the record schema is the same to improve performance"); + } + + return remove; + } + } + + @VisibleForTesting + Cache getInternalCache() { + return cache; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java new file mode 100644 index 000000000000..7be1d247f5d1 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -0,0 +1,208 @@ +/* + * 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.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TableUpdater { + private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); + private final TableDataCache cache; + private final Catalog catalog; + + TableUpdater(TableDataCache cache, Catalog catalog) { + this.cache = cache; + this.catalog = catalog; + } + + /** + * Creates or updates a table to make sure that the given branch, schema, spec exists. + * + * @return a {@link Tuple3} of the new {@link Schema}, the status of the schema compared to the + * requested one, and the new {@link PartitionSpec#specId()}. + */ + Tuple3 update( + TableIdentifier tableIdentifier, String branch, Schema schema, PartitionSpec spec) { + findOrCreateTable(tableIdentifier, schema, spec); + findOrCreateBranch(tableIdentifier, branch); + Tuple2 newSchema = + findOrCreateSchema(tableIdentifier, schema); + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(newSchema.f0, spec); + PartitionSpec newSpec = findOrCreateSpec(tableIdentifier, adjustedSpec, newSchema.f0); + return Tuple3.of(newSchema.f0, newSchema.f1, newSpec); + } + + private void findOrCreateTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { + Tuple2 exists = cache.exists(identifier); + if (Boolean.FALSE.equals(exists.f0)) { + if (exists.f1 instanceof NoSuchNamespaceException) { + SupportsNamespaces catalogWithNameSpace = (SupportsNamespaces) catalog; + LOG.info("Namespace {} not found during table search. Creating namespace", identifier); + try { + catalogWithNameSpace.createNamespace(identifier.namespace()); + } catch (AlreadyExistsException e) { + LOG.debug("Namespace {} created concurrently", identifier.namespace(), e); + } + + createTable(identifier, schema, spec); + } else { + LOG.info("Table {} not found during table search. Creating table.", identifier); + createTable(identifier, schema, spec); + } + } + } + + private void createTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { + try { + Table table = catalog.createTable(identifier, schema, spec); + cache.update(identifier, table); + } catch (AlreadyExistsException e) { + LOG.info("Table {} created concurrently. Skipping creation.", identifier, e); + } + } + + private void findOrCreateBranch(TableIdentifier identifier, String branch) { + String fromCache = cache.branch(identifier, branch); + if (fromCache == null) { + try { + // TODO: Which snapshot should be used to create the branch? + catalog.loadTable(identifier).manageSnapshots().createBranch(branch).commit(); + LOG.info("Branch {} for {} created", branch, identifier); + } catch (Exception e) { + LOG.info( + "Failed to create branch {} for {}. Maybe created concurrently?", + branch, + identifier, + e); + } + } + } + + private Tuple2 findOrCreateSchema( + TableIdentifier identifier, Schema schema) { + Tuple2 fromCache = cache.schema(identifier, schema); + if (fromCache.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { + return fromCache; + } else { + Table table = catalog.loadTable(identifier); + Schema tableSchema = table.schema(); + CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); + switch (result) { + case SAME: + case CONVERSION_NEEDED: + cache.update(identifier, table); + return Tuple2.of(tableSchema, result); + case INCOMPATIBLE: + LOG.info( + "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); + UpdateSchema updateApi = table.updateSchema(); + EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); + + try { + updateApi.commit(); + cache.invalidate(identifier); + Tuple2 comparisonAfterMigration = + cache.schema(identifier, schema); + Schema newSchema = comparisonAfterMigration.f0; + LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); + return comparisonAfterMigration; + } catch (CommitFailedException e) { + LOG.info( + "Schema update failed for {} from {} to {}", identifier, tableSchema, schema, e); + Tuple2 newSchema = + cache.schema(identifier, schema); + if (newSchema.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { + LOG.info("Table {} schema updated concurrently to {}", identifier, schema); + return newSchema; + } else { + throw e; + } + } + default: + throw new IllegalArgumentException("Unknown comparison result"); + } + } + } + + private PartitionSpec findOrCreateSpec( + TableIdentifier identifier, PartitionSpec targetSpec, Schema newSchema) { + PartitionSpec currentSpec = cache.spec(identifier, targetSpec); + if (currentSpec != null) { + return currentSpec; + } + + Table table = catalog.loadTable(identifier); + currentSpec = table.spec(); + + PartitionSpecEvolver.PartitionSpecEvolverResult result = + PartitionSpecEvolver.evolve(currentSpec, targetSpec, newSchema); + if (result.isEmpty()) { + LOG.info("Returning equivalent existing spec {} for {}", currentSpec, targetSpec); + return currentSpec; + } + + LOG.info( + "Spec for table {} has been altered. Updating from {} to {}", + identifier, + currentSpec, + targetSpec); + UpdatePartitionSpec updater = table.updateSpec(); + result.termsToRemove().forEach(updater::removeField); + result.termsToAdd().forEach(updater::addField); + + try { + updater.commit(); + } catch (CommitFailedException e) { + LOG.info( + "Partition spec update failed for {} from {} to {}", + identifier, + currentSpec, + targetSpec, + e); + PartitionSpec newSpec = cache.spec(identifier, targetSpec); + Schema maybeUpdatedSchema = cache.schema(identifier, newSchema).f0; + result = PartitionSpecEvolver.evolve(targetSpec, newSpec, maybeUpdatedSchema); + if (result.isEmpty()) { + LOG.info("Table {} partition spec updated concurrently to {}", identifier, newSpec); + return newSpec; + } else { + throw e; + } + } + + cache.invalidate(identifier); + return cache.spec(identifier, targetSpec); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java new file mode 100644 index 000000000000..43e94c2274a4 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -0,0 +1,144 @@ +/* + * 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(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 1767f774922a..ffa1db71d652 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -82,12 +82,19 @@ private SimpleDataUtil() {} Types.NestedField.optional(1, "id", Types.IntegerType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); + public static final Schema SCHEMA2 = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get())); + public static final TableSchema FLINK_SCHEMA = TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); public static final Record RECORD = GenericRecord.create(SCHEMA); + public static final Record RECORD2 = GenericRecord.create(SCHEMA2); public static Table createTable( String path, Map properties, boolean partitioned) { @@ -107,6 +114,14 @@ public static Record createRecord(Integer id, String data) { return record; } + public static Record createRecord(Integer id, String data, String extra) { + Record record = RECORD2.copy(); + record.setField("id", id); + record.setField("data", data); + record.setField("extra", extra); + return record; + } + public static RowData createRowData(Integer id, String data) { return GenericRowData.of(id, StringData.fromString(data)); } @@ -224,7 +239,12 @@ private static List convertToRecords(List rows) { for (RowData row : rows) { Integer id = row.isNullAt(0) ? null : row.getInt(0); String data = row.isNullAt(1) ? null : row.getString(1).toString(); - records.add(createRecord(id, data)); + if (row.getArity() == 2) { + records.add(createRecord(id, data)); + } else { + String extra = row.isNullAt(2) ? null : row.getString(2).toString(); + records.add(createRecord(id, data, extra)); + } } return records; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 6f8fc518a8f9..90dcfe500417 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -131,7 +131,7 @@ public static List convertRowDataToRow(List rowDataList, RowType r .collect(Collectors.toList()); } - private static List convertRecordToRow(List expectedRecords, Schema schema) { + public static List convertRecordToRow(List expectedRecords, Schema schema) { List expected = Lists.newArrayList(); @SuppressWarnings("unchecked") DataStructureConverter converter = diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index 9ce36cc1e8d0..29db9f6158c8 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.TestFixtures.DATABASE; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -64,8 +65,8 @@ public class TestFlinkIcebergSinkBase { protected Table table; protected StreamExecutionEnvironment env; - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(Collections.singletonList(rows)); } protected List createRows(String prefix) { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java new file mode 100644 index 000000000000..d14eebecef44 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -0,0 +1,76 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.required; + +import java.util.Collections; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test base for DynamicRecordInternalSerializer which allows to instantiate different serializer + * version, e.g. with writing the schema itself or just the schema id. + */ +abstract class DynamicRecordInternalSerializerTestBase + extends SerializerTestBase { + + static final String TABLE = "myTable"; + static final String BRANCH = "myBranch"; + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", TABLE); + + static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "number", Types.FloatType.get())); + + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); + + @Override + protected DynamicRecordInternal[] getTestData() { + GenericRowData rowData = new GenericRowData(3); + rowData.setField(0, 123L); + rowData.setField(1, StringData.fromString("test")); + rowData.setField(2, 1.23f); + + return new DynamicRecordInternal[] { + new DynamicRecordInternal( + TABLE, BRANCH, SCHEMA, SPEC, 42, rowData, false, Collections.emptyList()) + }; + } + + @Override + protected Class getTypeClass() { + return DynamicRecordInternal.class; + } + + @Override + protected int getLength() { + return -1; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java new file mode 100644 index 000000000000..0bf454edc2e9 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java @@ -0,0 +1,41 @@ +/* + * 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.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.BeforeEach; + +/** Test writing only the schema id on serialization. */ +class DynamicRecordInternalSerializerWriteSchemaIdTest + extends DynamicRecordInternalSerializerTestBase { + + final boolean writeSchemaAndSpec = false; + + @BeforeEach + void before() { + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.parse(TABLE), SCHEMA, SPEC); + } + + @Override + protected TypeSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java new file mode 100644 index 000000000000..c51e54b4b804 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java @@ -0,0 +1,34 @@ +/* + * 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.api.common.typeutils.TypeSerializer; + +/** Test the full schema */ +class DynamicRecordInternalSerializerWriteSchemaTest + extends DynamicRecordInternalSerializerTestBase { + + final boolean writeSchemaAndSpec = true; + + @Override + protected TypeSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java new file mode 100644 index 000000000000..6edebaecce2e --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java @@ -0,0 +1,209 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.junit.jupiter.api.Test; + +class TestCompareSchemasVisitor { + + @Test + void testSchema() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get(), "comment"), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get(), "comment"), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testSchemaDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testSchemaDifferent() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())), + new Schema( + optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testSchemaWithMoreColumns() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())), + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "data", StringType.get()), + optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); + } + + @Test + void testDifferentType() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testCompatibleType() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())), + new Schema( + optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())))) + .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); + } + + @Test + void testWithRequiredChange() { + Schema dataSchema = + new Schema(optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); + Schema tableSchema = + new Schema(required(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema)) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + assertThat(CompareSchemasVisitor.visit(tableSchema, dataSchema)) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testStructDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))), + new Schema( + optional(0, "id", IntegerType.get()), + optional( + 1, "struct1", StructType.of(optional(2, "extra", IntegerType.get())))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testStructChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "struct1", StructType.of(optional(2, "extra", LongType.get())))), + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testMapDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))), + new Schema( + optional(0, "id", IntegerType.get()), + optional( + 1, "map1", MapType.ofOptional(2, 3, IntegerType.get(), StringType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testMapChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, LongType.get(), StringType.get()))), + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } + + @Test + void testListDifferentId() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))), + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "list1", ListType.ofOptional(2, IntegerType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testListChanged() { + assertThat( + CompareSchemasVisitor.visit( + new Schema( + optional(0, "id", IntegerType.get()), + optional(1, "list1", ListType.ofOptional(2, LongType.get()))), + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java new file mode 100644 index 000000000000..e6069656bd65 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -0,0 +1,591 @@ +/* + * 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.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestDynamicIcebergSink extends TestFlinkIcebergSinkBase { + + private static long seed; + + @BeforeEach + void before() { + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(2); + seed = 0; + } + + private static class DynamicIcebergDataImpl implements Serializable { + Row rowProvided; + Row rowExpected; + Schema schemaProvided; + Schema schemaExpected; + String tableName; + String branch; + PartitionSpec partitionSpec; + boolean upsertMode; + List equalityFields; + + private DynamicIcebergDataImpl( + Schema schemaProvided, String tableName, String branch, PartitionSpec partitionSpec) { + this( + schemaProvided, + schemaProvided, + tableName, + branch, + partitionSpec, + false, + Collections.emptyList(), + false); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + Schema schemaExpected, + String tableName, + String branch, + PartitionSpec partitionSpec) { + this( + schemaProvided, + schemaExpected, + tableName, + branch, + partitionSpec, + false, + Collections.emptyList(), + false); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + String tableName, + String branch, + PartitionSpec partitionSpec, + boolean upsertMode, + List equalityFields, + boolean isDuplicate) { + this( + schemaProvided, + schemaProvided, + tableName, + branch, + partitionSpec, + upsertMode, + equalityFields, + isDuplicate); + } + + private DynamicIcebergDataImpl( + Schema schemaProvided, + Schema schemaExpected, + String tableName, + String branch, + PartitionSpec partitionSpec, + boolean upsertMode, + List equalityFields, + boolean isDuplicate) { + this.rowProvided = randomRow(schemaProvided, isDuplicate ? seed : ++seed); + this.rowExpected = isDuplicate ? null : rowProvided; + this.schemaProvided = schemaProvided; + this.schemaExpected = schemaExpected; + this.tableName = tableName; + this.branch = branch; + this.partitionSpec = partitionSpec; + this.upsertMode = upsertMode; + this.equalityFields = equalityFields; + } + } + + private static class Converter implements DynamicRecordConverter { + + @Override + public void convert(DynamicIcebergDataImpl row, Collector out) { + TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, row.tableName); + String branch = row.branch; + Schema schema = row.schemaProvided; + PartitionSpec spec = row.partitionSpec; + DynamicRecord dynamicRecord = + new DynamicRecord( + tableIdentifier, + branch, + schema, + spec, + converter(schema).toInternal(row.rowProvided), + spec.isPartitioned() ? DistributionMode.HASH : DistributionMode.NONE, + 10); + dynamicRecord.setUpsertMode(row.upsertMode); + dynamicRecord.setEqualityFields(row.equalityFields); + out.collect(dynamicRecord); + } + } + + private static DataFormatConverters.RowConverter converter(Schema schema) { + RowType rowType = FlinkSchemaUtil.convert(schema); + TableSchema tableSchema = FlinkSchemaUtil.toSchema(rowType); + return new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); + } + + @Test + void testWrite() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testWritePartitioned() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); + + runTest(rows); + } + + @Test + void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { + Schema schema = + new Schema( + // Use zero-based schema field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + Schema schema2 = + new Schema( + // Use zero-based schema field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get()), + Types.NestedField.optional(2, "extra", Types.StringType.get())); + PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("extra", 23).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema, "t1", "main", spec), + new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), + new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); + + runTest(rows); + } + + @Test + void testSchemaEvolutionFieldOrderChanges() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + Schema expectedSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + Schema schema2 = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "extra", Types.StringType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + Schema expectedSchema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); + + for (DynamicIcebergDataImpl row : rows) { + if (row.schemaExpected == expectedSchema) { + // We manually adjust the expected Row to match the second expected schema + row.rowExpected = Row.of(row.rowProvided.getField(0), null, row.rowProvided.getField(1)); + } + } + + runTest(rows); + } + + @Test + void testMultipleTables() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testMultipleTablesPartitioned() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); + + runTest(rows); + } + + @Test + void testSchemaEvolutionAddField() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows, this.env, 1); + } + + @Test + void testRowEvolutionNullMissingOptionalField() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows, this.env, 1); + } + + @Test + void testSchemaEvolutionNonBackwardsCompatible() throws Exception { + Schema backwardsIncompatibleSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + // Required column is missing in this schema + Schema erroringSchema = + new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + backwardsIncompatibleSchema, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); + + try { + runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); + Assert.fail(); + } catch (JobExecutionException e) { + assertThat( + ExceptionUtils.findThrowable( + e, + t -> + t.getMessage() + .contains( + "Field 2 in target schema ROW<`id` INT NOT NULL, `data` STRING NOT NULL> is non-nullable but does not exist in source schema."))) + .isNotEmpty(); + } + } + + @Test + void testPartitionSpecEvolution() throws Exception { + PartitionSpec spec1 = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 5).identity("data").build(); + + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); + + runTest(rows); + } + + @Test + void testMultipleBranches() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testWriteDynamicRowData() throws Exception { + List rows = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + + runTest(rows); + } + + @Test + void testUpsert() throws Exception { + List rows = + Lists.newArrayList( + // Insert one rows + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + false), + // Remaining rows are duplicates + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + "main", + PartitionSpec.unpartitioned(), + true, + Lists.newArrayList("id"), + true)); + + executeDynamicSink(rows, env, true, 1); + + try (CloseableIterable iterable = + IcebergGenerics.read( + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.of("default", "t1"))) + .build()) { + List records = Lists.newArrayList(); + for (Record record : iterable) { + records.add(record); + } + + assertThat(records.size()).isEqualTo(1); + Record actual = records.get(0); + DynamicIcebergDataImpl input = rows.get(0); + assertThat(actual.get(0)).isEqualTo(input.rowProvided.getField(0)); + assertThat(actual.get(1)).isEqualTo(input.rowProvided.getField(1)); + // There is an additional _pos field which gets added + } + } + + private void runTest(List dynamicData) throws Exception { + runTest(dynamicData, this.env, 2); + } + + private void runTest( + List dynamicData, StreamExecutionEnvironment env, int parallelism) + throws Exception { + runTest(dynamicData, env, true, parallelism); + runTest(dynamicData, env, false, parallelism); + } + + private void runTest( + List dynamicData, + StreamExecutionEnvironment env, + boolean immediateUpdate, + int parallelism) + throws Exception { + executeDynamicSink(dynamicData, env, immediateUpdate, parallelism); + verifyResults(dynamicData); + } + + private void executeDynamicSink( + List dynamicData, + StreamExecutionEnvironment env, + boolean immediateUpdate, + int parallelism) + throws Exception { + DataStream dataStream = + env.addSource(createBoundedSource(dynamicData), TypeInformation.of(new TypeHint<>() {})); + env.setParallelism(parallelism); + + DynamicIcebergSink.forInput(dataStream) + .withConverter(new Converter()) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .writeParallelism(parallelism) + .immediateTableUpdate(immediateUpdate) + .append(); + + // Write the data + env.execute("Test Iceberg DataStream"); + } + + private void verifyResults(List dynamicData) throws IOException { + // Calculate the expected result + Map, List> expectedData = Maps.newHashMap(); + Map expectedSchema = Maps.newHashMap(); + dynamicData.forEach( + r -> { + Schema oldSchema = expectedSchema.get(r.tableName); + if (oldSchema == null || oldSchema.columns().size() < r.schemaProvided.columns().size()) { + expectedSchema.put(r.tableName, r.schemaExpected); + } + }); + + dynamicData.forEach( + r -> { + List data = + expectedData.computeIfAbsent( + Tuple2.of(r.tableName, r.branch), unused -> Lists.newArrayList()); + data.addAll( + convertToRowData(expectedSchema.get(r.tableName), ImmutableList.of(r.rowExpected))); + }); + + // Check the expected result + int count = dynamicData.size(); + for (Map.Entry, List> e : expectedData.entrySet()) { + SimpleDataUtil.assertTableRows( + CATALOG_EXTENSION + .catalogLoader() + .loadCatalog() + .loadTable(TableIdentifier.of(DATABASE, e.getKey().f0)), + e.getValue(), + e.getKey().f1); + count -= e.getValue().size(); + } + + // Found every record + assertThat(count).isZero(); + } + + private List convertToRowData(Schema schema, List rows) { + DataFormatConverters.RowConverter converter = converter(schema); + return rows.stream() + .map( + r -> { + Row updateRow = r; + // We need conversion to generate the missing columns + if (r.getArity() != schema.columns().size()) { + updateRow = new Row(schema.columns().size()); + for (int i = 0; i < r.getArity(); ++i) { + updateRow.setField(i, r.getField(i)); + } + } + return converter.toInternal(updateRow); + }) + .collect(Collectors.toList()); + } + + private static Row randomRow(Schema schema, long seedOverride) { + return TestHelpers.convertRecordToRow( + RandomGenericData.generate(schema, 1, seedOverride), schema) + .get(0); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java new file mode 100644 index 000000000000..b7c145da2b77 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -0,0 +1,226 @@ +/* + * 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.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE; +import static org.apache.iceberg.flink.sink.dynamic.DynamicCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; + +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.util.Collector; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +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.TableIdentifier; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Disabled("Please enable manually for performance testing.") +class TestDynamicIcebergSinkPerf { + private static final Logger LOG = LoggerFactory.getLogger(TestDynamicIcebergSinkPerf.class); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TABLE); + + private static final int SAMPLE_SIZE = 50_000; + private static final int RECORD_SIZE = 5_000_000; + private static final int TABLE_NUM = 3; + private static final int PARALLELISM = 2; + private static final int WRITE_PARALLELISM = 2; + private static final TableIdentifier[] IDENTIFIERS = new TableIdentifier[TABLE_NUM]; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + private static final List RANGE = + IntStream.range(0, RECORD_SIZE).boxed().collect(Collectors.toList()); + + private static List rows; + private StreamExecutionEnvironment env; + + @BeforeEach + void before() { + for (int i = 0; i < TABLE_NUM; ++i) { + // So the table name hash difference is bigger than 1 + IDENTIFIERS[i] = TableIdentifier.of(DATABASE, TABLE + "_" + (i * 13)); + + Table table = + CATALOG_EXTENSION + .catalog() + .createTable( + IDENTIFIERS[i], + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); + + table.manageSnapshots().createBranch("main").commit(); + } + + List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); + rows = Lists.newArrayListWithCapacity(records.size()); + for (int i = 0; i < records.size(); ++i) { + rows.add( + new DynamicRecord( + IDENTIFIERS[i % TABLE_NUM], + "main", + SCHEMA, + PartitionSpec.unpartitioned(), + RowDataConverter.convert(SCHEMA, records.get(i)), + DistributionMode.NONE, + WRITE_PARALLELISM)); + } + + Configuration configuration = MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + configuration.setString("rest.flamegraph.enabled", "true"); + env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration) + .enableCheckpointing(100) + .setParallelism(PARALLELISM) + .setMaxParallelism(PARALLELISM); + env.getConfig().enableObjectReuse(); + } + + @AfterEach + void after() { + for (TableIdentifier identifier : IDENTIFIERS) { + CATALOG_EXTENSION.catalog().dropTable(identifier); + } + } + + private static class IdBasedConverter implements DynamicRecordConverter { + + @Override + public void convert(Integer id, Collector out) { + out.collect(rows.get(id % SAMPLE_SIZE)); + } + } + + @Test + void testDynamicSink() throws Exception { + // So we make sure that the writer threads are the same for the 2 tests + env.setMaxParallelism(PARALLELISM * TABLE_NUM * 2); + env.setParallelism(PARALLELISM * TABLE_NUM * 2); + runTest( + s -> { + DynamicIcebergSink.forInput(s) + .withConverter(new IdBasedConverter()) + .immediateTableUpdate(true) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .append(); + }); + } + + @Test + void testIcebergSink() throws Exception { + runTest( + s -> { + for (int i = 0; i < IDENTIFIERS.length; ++i) { + TableLoader tableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), IDENTIFIERS[i]); + final int finalInt = i; + IcebergSink.forRowData( + s.flatMap( + (FlatMapFunction) + (input, collector) -> { + if (input % TABLE_NUM == finalInt) { + collector.collect(rows.get(input % SAMPLE_SIZE).rowData()); + } + }) + .returns(InternalTypeInfo.of(FlinkSchemaUtil.convert(SCHEMA))) + .rebalance()) + .tableLoader(tableLoader) + .uidSuffix("Uid" + i) + .writeParallelism(WRITE_PARALLELISM) + .append(); + } + }); + } + + private void runTest(Consumer> sink) throws Exception { + DataStream dataStream = + env.addSource( + new BoundedTestSource<>( + ImmutableList.of( + RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE), + true), + TypeInformation.of(Integer.class)); + + sink.accept(dataStream); + + long before = System.currentTimeMillis(); + env.execute(); + + for (TableIdentifier identifier : IDENTIFIERS) { + Table table = CATALOG_EXTENSION.catalog().loadTable(identifier); + for (Snapshot snapshot : table.snapshots()) { + long records = 0; + for (DataFile dataFile : snapshot.addedDataFiles(table.io())) { + records += dataFile.recordCount(); + } + + LOG.info( + "TEST RESULT: For table {} snapshot {} written {} records in {} ms", + identifier, + snapshot.snapshotId(), + records, + snapshot.timestampMillis() - before); + before = snapshot.timestampMillis(); + } + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java new file mode 100644 index 000000000000..ad96cf9747c4 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -0,0 +1,106 @@ +/* + * 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, + 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; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java new file mode 100644 index 000000000000..05501b1e6e43 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java @@ -0,0 +1,607 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.of; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaUpdate; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.UUIDType; +import org.junit.Assert; +import org.junit.Test; + +public class TestEvolveSchemaVisitor { + + private static List primitiveTypes() { + return Lists.newArrayList( + StringType.get(), + TimeType.get(), + Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + UUIDType.get(), + Types.DateType.get(), + Types.BooleanType.get(), + Types.BinaryType.get(), + DoubleType.get(), + IntegerType.get(), + Types.FixedType.ofLength(10), + DecimalType.of(10, 2), + LongType.get(), + FloatType.get()); + } + + private static Types.NestedField[] primitiveFields( + Integer initialValue, List primitiveTypes) { + return primitiveFields(initialValue, primitiveTypes, true); + } + + private static Types.NestedField[] primitiveFields( + Integer initialValue, List primitiveTypes, boolean optional) { + AtomicInteger atomicInteger = new AtomicInteger(initialValue); + return primitiveTypes.stream() + .map( + type -> + of( + atomicInteger.incrementAndGet(), + optional, + type.toString(), + Types.fromPrimitiveString(type.toString()))) + .toArray(Types.NestedField[]::new); + } + + @Test + public void testAddTopLevelPrimitives() { + Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testMakeTopLevelPrimitivesOptional() { + Schema existingSchema = new Schema(primitiveFields(0, primitiveTypes(), false)); + Assert.assertTrue(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)); + + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); + Schema newSchema = updateApi.apply(); + Assert.assertEquals(14, newSchema.asStruct().fields().size()); + Assert.assertTrue(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)); + } + + @Test + public void testIdentifyFieldsByName() { + Schema existingSchema = + new Schema(Types.NestedField.optional(42, "myField", Types.LongType.get())); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + Schema newSchema = + new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); + EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); + Assert.assertTrue(updateApi.apply().sameSchema(existingSchema)); + } + + @Test + public void testChangeOrderTopLevelPrimitives() { + Schema existingSchema = + new Schema( + Arrays.asList(optional(1, "a", StringType.get()), optional(2, "b", StringType.get()))); + Schema targetSchema = + new Schema( + Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddTopLevelListOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testMakeTopLevelListOfPrimitivesOptional() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema existingSchema = + new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); + Schema targetSchema = new Schema(); + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Schema expectedSchema = + new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); + Assert.assertEquals(expectedSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddTopLevelMapOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddTopLevelStructOfPrimitives() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); + Assert.assertEquals(currentSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddNestedPrimitive() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); + Schema targetSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testMakeNestedPrimitiveOptional() { + for (PrimitiveType primitiveType : primitiveTypes()) { + Schema currentSchema = + new Schema( + optional(1, "aStruct", StructType.of(required(2, "primitive", primitiveType)))); + Schema targetSchema = + new Schema( + optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + } + + @Test + public void testAddNestedPrimitives() { + Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); + Schema targetSchema = + new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedLists() { + Schema targetSchema = + new Schema( + optional( + 1, + "aList", + ListType.ofOptional( + 2, + ListType.ofOptional( + 3, + ListType.ofOptional( + 4, + ListType.ofOptional( + 5, + ListType.ofOptional( + 6, + ListType.ofOptional( + 7, + ListType.ofOptional( + 8, + ListType.ofOptional( + 9, + ListType.ofOptional( + 10, DecimalType.of(11, 20)))))))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedStruct() { + Schema targetSchema = + new Schema( + optional( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "struct3", + StructType.of( + optional( + 4, + "struct4", + StructType.of( + optional( + 5, + "struct5", + StructType.of( + optional( + 6, + "struct6", + StructType.of( + optional( + 7, + "aString", + StringType.get())))))))))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddNestedMaps() { + Schema targetSchema = + new Schema( + optional( + 1, + "struct", + MapType.ofOptional( + 2, + 3, + StringType.get(), + MapType.ofOptional( + 4, + 5, + StringType.get(), + MapType.ofOptional( + 6, + 7, + StringType.get(), + MapType.ofOptional( + 8, + 9, + StringType.get(), + MapType.ofOptional( + 10, + 11, + StringType.get(), + MapType.ofOptional( + 12, 13, StringType.get(), StringType.get())))))))); + SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); + EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testDetectInvalidTopLevelList() { + Schema currentSchema = + new Schema(optional(1, "aList", ListType.ofOptional(2, StringType.get()))); + Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, LongType.get()))); + Assert.assertThrows( + "Cannot change column type: aList.element: string -> long", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 2), currentSchema, targetSchema)); + } + + @Test + public void testDetectInvalidTopLevelMapValue() { + + Schema currentSchema = + new Schema( + optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), LongType.get()))); + + Assert.assertThrows( + "Cannot change column type: aMap.value: string -> long", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + public void testDetectInvalidTopLevelMapKey() { + Schema currentSchema = + new Schema( + optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); + Schema targetSchema = + new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, UUIDType.get(), StringType.get()))); + Assert.assertThrows( + "Cannot change column type: aMap.key: string -> uuid", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + // int 32-bit signed integers -> Can promote to long + public void testTypePromoteIntegerToLong() { + Schema currentSchema = new Schema(required(1, "aCol", IntegerType.get())); + Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Schema applied = updateApi.apply(); + Assert.assertEquals(1, applied.asStruct().fields().size()); + Assert.assertEquals(LongType.get(), applied.asStruct().fields().get(0).type()); + } + + @Test + // float 32-bit IEEE 754 floating point -> Can promote to double + public void testTypePromoteFloatToDouble() { + Schema currentSchema = new Schema(required(1, "aCol", FloatType.get())); + Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Schema applied = updateApi.apply(); + Assert.assertEquals(1, applied.asStruct().fields().size()); + Assert.assertEquals(DoubleType.get(), applied.asStruct().fields().get(0).type()); + } + + @Test + public void testInvalidTypePromoteDoubleToFloat() { + Schema currentSchema = new Schema(required(1, "aCol", DoubleType.get())); + Schema targetSchema = new Schema(required(1, "aCol", FloatType.get())); + Assert.assertThrows( + "Cannot change column type: aCol: double -> float", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + // decimal(P,S) Fixed-point decimal; precision P, scale S -> Scale is fixed [1], precision must be + // 38 or less + public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { + Schema currentSchema = new Schema(required(1, "aCol", DecimalType.of(20, 1))); + Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAddPrimitiveToNestedStruct() { + Schema existingSchema = + new Schema( + required( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "list", + ListType.ofOptional( + 4, + StructType.of(optional(5, "number", IntegerType.get()))))))))); + + Schema targetSchema = + new Schema( + required( + 1, + "struct1", + StructType.of( + optional( + 2, + "struct2", + StructType.of( + optional( + 3, + "list", + ListType.ofOptional( + 4, + StructType.of( + optional(5, "number", LongType.get()), + optional(6, "time", TimeType.get()))))))))); + + SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 5); + EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testReplaceListWithPrimitive() { + Schema currentSchema = + new Schema(optional(1, "aColumn", ListType.ofOptional(2, StringType.get()))); + Schema targetSchema = new Schema(optional(1, "aColumn", StringType.get())); + Assert.assertThrows( + "Cannot change column type: aColumn: list -> string", + IllegalArgumentException.class, + () -> + EvolveSchemaVisitor.visit( + new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); + } + + @Test + public void addNewTopLevelStruct() { + Schema currentSchema = + new Schema( + optional( + 1, + "map1", + MapType.ofOptional( + 2, + 3, + StringType.get(), + ListType.ofOptional( + 4, StructType.of(optional(5, "string1", StringType.get())))))); + + Schema targetSchema = + new Schema( + optional( + 1, + "map1", + MapType.ofOptional( + 2, + 3, + StringType.get(), + ListType.ofOptional( + 4, StructType.of(optional(5, "string1", StringType.get()))))), + optional( + 6, + "struct1", + StructType.of( + optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 5); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testAppendNestedStruct() { + Schema currentSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional( + 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); + + Schema targetSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), + optional( + 5, + "repeat", + StructType.of( + optional( + 6, + "s1", + StructType.of( + optional( + 7, + "s2", + StructType.of( + optional( + 8, + "s3", + StructType.of( + optional( + 9, + "s4", + StringType.get())))))))))))))); + + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 4); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); + } + + @Test + public void testMakeNestedStructOptional() { + Schema currentSchema = getNestedSchemaWithOptionalModifier(false); + Schema targetSchema = + new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional( + 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); + SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 9); + EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + Assert.assertEquals( + updateApi.apply().asStruct(), getNestedSchemaWithOptionalModifier(true).asStruct()); + } + + private static Schema getNestedSchemaWithOptionalModifier(boolean nestedIsOptional) { + return new Schema( + required( + 1, + "s1", + StructType.of( + optional( + 2, + "s2", + StructType.of( + optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), + of( + 5, + nestedIsOptional, + "repeat", + StructType.of( + optional( + 6, + "s1", + StructType.of( + optional( + 7, + "s2", + StructType.of( + optional( + 8, + "s3", + StructType.of( + optional( + 9, "s4", StringType.get())))))))))))))); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java new file mode 100644 index 000000000000..b7cd95e00146 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java @@ -0,0 +1,57 @@ +/* + * 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 org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionSpecAdjustment { + + @Test + void testPartitionSpecSourceIdRemappingBasedOnFieldNames() { + Schema specSchema = + new Schema( + // Use zero-based field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required( + 1, + "data", + Types.StructType.of(Types.NestedField.required(2, "str", Types.StringType.get())))); + + PartitionSpec spec = PartitionSpec.builderFor(specSchema).bucket("id", 10).build(); + + Schema tableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required( + 2, + "data", + Types.StructType.of(Types.NestedField.required(3, "str", Types.StringType.get())))); + + PartitionSpec adjustedSpec = + PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(tableSchema, spec); + + assertThat(adjustedSpec) + .isEqualTo(PartitionSpec.builderFor(tableSchema).bucket("id", 10).build()); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java new file mode 100644 index 000000000000..2a2b4c385730 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java @@ -0,0 +1,248 @@ +/* + * 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.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; +import org.junit.Assert; +import org.junit.jupiter.api.Test; + +class TestRowDataEvolver { + @Test + void testPrimitiveTypes() { + DataGenerator generator = new DataGenerators.Primitives(); + assertThat( + RowDataEvolver.convert( + generator.generateFlinkRowData(), + generator.icebergSchema(), + generator.icebergSchema())) + .isEqualTo(generator.generateFlinkRowData()); + } + + @Test + void testAddColumn() { + assertThat( + RowDataEvolver.convert( + SimpleDataUtil.createRowData(1, "a"), + SimpleDataUtil.SCHEMA, + SimpleDataUtil.SCHEMA2)) + .isEqualTo(GenericRowData.of(1, StringData.fromString("a"), null)); + } + + @Test + void testAddRequiredColumn() { + Schema currentSchema = new Schema(Types.NestedField.optional(1, "id", Types.IntegerType.get())); + Schema targetSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get())); + + Assert.assertThrows( + IllegalArgumentException.class, + () -> RowDataEvolver.convert(GenericRowData.of(42), currentSchema, targetSchema)); + } + + @Test + void testIntToLong() { + Schema schemaWithLong = + new Schema( + Types.NestedField.optional(2, "id", Types.LongType.get()), + Types.NestedField.optional(4, "data", Types.StringType.get())); + + assertThat( + RowDataEvolver.convert( + SimpleDataUtil.createRowData(1, "a"), SimpleDataUtil.SCHEMA, schemaWithLong)) + .isEqualTo(GenericRowData.of(1L, StringData.fromString("a"))); + } + + @Test + void testFloatToDouble() { + Schema schemaWithFloat = + new Schema(Types.NestedField.optional(1, "float2double", Types.FloatType.get())); + Schema schemaWithDouble = + new Schema(Types.NestedField.optional(2, "float2double", Types.DoubleType.get())); + + assertThat(RowDataEvolver.convert(GenericRowData.of(1.5f), schemaWithFloat, schemaWithDouble)) + .isEqualTo(GenericRowData.of(1.5d)); + } + + @Test + void testDateToTimestamp() { + Schema schemaWithFloat = + new Schema(Types.NestedField.optional(1, "date2timestamp", Types.DateType.get())); + Schema schemaWithDouble = + new Schema( + Types.NestedField.optional(2, "date2timestamp", Types.TimestampType.withoutZone())); + + DateTime time = new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); + int days = + Days.daysBetween(new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC), time).getDays(); + + assertThat(RowDataEvolver.convert(GenericRowData.of(days), schemaWithFloat, schemaWithDouble)) + .isEqualTo(GenericRowData.of(TimestampData.fromEpochMillis(time.getMillis()))); + } + + @Test + void testIncreasePrecision() { + Schema before = + new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2))); + Schema after = + new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(10, 2))); + + assertThat( + RowDataEvolver.convert( + GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 9, 2)), + before, + after)) + .isEqualTo(GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 10, 2))); + } + + @Test + void testStructAddOptionalFields() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.required( + 10, + structField.name(), + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + optional(103, "optional", Types.StringType.get()), + required(102, "name", Types.StringType.get())))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of(1, null, StringData.fromString("Jane"))); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } + + @Test + void testStructAddRequiredFieldsWithOptionalRoot() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, + "newFieldOptionalField", + Types.StructType.of( + Types.NestedField.optional( + structField.fieldId(), + structField.name(), + Types.StructType.of( + optional(101, "id", Types.IntegerType.get()), + // Required columns which leads to nulling the entire struct + required(103, "required", Types.StringType.get()), + required(102, "name", Types.StringType.get())))))); + + RowData expectedData = GenericRowData.of(StringData.fromString("row_id_value"), null); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(expectedData); + } + + @Test + void testStructAddRequiredFields() { + DataGenerator generator = new DataGenerators.StructOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField structField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.required( + 10, + structField.name(), + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required(103, "required", Types.StringType.get()), + required(102, "name", Types.StringType.get())))); + + Assert.assertThrows( + IllegalArgumentException.class, + () -> RowDataEvolver.convert(oldData, oldSchema, newSchema)); + } + + @Test + void testMap() { + DataGenerator generator = new DataGenerators.MapOfPrimitives(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField mapField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, + mapField.name(), + Types.MapType.ofRequired(101, 102, Types.StringType.get(), Types.LongType.get()))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), 1L, StringData.fromString("Joe"), 2L))); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } + + @Test + void testArray() { + DataGenerator generator = new DataGenerators.ArrayOfPrimitive(); + RowData oldData = generator.generateFlinkRowData(); + Schema oldSchema = generator.icebergSchema(); + Types.NestedField arrayField = oldSchema.columns().get(1); + Schema newSchema = + new Schema( + oldSchema.columns().get(0), + Types.NestedField.optional( + 10, arrayField.name(), Types.ListType.ofOptional(101, Types.LongType.get()))); + RowData newData = + GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(new Long[] {1L, 2L, 3L})); + + assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java new file mode 100644 index 000000000000..c0ca74eba240 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java @@ -0,0 +1,85 @@ +/* + * 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.commons.lang3.SerializationUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestTableDataCache extends TestFlinkIcebergSinkBase { + + @Test + void testCaching() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + + Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); + assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) + .isEqualTo(schema1); + + assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2)) + .isEqualTo(TableDataCache.NOT_FOUND); + + schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) + .isEqualTo(schema1); + } + + @Test + void testCacheInvalidationAfterSchemaChange() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; + assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); + + catalog.dropTable(tableIdentifier); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()); + + Schema schema2 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f0; + assertThat(schema2.sameSchema(SimpleDataUtil.SCHEMA2)).isTrue(); + } + + @Test + void testCachingDisabled() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 0, Long.MAX_VALUE); + + // Cleanup routine doesn't run after every write + cache.getInternalCache().cleanUp(); + assertThat(cache.getInternalCache().estimatedSize()).isEqualTo(0); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java new file mode 100644 index 000000000000..8dbf630490b0 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestTableUpdater extends TestFlinkIcebergSinkBase { + + @Test + void testInvalidateOldCacheEntryOnUpdate() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + Schema updated = + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) + .f0; + assertThat(updated.sameSchema(SimpleDataUtil.SCHEMA2)); + assertThat( + cache + .schema(tableIdentifier, SimpleDataUtil.SCHEMA2) + .f0 + .sameSchema(SimpleDataUtil.SCHEMA2)) + .isTrue(); + } + + @Test + void testLastResultInvalidation() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); + TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); + TableUpdater tableUpdater = new TableUpdater(cache, catalog); + + // Initialize cache + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); + + // Update table behind the scenes + catalog.dropTable(tableIdentifier); + catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); + + // Cache still stores the old information + assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f1) + .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); + + assertThat( + tableUpdater.update( + tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) + .f1) + .isEqualTo(CompareSchemasVisitor.Result.SAME); + + // Last result cache should be cleared + assertThat( + cache + .getInternalCache() + .getIfPresent(tableIdentifier) + .getSchemaInfo() + .getLastResult(SimpleDataUtil.SCHEMA2)) + .isNull(); + } +} From 358712ffbbd9bd12ed6ee049317286c552d080e9 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Mon, 3 Mar 2025 09:22:32 +0100 Subject: [PATCH 4/6] Revert "Flink 1.19: Dynamic Iceberg Sink" This reverts commit 5a16da0ed6060b453e1e5c5c9a1ef33a67c4e6dc. --- flink/v1.19/build.gradle | 3 - ...RecordSerializerDeserializerBenchmark.java | 137 ---- .../apache/iceberg/flink/FlinkConfParser.java | 7 - .../apache/iceberg/flink/FlinkWriteConf.java | 4 - .../flink/sink/BucketPartitionerUtil.java | 2 +- .../iceberg/flink/sink/CommitSummary.java | 50 +- .../iceberg/flink/sink/DeltaManifests.java | 4 +- .../flink/sink/DeltaManifestsSerializer.java | 4 +- .../flink/sink/EqualityFieldKeySelector.java | 10 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 10 +- .../flink/sink/IcebergCommittable.java | 2 +- .../sink/IcebergCommittableSerializer.java | 2 +- .../iceberg/flink/sink/IcebergCommitter.java | 2 +- .../flink/sink/IcebergFilesCommitter.java | 2 +- .../sink/IcebergFilesCommitterMetrics.java | 10 +- .../iceberg/flink/sink/IcebergSink.java | 2 +- .../sink/IcebergStreamWriterMetrics.java | 8 +- .../flink/sink/IcebergWriteAggregator.java | 3 +- .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../flink/sink/NonThrowingKeySelector.java | 28 - .../flink/sink/PartitionKeySelector.java | 11 +- .../flink/sink/RowDataTaskWriterFactory.java | 27 +- .../flink/sink/WriteResultSerializer.java | 2 +- .../sink/dynamic/CompareSchemasVisitor.java | 254 -------- .../sink/dynamic/DynamicCommittable.java | 80 --- .../dynamic/DynamicCommittableSerializer.java | 73 --- .../flink/sink/dynamic/DynamicCommitter.java | 417 ------------ .../sink/dynamic/DynamicCommitterMetrics.java | 51 -- .../sink/dynamic/DynamicIcebergSink.java | 439 ------------- .../sink/dynamic/DynamicKeySelector.java | 384 ----------- .../flink/sink/dynamic/DynamicRecord.java | 128 ---- .../sink/dynamic/DynamicRecordConverter.java | 31 - .../sink/dynamic/DynamicRecordInternal.java | 164 ----- .../DynamicRecordInternalSerializer.java | 299 --------- .../dynamic/DynamicRecordInternalType.java | 102 --- .../sink/dynamic/DynamicRecordProcessor.java | 196 ------ .../dynamic/DynamicTableUpdateOperator.java | 71 -- .../sink/dynamic/DynamicWriteResult.java | 41 -- .../dynamic/DynamicWriteResultAggregator.java | 190 ------ .../dynamic/DynamicWriteResultSerializer.java | 63 -- .../flink/sink/dynamic/DynamicWriter.java | 218 ------- .../sink/dynamic/DynamicWriterMetrics.java | 50 -- .../sink/dynamic/EvolveSchemaVisitor.java | 184 ------ .../sink/dynamic/PartitionSpecAdjustment.java | 45 -- .../sink/dynamic/PartitionSpecEvolver.java | 119 ---- .../flink/sink/dynamic/RowDataEvolver.java | 169 ----- .../sink/dynamic/RowDataSerializerCache.java | 102 --- .../flink/sink/dynamic/TableDataCache.java | 257 -------- .../flink/sink/dynamic/TableUpdater.java | 208 ------ .../flink/sink/dynamic/WriteTarget.java | 144 ----- .../apache/iceberg/flink/SimpleDataUtil.java | 22 +- .../org/apache/iceberg/flink/TestHelpers.java | 2 +- .../flink/sink/TestFlinkIcebergSinkBase.java | 5 +- ...namicRecordInternalSerializerTestBase.java | 75 --- ...rdInternalSerializerWriteSchemaIdTest.java | 41 -- ...cordInternalSerializerWriteSchemaTest.java | 34 - .../dynamic/TestCompareSchemasVisitor.java | 209 ------ .../sink/dynamic/TestDynamicIcebergSink.java | 591 ----------------- .../dynamic/TestDynamicIcebergSinkPerf.java | 226 ------- .../flink/sink/dynamic/TestDynamicWriter.java | 106 --- .../sink/dynamic/TestEvolveSchemaVisitor.java | 607 ------------------ .../dynamic/TestPartitionSpecAdjustment.java | 57 -- .../sink/dynamic/TestRowDataEvolver.java | 248 ------- .../sink/dynamic/TestTableDataCache.java | 85 --- .../flink/sink/dynamic/TestTableUpdater.java | 90 --- 65 files changed, 67 insertions(+), 7144 deletions(-) delete mode 100644 flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 7723d5fc6712..599ba085e4c4 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -68,9 +68,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation libs.datasketches - // for caching in DynamicSink - implementation libs.caffeine - testImplementation libs.flink119.connector.test.utils testImplementation libs.flink119.core testImplementation libs.flink119.runtime diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java deleted file mode 100644 index 2d102449b5b3..000000000000 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.serialization.SerializerConfig; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; -import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.RunnerException; -import org.openjdk.jmh.runner.options.Options; -import org.openjdk.jmh.runner.options.OptionsBuilder; - -@Fork(1) -@State(Scope.Benchmark) -@Warmup(iterations = 3) -@Measurement(iterations = 5) -@BenchmarkMode(Mode.SingleShotTime) -public class DynamicRecordSerializerDeserializerBenchmark { - private static final int SAMPLE_SIZE = 100_000; - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name2", Types.StringType.get()), - Types.NestedField.required(3, "name3", Types.StringType.get()), - Types.NestedField.required(4, "name4", Types.StringType.get()), - Types.NestedField.required(5, "name5", Types.StringType.get()), - Types.NestedField.required(6, "name6", Types.StringType.get()), - Types.NestedField.required(7, "name7", Types.StringType.get()), - Types.NestedField.required(8, "name8", Types.StringType.get()), - Types.NestedField.required(9, "name9", Types.StringType.get())); - - private List rows = Lists.newArrayListWithExpectedSize(SAMPLE_SIZE); - private DynamicRecordInternalType type; - - public static void main(String[] args) throws RunnerException { - Options options = - new OptionsBuilder() - .include(DynamicRecordSerializerDeserializerBenchmark.class.getSimpleName()) - .build(); - new Runner(options).run(); - } - - @Setup - public void setupBenchmark() throws IOException { - List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); - this.rows = - records.stream() - .map( - r -> - new DynamicRecordInternal( - "t", - "main", - SCHEMA, - PartitionSpec.unpartitioned(), - 1, - RowDataConverter.convert(SCHEMA, r), - false, - List.of())) - .collect(Collectors.toList()); - - File warehouse = Files.createTempFile("perf-bench", null).toFile(); - CatalogLoader catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getPath())); - this.type = new DynamicRecordInternalType(catalogLoader, true, 100); - } - - @Benchmark - @Threads(1) - public void testSerialize(Blackhole blackhole) throws IOException { - TypeSerializer serializer = - type.createSerializer((SerializerConfig) null); - DataOutputSerializer outputView = new DataOutputSerializer(1024); - for (int i = 0; i < SAMPLE_SIZE; ++i) { - serializer.serialize(rows.get(i), outputView); - } - } - - @Benchmark - @Threads(1) - public void testSerializeAndDeserialize(Blackhole blackhole) throws IOException { - TypeSerializer serializer = - type.createSerializer((SerializerConfig) null); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - for (int i = 0; i < SAMPLE_SIZE; ++i) { - serializer.serialize(rows.get(i), outputView); - serializer.deserialize(new DataInputDeserializer(outputView.getSharedBuffer())); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index b9b15e4d730e..d5eea6706b39 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -27,7 +27,6 @@ import org.apache.flink.util.TimeUtils; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; class FlinkConfParser { @@ -42,12 +41,6 @@ class FlinkConfParser { this.readableConfig = readableConfig; } - FlinkConfParser(Map options, ReadableConfig readableConfig) { - this.tableProperties = ImmutableMap.of(); - this.options = options; - this.readableConfig = readableConfig; - } - public BooleanConfParser booleanConf() { return new BooleanConfParser(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index d72c05e2ef1f..a31902d49a8b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -55,10 +55,6 @@ public FlinkWriteConf( this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); } - public FlinkWriteConf(Map writeOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(writeOptions, readableConfig); - } - public boolean overwriteMode() { return confParser .booleanConf() diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index 22dd40a61b9d..c33207728d3e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -26,7 +26,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.PartitionSpecVisitor; -public final class BucketPartitionerUtil { +final class BucketPartitionerUtil { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = "Invalid number of buckets: %s (must be 1)"; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 62de3170d55c..2109c91bddf7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -19,14 +19,13 @@ 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; -public class CommitSummary { +class CommitSummary { private final AtomicLong dataFilesCount = new AtomicLong(); private final AtomicLong dataFilesRecordCount = new AtomicLong(); @@ -35,35 +34,30 @@ public class CommitSummary { private final AtomicLong deleteFilesRecordCount = new AtomicLong(); private final AtomicLong deleteFilesByteCount = new AtomicLong(); - 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()) + CommitSummary(NavigableMap pendingResults) { + pendingResults + .values() .forEach( - deleteFile -> { - deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); - deleteFilesByteCount.addAndGet(deleteBytes); + 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); + }); }); } - public long dataFilesCount() { + long dataFilesCount() { return dataFilesCount.get(); } @@ -75,7 +69,7 @@ long dataFilesByteCount() { return dataFilesByteCount.get(); } - public long deleteFilesCount() { + long deleteFilesCount() { return deleteFilesCount.get(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index 89c26692b113..036970c06d5b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ b/flink/v1.19/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; -public class DeltaManifests { +class DeltaManifests { private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; @@ -56,7 +56,7 @@ CharSequence[] referencedDataFiles() { return referencedDataFiles; } - public List manifests() { + List manifests() { List manifests = Lists.newArrayListWithCapacity(2); if (dataManifest != null) { manifests.add(dataManifest); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java index 59de599a6886..92ca284b12ba 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ b/flink/v1.19/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; -public class DeltaManifestsSerializer implements SimpleVersionedSerializer { +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]; - public static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); @Override public int getVersion() { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 5b971415a9d9..18b269d6c3e9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink; import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.Schema; @@ -29,10 +30,10 @@ import org.apache.iceberg.util.StructProjection; /** - * Create a {@link NonThrowingKeySelector} to shuffle by equality fields, to ensure same equality - * fields record will be emitted to same writer in order. + * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record + * will be emitted to same writer in order. */ -public class EqualityFieldKeySelector implements NonThrowingKeySelector { +class EqualityFieldKeySelector implements KeySelector { private final Schema schema; private final RowType flinkSchema; @@ -42,8 +43,7 @@ public class EqualityFieldKeySelector implements NonThrowingKeySelector equalityFieldIds) { + EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { this.schema = schema; this.flinkSchema = flinkSchema; this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index d107c2739b04..9571efdc5268 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -38,7 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class FlinkManifestUtil { +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( } } - public static ManifestOutputFileFactory createOutputFileFactory( + static ManifestOutputFileFactory createOutputFileFactory( Supplier

tableSupplier, Map tableProps, String flinkJobId, @@ -83,7 +83,7 @@ public static ManifestOutputFileFactory createOutputFileFactory( * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same * partition spec */ - public static DeltaManifests writeCompletedFiles( + static DeltaManifests writeCompletedFiles( WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) throws IOException { @@ -114,7 +114,7 @@ public static DeltaManifests writeCompletedFiles( return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); } - public static WriteResult readCompletedFiles( + static WriteResult readCompletedFiles( DeltaManifests deltaManifests, FileIO io, Map specsById) throws IOException { WriteResult.Builder builder = WriteResult.builder(); @@ -135,7 +135,7 @@ public static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } - public static void deleteCommittedManifests( + static void deleteCommittedManifests( Table table, List manifests, String newFlinkJobId, long checkpointId) { for (ManifestFile manifest : manifests) { try { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java index 8b06949a5519..408c3e9a9d5f 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -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. */ -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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java index 1d83c211e001..e2b388a83c75 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -30,7 +30,7 @@ * *

In both cases only the respective part is serialized. */ -public class IcebergCommittableSerializer implements SimpleVersionedSerializer { +class IcebergCommittableSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; @Override diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java index d7c35aa4f708..2245b36f15a7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -57,7 +57,7 @@ * same jobId-operatorId-checkpointId triplet * */ -public class IcebergCommitter implements Committer { +class IcebergCommitter implements Committer { private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; public static final WriteResult EMPTY_WRITE_RESULT = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 9a50b7a900c5..609deb621f43 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -76,7 +76,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // the max committed one to iceberg table, for avoiding committing the same data files twice. This // id will be attached to iceberg's meta when committing the iceberg transaction. private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - public static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; // TableLoader to load iceberg table lazily. private final TableLoader tableLoader; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 86af9c5154f7..5b28c4acb1c5 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v1.19/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; -public class IcebergFilesCommitterMetrics { +class IcebergFilesCommitterMetrics { private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); private final AtomicLong lastCommitDurationMs = new AtomicLong(); private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; @@ -35,7 +35,7 @@ public class IcebergFilesCommitterMetrics { private final Counter committedDeleteFilesRecordCount; private final Counter committedDeleteFilesByteCount; - public IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { + IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { MetricGroup committerMetrics = metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); @@ -52,16 +52,16 @@ public IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); } - public void checkpointDuration(long checkpointDurationMs) { + void checkpointDuration(long checkpointDurationMs) { lastCheckpointDurationMs.set(checkpointDurationMs); } - public void commitDuration(long commitDurationMs) { + void commitDuration(long commitDurationMs) { lastCommitDurationMs.set(commitDurationMs); } /** This is called upon a successful commit. */ - public void updateCommitSummary(CommitSummary stats) { + void updateCommitSummary(CommitSummary stats) { elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); committedDataFilesCount.inc(stats.dataFilesCount()); committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index c88de85a25ed..d080169544cd 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -571,7 +571,7 @@ private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table return (SerializableTable) SerializableTable.copyOf(table); } - public static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { if (requestedSchema != null) { // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing // iceberg schema. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index 04ea868c0a38..ab458ad2e7cb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.19/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; -public class IcebergStreamWriterMetrics { +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 @@ public class IcebergStreamWriterMetrics { private final Histogram dataFilesSizeHistogram; private final Histogram deleteFilesSizeHistogram; - public IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { + IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { MetricGroup writerMetrics = metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); @@ -63,7 +63,7 @@ public IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); } - public void updateFlushResult(WriteResult result) { + void updateFlushResult(WriteResult result) { flushedDataFiles.inc(result.dataFiles().length); flushedDeleteFiles.inc(result.deleteFiles().length); flushedReferencedDataFiles.inc(result.referencedDataFiles().length); @@ -84,7 +84,7 @@ public void updateFlushResult(WriteResult result) { }); } - public void flushDuration(long flushDurationMs) { + void flushDuration(long flushDurationMs) { lastFlushDurationMs.set(flushDurationMs); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java index 2b3c53b0e31a..794ade577976 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -40,8 +40,7 @@ * IcebergCommittable} per checkpoint (storing the serialized {@link * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) */ -public class IcebergWriteAggregator - extends AbstractStreamOperator> +class IcebergWriteAggregator extends AbstractStreamOperator> implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 81434ad171fb..30517cd38216 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -30,7 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; -public class ManifestOutputFileFactory { +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())); } - public OutputFile create(long checkpointId) { + OutputFile create(long checkpointId) { String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java deleted file mode 100644 index a9953ea8bd01..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink; - -import org.apache.flink.api.java.functions.KeySelector; - -/** A non-throwing variant of Flink's {@link KeySelector}. */ -public interface NonThrowingKeySelector extends KeySelector { - - @Override - K getKey(I value); -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java index f10dc710200d..df951684b446 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.PartitionKey; @@ -26,11 +27,11 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * Create a {@link NonThrowingKeySelector} to shuffle by partition key, then each partition/bucket - * will be wrote by only one task. That will reduce lots of small files in partitioned fanout write - * policy for {@link FlinkSink}. + * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be + * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy + * for {@link FlinkSink}. */ -public class PartitionKeySelector implements NonThrowingKeySelector { +class PartitionKeySelector implements KeySelector { private final Schema schema; private final PartitionKey partitionKey; @@ -38,7 +39,7 @@ public class PartitionKeySelector implements NonThrowingKeySelector 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; @@ -112,9 +90,9 @@ public RowDataTaskWriterFactory( table = tableSupplier.get(); } - this.schema = schema; + this.schema = table.schema(); this.flinkSchema = flinkSchema; - this.spec = spec; + this.spec = table.spec(); this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; this.equalityFieldIds = equalityFieldIds; @@ -170,7 +148,6 @@ 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java index 34868458b0e0..5a44373cccaa 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -26,7 +26,7 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.iceberg.io.WriteResult; -public class WriteResultSerializer implements SimpleVersionedSerializer { +class WriteResultSerializer implements SimpleVersionedSerializer { private static final int VERSION = 1; @Override diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java deleted file mode 100644 index 99abc95d7102..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java +++ /dev/null @@ -1,254 +0,0 @@ -/* - * 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.List; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.schema.SchemaWithPartnerVisitor; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -/** Visitor class which compares two schemas and decides whether they are compatible. */ -public class CompareSchemasVisitor - extends SchemaWithPartnerVisitor { - - private final Schema tableSchema; - - private CompareSchemasVisitor(Schema tableSchema) { - this.tableSchema = tableSchema; - } - - public static Result visit(Schema dataSchema, Schema tableSchema) { - return visit(dataSchema, tableSchema, true); - } - - public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { - return visit( - dataSchema, - -1, - new CompareSchemasVisitor(tableSchema), - new PartnerIdByNameAccessors(tableSchema, caseSensitive)); - } - - @Override - public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - return downstream; - } - - @Override - public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - Result result = fields.stream().reduce(Result::merge).orElse(Result.INCOMPATIBLE); - - if (result == Result.INCOMPATIBLE) { - return Result.INCOMPATIBLE; - } - - Type tableSchemaType = - tableSchemaId == -1 ? tableSchema.asStruct() : tableSchema.findField(tableSchemaId).type(); - if (!tableSchemaType.isStructType()) { - return Result.INCOMPATIBLE; - } - - if (struct.fields().size() != tableSchemaType.asStructType().fields().size()) { - return Result.CONVERSION_NEEDED; - } - - for (int i = 0; i < struct.fields().size(); ++i) { - if (!struct - .fields() - .get(i) - .name() - .equals(tableSchemaType.asStructType().fields().get(i).name())) { - return Result.CONVERSION_NEEDED; - } - } - - return result; - } - - @Override - public Result field(Types.NestedField field, Integer tableSchemaId, Result typeResult) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - if (typeResult != Result.SAME) { - return typeResult; - } - - if (tableSchema.findField(tableSchemaId).isRequired() && field.isOptional()) { - return Result.INCOMPATIBLE; - } else { - return Result.SAME; - } - } - - @Override - public Result list(Types.ListType list, Integer tableSchemaId, Result elementsResult) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - return elementsResult; - } - - @Override - public Result map( - Types.MapType map, Integer tableSchemaId, Result keyResult, Result valueResult) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - return keyResult.merge(valueResult); - } - - @Override - @SuppressWarnings("checkstyle:CyclomaticComplexity") - public Result primitive(Type.PrimitiveType primitive, Integer tableSchemaId) { - if (tableSchemaId == null) { - return Result.INCOMPATIBLE; - } - - Type tableSchemaType = tableSchema.findField(tableSchemaId).type(); - if (!tableSchemaType.isPrimitiveType()) { - return Result.INCOMPATIBLE; - } - - Type.PrimitiveType tableSchemaPrimitiveType = tableSchemaType.asPrimitiveType(); - if (primitive.equals(tableSchemaPrimitiveType)) { - return Result.SAME; - } else if (primitive.equals(Types.IntegerType.get()) - && tableSchemaPrimitiveType.equals(Types.LongType.get())) { - return Result.CONVERSION_NEEDED; - } else if (primitive.equals(Types.FloatType.get()) - && tableSchemaPrimitiveType.equals(Types.DoubleType.get())) { - return Result.CONVERSION_NEEDED; - } else if (primitive.equals(Types.DateType.get()) - && tableSchemaPrimitiveType.equals(Types.TimestampType.withoutZone())) { - return Result.CONVERSION_NEEDED; - } else if (primitive.typeId() == Type.TypeID.DECIMAL - && tableSchemaPrimitiveType.typeId() == Type.TypeID.DECIMAL) { - Types.DecimalType dataType = (Types.DecimalType) primitive; - Types.DecimalType tableType = (Types.DecimalType) tableSchemaPrimitiveType; - return dataType.scale() == tableType.scale() && dataType.precision() < tableType.precision() - ? Result.CONVERSION_NEEDED - : Result.INCOMPATIBLE; - } else { - return Result.INCOMPATIBLE; - } - } - - static class PartnerIdByNameAccessors implements PartnerAccessors { - private final Schema tableSchema; - private boolean caseSensitive = true; - - PartnerIdByNameAccessors(Schema tableSchema) { - this.tableSchema = tableSchema; - } - - private PartnerIdByNameAccessors(Schema tableSchema, boolean caseSensitive) { - this(tableSchema); - this.caseSensitive = caseSensitive; - } - - @Override - public Integer fieldPartner(Integer tableSchemaFieldId, int fieldId, String name) { - Types.StructType struct; - if (tableSchemaFieldId == -1) { - struct = tableSchema.asStruct(); - } else { - struct = tableSchema.findField(tableSchemaFieldId).type().asStructType(); - } - - Types.NestedField field = - caseSensitive ? struct.field(name) : struct.caseInsensitiveField(name); - if (field != null) { - return field.fieldId(); - } - - return null; - } - - @Override - public Integer mapKeyPartner(Integer tableSchemaMapId) { - Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); - if (mapField != null) { - return mapField.type().asMapType().fields().get(0).fieldId(); - } - - return null; - } - - @Override - public Integer mapValuePartner(Integer tableSchemaMapId) { - Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); - if (mapField != null) { - return mapField.type().asMapType().fields().get(1).fieldId(); - } - - return null; - } - - @Override - public Integer listElementPartner(Integer tableSchemaListId) { - Types.NestedField listField = tableSchema.findField(tableSchemaListId); - if (listField != null) { - return listField.type().asListType().fields().get(0).fieldId(); - } - - return null; - } - } - - public enum Result { - SAME(0), - CONVERSION_NEEDED(1), - INCOMPATIBLE(2); - - private static final Map BY_ID = Maps.newHashMap(); - - static { - for (Result e : Result.values()) { - if (BY_ID.put(e.id, e) != null) { - throw new IllegalArgumentException("Duplicate id: " + e.id); - } - } - } - - private final int id; - - Result(int id) { - this.id = id; - } - - private Result merge(Result other) { - return BY_ID.get(Math.max(this.id, other.id)); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java deleted file mode 100644 index 5f26d3406cd1..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java deleted file mode 100644 index c885e048127f..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java deleted file mode 100644 index 205b74384710..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ /dev/null @@ -1,417 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java deleted file mode 100644 index 8d815d6f6923..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java deleted file mode 100644 index dd0442bc21ea..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java +++ /dev/null @@ -1,439 +0,0 @@ -/* - * 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.apache.iceberg.TableProperties.AVRO_COMPRESSION; -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; - -import java.util.Map; -import java.util.Optional; -import java.util.UUID; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.sink2.Committer; -import org.apache.flink.api.connector.sink2.CommitterInitContext; -import org.apache.flink.api.connector.sink2.Sink; -import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.connector.sink2.SupportsCommitter; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; -import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; -import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; -import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; -import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.OutputTag; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.sink.IcebergSink; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -/** - * Dynamic version of the IcebergSink which supports: - * - *
    - *
  1. Writing to any number of tables (No more 1:1 sink/topic relationship). - *
  2. Creating and updating tables based on the user-supplied routing. - *
  3. Updating the schema and partition spec of tables based on the user-supplied specification. - *
- */ -@Experimental -public class DynamicIcebergSink - implements Sink, - SupportsPreWriteTopology, - SupportsCommitter, - SupportsPreCommitTopology, - SupportsPostCommitTopology { - - private final CatalogLoader catalogLoader; - private final Map snapshotProperties; - private final String uidPrefix; - private final String sinkId; - private final Map writeProperties; - private final transient FlinkWriteConf flinkWriteConf; - private final FileFormat dataFileFormat; - private final long targetDataFileSize; - private final boolean overwriteMode; - private final int workerPoolSize; - - private DynamicIcebergSink( - CatalogLoader catalogLoader, - Map snapshotProperties, - String uidPrefix, - Map writeProperties, - FlinkWriteConf flinkWriteConf) { - this.catalogLoader = catalogLoader; - this.snapshotProperties = snapshotProperties; - this.uidPrefix = uidPrefix; - this.writeProperties = writeProperties; - this.flinkWriteConf = flinkWriteConf; - this.dataFileFormat = flinkWriteConf.dataFileFormat(); - this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); - this.overwriteMode = flinkWriteConf.overwriteMode(); - this.workerPoolSize = flinkWriteConf.workerPoolSize(); - // We generate a random UUID every time when a sink is created. - // This is used to separate files generated by different sinks writing the same table. - // Also used to generate the aggregator operator name - this.sinkId = UUID.randomUUID().toString(); - } - - @Override - public SinkWriter createWriter(InitContext context) { - return new DynamicWriter( - catalogLoader.loadCatalog(), - dataFileFormat, - targetDataFileSize, - writeProperties, - new DynamicWriterMetrics(context.metricGroup()), - context.getTaskInfo().getIndexOfThisSubtask(), - context.getTaskInfo().getAttemptNumber()); - } - - @Override - public Committer createCommitter(CommitterInitContext context) { - DynamicCommitterMetrics metrics = new DynamicCommitterMetrics(context.metricGroup()); - return new DynamicCommitter( - catalogLoader.loadCatalog(), - snapshotProperties, - overwriteMode, - workerPoolSize, - sinkId, - metrics); - } - - @Override - public SimpleVersionedSerializer getCommittableSerializer() { - return new DynamicCommittableSerializer(); - } - - @Override - public void addPostCommitTopology( - DataStream> committables) {} - - @Override - public DataStream addPreWriteTopology( - DataStream inputDataStream) { - return distributeDataStream(inputDataStream); - } - - @Override - public DataStream> addPreCommitTopology( - DataStream> writeResults) { - TypeInformation> typeInformation = - CommittableMessageTypeInfo.of(this::getCommittableSerializer); - - return writeResults - .keyBy( - committable -> { - if (committable instanceof CommittableSummary) { - return "__summary"; - } else { - CommittableWithLineage result = - (CommittableWithLineage) committable; - return result.getCommittable().key().tableName(); - } - }) - .transform( - prefixIfNotNull(uidPrefix, sinkId + " Pre Commit"), - typeInformation, - new DynamicWriteResultAggregator(catalogLoader)) - .uid(prefixIfNotNull(uidPrefix, sinkId + "-pre-commit-topology")); - } - - @Override - public SimpleVersionedSerializer getWriteResultSerializer() { - return new DynamicWriteResultSerializer(); - } - - public static class Builder { - private DataStream input; - private DynamicRecordConverter converter; - private CatalogLoader catalogLoader; - private String uidPrefix = null; - private final Map writeOptions = Maps.newHashMap(); - private final Map snapshotSummary = Maps.newHashMap(); - private ReadableConfig readableConfig = new Configuration(); - private boolean immediateUpdate = false; - private int cacheMaximumSize = 100; - private long cacheRefreshMs = 1_000; - - private Builder() {} - - public Builder forInput(DataStream inputStream) { - this.input = inputStream; - return this; - } - - public Builder withConverter(DynamicRecordConverter inputConverter) { - this.converter = inputConverter; - return this; - } - - /** - * The catalog loader is used for loading tables in {@link DynamicCommitter} lazily, we need - * this loader because {@link Table} is not serializable and could not just use the loaded table - * from Builder#table in the remote task manager. - * - * @param newCatalogLoader to load iceberg table inside tasks. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder catalogLoader(CatalogLoader newCatalogLoader) { - this.catalogLoader = newCatalogLoader; - return this; - } - - /** - * Set the write properties for IcebergSink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder set(String property, String value) { - writeOptions.put(property, value); - return this; - } - - /** - * Set the write properties for IcebergSink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder setAll(Map properties) { - writeOptions.putAll(properties); - return this; - } - - public Builder overwrite(boolean newOverwrite) { - writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - /** - * Configuring the write parallel number for iceberg stream writer. - * - * @param newWriteParallelism the number of parallel iceberg stream writer. - * @return {@link DynamicIcebergSink.Builder} to connect the iceberg table. - */ - public Builder writeParallelism(int newWriteParallelism) { - writeOptions.put( - FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); - return this; - } - - /** - * Set the uid prefix for IcebergSink operators. Note that IcebergSink internally consists of - * multiple operators (like writer, committer, aggregator) Actual operator uid will be appended - * with a suffix like "uidPrefix-writer". - * - *

If provided, this prefix is also applied to operator names. - * - *

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

Be careful with setting this for an existing job, because now we are changing the operator - * uid from an auto-generated one to this new value. When deploying the change with a - * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more - * specifically the committer operator state). You need to use {@code --allowNonRestoredState} - * to ignore the previous sink state. During restore IcebergSink state is used to check if last - * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss - * if the Iceberg commit failed in the last completed checkpoint. - * - * @param newPrefix prefix for Flink sink operator uid and name - * @return {@link Builder} to connect the iceberg table. - */ - public Builder uidPrefix(String newPrefix) { - this.uidPrefix = newPrefix; - return this; - } - - public Builder snapshotProperties(Map properties) { - snapshotSummary.putAll(properties); - return this; - } - - public Builder setSnapshotProperty(String property, String value) { - snapshotSummary.put(property, value); - return this; - } - - public Builder toBranch(String branch) { - writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); - return this; - } - - public Builder immediateTableUpdate(boolean newImmediateUpdate) { - this.immediateUpdate = newImmediateUpdate; - return this; - } - - /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ - public Builder cacheMaxSize(int maxSize) { - this.cacheMaximumSize = maxSize; - return this; - } - - /** Maximum interval for cache items renewals. */ - public Builder cacheRefreshMs(long refreshMs) { - this.cacheRefreshMs = refreshMs; - return this; - } - - private String operatorName(String suffix) { - return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; - } - - public DynamicIcebergSink build() { - - Preconditions.checkArgument( - converter != null, "Please use withConverter() to convert the input DataStream."); - Preconditions.checkNotNull(catalogLoader, "Catalog loader shouldn't be null"); - - // Init the `flinkWriteConf` here, so we can do the checks - FlinkWriteConf flinkWriteConf = new FlinkWriteConf(writeOptions, readableConfig); - - Map writeProperties = - writeProperties(flinkWriteConf.dataFileFormat(), flinkWriteConf); - - uidPrefix = Optional.ofNullable(uidPrefix).orElse(""); - - // FlinkWriteConf properties needed to be set separately, so we do not have to serialize the - // full conf - return new DynamicIcebergSink( - catalogLoader, snapshotSummary, uidPrefix, writeProperties, flinkWriteConf); - } - - /** - * Append the iceberg sink operators to write records to iceberg table. - * - * @return {@link DataStreamSink} for sink. - */ - public DataStreamSink append() { - DynamicRecordInternalType type = - new DynamicRecordInternalType(catalogLoader, false, cacheMaximumSize); - DynamicIcebergSink sink = build(); - SingleOutputStreamOperator converted = - input - .process( - new DynamicRecordProcessor<>( - converter, catalogLoader, immediateUpdate, cacheMaximumSize, cacheRefreshMs)) - .uid(prefixIfNotNull(uidPrefix, "-converter")) - .name(operatorName("Converter")) - .returns(type); - - DataStreamSink rowDataDataStreamSink = - converted - .getSideOutput( - new OutputTag<>( - DynamicRecordProcessor.DYNAMIC_TABLE_UPDATE_STREAM, - new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize))) - .keyBy((KeySelector) DynamicRecordInternal::tableName) - .map(new DynamicTableUpdateOperator(catalogLoader, cacheMaximumSize, cacheRefreshMs)) - .uid(prefixIfNotNull(uidPrefix, "-updater")) - .name(operatorName("Updater")) - .returns(type) - .union(converted) - .sinkTo(sink) - .uid(prefixIfNotNull(uidPrefix, "-sink")); - if (sink.flinkWriteConf.writeParallelism() != null) { - rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); - } - - return rowDataDataStreamSink; - } - } - - /** - * Based on the {@link FileFormat} overwrites the table level compression properties for the table - * write. - * - * @param format The FileFormat to use - * @param conf The write configuration - * @return The properties to use for writing - */ - private static Map writeProperties(FileFormat format, FlinkWriteConf conf) { - Map writeProperties = Maps.newHashMap(); - - switch (format) { - case PARQUET: - writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); - String parquetCompressionLevel = conf.parquetCompressionLevel(); - if (parquetCompressionLevel != null) { - writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); - } - - break; - case AVRO: - writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); - String avroCompressionLevel = conf.avroCompressionLevel(); - if (avroCompressionLevel != null) { - writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); - } - - break; - case ORC: - writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); - writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); - break; - default: - throw new IllegalArgumentException(String.format("Unknown file format %s", format)); - } - - return writeProperties; - } - - DataStream distributeDataStream(DataStream input) { - return input.keyBy(DynamicRecordInternal::writerKey); - } - - private static String prefixIfNotNull(String uidPrefix, String suffix) { - return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; - } - - /** - * Initialize a {@link IcebergSink.Builder} to export the data from input data stream with {@link - * RowData}s into iceberg table. - * - * @param input the source input data stream with {@link RowData}s. - * @return {@link IcebergSink.Builder} to connect the iceberg table. - */ - public static Builder forInput(DataStream input) { - return new Builder().forInput(input); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java deleted file mode 100644 index 9a79be6838d2..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java +++ /dev/null @@ -1,384 +0,0 @@ -/* - * 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.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; - -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.sink.EqualityFieldKeySelector; -import org.apache.iceberg.flink.sink.NonThrowingKeySelector; -import org.apache.iceberg.flink.sink.PartitionKeySelector; -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.Sets; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -class DynamicKeySelector implements NonThrowingKeySelector { - private static final Logger LOG = LoggerFactory.getLogger(DynamicKeySelector.class); - - private final int maxWriteParallelism; - private final Cache> keySelectorCache; - - DynamicKeySelector(int maxCacheSize, int maxWriteParallelism) { - this.maxWriteParallelism = maxWriteParallelism; - this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); - } - - @Override - public Integer getKey(Input input) { - SelectorKey cacheKey = new SelectorKey(input); - return keySelectorCache - .get( - cacheKey, - k -> - getKeySelector( - input.tableName, - input.schema, - input.spec, - input.mode, - input.equalityFields, - input.writeParallelism)) - .getKey(input.rowData); - } - - public NonThrowingKeySelector getKeySelector( - String tableName, - Schema schema, - PartitionSpec spec, - DistributionMode mode, - List equalityFields, - int writeParallelism) { - LOG.info("Write distribution mode is '{}'", mode.modeName()); - switch (mode) { - case NONE: - if (equalityFields.isEmpty()) { - return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); - } else { - LOG.info("Distribute rows by equality fields, because there are equality fields set"); - return equalityFieldKeySelector( - tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); - } - - case HASH: - if (equalityFields.isEmpty()) { - if (spec.isUnpartitioned()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and table is unpartitioned"); - return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); - } else { - return partitionKeySelector( - tableName, schema, spec, writeParallelism, maxWriteParallelism); - } - } else { - if (spec.isUnpartitioned()) { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and table is unpartitioned"); - return equalityFieldKeySelector( - tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); - } else { - for (PartitionField partitionField : spec.fields()) { - Preconditions.checkState( - equalityFields.contains(partitionField.name()), - "In 'hash' distribution mode with equality fields set, partition field '%s' " - + "should be included in equality fields: '%s'", - partitionField, - schema.columns().stream() - .filter(c -> equalityFields.contains(c.name())) - .collect(Collectors.toList())); - } - return partitionKeySelector( - tableName, schema, spec, writeParallelism, maxWriteParallelism); - } - } - - case RANGE: - if (schema.identifierFieldIds().isEmpty()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return equalityFieldKeySelector( - tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); - } - - default: - throw new IllegalArgumentException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); - } - } - - private static NonThrowingKeySelector equalityFieldKeySelector( - String tableName, - Schema schema, - List equalityFields, - int writeParallelism, - int maxWriteParallelism) { - return new TargetLimitedKeySelector( - new EqualityFieldKeySelector( - schema, - FlinkSchemaUtil.convert(schema), - DynamicRecordProcessor.getEqualityFieldIds(equalityFields, schema)), - tableName.hashCode(), - writeParallelism, - maxWriteParallelism); - } - - private static NonThrowingKeySelector partitionKeySelector( - String tableName, - Schema schema, - PartitionSpec spec, - int writeParallelism, - int maxWriteParallelism) { - NonThrowingKeySelector inner = - new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); - return new TargetLimitedKeySelector( - in -> inner.getKey(in).hashCode(), - tableName.hashCode(), - writeParallelism, - maxWriteParallelism); - } - - private static NonThrowingKeySelector tableKeySelector( - String tableName, int writeParallelism, int maxWriteParallelism) { - return new TargetLimitedKeySelector( - new RoundRobinKeySelector<>(writeParallelism), - tableName.hashCode(), - writeParallelism, - maxWriteParallelism); - } - - /** - * Generates a new key using the salt as a base, and reduces the target key range of the {@link - * #wrapped} {@link NonThrowingKeySelector} to {@link #writeParallelism}. - */ - private static class TargetLimitedKeySelector - implements NonThrowingKeySelector { - private final NonThrowingKeySelector wrapped; - private final int writeParallelism; - private final int[] distinctKeys; - - @SuppressWarnings("checkstyle:ParameterAssignment") - TargetLimitedKeySelector( - NonThrowingKeySelector wrapped, - int salt, - int writeParallelism, - int maxWriteParallelism) { - if (writeParallelism > maxWriteParallelism) { - LOG.warn( - "writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", - writeParallelism, - maxWriteParallelism, - maxWriteParallelism); - writeParallelism = maxWriteParallelism; - } - this.wrapped = wrapped; - this.writeParallelism = writeParallelism; - this.distinctKeys = new int[writeParallelism]; - - // Ensures that the generated keys are always result in unique slotId - Set targetSlots = Sets.newHashSetWithExpectedSize(writeParallelism); - int nextKey = salt; - for (int i = 0; i < writeParallelism; ++i) { - int subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); - while (targetSlots.contains(subtaskId)) { - ++nextKey; - subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); - } - - targetSlots.add(subtaskId); - distinctKeys[i] = nextKey; - ++nextKey; - } - } - - @Override - public Integer getKey(RowData value) { - return distinctKeys[Math.abs(wrapped.getKey(value).hashCode()) % writeParallelism]; - } - - private static int subtaskId(int key, int writeParallelism, int maxWriteParallelism) { - return KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( - maxWriteParallelism, - writeParallelism, - KeyGroupRangeAssignment.computeKeyGroupForKeyHash(key, maxWriteParallelism)); - } - } - - /** - * Generates evenly distributed keys between [0..{@link #maxTarget}) range using round-robin - * algorithm. - * - * @param unused input for key generation - */ - private static class RoundRobinKeySelector implements NonThrowingKeySelector { - private final int maxTarget; - private int lastTarget = 0; - - RoundRobinKeySelector(int maxTarget) { - this.maxTarget = maxTarget; - } - - @Override - public Integer getKey(T value) { - lastTarget = (lastTarget + 1) % maxTarget; - return lastTarget; - } - } - - static class Input { - private final String tableName; - private final String branch; - private final Integer schemaId; - private final Integer specId; - private final Schema schema; - private final PartitionSpec spec; - private final DistributionMode mode; - private final int writeParallelism; - private final List equalityFields; - private final RowData rowData; - - Input( - DynamicRecord dynamicRecord, - Schema schemaOverride, - PartitionSpec specOverride, - RowData rowDataOverride) { - this( - dynamicRecord.tableIdentifier().toString(), - dynamicRecord.branch(), - schemaOverride != null ? schemaOverride.schemaId() : null, - specOverride != null ? specOverride.specId() : null, - schemaOverride, - specOverride, - dynamicRecord.mode(), - dynamicRecord.writeParallelism(), - dynamicRecord.equalityFields() != null - ? dynamicRecord.equalityFields() - : Collections.emptyList(), - rowDataOverride); - } - - private Input( - String tableName, - String branch, - Integer schemaId, - Integer specId, - Schema schema, - PartitionSpec spec, - DistributionMode mode, - int writeParallelism, - List equalityFields, - RowData rowData) { - this.tableName = tableName; - this.branch = branch; - this.schemaId = schemaId; - this.specId = specId; - this.schema = schema; - this.spec = spec; - this.mode = mode; - this.writeParallelism = writeParallelism; - this.equalityFields = equalityFields; - this.rowData = rowData; - } - } - - /** - * Cache key for the {@link NonThrowingKeySelector}. Only contains the {@link Schema} and the - * {@link PartitionSpec} if the ids are not available. - */ - private static class SelectorKey { - private final String tableName; - private final String branch; - private final Integer schemaId; - private final Integer specId; - private final Schema schema; - private final PartitionSpec spec; - private final List equalityFields; - - private SelectorKey(Input input) { - this.tableName = input.tableName; - this.branch = input.branch; - this.schemaId = input.schemaId; - this.schema = schemaId == null ? input.schema : null; - this.specId = input.specId; - this.spec = specId == null ? input.spec : null; - this.equalityFields = input.equalityFields; - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - SelectorKey that = (SelectorKey) other; - return Objects.equals(tableName, that.tableName) - && Objects.equals(branch, that.branch) - && Objects.equals(schemaId, that.schemaId) - && Objects.equals(specId, that.specId) - && Objects.equals(schema, that.schema) - && Objects.equals(spec, that.spec) - && Objects.equals(equalityFields, that.equalityFields); - } - - @Override - public int hashCode() { - return Objects.hash(tableName, branch, schemaId, specId, schema, spec, equalityFields); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("tableName", tableName) - .add("branch", branch) - .add("schemaId", schemaId) - .add("specId", specId) - .add("schema", schema) - .add("spec", spec) - .add("eqalityFields", equalityFields) - .toString(); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java deleted file mode 100644 index 193d9bfc7f66..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.List; -import javax.annotation.Nullable; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.TableIdentifier; - -public class DynamicRecord { - private TableIdentifier tableIdentifier; - private String branch; - private Schema schema; - private PartitionSpec spec; - private RowData rowData; - private DistributionMode mode; - private int writeParallelism; - private boolean upsertMode; - @Nullable private List equalityFields; - - public DynamicRecord( - TableIdentifier tableIdentifier, - String branch, - Schema schema, - PartitionSpec spec, - RowData rowData, - DistributionMode mode, - int writeParallelism) { - this.tableIdentifier = tableIdentifier; - this.branch = branch; - this.schema = schema; - this.spec = spec; - this.rowData = rowData; - this.mode = mode; - this.writeParallelism = writeParallelism; - } - - public TableIdentifier tableIdentifier() { - return tableIdentifier; - } - - public void setTableIdentifier(TableIdentifier tableIdentifier) { - this.tableIdentifier = tableIdentifier; - } - - public String branch() { - return branch; - } - - public void setBranch(String branch) { - this.branch = branch; - } - - public Schema schema() { - return schema; - } - - public void setSchema(Schema schema) { - this.schema = schema; - } - - public PartitionSpec spec() { - return spec; - } - - public void setSpec(PartitionSpec spec) { - this.spec = spec; - } - - public RowData rowData() { - return rowData; - } - - public void setRowData(RowData rowData) { - this.rowData = rowData; - } - - public DistributionMode mode() { - return mode; - } - - public void setMode(DistributionMode mode) { - this.mode = mode; - } - - public int writeParallelism() { - return writeParallelism; - } - - public void writeParallelism(int parallelism) { - this.writeParallelism = parallelism; - } - - public boolean upsertMode() { - return upsertMode; - } - - public void setUpsertMode(boolean upsertMode) { - this.upsertMode = upsertMode; - } - - public List equalityFields() { - return equalityFields; - } - - public void setEqualityFields(List equalityFields) { - this.equalityFields = equalityFields; - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java deleted file mode 100644 index 612b94ffb0cf..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordConverter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.api.common.functions.OpenContext; -import org.apache.flink.util.Collector; - -/** Conversion method to return input type into a DynamicRecord */ -public interface DynamicRecordConverter extends Serializable { - default void open(OpenContext openContext) throws Exception {} - - /** Takes a user-defined input type and converts it one or multiple {@link DynamicRecord}s. */ - void convert(T inputRecord, Collector out) throws Exception; -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java deleted file mode 100644 index 25e61bd20e81..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * 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.List; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; - -@Internal -class DynamicRecordInternal { - private String tableName; - private String branch; - private Schema schema; - private PartitionSpec spec; - private int writerKey; - private RowData rowData; - private boolean upsertMode; - private List equalityFieldIds; - - DynamicRecordInternal() {} - - DynamicRecordInternal( - String tableName, - String branch, - Schema schema, - PartitionSpec spec, - int writerKey, - RowData rowData, - boolean upsertMode, - List equalityFieldsIds) { - this.tableName = tableName; - this.branch = branch; - this.schema = schema; - this.spec = spec; - this.writerKey = writerKey; - this.rowData = rowData; - this.upsertMode = upsertMode; - this.equalityFieldIds = equalityFieldsIds; - } - - public String tableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String branch() { - return branch; - } - - public void setBranch(String branch) { - this.branch = branch; - } - - public Schema schema() { - return schema; - } - - public void setSchema(Schema schema) { - this.schema = schema; - } - - public PartitionSpec spec() { - return spec; - } - - public void setSpec(PartitionSpec spec) { - this.spec = spec; - } - - public int writerKey() { - return writerKey; - } - - public void setWriterKey(int writerKey) { - this.writerKey = writerKey; - } - - public RowData rowData() { - return rowData; - } - - public void setRowData(RowData rowData) { - this.rowData = rowData; - } - - public boolean upsertMode() { - return upsertMode; - } - - public void setUpsertMode(boolean upsertMode) { - this.upsertMode = upsertMode; - } - - public List equalityFields() { - return equalityFieldIds; - } - - public void setEqualityFieldIds(List equalityFieldIds) { - this.equalityFieldIds = equalityFieldIds; - } - - @Override - public int hashCode() { - return Objects.hash( - tableName, branch, schema, spec, writerKey, rowData, upsertMode, equalityFieldIds); - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - DynamicRecordInternal that = (DynamicRecordInternal) other; - boolean tableFieldsMatch = - Objects.equals(tableName, that.tableName) - && Objects.equals(branch, that.branch) - && schema.schemaId() == that.schema.schemaId() - && Objects.equals(spec, that.spec) - && writerKey == that.writerKey - && upsertMode == that.upsertMode - && Objects.equals(equalityFieldIds, that.equalityFieldIds); - if (!tableFieldsMatch) { - return false; - } - - if (rowData.getClass().equals(that.rowData.getClass())) { - return Objects.equals(rowData, that.rowData); - } else { - RowDataSerializer rowDataSerializer = new RowDataSerializer(FlinkSchemaUtil.convert(schema)); - return rowDataSerializer - .toBinaryRow(rowData) - .equals(rowDataSerializer.toBinaryRow(that.rowData)); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java deleted file mode 100644 index 250532dfec17..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * 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.util.Collections; -import java.util.List; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -class DynamicRecordInternalSerializer extends TypeSerializer { - private static final long serialVersionUID = 1L; - - private final RowDataSerializerCache serializerCache; - private final boolean writeSchemaAndSpec; - - DynamicRecordInternalSerializer( - RowDataSerializerCache serializerCache, boolean writeSchemaAndSpec) { - this.serializerCache = serializerCache; - this.writeSchemaAndSpec = writeSchemaAndSpec; - } - - @Override - public TypeSerializer duplicate() { - return new DynamicRecordInternalSerializer( - new RowDataSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), - writeSchemaAndSpec); - } - - @Override - public DynamicRecordInternal createInstance() { - return new DynamicRecordInternal(); - } - - @Override - public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutputView) - throws IOException { - dataOutputView.writeUTF(toSerialize.tableName()); - dataOutputView.writeUTF(toSerialize.branch()); - if (writeSchemaAndSpec) { - dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); - dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); - } else { - dataOutputView.writeInt(toSerialize.schema().schemaId()); - dataOutputView.writeInt(toSerialize.spec().specId()); - } - dataOutputView.writeInt(toSerialize.writerKey()); - final Tuple3 rowDataSerializer; - if (writeSchemaAndSpec) { - rowDataSerializer = - serializerCache.serializer( - toSerialize.tableName(), toSerialize.schema(), toSerialize.spec(), null, null); - } else { - // Check that the schema id can be resolved. Not strictly necessary for serialization. - rowDataSerializer = - serializerCache.serializer( - toSerialize.tableName(), - null, - null, - toSerialize.schema().schemaId(), - toSerialize.spec().specId()); - } - rowDataSerializer.f0.serialize(toSerialize.rowData(), dataOutputView); - dataOutputView.writeBoolean(toSerialize.upsertMode()); - dataOutputView.writeInt(toSerialize.equalityFields().size()); - for (Integer equalityField : toSerialize.equalityFields()) { - dataOutputView.writeInt(equalityField); - } - } - - @Override - public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOException { - String tableName = dataInputView.readUTF(); - String branch = dataInputView.readUTF(); - Schema schema = null; - PartitionSpec spec = null; - Integer schemaId = null; - Integer specId = null; - if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); - spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); - } else { - schemaId = dataInputView.readInt(); - specId = dataInputView.readInt(); - } - - int writerKey = dataInputView.readInt(); - Tuple3 rowDataSerializer = - serializerCache.serializer(tableName, schema, spec, schemaId, specId); - RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); - boolean upsertMode = dataInputView.readBoolean(); - int numEqualityFields = dataInputView.readInt(); - final List equalityFieldIds; - if (numEqualityFields > 0) { - equalityFieldIds = Lists.newArrayList(); - } else { - equalityFieldIds = Collections.emptyList(); - } - for (int i = 0; i < numEqualityFields; i++) { - equalityFieldIds.add(dataInputView.readInt()); - } - return new DynamicRecordInternal( - tableName, - branch, - rowDataSerializer.f1, - rowDataSerializer.f2, - writerKey, - rowData, - upsertMode, - equalityFieldIds); - } - - @Override - public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputView dataInputView) - throws IOException { - String tableName = dataInputView.readUTF(); - reuse.setTableName(tableName); - String branch = dataInputView.readUTF(); - reuse.setBranch(branch); - - Schema schema = null; - PartitionSpec spec = null; - Integer schemaId = null; - Integer specId = null; - if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); - spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); - reuse.setSchema(schema); - reuse.setSpec(spec); - } else { - schemaId = dataInputView.readInt(); - specId = dataInputView.readInt(); - } - - int writerKey = dataInputView.readInt(); - reuse.setWriterKey(writerKey); - Tuple3 rowDataSerializer = - serializerCache.serializer(tableName, schema, spec, schemaId, specId); - RowData rowData = rowDataSerializer.f0.deserialize(dataInputView); - boolean upsertMode = dataInputView.readBoolean(); - int numEqualityFields = dataInputView.readInt(); - final List equalityFieldIds; - if (numEqualityFields > 0) { - equalityFieldIds = Lists.newArrayList(); - } else { - equalityFieldIds = Collections.emptyList(); - } - for (int i = 0; i < numEqualityFields; i++) { - equalityFieldIds.add(dataInputView.readInt()); - } - return new DynamicRecordInternal( - tableName, - branch, - rowDataSerializer.f1, - rowDataSerializer.f2, - writerKey, - rowData, - upsertMode, - equalityFieldIds); - } - - @Override - public DynamicRecordInternal copy(DynamicRecordInternal from) { - return new DynamicRecordInternal( - from.tableName(), - from.branch(), - from.schema(), - from.spec(), - from.writerKey(), - from.rowData(), - from.upsertMode(), - from.equalityFields()); - } - - @Override - public DynamicRecordInternal copy(DynamicRecordInternal from, DynamicRecordInternal reuse) { - reuse.setTableName(from.tableName()); - reuse.setBranch(from.branch()); - reuse.setSchema(from.schema()); - reuse.setSpec(from.spec()); - reuse.setWriterKey(from.writerKey()); - reuse.setRowData(from.rowData()); - reuse.setUpsertMode(from.upsertMode()); - reuse.setEqualityFieldIds(from.equalityFields()); - return reuse; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj instanceof DynamicRecordInternalSerializer) { - DynamicRecordInternalSerializer other = (DynamicRecordInternalSerializer) obj; - return writeSchemaAndSpec == other.writeSchemaAndSpec; - } - return false; - } - - @Override - public int hashCode() { - return Objects.hashCode(writeSchemaAndSpec); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); - } - - public static class DynamicRecordInternalTypeSerializerSnapshot - implements TypeSerializerSnapshot { - - private boolean writeSchemaAndSpec; - - // Zero args constructor is required to instantiate this class on restore - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public DynamicRecordInternalTypeSerializerSnapshot() {} - - DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { - this.writeSchemaAndSpec = writeSchemaAndSpec; - } - - @Override - public int getCurrentVersion() { - return 0; - } - - @Override - public void writeSnapshot(DataOutputView out) throws IOException { - out.writeBoolean(writeSchemaAndSpec); - } - - @Override - public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) - throws IOException { - this.writeSchemaAndSpec = in.readBoolean(); - } - - @Override - public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - TypeSerializerSnapshot oldSerializerSnapshot) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - @Override - public TypeSerializer restoreSerializer() { - // Note: We pass in a null serializer cache which would create issues if we tried to use this - // restored serializer, but since we are using {@code - // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be - // used. A new one will be created via {@code DynamicRecordInternalType}. - return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java deleted file mode 100644 index 8b22ca1a57cb..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.serialization.SerializerConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.flink.CatalogLoader; - -@Internal -class DynamicRecordInternalType extends TypeInformation { - private final CatalogLoader catalogLoader; - private final boolean writeSchemaAndSpec; - private final int cacheSize; - - DynamicRecordInternalType( - CatalogLoader catalogLoader, boolean writeSchemaAndSpec, int cacheSize) { - this.catalogLoader = catalogLoader; - this.writeSchemaAndSpec = writeSchemaAndSpec; - this.cacheSize = cacheSize; - } - - @Override - public boolean isBasicType() { - return false; - } - - @Override - public boolean isTupleType() { - return false; - } - - @Override - public int getArity() { - return 0; - } - - @Override - public int getTotalFields() { - return 1; - } - - @Override - public Class getTypeClass() { - return DynamicRecordInternal.class; - } - - @Override - public boolean isKeyType() { - return false; - } - - @Override - public TypeSerializer createSerializer(SerializerConfig serializerConfig) { - return new DynamicRecordInternalSerializer( - new RowDataSerializerCache(catalogLoader, cacheSize), writeSchemaAndSpec); - } - - @Override - @Deprecated - public TypeSerializer createSerializer(ExecutionConfig config) { - return this.createSerializer(config.getSerializerConfig()); - } - - @Override - public String toString() { - return getClass().getName(); - } - - @Override - public boolean equals(Object o) { - return canEqual(o); - } - - @Override - public int hashCode() { - return getClass().getName().hashCode(); - } - - @Override - public boolean canEqual(Object o) { - return o instanceof DynamicRecordInternalType; - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java deleted file mode 100644 index 86536e24b08e..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ /dev/null @@ -1,196 +0,0 @@ -/* - * 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.Collections; -import java.util.List; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; - -@Internal -class DynamicRecordProcessor extends ProcessFunction - implements Collector { - static final String DYNAMIC_TABLE_UPDATE_STREAM = "dynamic-table-update-stream"; - - private final DynamicRecordConverter converter; - private final CatalogLoader catalogLoader; - private final boolean immediateUpdate; - private final int cacheMaximumSize; - private final long cacheRefreshMs; - - private transient TableDataCache tableCache; - private transient DynamicKeySelector selector; - private transient TableUpdater updater; - private transient OutputTag updateStream; - private transient Collector collector; - private transient Context context; - - DynamicRecordProcessor( - DynamicRecordConverter converter, - CatalogLoader catalogLoader, - boolean immediateUpdate, - int cacheMaximumSize, - long cacheRefreshMs) { - this.converter = converter; - this.catalogLoader = catalogLoader; - this.immediateUpdate = immediateUpdate; - this.cacheMaximumSize = cacheMaximumSize; - this.cacheRefreshMs = cacheRefreshMs; - } - - @Override - public void open(OpenContext openContext) throws Exception { - super.open(openContext); - Catalog catalog = catalogLoader.loadCatalog(); - this.tableCache = new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs); - this.selector = - new DynamicKeySelector( - cacheMaximumSize, getRuntimeContext().getTaskInfo().getMaxNumberOfParallelSubtasks()); - if (immediateUpdate) { - updater = new TableUpdater(tableCache, catalog); - } - - updateStream = - new OutputTag<>( - DYNAMIC_TABLE_UPDATE_STREAM, - new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize)) {}; - - converter.open(openContext); - } - - @Override - public void processElement(T element, Context ctx, Collector out) - throws Exception { - this.context = ctx; - this.collector = out; - converter.convert(element, this); - } - - @Override - public void collect(DynamicRecord data) { - boolean exists = tableCache.exists(data.tableIdentifier()).f0; - String foundBranch = exists ? tableCache.branch(data.tableIdentifier(), data.branch()) : null; - - Tuple2 foundSchema = - exists - ? tableCache.schema(data.tableIdentifier(), data.schema()) - : TableDataCache.NOT_FOUND; - - adjustPartitionSpecToTableSchema(foundSchema.f0, data); - PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; - - if (!exists - || foundBranch == null - || foundSpec == null - || foundSchema.f1 == CompareSchemasVisitor.Result.INCOMPATIBLE) { - if (immediateUpdate) { - Tuple3 newData = - updater.update(data.tableIdentifier(), data.branch(), data.schema(), data.spec()); - emit(collector, data, newData.f0, newData.f1, newData.f2); - } else { - int writerKey = - selector.getKey( - new DynamicKeySelector.Input( - data, - foundSchema.f0 != null ? foundSchema.f0 : data.schema(), - foundSpec != null ? foundSpec : data.spec(), - data.rowData())); - context.output( - updateStream, - new DynamicRecordInternal( - data.tableIdentifier().toString(), - data.branch(), - data.schema(), - data.spec(), - writerKey, - data.rowData(), - data.upsertMode(), - getEqualityFieldIds(data.equalityFields(), data.schema()))); - } - } else { - emit(collector, data, foundSchema.f0, foundSchema.f1, foundSpec); - } - } - - private static void adjustPartitionSpecToTableSchema(Schema schema, DynamicRecord data) { - if (schema != null) { - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(schema, data.spec()); - data.setSpec(adjustedSpec); - } - } - - private void emit( - Collector out, - DynamicRecord data, - Schema schema, - CompareSchemasVisitor.Result result, - PartitionSpec spec) { - RowData rowData = - result == CompareSchemasVisitor.Result.SAME - ? data.rowData() - : RowDataEvolver.convert(data.rowData(), data.schema(), schema); - int writerKey = selector.getKey(new DynamicKeySelector.Input(data, schema, spec, rowData)); - String tableName = data.tableIdentifier().toString(); - out.collect( - new DynamicRecordInternal( - tableName, - data.branch(), - schema, - spec, - writerKey, - rowData, - data.upsertMode(), - getEqualityFieldIds(data.equalityFields(), schema))); - } - - static List getEqualityFieldIds(List equalityFields, Schema schema) { - if (equalityFields == null || equalityFields.isEmpty()) { - if (!schema.identifierFieldIds().isEmpty()) { - return Lists.newArrayList(schema.identifierFieldIds()); - } else { - return Collections.emptyList(); - } - } - List equalityFieldIds = Lists.newArrayList(equalityFields.size()); - for (String equalityField : equalityFields) { - Types.NestedField field = schema.findField(equalityField); - Preconditions.checkNotNull( - field, "Equality field %s does not exist in schema", equalityField); - equalityFieldIds.add(field.fieldId()); - } - return equalityFieldIds; - } - - @Override - public void close() {} -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java deleted file mode 100644 index 17ee853104b6..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; - -@Internal -class DynamicTableUpdateOperator - extends RichMapFunction { - private final CatalogLoader catalogLoader; - private final int cacheMaximumSize; - private final long cacheRefreshMs; - private transient TableUpdater updater; - - DynamicTableUpdateOperator( - CatalogLoader catalogLoader, int cacheMaximumSize, long cacheRefreshMs) { - this.catalogLoader = catalogLoader; - this.cacheMaximumSize = cacheMaximumSize; - this.cacheRefreshMs = cacheRefreshMs; - } - - @Override - public void open(OpenContext openContext) throws Exception { - super.open(openContext); - Catalog catalog = catalogLoader.loadCatalog(); - this.updater = - new TableUpdater(new TableDataCache(catalog, cacheMaximumSize, cacheRefreshMs), catalog); - } - - @Override - public DynamicRecordInternal map(DynamicRecordInternal data) throws Exception { - Tuple3 newData = - updater.update( - TableIdentifier.parse(data.tableName()), data.branch(), data.schema(), data.spec()); - - data.setSchema(newData.f0); - data.setSpec(newData.f2); - - if (newData.f1 == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { - RowData newRowData = RowDataEvolver.convert(data.rowData(), data.schema(), newData.f0); - data.setRowData(newRowData); - } - - return data; - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java deleted file mode 100644 index f7f6af86e39a..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java deleted file mode 100644 index 566cfb9963b7..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java deleted file mode 100644 index 4432abd19199..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java deleted file mode 100644 index bc54898bd644..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * 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 static final int CACHE_MAXIMUM_SIZE = 100; - - 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, - 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(CACHE_MAXIMUM_SIZE).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/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java deleted file mode 100644 index 7112fc40f310..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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); - } - - private IcebergStreamWriterMetrics writerMetrics(String fullTableName) { - return metrics.computeIfAbsent( - fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java deleted file mode 100644 index 6b020a5bda73..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.UpdateSchema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.schema.SchemaWithPartnerVisitor; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -/** - * Visitor class that accumulates the set of changes needed to evolve an existing schema into the - * target schema. Changes are applied to an {@link UpdateSchema} operation. - */ -public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { - - private final UpdateSchema api; - private final Schema existingSchema; - private final Schema targetSchema; - - private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { - this.api = api; - this.existingSchema = existingSchema; - this.targetSchema = targetSchema; - } - - /** - * Adds changes needed to produce the target schema to an {@link UpdateSchema} operation. - * - *

Changes are accumulated to evolve the existingSchema into a targetSchema. - * - * @param api an UpdateSchema for adding changes - * @param existingSchema an existing schema - * @param targetSchema a new schema to compare with the existing - */ - public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { - visit( - targetSchema, - -1, - new EvolveSchemaVisitor(api, existingSchema, targetSchema), - new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); - } - - @Override - public Boolean struct(Types.StructType struct, Integer partnerId, List existingFields) { - if (partnerId == null) { - return true; - } - - // Add, update and order fields in the struct - Types.StructType partnerStruct = findFieldType(partnerId).asStructType(); - String after = null; - for (Types.NestedField targetField : struct.fields()) { - Types.NestedField nestedField = partnerStruct.field(targetField.name()); - final String columnName; - if (nestedField != null) { - updateColumn(nestedField, targetField); - columnName = this.existingSchema.findColumnName(nestedField.fieldId()); - } else { - addColumn(partnerId, targetField); - columnName = this.targetSchema.findColumnName(targetField.fieldId()); - } - - setPosition(columnName, after); - after = columnName; - } - - // Ensure that unused fields are made optional - for (Types.NestedField existingField : partnerStruct.fields()) { - if (struct.field(existingField.name()) == null) { - if (existingField.isRequired()) { - this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); - } - } - } - - return false; - } - - @Override - public Boolean field(Types.NestedField field, Integer partnerId, Boolean isFieldMissing) { - return partnerId == null; - } - - @Override - public Boolean list(Types.ListType list, Integer partnerId, Boolean isElementMissing) { - if (partnerId == null) { - return true; - } - - Preconditions.checkState( - !isElementMissing, "Error traversing schemas: element is missing, but list is present"); - - Types.ListType partnerList = findFieldType(partnerId).asListType(); - updateColumn(partnerList.fields().get(0), list.fields().get(0)); - - return false; - } - - @Override - public Boolean map( - Types.MapType map, Integer partnerId, Boolean isKeyMissing, Boolean isValueMissing) { - if (partnerId == null) { - return true; - } - - Preconditions.checkState( - !isKeyMissing, "Error traversing schemas: key is missing, but map is present"); - Preconditions.checkState( - !isValueMissing, "Error traversing schemas: value is missing, but map is present"); - - Types.MapType partnerMap = findFieldType(partnerId).asMapType(); - updateColumn(partnerMap.fields().get(0), map.fields().get(0)); - updateColumn(partnerMap.fields().get(1), map.fields().get(1)); - - return false; - } - - @Override - public Boolean primitive(Type.PrimitiveType primitive, Integer partnerId) { - return partnerId == null; - } - - private Type findFieldType(int fieldId) { - if (fieldId == -1) { - return existingSchema.asStruct(); - } else { - return existingSchema.findField(fieldId).type(); - } - } - - private void addColumn(int parentId, Types.NestedField field) { - String parentName = targetSchema.findColumnName(parentId); - api.addColumn(parentName, field.name(), field.type(), field.doc()); - } - - private void updateColumn(Types.NestedField existingField, Types.NestedField targetField) { - String existingColumnName = this.existingSchema.findColumnName(existingField.fieldId()); - - boolean needsOptionalUpdate = targetField.isOptional() && existingField.isRequired(); - boolean needsTypeUpdate = - targetField.type().isPrimitiveType() && !targetField.type().equals(existingField.type()); - boolean needsDocUpdate = - targetField.doc() != null && !targetField.doc().equals(existingField.doc()); - - if (needsOptionalUpdate) { - api.makeColumnOptional(existingColumnName); - } - - if (needsTypeUpdate) { - api.updateColumn(existingColumnName, targetField.type().asPrimitiveType()); - } - - if (needsDocUpdate) { - api.updateColumnDoc(existingColumnName, targetField.doc()); - } - } - - private void setPosition(String columnName, String after) { - if (after == null) { - this.api.moveFirst(columnName); - } else { - this.api.moveAfter(columnName, after); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java deleted file mode 100644 index 12fa3f9e7f3b..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.UnboundPartitionSpec; - -public class PartitionSpecAdjustment { - - private PartitionSpecAdjustment() {} - - public static PartitionSpec adjustPartitionSpecToTableSchema( - Schema tableSchema, PartitionSpec userSpec) { - if (userSpec.isUnpartitioned()) { - return userSpec; - } - UnboundPartitionSpec.Builder builder = - UnboundPartitionSpec.builder().withSpecId(userSpec.specId()); - for (PartitionField field : userSpec.fields()) { - String sourceFieldName = userSpec.schema().idToName().get(field.sourceId()); - int adjustedSourceId = tableSchema.findField(sourceFieldName).fieldId(); - builder.addField( - field.transform().toString(), adjustedSourceId, field.fieldId(), field.name()); - } - return builder.build().bind(tableSchema); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java deleted file mode 100644 index e3746b82ce9f..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * 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.List; -import java.util.Objects; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.NamedReference; -import org.apache.iceberg.expressions.Term; -import org.apache.iceberg.expressions.UnboundTransform; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** Checks compatibility of PartitionSpecs and evolves one into the other. */ -public class PartitionSpecEvolver { - - private PartitionSpecEvolver() {} - - /** - * Checks whether two PartitionSpecs are compatible with each other. Less strict than {@code - * PartitionSpec#compatible} in the sense that it tolerates differently named partition fields, as - * long as their transforms and source ids match. - */ - public static boolean checkCompatibility(PartitionSpec first, PartitionSpec second) { - if (first.equals(second)) { - return true; - } - - if (first.fields().size() != second.fields().size()) { - return false; - } - - for (int i = 0; i < first.fields().size(); i++) { - PartitionField firstField = first.fields().get(i); - PartitionField secondField = second.fields().get(i); - if (firstField.sourceId() != secondField.sourceId() - || !firstField.transform().toString().equals(secondField.transform().toString())) { - return false; - } - } - - return true; - } - - public static PartitionSpecEvolverResult evolve( - PartitionSpec currentSpec, PartitionSpec targetSpec, Schema schema) { - if (currentSpec.compatibleWith(targetSpec)) { - return new PartitionSpecEvolverResult(); - } - - PartitionSpecEvolverResult result = new PartitionSpecEvolverResult(); - - int maxNumFields = Math.max(currentSpec.fields().size(), targetSpec.fields().size()); - for (int i = 0; i < maxNumFields; i++) { - PartitionField currentField = Iterables.get(currentSpec.fields(), i, null); - PartitionField targetField = Iterables.get(targetSpec.fields(), i, null); - - if (!Objects.equals(currentField, targetField)) { - if (currentField != null) { - result.remove(toTerm(currentField, schema)); - } - - if (targetField != null) { - result.add(toTerm(targetField, schema)); - } - } - } - - return result; - } - - public static class PartitionSpecEvolverResult { - private final List termsToAdd = Lists.newArrayList(); - private final List termsToRemove = Lists.newArrayList(); - - public void add(Term term) { - termsToAdd.add(term); - } - - public void remove(Term term) { - termsToRemove.add(term); - } - - public List termsToAdd() { - return termsToAdd; - } - - public List termsToRemove() { - return termsToRemove; - } - - public boolean isEmpty() { - return termsToAdd.isEmpty() && termsToRemove.isEmpty(); - } - } - - private static Term toTerm(PartitionField field, Schema schema) { - String sourceName = schema.findField(field.sourceId()).name(); - return new UnboundTransform<>(new NamedReference<>(sourceName), field.transform()); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java deleted file mode 100644 index f5f7cd2c4c3d..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * 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.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public class RowDataEvolver { - private RowDataEvolver() {} - - public static RowData convert(RowData sourceData, Schema sourceSchema, Schema targetSchema) { - return convertStruct( - sourceData, FlinkSchemaUtil.convert(sourceSchema), FlinkSchemaUtil.convert(targetSchema)); - } - - private static Object convert(Object object, LogicalType sourceType, LogicalType targetType) { - if (object == null) { - return null; - } - - switch (targetType.getTypeRoot()) { - case BOOLEAN: - case INTEGER: - case FLOAT: - case VARCHAR: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - case BINARY: - case VARBINARY: - return object; - case DOUBLE: - if (object instanceof Float) { - return ((Float) object).doubleValue(); - } else { - return object; - } - case BIGINT: - if (object instanceof Integer) { - return ((Integer) object).longValue(); - } else { - return object; - } - case DECIMAL: - DecimalType toDecimalType = (DecimalType) targetType; - DecimalData decimalData = (DecimalData) object; - if (((DecimalType) sourceType).getPrecision() == toDecimalType.getPrecision()) { - return object; - } else { - return DecimalData.fromBigDecimal( - decimalData.toBigDecimal(), toDecimalType.getPrecision(), toDecimalType.getScale()); - } - case TIMESTAMP_WITHOUT_TIME_ZONE: - if (object instanceof Integer) { - LocalDateTime dateTime = - LocalDateTime.of(LocalDate.ofEpochDay((Integer) object), LocalTime.MIN); - return TimestampData.fromLocalDateTime(dateTime); - } else { - return object; - } - case ROW: - return convertStruct((RowData) object, (RowType) sourceType, (RowType) targetType); - case ARRAY: - return convertArray((ArrayData) object, (ArrayType) sourceType, (ArrayType) targetType); - case MAP: - return convertMap((MapData) object, (MapType) sourceType, (MapType) targetType); - default: - throw new UnsupportedOperationException("Not a supported type: " + targetType); - } - } - - private static RowData convertStruct(RowData sourceData, RowType sourceType, RowType targetType) { - GenericRowData targetData = new GenericRowData(targetType.getFields().size()); - List targetFields = targetType.getFields(); - for (int i = 0; i < targetFields.size(); i++) { - RowType.RowField targetField = targetFields.get(i); - - int sourceFieldId = sourceType.getFieldIndex(targetField.getName()); - if (sourceFieldId == -1) { - if (targetField.getType().isNullable()) { - targetData.setField(i, null); - } else { - throw new IllegalArgumentException( - String.format( - "Field %s in target schema %s is non-nullable but does not exist in source schema.", - i + 1, targetType)); - } - } else { - RowData.FieldGetter getter = - RowData.createFieldGetter(sourceType.getTypeAt(sourceFieldId), sourceFieldId); - targetData.setField( - i, - convert( - getter.getFieldOrNull(sourceData), - sourceType.getFields().get(sourceFieldId).getType(), - targetField.getType())); - } - } - - return targetData; - } - - private static ArrayData convertArray( - ArrayData sourceData, ArrayType sourceType, ArrayType targetType) { - LogicalType fromElementType = sourceType.getElementType(); - LogicalType toElementType = targetType.getElementType(); - ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(fromElementType); - Object[] convertedArray = new Object[sourceData.size()]; - for (int i = 0; i < convertedArray.length; i++) { - convertedArray[i] = - convert(elementGetter.getElementOrNull(sourceData, i), fromElementType, toElementType); - } - return new GenericArrayData(convertedArray); - } - - private static MapData convertMap(MapData sourceData, MapType sourceType, MapType targetType) { - LogicalType fromMapKeyType = sourceType.getKeyType(); - LogicalType fromMapValueType = sourceType.getValueType(); - LogicalType toMapKeyType = targetType.getKeyType(); - LogicalType toMapValueType = targetType.getValueType(); - ArrayData keyArray = sourceData.keyArray(); - ArrayData valueArray = sourceData.valueArray(); - ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(fromMapKeyType); - ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(fromMapValueType); - Map convertedMap = Maps.newLinkedHashMap(); - for (int i = 0; i < keyArray.size(); ++i) { - convertedMap.put( - convert(keyGetter.getElementOrNull(keyArray, i), fromMapKeyType, toMapKeyType), - convert(valueGetter.getElementOrNull(valueArray, i), fromMapValueType, toMapValueType)); - } - - return new GenericMapData(convertedMap); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java deleted file mode 100644 index 68269a32ee6e..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.Serializable; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class RowDataSerializerCache implements Serializable { - private final CatalogLoader catalogLoader; - private final int maximumSize; - private transient Cache serializers; - - RowDataSerializerCache(CatalogLoader catalogLoader, int maximumSize) { - this.catalogLoader = catalogLoader; - this.maximumSize = maximumSize; - } - - Tuple3 serializer( - String tableName, - Schema unknownSchema, - PartitionSpec unknownSpec, - Integer schemaId, - Integer specId) { - if (serializers == null) { - // We need to initialize the cache at the first time - this.serializers = Caffeine.newBuilder().maximumSize(maximumSize).build(); - } - - SerializerInfo info = serializers.get(tableName, SerializerInfo::new); - Schema schema = unknownSchema != null ? unknownSchema : info.schemas.get(schemaId); - PartitionSpec spec = unknownSpec != null ? unknownSpec : info.specs.get(specId); - - if (schema == null || spec == null) { - info.update(); - schema = info.schemas.get(schemaId); - spec = info.specs.get(specId); - } - - RowDataSerializer serializer = - info.serializers.computeIfAbsent( - schema, s -> new RowDataSerializer(FlinkSchemaUtil.convert(s))); - - return Tuple3.of(serializer, schema, spec); - } - - CatalogLoader catalogLoader() { - return catalogLoader; - } - - int maximumSize() { - return maximumSize; - } - - private class SerializerInfo { - private final String tableName; - private final Map serializers; - private Map schemas; - private Map specs; - - SerializerInfo(String tableName) { - this.tableName = tableName; - this.serializers = Maps.newHashMapWithExpectedSize(2); - this.schemas = Maps.newHashMapWithExpectedSize(0); - this.specs = Maps.newHashMapWithExpectedSize(0); - } - - private void update() { - Table table = catalogLoader.loadCatalog().loadTable(TableIdentifier.parse(tableName)); - schemas = table.schemas(); - specs = table.specs(); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java deleted file mode 100644 index a86a4b878405..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java +++ /dev/null @@ -1,257 +0,0 @@ -/* - * 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.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -class TableDataCache { - private static final Logger LOG = LoggerFactory.getLogger(TableDataCache.class); - private static final int MAX_SIZE = 10; - private static final Tuple2 EXISTS = Tuple2.of(true, null); - private static final Tuple2 NOT_EXISTS = Tuple2.of(false, null); - static final Tuple2 NOT_FOUND = - Tuple2.of(null, CompareSchemasVisitor.Result.INCOMPATIBLE); - - private final Catalog catalog; - private final long refreshMs; - private final Cache cache; - - TableDataCache(Catalog catalog, int maximumSize, long refreshMs) { - this.catalog = catalog; - this.refreshMs = refreshMs; - this.cache = Caffeine.newBuilder().maximumSize(maximumSize).build(); - } - - Tuple2 exists(TableIdentifier identifier) { - CacheItem cached = cache.getIfPresent(identifier); - if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { - return EXISTS; - } else if (needsRefresh(cached, true)) { - return refreshTable(identifier); - } else { - return NOT_EXISTS; - } - } - - String branch(TableIdentifier identifier, String branch) { - return branch(identifier, branch, true); - } - - Tuple2 schema(TableIdentifier identifier, Schema input) { - return schema(identifier, input, true); - } - - PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { - return spec(identifier, spec, true); - } - - void update(TableIdentifier identifier, Table table) { - cache.put( - identifier, - new CacheItem(true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); - } - - private String branch(TableIdentifier identifier, String branch, boolean allowRefresh) { - CacheItem cached = cache.getIfPresent(identifier); - if (cached != null && cached.tableExists && cached.branches.contains(branch)) { - return branch; - } - - if (needsRefresh(cached, allowRefresh)) { - refreshTable(identifier); - return branch(identifier, branch, false); - } else { - return null; - } - } - - private Tuple2 schema( - TableIdentifier identifier, Schema input, boolean allowRefresh) { - CacheItem cached = cache.getIfPresent(identifier); - Schema compatible = null; - if (cached != null && cached.tableExists) { - // This only works if the {@link Schema#equals(Object)} returns true for the old schema - // and a new schema. Performance is paramount as this code is on the hot path. Every other - // way for comparing 2 schemas were performing worse than the - // {@link CompareByNameVisitor#visit(Schema, Schema, boolean)}, so caching was useless. - Tuple2 lastResult = - cached.schema.lastResults.get(input); - if (lastResult != null) { - return lastResult; - } - - for (Map.Entry tableSchema : cached.schema.schemas.entrySet()) { - CompareSchemasVisitor.Result result = - CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); - if (result == CompareSchemasVisitor.Result.SAME) { - Tuple2 newResult = - Tuple2.of(tableSchema.getValue(), CompareSchemasVisitor.Result.SAME); - cached.schema.update(input, newResult); - return newResult; - } else if (compatible == null && result == CompareSchemasVisitor.Result.CONVERSION_NEEDED) { - compatible = tableSchema.getValue(); - } - } - } - - if (needsRefresh(cached, allowRefresh)) { - refreshTable(identifier); - return schema(identifier, input, false); - } else if (compatible != null) { - Tuple2 newResult = - Tuple2.of(compatible, CompareSchemasVisitor.Result.CONVERSION_NEEDED); - cached.schema.update(input, newResult); - return newResult; - } else if (cached != null && cached.tableExists) { - cached.schema.update(input, NOT_FOUND); - return NOT_FOUND; - } else { - return NOT_FOUND; - } - } - - private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boolean allowRefresh) { - CacheItem cached = cache.getIfPresent(identifier); - if (cached != null && cached.tableExists) { - for (PartitionSpec tableSpec : cached.specs.values()) { - if (PartitionSpecEvolver.checkCompatibility(tableSpec, spec)) { - return tableSpec; - } - } - } - - if (needsRefresh(cached, allowRefresh)) { - refreshTable(identifier); - return spec(identifier, spec, false); - } else { - return null; - } - } - - private Tuple2 refreshTable(TableIdentifier identifier) { - try { - Table table = catalog.loadTable(identifier); - cache.put( - identifier, - new CacheItem( - true, table.refs().keySet(), new SchemaInfo(table.schemas()), table.specs())); - return EXISTS; - } catch (NoSuchTableException e) { - LOG.debug("Table doesn't exist {}", identifier, e); - cache.put(identifier, new CacheItem(false, null, null, null)); - return Tuple2.of(false, e); - } - } - - private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { - return allowRefresh - && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); - } - - public void invalidate(TableIdentifier identifier) { - cache.invalidate(identifier); - } - - /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ - static class CacheItem { - private final long created = System.currentTimeMillis(); - - private final boolean tableExists; - private final Set branches; - private final SchemaInfo schema; - private final Map specs; - - private CacheItem( - boolean tableExists, - Set branches, - SchemaInfo schema, - Map specs) { - this.tableExists = tableExists; - this.branches = branches; - this.schema = schema; - this.specs = specs; - } - - @VisibleForTesting - SchemaInfo getSchemaInfo() { - return schema; - } - } - - /** - * Stores precalculated results for {@link CompareSchemasVisitor#visit(Schema, Schema, boolean)} - * in the cache. - */ - static class SchemaInfo { - private final Map schemas; - private final Map> lastResults; - - private SchemaInfo(Map schemas) { - this.schemas = schemas; - this.lastResults = new LimitedLinkedHashMap<>(); - } - - private void update( - Schema newLastSchema, Tuple2 newLastResult) { - lastResults.put(newLastSchema, newLastResult); - } - - @VisibleForTesting - Tuple2 getLastResult(Schema schema) { - return lastResults.get(schema); - } - } - - @SuppressWarnings("checkstyle:IllegalType") - private static class LimitedLinkedHashMap extends LinkedHashMap { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - boolean remove = size() > MAX_SIZE; - if (remove) { - LOG.warn( - "Performance degraded as records with different schema is generated for the same table. " - + "Likely the DynamicRecord.schema is not reused. " - + "Reuse the same instance if the record schema is the same to improve performance"); - } - - return remove; - } - } - - @VisibleForTesting - Cache getInternalCache() { - return cache; - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java deleted file mode 100644 index 7be1d247f5d1..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.UpdatePartitionSpec; -import org.apache.iceberg.UpdateSchema; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.exceptions.CommitFailedException; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -class TableUpdater { - private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); - private final TableDataCache cache; - private final Catalog catalog; - - TableUpdater(TableDataCache cache, Catalog catalog) { - this.cache = cache; - this.catalog = catalog; - } - - /** - * Creates or updates a table to make sure that the given branch, schema, spec exists. - * - * @return a {@link Tuple3} of the new {@link Schema}, the status of the schema compared to the - * requested one, and the new {@link PartitionSpec#specId()}. - */ - Tuple3 update( - TableIdentifier tableIdentifier, String branch, Schema schema, PartitionSpec spec) { - findOrCreateTable(tableIdentifier, schema, spec); - findOrCreateBranch(tableIdentifier, branch); - Tuple2 newSchema = - findOrCreateSchema(tableIdentifier, schema); - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(newSchema.f0, spec); - PartitionSpec newSpec = findOrCreateSpec(tableIdentifier, adjustedSpec, newSchema.f0); - return Tuple3.of(newSchema.f0, newSchema.f1, newSpec); - } - - private void findOrCreateTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { - Tuple2 exists = cache.exists(identifier); - if (Boolean.FALSE.equals(exists.f0)) { - if (exists.f1 instanceof NoSuchNamespaceException) { - SupportsNamespaces catalogWithNameSpace = (SupportsNamespaces) catalog; - LOG.info("Namespace {} not found during table search. Creating namespace", identifier); - try { - catalogWithNameSpace.createNamespace(identifier.namespace()); - } catch (AlreadyExistsException e) { - LOG.debug("Namespace {} created concurrently", identifier.namespace(), e); - } - - createTable(identifier, schema, spec); - } else { - LOG.info("Table {} not found during table search. Creating table.", identifier); - createTable(identifier, schema, spec); - } - } - } - - private void createTable(TableIdentifier identifier, Schema schema, PartitionSpec spec) { - try { - Table table = catalog.createTable(identifier, schema, spec); - cache.update(identifier, table); - } catch (AlreadyExistsException e) { - LOG.info("Table {} created concurrently. Skipping creation.", identifier, e); - } - } - - private void findOrCreateBranch(TableIdentifier identifier, String branch) { - String fromCache = cache.branch(identifier, branch); - if (fromCache == null) { - try { - // TODO: Which snapshot should be used to create the branch? - catalog.loadTable(identifier).manageSnapshots().createBranch(branch).commit(); - LOG.info("Branch {} for {} created", branch, identifier); - } catch (Exception e) { - LOG.info( - "Failed to create branch {} for {}. Maybe created concurrently?", - branch, - identifier, - e); - } - } - } - - private Tuple2 findOrCreateSchema( - TableIdentifier identifier, Schema schema) { - Tuple2 fromCache = cache.schema(identifier, schema); - if (fromCache.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { - return fromCache; - } else { - Table table = catalog.loadTable(identifier); - Schema tableSchema = table.schema(); - CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); - switch (result) { - case SAME: - case CONVERSION_NEEDED: - cache.update(identifier, table); - return Tuple2.of(tableSchema, result); - case INCOMPATIBLE: - LOG.info( - "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); - UpdateSchema updateApi = table.updateSchema(); - EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); - - try { - updateApi.commit(); - cache.invalidate(identifier); - Tuple2 comparisonAfterMigration = - cache.schema(identifier, schema); - Schema newSchema = comparisonAfterMigration.f0; - LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); - return comparisonAfterMigration; - } catch (CommitFailedException e) { - LOG.info( - "Schema update failed for {} from {} to {}", identifier, tableSchema, schema, e); - Tuple2 newSchema = - cache.schema(identifier, schema); - if (newSchema.f1 != CompareSchemasVisitor.Result.INCOMPATIBLE) { - LOG.info("Table {} schema updated concurrently to {}", identifier, schema); - return newSchema; - } else { - throw e; - } - } - default: - throw new IllegalArgumentException("Unknown comparison result"); - } - } - } - - private PartitionSpec findOrCreateSpec( - TableIdentifier identifier, PartitionSpec targetSpec, Schema newSchema) { - PartitionSpec currentSpec = cache.spec(identifier, targetSpec); - if (currentSpec != null) { - return currentSpec; - } - - Table table = catalog.loadTable(identifier); - currentSpec = table.spec(); - - PartitionSpecEvolver.PartitionSpecEvolverResult result = - PartitionSpecEvolver.evolve(currentSpec, targetSpec, newSchema); - if (result.isEmpty()) { - LOG.info("Returning equivalent existing spec {} for {}", currentSpec, targetSpec); - return currentSpec; - } - - LOG.info( - "Spec for table {} has been altered. Updating from {} to {}", - identifier, - currentSpec, - targetSpec); - UpdatePartitionSpec updater = table.updateSpec(); - result.termsToRemove().forEach(updater::removeField); - result.termsToAdd().forEach(updater::addField); - - try { - updater.commit(); - } catch (CommitFailedException e) { - LOG.info( - "Partition spec update failed for {} from {} to {}", - identifier, - currentSpec, - targetSpec, - e); - PartitionSpec newSpec = cache.spec(identifier, targetSpec); - Schema maybeUpdatedSchema = cache.schema(identifier, newSchema).f0; - result = PartitionSpecEvolver.evolve(targetSpec, newSpec, maybeUpdatedSchema); - if (result.isEmpty()) { - LOG.info("Table {} partition spec updated concurrently to {}", identifier, newSpec); - return newSpec; - } else { - throw e; - } - } - - cache.invalidate(identifier); - return cache.spec(identifier, targetSpec); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java deleted file mode 100644 index 43e94c2274a4..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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(); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ffa1db71d652..1767f774922a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -82,19 +82,12 @@ private SimpleDataUtil() {} Types.NestedField.optional(1, "id", Types.IntegerType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); - public static final Schema SCHEMA2 = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "extra", Types.StringType.get())); - public static final TableSchema FLINK_SCHEMA = TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); public static final Record RECORD = GenericRecord.create(SCHEMA); - public static final Record RECORD2 = GenericRecord.create(SCHEMA2); public static Table createTable( String path, Map properties, boolean partitioned) { @@ -114,14 +107,6 @@ public static Record createRecord(Integer id, String data) { return record; } - public static Record createRecord(Integer id, String data, String extra) { - Record record = RECORD2.copy(); - record.setField("id", id); - record.setField("data", data); - record.setField("extra", extra); - return record; - } - public static RowData createRowData(Integer id, String data) { return GenericRowData.of(id, StringData.fromString(data)); } @@ -239,12 +224,7 @@ private static List convertToRecords(List rows) { for (RowData row : rows) { Integer id = row.isNullAt(0) ? null : row.getInt(0); String data = row.isNullAt(1) ? null : row.getString(1).toString(); - if (row.getArity() == 2) { - records.add(createRecord(id, data)); - } else { - String extra = row.isNullAt(2) ? null : row.getString(2).toString(); - records.add(createRecord(id, data, extra)); - } + records.add(createRecord(id, data)); } return records; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 90dcfe500417..6f8fc518a8f9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -131,7 +131,7 @@ public static List convertRowDataToRow(List rowDataList, RowType r .collect(Collectors.toList()); } - public static List convertRecordToRow(List expectedRecords, Schema schema) { + private static List convertRecordToRow(List expectedRecords, Schema schema) { List expected = Lists.newArrayList(); @SuppressWarnings("unchecked") DataStructureConverter converter = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index 29db9f6158c8..9ce36cc1e8d0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -21,7 +21,6 @@ import static org.apache.iceberg.flink.TestFixtures.DATABASE; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -65,8 +64,8 @@ public class TestFlinkIcebergSinkBase { protected Table table; protected StreamExecutionEnvironment env; - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(Collections.singletonList(rows)); + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); } protected List createRows(String prefix) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java deleted file mode 100644 index 0c5794f18c01..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.apache.iceberg.types.Types.NestedField.required; - -import java.util.List; -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.extension.RegisterExtension; - -/** - * Test base for DynamicRecordInternalSerializer which allows to instantiate different serializer - * version, e.g. with writing the schema itself or just the schema id. - */ -abstract class DynamicRecordInternalSerializerTestBase - extends SerializerTestBase { - - static final String TABLE = "myTable"; - static final String BRANCH = "myBranch"; - - @RegisterExtension - static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", TABLE); - - static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - required(2, "data", Types.StringType.get()), - required(3, "number", Types.FloatType.get())); - - static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); - - @Override - protected DynamicRecordInternal[] getTestData() { - GenericRowData rowData = new GenericRowData(3); - rowData.setField(0, 123L); - rowData.setField(1, StringData.fromString("test")); - rowData.setField(2, 1.23f); - - return new DynamicRecordInternal[] { - new DynamicRecordInternal(TABLE, BRANCH, SCHEMA, SPEC, 42, rowData, true, List.of(1, 3)) - }; - } - - @Override - protected Class getTypeClass() { - return DynamicRecordInternal.class; - } - - @Override - protected int getLength() { - return -1; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java deleted file mode 100644 index 0bf454edc2e9..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaIdTest.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.jupiter.api.BeforeEach; - -/** Test writing only the schema id on serialization. */ -class DynamicRecordInternalSerializerWriteSchemaIdTest - extends DynamicRecordInternalSerializerTestBase { - - final boolean writeSchemaAndSpec = false; - - @BeforeEach - void before() { - CATALOG_EXTENSION.catalog().createTable(TableIdentifier.parse(TABLE), SCHEMA, SPEC); - } - - @Override - protected TypeSerializer createSerializer() { - return new DynamicRecordInternalSerializer( - new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java deleted file mode 100644 index c51e54b4b804..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerWriteSchemaTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.api.common.typeutils.TypeSerializer; - -/** Test the full schema */ -class DynamicRecordInternalSerializerWriteSchemaTest - extends DynamicRecordInternalSerializerTestBase { - - final boolean writeSchemaAndSpec = true; - - @Override - protected TypeSerializer createSerializer() { - return new DynamicRecordInternalSerializer( - new RowDataSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeSchemaAndSpec); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java deleted file mode 100644 index 6edebaecce2e..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * 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.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Types.IntegerType; -import org.apache.iceberg.types.Types.ListType; -import org.apache.iceberg.types.Types.LongType; -import org.apache.iceberg.types.Types.MapType; -import org.apache.iceberg.types.Types.StringType; -import org.apache.iceberg.types.Types.StructType; -import org.junit.jupiter.api.Test; - -class TestCompareSchemasVisitor { - - @Test - void testSchema() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get(), "comment"), - optional(2, "data", StringType.get()), - optional(3, "extra", StringType.get())), - new Schema( - optional(1, "id", IntegerType.get(), "comment"), - optional(2, "data", StringType.get()), - optional(3, "extra", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testSchemaDifferentId() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "data", StringType.get()), - optional(2, "extra", StringType.get())), - new Schema( - optional(1, "id", IntegerType.get()), - optional(2, "data", StringType.get()), - optional(3, "extra", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testSchemaDifferent() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "data", StringType.get()), - optional(2, "extra", StringType.get())), - new Schema( - optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - } - - @Test - void testSchemaWithMoreColumns() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(0, "id", IntegerType.get()), optional(1, "data", StringType.get())), - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "data", StringType.get()), - optional(2, "extra", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); - } - - @Test - void testDifferentType() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())), - new Schema( - optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - } - - @Test - void testCompatibleType() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())), - new Schema( - optional(1, "id", LongType.get()), optional(2, "extra", StringType.get())))) - .isEqualTo(CompareSchemasVisitor.Result.CONVERSION_NEEDED); - } - - @Test - void testWithRequiredChange() { - Schema dataSchema = - new Schema(optional(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); - Schema tableSchema = - new Schema(required(1, "id", IntegerType.get()), optional(2, "extra", StringType.get())); - assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema)) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - assertThat(CompareSchemasVisitor.visit(tableSchema, dataSchema)) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testStructDifferentId() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get()), - optional(2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))), - new Schema( - optional(0, "id", IntegerType.get()), - optional( - 1, "struct1", StructType.of(optional(2, "extra", IntegerType.get())))))) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testStructChanged() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "struct1", StructType.of(optional(2, "extra", LongType.get())))), - new Schema( - optional(1, "id", IntegerType.get()), - optional( - 2, "struct1", StructType.of(optional(3, "extra", IntegerType.get())))))) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - } - - @Test - void testMapDifferentId() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get()), - optional( - 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))), - new Schema( - optional(0, "id", IntegerType.get()), - optional( - 1, "map1", MapType.ofOptional(2, 3, IntegerType.get(), StringType.get()))))) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testMapChanged() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get()), - optional( - 2, "map1", MapType.ofOptional(3, 4, LongType.get(), StringType.get()))), - new Schema( - optional(1, "id", IntegerType.get()), - optional( - 2, "map1", MapType.ofOptional(3, 4, IntegerType.get(), StringType.get()))))) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - } - - @Test - void testListDifferentId() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(1, "id", IntegerType.get()), - optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))), - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "list1", ListType.ofOptional(2, IntegerType.get()))))) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - } - - @Test - void testListChanged() { - assertThat( - CompareSchemasVisitor.visit( - new Schema( - optional(0, "id", IntegerType.get()), - optional(1, "list1", ListType.ofOptional(2, LongType.get()))), - new Schema( - optional(1, "id", IntegerType.get()), - optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java deleted file mode 100644 index e6069656bd65..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ /dev/null @@ -1,591 +0,0 @@ -/* - * 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.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; -import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class TestDynamicIcebergSink extends TestFlinkIcebergSinkBase { - - private static long seed; - - @BeforeEach - void before() { - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(2); - seed = 0; - } - - private static class DynamicIcebergDataImpl implements Serializable { - Row rowProvided; - Row rowExpected; - Schema schemaProvided; - Schema schemaExpected; - String tableName; - String branch; - PartitionSpec partitionSpec; - boolean upsertMode; - List equalityFields; - - private DynamicIcebergDataImpl( - Schema schemaProvided, String tableName, String branch, PartitionSpec partitionSpec) { - this( - schemaProvided, - schemaProvided, - tableName, - branch, - partitionSpec, - false, - Collections.emptyList(), - false); - } - - private DynamicIcebergDataImpl( - Schema schemaProvided, - Schema schemaExpected, - String tableName, - String branch, - PartitionSpec partitionSpec) { - this( - schemaProvided, - schemaExpected, - tableName, - branch, - partitionSpec, - false, - Collections.emptyList(), - false); - } - - private DynamicIcebergDataImpl( - Schema schemaProvided, - String tableName, - String branch, - PartitionSpec partitionSpec, - boolean upsertMode, - List equalityFields, - boolean isDuplicate) { - this( - schemaProvided, - schemaProvided, - tableName, - branch, - partitionSpec, - upsertMode, - equalityFields, - isDuplicate); - } - - private DynamicIcebergDataImpl( - Schema schemaProvided, - Schema schemaExpected, - String tableName, - String branch, - PartitionSpec partitionSpec, - boolean upsertMode, - List equalityFields, - boolean isDuplicate) { - this.rowProvided = randomRow(schemaProvided, isDuplicate ? seed : ++seed); - this.rowExpected = isDuplicate ? null : rowProvided; - this.schemaProvided = schemaProvided; - this.schemaExpected = schemaExpected; - this.tableName = tableName; - this.branch = branch; - this.partitionSpec = partitionSpec; - this.upsertMode = upsertMode; - this.equalityFields = equalityFields; - } - } - - private static class Converter implements DynamicRecordConverter { - - @Override - public void convert(DynamicIcebergDataImpl row, Collector out) { - TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, row.tableName); - String branch = row.branch; - Schema schema = row.schemaProvided; - PartitionSpec spec = row.partitionSpec; - DynamicRecord dynamicRecord = - new DynamicRecord( - tableIdentifier, - branch, - schema, - spec, - converter(schema).toInternal(row.rowProvided), - spec.isPartitioned() ? DistributionMode.HASH : DistributionMode.NONE, - 10); - dynamicRecord.setUpsertMode(row.upsertMode); - dynamicRecord.setEqualityFields(row.equalityFields); - out.collect(dynamicRecord); - } - } - - private static DataFormatConverters.RowConverter converter(Schema schema) { - RowType rowType = FlinkSchemaUtil.convert(schema); - TableSchema tableSchema = FlinkSchemaUtil.toSchema(rowType); - return new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); - } - - @Test - void testWrite() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); - - runTest(rows); - } - - @Test - void testWritePartitioned() throws Exception { - PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); - - runTest(rows); - } - - @Test - void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { - Schema schema = - new Schema( - // Use zero-based schema field ids - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required(1, "data", Types.StringType.get())); - PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 10).build(); - Schema schema2 = - new Schema( - // Use zero-based schema field ids - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required(1, "data", Types.StringType.get()), - Types.NestedField.optional(2, "extra", Types.StringType.get())); - PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("extra", 23).build(); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); - - runTest(rows); - } - - @Test - void testSchemaEvolutionFieldOrderChanges() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required(1, "data", Types.StringType.get())); - Schema expectedSchema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - - Schema schema2 = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required(1, "extra", Types.StringType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - Schema expectedSchema2 = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(3, "extra", Types.StringType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); - - for (DynamicIcebergDataImpl row : rows) { - if (row.schemaExpected == expectedSchema) { - // We manually adjust the expected Row to match the second expected schema - row.rowExpected = Row.of(row.rowProvided.getField(0), null, row.rowProvided.getField(1)); - } - } - - runTest(rows); - } - - @Test - void testMultipleTables() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); - - runTest(rows); - } - - @Test - void testMultipleTablesPartitioned() throws Exception { - PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); - - runTest(rows); - } - - @Test - void testSchemaEvolutionAddField() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); - - runTest(rows, this.env, 1); - } - - @Test - void testRowEvolutionNullMissingOptionalField() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); - - runTest(rows, this.env, 1); - } - - @Test - void testSchemaEvolutionNonBackwardsCompatible() throws Exception { - Schema backwardsIncompatibleSchema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - // Required column is missing in this schema - Schema erroringSchema = - new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - backwardsIncompatibleSchema, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); - - try { - runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); - Assert.fail(); - } catch (JobExecutionException e) { - assertThat( - ExceptionUtils.findThrowable( - e, - t -> - t.getMessage() - .contains( - "Field 2 in target schema ROW<`id` INT NOT NULL, `data` STRING NOT NULL> is non-nullable but does not exist in source schema."))) - .isNotEmpty(); - } - } - - @Test - void testPartitionSpecEvolution() throws Exception { - PartitionSpec spec1 = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 5).identity("data").build(); - - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); - - runTest(rows); - } - - @Test - void testMultipleBranches() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); - - runTest(rows); - } - - @Test - void testWriteDynamicRowData() throws Exception { - List rows = - Lists.newArrayList( - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); - - runTest(rows); - } - - @Test - void testUpsert() throws Exception { - List rows = - Lists.newArrayList( - // Insert one rows - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, - "t1", - "main", - PartitionSpec.unpartitioned(), - true, - Lists.newArrayList("id"), - false), - // Remaining rows are duplicates - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, - "t1", - "main", - PartitionSpec.unpartitioned(), - true, - Lists.newArrayList("id"), - true), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, - "t1", - "main", - PartitionSpec.unpartitioned(), - true, - Lists.newArrayList("id"), - true), - new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, - "t1", - "main", - PartitionSpec.unpartitioned(), - true, - Lists.newArrayList("id"), - true)); - - executeDynamicSink(rows, env, true, 1); - - try (CloseableIterable iterable = - IcebergGenerics.read( - CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.of("default", "t1"))) - .build()) { - List records = Lists.newArrayList(); - for (Record record : iterable) { - records.add(record); - } - - assertThat(records.size()).isEqualTo(1); - Record actual = records.get(0); - DynamicIcebergDataImpl input = rows.get(0); - assertThat(actual.get(0)).isEqualTo(input.rowProvided.getField(0)); - assertThat(actual.get(1)).isEqualTo(input.rowProvided.getField(1)); - // There is an additional _pos field which gets added - } - } - - private void runTest(List dynamicData) throws Exception { - runTest(dynamicData, this.env, 2); - } - - private void runTest( - List dynamicData, StreamExecutionEnvironment env, int parallelism) - throws Exception { - runTest(dynamicData, env, true, parallelism); - runTest(dynamicData, env, false, parallelism); - } - - private void runTest( - List dynamicData, - StreamExecutionEnvironment env, - boolean immediateUpdate, - int parallelism) - throws Exception { - executeDynamicSink(dynamicData, env, immediateUpdate, parallelism); - verifyResults(dynamicData); - } - - private void executeDynamicSink( - List dynamicData, - StreamExecutionEnvironment env, - boolean immediateUpdate, - int parallelism) - throws Exception { - DataStream dataStream = - env.addSource(createBoundedSource(dynamicData), TypeInformation.of(new TypeHint<>() {})); - env.setParallelism(parallelism); - - DynamicIcebergSink.forInput(dataStream) - .withConverter(new Converter()) - .catalogLoader(CATALOG_EXTENSION.catalogLoader()) - .writeParallelism(parallelism) - .immediateTableUpdate(immediateUpdate) - .append(); - - // Write the data - env.execute("Test Iceberg DataStream"); - } - - private void verifyResults(List dynamicData) throws IOException { - // Calculate the expected result - Map, List> expectedData = Maps.newHashMap(); - Map expectedSchema = Maps.newHashMap(); - dynamicData.forEach( - r -> { - Schema oldSchema = expectedSchema.get(r.tableName); - if (oldSchema == null || oldSchema.columns().size() < r.schemaProvided.columns().size()) { - expectedSchema.put(r.tableName, r.schemaExpected); - } - }); - - dynamicData.forEach( - r -> { - List data = - expectedData.computeIfAbsent( - Tuple2.of(r.tableName, r.branch), unused -> Lists.newArrayList()); - data.addAll( - convertToRowData(expectedSchema.get(r.tableName), ImmutableList.of(r.rowExpected))); - }); - - // Check the expected result - int count = dynamicData.size(); - for (Map.Entry, List> e : expectedData.entrySet()) { - SimpleDataUtil.assertTableRows( - CATALOG_EXTENSION - .catalogLoader() - .loadCatalog() - .loadTable(TableIdentifier.of(DATABASE, e.getKey().f0)), - e.getValue(), - e.getKey().f1); - count -= e.getValue().size(); - } - - // Found every record - assertThat(count).isZero(); - } - - private List convertToRowData(Schema schema, List rows) { - DataFormatConverters.RowConverter converter = converter(schema); - return rows.stream() - .map( - r -> { - Row updateRow = r; - // We need conversion to generate the missing columns - if (r.getArity() != schema.columns().size()) { - updateRow = new Row(schema.columns().size()); - for (int i = 0; i < r.getArity(); ++i) { - updateRow.setField(i, r.getField(i)); - } - } - return converter.toInternal(updateRow); - }) - .collect(Collectors.toList()); - } - - private static Row randomRow(Schema schema, long seedOverride) { - return TestHelpers.convertRecordToRow( - RandomGenericData.generate(schema, 1, seedOverride), schema) - .get(0); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java deleted file mode 100644 index b7c145da2b77..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * 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.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.apache.iceberg.flink.TestFixtures.TABLE; -import static org.apache.iceberg.flink.sink.dynamic.DynamicCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; - -import java.util.List; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.util.Collector; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -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.TableIdentifier; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.sink.IcebergSink; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Disabled("Please enable manually for performance testing.") -class TestDynamicIcebergSinkPerf { - private static final Logger LOG = LoggerFactory.getLogger(TestDynamicIcebergSinkPerf.class); - - @RegisterExtension - protected static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TABLE); - - private static final int SAMPLE_SIZE = 50_000; - private static final int RECORD_SIZE = 5_000_000; - private static final int TABLE_NUM = 3; - private static final int PARALLELISM = 2; - private static final int WRITE_PARALLELISM = 2; - private static final TableIdentifier[] IDENTIFIERS = new TableIdentifier[TABLE_NUM]; - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name2", Types.StringType.get()), - Types.NestedField.required(3, "name3", Types.StringType.get()), - Types.NestedField.required(4, "name4", Types.StringType.get()), - Types.NestedField.required(5, "name5", Types.StringType.get()), - Types.NestedField.required(6, "name6", Types.StringType.get()), - Types.NestedField.required(7, "name7", Types.StringType.get()), - Types.NestedField.required(8, "name8", Types.StringType.get()), - Types.NestedField.required(9, "name9", Types.StringType.get())); - private static final List RANGE = - IntStream.range(0, RECORD_SIZE).boxed().collect(Collectors.toList()); - - private static List rows; - private StreamExecutionEnvironment env; - - @BeforeEach - void before() { - for (int i = 0; i < TABLE_NUM; ++i) { - // So the table name hash difference is bigger than 1 - IDENTIFIERS[i] = TableIdentifier.of(DATABASE, TABLE + "_" + (i * 13)); - - Table table = - CATALOG_EXTENSION - .catalog() - .createTable( - IDENTIFIERS[i], - SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); - - table.manageSnapshots().createBranch("main").commit(); - } - - List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); - rows = Lists.newArrayListWithCapacity(records.size()); - for (int i = 0; i < records.size(); ++i) { - rows.add( - new DynamicRecord( - IDENTIFIERS[i % TABLE_NUM], - "main", - SCHEMA, - PartitionSpec.unpartitioned(), - RowDataConverter.convert(SCHEMA, records.get(i)), - DistributionMode.NONE, - WRITE_PARALLELISM)); - } - - Configuration configuration = MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; - configuration.setString("rest.flamegraph.enabled", "true"); - env = - StreamExecutionEnvironment.getExecutionEnvironment(configuration) - .enableCheckpointing(100) - .setParallelism(PARALLELISM) - .setMaxParallelism(PARALLELISM); - env.getConfig().enableObjectReuse(); - } - - @AfterEach - void after() { - for (TableIdentifier identifier : IDENTIFIERS) { - CATALOG_EXTENSION.catalog().dropTable(identifier); - } - } - - private static class IdBasedConverter implements DynamicRecordConverter { - - @Override - public void convert(Integer id, Collector out) { - out.collect(rows.get(id % SAMPLE_SIZE)); - } - } - - @Test - void testDynamicSink() throws Exception { - // So we make sure that the writer threads are the same for the 2 tests - env.setMaxParallelism(PARALLELISM * TABLE_NUM * 2); - env.setParallelism(PARALLELISM * TABLE_NUM * 2); - runTest( - s -> { - DynamicIcebergSink.forInput(s) - .withConverter(new IdBasedConverter()) - .immediateTableUpdate(true) - .catalogLoader(CATALOG_EXTENSION.catalogLoader()) - .append(); - }); - } - - @Test - void testIcebergSink() throws Exception { - runTest( - s -> { - for (int i = 0; i < IDENTIFIERS.length; ++i) { - TableLoader tableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), IDENTIFIERS[i]); - final int finalInt = i; - IcebergSink.forRowData( - s.flatMap( - (FlatMapFunction) - (input, collector) -> { - if (input % TABLE_NUM == finalInt) { - collector.collect(rows.get(input % SAMPLE_SIZE).rowData()); - } - }) - .returns(InternalTypeInfo.of(FlinkSchemaUtil.convert(SCHEMA))) - .rebalance()) - .tableLoader(tableLoader) - .uidSuffix("Uid" + i) - .writeParallelism(WRITE_PARALLELISM) - .append(); - } - }); - } - - private void runTest(Consumer> sink) throws Exception { - DataStream dataStream = - env.addSource( - new BoundedTestSource<>( - ImmutableList.of( - RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE, RANGE), - true), - TypeInformation.of(Integer.class)); - - sink.accept(dataStream); - - long before = System.currentTimeMillis(); - env.execute(); - - for (TableIdentifier identifier : IDENTIFIERS) { - Table table = CATALOG_EXTENSION.catalog().loadTable(identifier); - for (Snapshot snapshot : table.snapshots()) { - long records = 0; - for (DataFile dataFile : snapshot.addedDataFiles(table.io())) { - records += dataFile.recordCount(); - } - - LOG.info( - "TEST RESULT: For table {} snapshot {} written {} records in {} ms", - identifier, - snapshot.snapshotId(), - records, - snapshot.timestampMillis() - before); - before = snapshot.timestampMillis(); - } - } - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java deleted file mode 100644 index ad96cf9747c4..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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, - 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; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java deleted file mode 100644 index 05501b1e6e43..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java +++ /dev/null @@ -1,607 +0,0 @@ -/* - * 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.apache.iceberg.types.Types.NestedField.of; -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaUpdate; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type.PrimitiveType; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.types.Types.DecimalType; -import org.apache.iceberg.types.Types.DoubleType; -import org.apache.iceberg.types.Types.FloatType; -import org.apache.iceberg.types.Types.IntegerType; -import org.apache.iceberg.types.Types.ListType; -import org.apache.iceberg.types.Types.LongType; -import org.apache.iceberg.types.Types.MapType; -import org.apache.iceberg.types.Types.StringType; -import org.apache.iceberg.types.Types.StructType; -import org.apache.iceberg.types.Types.TimeType; -import org.apache.iceberg.types.Types.UUIDType; -import org.junit.Assert; -import org.junit.Test; - -public class TestEvolveSchemaVisitor { - - private static List primitiveTypes() { - return Lists.newArrayList( - StringType.get(), - TimeType.get(), - Types.TimestampType.withoutZone(), - Types.TimestampType.withZone(), - UUIDType.get(), - Types.DateType.get(), - Types.BooleanType.get(), - Types.BinaryType.get(), - DoubleType.get(), - IntegerType.get(), - Types.FixedType.ofLength(10), - DecimalType.of(10, 2), - LongType.get(), - FloatType.get()); - } - - private static Types.NestedField[] primitiveFields( - Integer initialValue, List primitiveTypes) { - return primitiveFields(initialValue, primitiveTypes, true); - } - - private static Types.NestedField[] primitiveFields( - Integer initialValue, List primitiveTypes, boolean optional) { - AtomicInteger atomicInteger = new AtomicInteger(initialValue); - return primitiveTypes.stream() - .map( - type -> - of( - atomicInteger.incrementAndGet(), - optional, - type.toString(), - Types.fromPrimitiveString(type.toString()))) - .toArray(Types.NestedField[]::new); - } - - @Test - public void testAddTopLevelPrimitives() { - Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testMakeTopLevelPrimitivesOptional() { - Schema existingSchema = new Schema(primitiveFields(0, primitiveTypes(), false)); - Assert.assertTrue(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)); - - SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); - EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); - Schema newSchema = updateApi.apply(); - Assert.assertEquals(14, newSchema.asStruct().fields().size()); - Assert.assertTrue(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)); - } - - @Test - public void testIdentifyFieldsByName() { - Schema existingSchema = - new Schema(Types.NestedField.optional(42, "myField", Types.LongType.get())); - SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); - Schema newSchema = - new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); - EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); - Assert.assertTrue(updateApi.apply().sameSchema(existingSchema)); - } - - @Test - public void testChangeOrderTopLevelPrimitives() { - Schema existingSchema = - new Schema( - Arrays.asList(optional(1, "a", StringType.get()), optional(2, "b", StringType.get()))); - Schema targetSchema = - new Schema( - Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); - SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAddTopLevelListOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testMakeTopLevelListOfPrimitivesOptional() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema existingSchema = - new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); - Schema targetSchema = new Schema(); - SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 0); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); - Schema expectedSchema = - new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); - Assert.assertEquals(expectedSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testAddTopLevelMapOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema targetSchema = - new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testAddTopLevelStructOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema currentSchema = - new Schema( - optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); - Assert.assertEquals(currentSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testAddNestedPrimitive() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); - Schema targetSchema = - new Schema( - optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testMakeNestedPrimitiveOptional() { - for (PrimitiveType primitiveType : primitiveTypes()) { - Schema currentSchema = - new Schema( - optional(1, "aStruct", StructType.of(required(2, "primitive", primitiveType)))); - Schema targetSchema = - new Schema( - optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - } - - @Test - public void testAddNestedPrimitives() { - Schema currentSchema = new Schema(optional(1, "aStruct", StructType.of())); - Schema targetSchema = - new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAddNestedLists() { - Schema targetSchema = - new Schema( - optional( - 1, - "aList", - ListType.ofOptional( - 2, - ListType.ofOptional( - 3, - ListType.ofOptional( - 4, - ListType.ofOptional( - 5, - ListType.ofOptional( - 6, - ListType.ofOptional( - 7, - ListType.ofOptional( - 8, - ListType.ofOptional( - 9, - ListType.ofOptional( - 10, DecimalType.of(11, 20)))))))))))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAddNestedStruct() { - Schema targetSchema = - new Schema( - optional( - 1, - "struct1", - StructType.of( - optional( - 2, - "struct2", - StructType.of( - optional( - 3, - "struct3", - StructType.of( - optional( - 4, - "struct4", - StructType.of( - optional( - 5, - "struct5", - StructType.of( - optional( - 6, - "struct6", - StructType.of( - optional( - 7, - "aString", - StringType.get())))))))))))))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAddNestedMaps() { - Schema targetSchema = - new Schema( - optional( - 1, - "struct", - MapType.ofOptional( - 2, - 3, - StringType.get(), - MapType.ofOptional( - 4, - 5, - StringType.get(), - MapType.ofOptional( - 6, - 7, - StringType.get(), - MapType.ofOptional( - 8, - 9, - StringType.get(), - MapType.ofOptional( - 10, - 11, - StringType.get(), - MapType.ofOptional( - 12, 13, StringType.get(), StringType.get())))))))); - SchemaUpdate updateApi = new SchemaUpdate(new Schema(), 0); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testDetectInvalidTopLevelList() { - Schema currentSchema = - new Schema(optional(1, "aList", ListType.ofOptional(2, StringType.get()))); - Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, LongType.get()))); - Assert.assertThrows( - "Cannot change column type: aList.element: string -> long", - IllegalArgumentException.class, - () -> - EvolveSchemaVisitor.visit( - new SchemaUpdate(currentSchema, 2), currentSchema, targetSchema)); - } - - @Test - public void testDetectInvalidTopLevelMapValue() { - - Schema currentSchema = - new Schema( - optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); - Schema targetSchema = - new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), LongType.get()))); - - Assert.assertThrows( - "Cannot change column type: aMap.value: string -> long", - IllegalArgumentException.class, - () -> - EvolveSchemaVisitor.visit( - new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); - } - - @Test - public void testDetectInvalidTopLevelMapKey() { - Schema currentSchema = - new Schema( - optional(1, "aMap", MapType.ofOptional(2, 3, StringType.get(), StringType.get()))); - Schema targetSchema = - new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, UUIDType.get(), StringType.get()))); - Assert.assertThrows( - "Cannot change column type: aMap.key: string -> uuid", - IllegalArgumentException.class, - () -> - EvolveSchemaVisitor.visit( - new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); - } - - @Test - // int 32-bit signed integers -> Can promote to long - public void testTypePromoteIntegerToLong() { - Schema currentSchema = new Schema(required(1, "aCol", IntegerType.get())); - Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); - - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Schema applied = updateApi.apply(); - Assert.assertEquals(1, applied.asStruct().fields().size()); - Assert.assertEquals(LongType.get(), applied.asStruct().fields().get(0).type()); - } - - @Test - // float 32-bit IEEE 754 floating point -> Can promote to double - public void testTypePromoteFloatToDouble() { - Schema currentSchema = new Schema(required(1, "aCol", FloatType.get())); - Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); - - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 0); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Schema applied = updateApi.apply(); - Assert.assertEquals(1, applied.asStruct().fields().size()); - Assert.assertEquals(DoubleType.get(), applied.asStruct().fields().get(0).type()); - } - - @Test - public void testInvalidTypePromoteDoubleToFloat() { - Schema currentSchema = new Schema(required(1, "aCol", DoubleType.get())); - Schema targetSchema = new Schema(required(1, "aCol", FloatType.get())); - Assert.assertThrows( - "Cannot change column type: aCol: double -> float", - IllegalArgumentException.class, - () -> - EvolveSchemaVisitor.visit( - new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); - } - - @Test - // decimal(P,S) Fixed-point decimal; precision P, scale S -> Scale is fixed [1], precision must be - // 38 or less - public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { - Schema currentSchema = new Schema(required(1, "aCol", DecimalType.of(20, 1))); - Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); - - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 1); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAddPrimitiveToNestedStruct() { - Schema existingSchema = - new Schema( - required( - 1, - "struct1", - StructType.of( - optional( - 2, - "struct2", - StructType.of( - optional( - 3, - "list", - ListType.ofOptional( - 4, - StructType.of(optional(5, "number", IntegerType.get()))))))))); - - Schema targetSchema = - new Schema( - required( - 1, - "struct1", - StructType.of( - optional( - 2, - "struct2", - StructType.of( - optional( - 3, - "list", - ListType.ofOptional( - 4, - StructType.of( - optional(5, "number", LongType.get()), - optional(6, "time", TimeType.get()))))))))); - - SchemaUpdate updateApi = new SchemaUpdate(existingSchema, 5); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testReplaceListWithPrimitive() { - Schema currentSchema = - new Schema(optional(1, "aColumn", ListType.ofOptional(2, StringType.get()))); - Schema targetSchema = new Schema(optional(1, "aColumn", StringType.get())); - Assert.assertThrows( - "Cannot change column type: aColumn: list -> string", - IllegalArgumentException.class, - () -> - EvolveSchemaVisitor.visit( - new SchemaUpdate(currentSchema, 3), currentSchema, targetSchema)); - } - - @Test - public void addNewTopLevelStruct() { - Schema currentSchema = - new Schema( - optional( - 1, - "map1", - MapType.ofOptional( - 2, - 3, - StringType.get(), - ListType.ofOptional( - 4, StructType.of(optional(5, "string1", StringType.get())))))); - - Schema targetSchema = - new Schema( - optional( - 1, - "map1", - MapType.ofOptional( - 2, - 3, - StringType.get(), - ListType.ofOptional( - 4, StructType.of(optional(5, "string1", StringType.get()))))), - optional( - 6, - "struct1", - StructType.of( - optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); - - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 5); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testAppendNestedStruct() { - Schema currentSchema = - new Schema( - required( - 1, - "s1", - StructType.of( - optional( - 2, - "s2", - StructType.of( - optional( - 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); - - Schema targetSchema = - new Schema( - required( - 1, - "s1", - StructType.of( - optional( - 2, - "s2", - StructType.of( - optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), - optional( - 5, - "repeat", - StructType.of( - optional( - 6, - "s1", - StructType.of( - optional( - 7, - "s2", - StructType.of( - optional( - 8, - "s3", - StructType.of( - optional( - 9, - "s4", - StringType.get())))))))))))))); - - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 4); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals(targetSchema.asStruct(), updateApi.apply().asStruct()); - } - - @Test - public void testMakeNestedStructOptional() { - Schema currentSchema = getNestedSchemaWithOptionalModifier(false); - Schema targetSchema = - new Schema( - required( - 1, - "s1", - StructType.of( - optional( - 2, - "s2", - StructType.of( - optional( - 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); - SchemaUpdate updateApi = new SchemaUpdate(currentSchema, 9); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); - Assert.assertEquals( - updateApi.apply().asStruct(), getNestedSchemaWithOptionalModifier(true).asStruct()); - } - - private static Schema getNestedSchemaWithOptionalModifier(boolean nestedIsOptional) { - return new Schema( - required( - 1, - "s1", - StructType.of( - optional( - 2, - "s2", - StructType.of( - optional(3, "s3", StructType.of(optional(4, "s4", StringType.get()))), - of( - 5, - nestedIsOptional, - "repeat", - StructType.of( - optional( - 6, - "s1", - StructType.of( - optional( - 7, - "s2", - StructType.of( - optional( - 8, - "s3", - StructType.of( - optional( - 9, "s4", StringType.get())))))))))))))); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java deleted file mode 100644 index b7cd95e00146..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestPartitionSpecAdjustment { - - @Test - void testPartitionSpecSourceIdRemappingBasedOnFieldNames() { - Schema specSchema = - new Schema( - // Use zero-based field ids - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required( - 1, - "data", - Types.StructType.of(Types.NestedField.required(2, "str", Types.StringType.get())))); - - PartitionSpec spec = PartitionSpec.builderFor(specSchema).bucket("id", 10).build(); - - Schema tableSchema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required( - 2, - "data", - Types.StructType.of(Types.NestedField.required(3, "str", Types.StringType.get())))); - - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(tableSchema, spec); - - assertThat(adjustedSpec) - .isEqualTo(PartitionSpec.builderFor(tableSchema).bucket("id", 10).build()); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java deleted file mode 100644 index 2a2b4c385730..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java +++ /dev/null @@ -1,248 +0,0 @@ -/* - * 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.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assertions.assertThat; - -import java.math.BigDecimal; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; -import org.junit.Assert; -import org.junit.jupiter.api.Test; - -class TestRowDataEvolver { - @Test - void testPrimitiveTypes() { - DataGenerator generator = new DataGenerators.Primitives(); - assertThat( - RowDataEvolver.convert( - generator.generateFlinkRowData(), - generator.icebergSchema(), - generator.icebergSchema())) - .isEqualTo(generator.generateFlinkRowData()); - } - - @Test - void testAddColumn() { - assertThat( - RowDataEvolver.convert( - SimpleDataUtil.createRowData(1, "a"), - SimpleDataUtil.SCHEMA, - SimpleDataUtil.SCHEMA2)) - .isEqualTo(GenericRowData.of(1, StringData.fromString("a"), null)); - } - - @Test - void testAddRequiredColumn() { - Schema currentSchema = new Schema(Types.NestedField.optional(1, "id", Types.IntegerType.get())); - Schema targetSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - required(2, "data", Types.StringType.get())); - - Assert.assertThrows( - IllegalArgumentException.class, - () -> RowDataEvolver.convert(GenericRowData.of(42), currentSchema, targetSchema)); - } - - @Test - void testIntToLong() { - Schema schemaWithLong = - new Schema( - Types.NestedField.optional(2, "id", Types.LongType.get()), - Types.NestedField.optional(4, "data", Types.StringType.get())); - - assertThat( - RowDataEvolver.convert( - SimpleDataUtil.createRowData(1, "a"), SimpleDataUtil.SCHEMA, schemaWithLong)) - .isEqualTo(GenericRowData.of(1L, StringData.fromString("a"))); - } - - @Test - void testFloatToDouble() { - Schema schemaWithFloat = - new Schema(Types.NestedField.optional(1, "float2double", Types.FloatType.get())); - Schema schemaWithDouble = - new Schema(Types.NestedField.optional(2, "float2double", Types.DoubleType.get())); - - assertThat(RowDataEvolver.convert(GenericRowData.of(1.5f), schemaWithFloat, schemaWithDouble)) - .isEqualTo(GenericRowData.of(1.5d)); - } - - @Test - void testDateToTimestamp() { - Schema schemaWithFloat = - new Schema(Types.NestedField.optional(1, "date2timestamp", Types.DateType.get())); - Schema schemaWithDouble = - new Schema( - Types.NestedField.optional(2, "date2timestamp", Types.TimestampType.withoutZone())); - - DateTime time = new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); - int days = - Days.daysBetween(new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC), time).getDays(); - - assertThat(RowDataEvolver.convert(GenericRowData.of(days), schemaWithFloat, schemaWithDouble)) - .isEqualTo(GenericRowData.of(TimestampData.fromEpochMillis(time.getMillis()))); - } - - @Test - void testIncreasePrecision() { - Schema before = - new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2))); - Schema after = - new Schema(Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(10, 2))); - - assertThat( - RowDataEvolver.convert( - GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 9, 2)), - before, - after)) - .isEqualTo(GenericRowData.of(DecimalData.fromBigDecimal(new BigDecimal("-1.50"), 10, 2))); - } - - @Test - void testStructAddOptionalFields() { - DataGenerator generator = new DataGenerators.StructOfPrimitive(); - RowData oldData = generator.generateFlinkRowData(); - Schema oldSchema = generator.icebergSchema(); - Types.NestedField structField = oldSchema.columns().get(1); - Schema newSchema = - new Schema( - oldSchema.columns().get(0), - Types.NestedField.required( - 10, - structField.name(), - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - optional(103, "optional", Types.StringType.get()), - required(102, "name", Types.StringType.get())))); - RowData newData = - GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of(1, null, StringData.fromString("Jane"))); - - assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); - } - - @Test - void testStructAddRequiredFieldsWithOptionalRoot() { - DataGenerator generator = new DataGenerators.StructOfPrimitive(); - RowData oldData = generator.generateFlinkRowData(); - Schema oldSchema = generator.icebergSchema(); - Types.NestedField structField = oldSchema.columns().get(1); - Schema newSchema = - new Schema( - oldSchema.columns().get(0), - Types.NestedField.optional( - 10, - "newFieldOptionalField", - Types.StructType.of( - Types.NestedField.optional( - structField.fieldId(), - structField.name(), - Types.StructType.of( - optional(101, "id", Types.IntegerType.get()), - // Required columns which leads to nulling the entire struct - required(103, "required", Types.StringType.get()), - required(102, "name", Types.StringType.get())))))); - - RowData expectedData = GenericRowData.of(StringData.fromString("row_id_value"), null); - - assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(expectedData); - } - - @Test - void testStructAddRequiredFields() { - DataGenerator generator = new DataGenerators.StructOfPrimitive(); - RowData oldData = generator.generateFlinkRowData(); - Schema oldSchema = generator.icebergSchema(); - Types.NestedField structField = oldSchema.columns().get(1); - Schema newSchema = - new Schema( - oldSchema.columns().get(0), - Types.NestedField.required( - 10, - structField.name(), - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required(103, "required", Types.StringType.get()), - required(102, "name", Types.StringType.get())))); - - Assert.assertThrows( - IllegalArgumentException.class, - () -> RowDataEvolver.convert(oldData, oldSchema, newSchema)); - } - - @Test - void testMap() { - DataGenerator generator = new DataGenerators.MapOfPrimitives(); - RowData oldData = generator.generateFlinkRowData(); - Schema oldSchema = generator.icebergSchema(); - Types.NestedField mapField = oldSchema.columns().get(1); - Schema newSchema = - new Schema( - oldSchema.columns().get(0), - Types.NestedField.optional( - 10, - mapField.name(), - Types.MapType.ofRequired(101, 102, Types.StringType.get(), Types.LongType.get()))); - RowData newData = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), 1L, StringData.fromString("Joe"), 2L))); - - assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); - } - - @Test - void testArray() { - DataGenerator generator = new DataGenerators.ArrayOfPrimitive(); - RowData oldData = generator.generateFlinkRowData(); - Schema oldSchema = generator.icebergSchema(); - Types.NestedField arrayField = oldSchema.columns().get(1); - Schema newSchema = - new Schema( - oldSchema.columns().get(0), - Types.NestedField.optional( - 10, arrayField.name(), Types.ListType.ofOptional(101, Types.LongType.get()))); - RowData newData = - GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(new Long[] {1L, 2L, 3L})); - - assertThat(RowDataEvolver.convert(oldData, oldSchema, newSchema)).isEqualTo(newData); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java deleted file mode 100644 index c0ca74eba240..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableDataCache.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.commons.lang3.SerializationUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -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.junit.jupiter.api.Test; - -public class TestTableDataCache extends TestFlinkIcebergSinkBase { - - @Test - void testCaching() { - Catalog catalog = CATALOG_EXTENSION.catalog(); - TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); - TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); - - Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; - assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); - assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) - .isEqualTo(schema1); - - assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2)) - .isEqualTo(TableDataCache.NOT_FOUND); - - schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; - assertThat(cache.schema(tableIdentifier, SerializationUtils.clone(SimpleDataUtil.SCHEMA)).f0) - .isEqualTo(schema1); - } - - @Test - void testCacheInvalidationAfterSchemaChange() { - Catalog catalog = CATALOG_EXTENSION.catalog(); - TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); - TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); - - Schema schema1 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA).f0; - assertThat(schema1.sameSchema(SimpleDataUtil.SCHEMA)).isTrue(); - - catalog.dropTable(tableIdentifier); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); - tableUpdater.update( - tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()); - - Schema schema2 = cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f0; - assertThat(schema2.sameSchema(SimpleDataUtil.SCHEMA2)).isTrue(); - } - - @Test - void testCachingDisabled() { - Catalog catalog = CATALOG_EXTENSION.catalog(); - TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); - TableDataCache cache = new TableDataCache(catalog, 0, Long.MAX_VALUE); - - // Cleanup routine doesn't run after every write - cache.getInternalCache().cleanUp(); - assertThat(cache.getInternalCache().estimatedSize()).isEqualTo(0); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java deleted file mode 100644 index 8dbf630490b0..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -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.junit.jupiter.api.Test; - -public class TestTableUpdater extends TestFlinkIcebergSinkBase { - - @Test - void testInvalidateOldCacheEntryOnUpdate() { - Catalog catalog = CATALOG_EXTENSION.catalog(); - TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); - TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); - cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); - - Schema updated = - tableUpdater.update( - tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) - .f0; - assertThat(updated.sameSchema(SimpleDataUtil.SCHEMA2)); - assertThat( - cache - .schema(tableIdentifier, SimpleDataUtil.SCHEMA2) - .f0 - .sameSchema(SimpleDataUtil.SCHEMA2)) - .isTrue(); - } - - @Test - void testLastResultInvalidation() { - Catalog catalog = CATALOG_EXTENSION.catalog(); - TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA); - TableDataCache cache = new TableDataCache(catalog, 10, Long.MAX_VALUE); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); - - // Initialize cache - tableUpdater.update( - tableIdentifier, "main", SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); - - // Update table behind the scenes - catalog.dropTable(tableIdentifier); - catalog.createTable(tableIdentifier, SimpleDataUtil.SCHEMA2); - - // Cache still stores the old information - assertThat(cache.schema(tableIdentifier, SimpleDataUtil.SCHEMA2).f1) - .isEqualTo(CompareSchemasVisitor.Result.INCOMPATIBLE); - - assertThat( - tableUpdater.update( - tableIdentifier, "main", SimpleDataUtil.SCHEMA2, PartitionSpec.unpartitioned()) - .f1) - .isEqualTo(CompareSchemasVisitor.Result.SAME); - - // Last result cache should be cleared - assertThat( - cache - .getInternalCache() - .getIfPresent(tableIdentifier) - .getSchemaInfo() - .getLastResult(SimpleDataUtil.SCHEMA2)) - .isNull(); - } -} From b1229ff9965195625ae617a0d91e093d497a76bd Mon Sep 17 00:00:00 2001 From: Max Michels Date: Tue, 4 Mar 2025 12:15:55 +0100 Subject: [PATCH 5/6] Consolidate partition spec adjustment logic with PartitionSpecEvolution --- .../sink/dynamic/DynamicRecordProcessor.java | 9 - .../sink/dynamic/PartitionSpecAdjustment.java | 45 ----- ...olver.java => PartitionSpecEvolution.java} | 62 ++++-- .../flink/sink/dynamic/TableDataCache.java | 2 +- .../flink/sink/dynamic/TableUpdater.java | 14 +- .../dynamic/TestPartitionSpecAdjustment.java | 57 ------ .../dynamic/TestPartitionSpecEvolution.java | 188 ++++++++++++++++++ 7 files changed, 235 insertions(+), 142 deletions(-) delete mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/{PartitionSpecEvolver.java => PartitionSpecEvolution.java} (60%) delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index 86536e24b08e..afb173c3b9d2 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -105,7 +105,6 @@ public void collect(DynamicRecord data) { ? tableCache.schema(data.tableIdentifier(), data.schema()) : TableDataCache.NOT_FOUND; - adjustPartitionSpecToTableSchema(foundSchema.f0, data); PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; if (!exists @@ -141,14 +140,6 @@ public void collect(DynamicRecord data) { } } - private static void adjustPartitionSpecToTableSchema(Schema schema, DynamicRecord data) { - if (schema != null) { - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(schema, data.spec()); - data.setSpec(adjustedSpec); - } - } - private void emit( Collector out, DynamicRecord data, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java deleted file mode 100644 index 12fa3f9e7f3b..000000000000 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecAdjustment.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.UnboundPartitionSpec; - -public class PartitionSpecAdjustment { - - private PartitionSpecAdjustment() {} - - public static PartitionSpec adjustPartitionSpecToTableSchema( - Schema tableSchema, PartitionSpec userSpec) { - if (userSpec.isUnpartitioned()) { - return userSpec; - } - UnboundPartitionSpec.Builder builder = - UnboundPartitionSpec.builder().withSpecId(userSpec.specId()); - for (PartitionField field : userSpec.fields()) { - String sourceFieldName = userSpec.schema().idToName().get(field.sourceId()); - int adjustedSourceId = tableSchema.findField(sourceFieldName).fieldId(); - builder.addField( - field.transform().toString(), adjustedSourceId, field.fieldId(), field.name()); - } - return builder.build().bind(tableSchema); - } -} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java similarity index 60% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java index e3746b82ce9f..760340686a8c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolver.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.sink.dynamic; import java.util.List; -import java.util.Objects; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -30,29 +29,28 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Checks compatibility of PartitionSpecs and evolves one into the other. */ -public class PartitionSpecEvolver { +public class PartitionSpecEvolution { - private PartitionSpecEvolver() {} + private PartitionSpecEvolution() {} /** * Checks whether two PartitionSpecs are compatible with each other. Less strict than {@code * PartitionSpec#compatible} in the sense that it tolerates differently named partition fields, as - * long as their transforms and source ids match. + * long as their transforms and field names corresponding to their source ids match. */ - public static boolean checkCompatibility(PartitionSpec first, PartitionSpec second) { - if (first.equals(second)) { + public static boolean checkCompatibility(PartitionSpec spec1, PartitionSpec spec2) { + if (spec1.equals(spec2)) { return true; } - if (first.fields().size() != second.fields().size()) { + if (spec1.fields().size() != spec2.fields().size()) { return false; } - for (int i = 0; i < first.fields().size(); i++) { - PartitionField firstField = first.fields().get(i); - PartitionField secondField = second.fields().get(i); - if (firstField.sourceId() != secondField.sourceId() - || !firstField.transform().toString().equals(secondField.transform().toString())) { + for (int i = 0; i < spec1.fields().size(); i++) { + PartitionField field1 = spec1.fields().get(i); + PartitionField field2 = spec2.fields().get(i); + if (!specFieldsAreCompatible(field1, spec1.schema(), field2, spec2.schema())) { return false; } } @@ -60,26 +58,27 @@ public static boolean checkCompatibility(PartitionSpec first, PartitionSpec seco return true; } - public static PartitionSpecEvolverResult evolve( - PartitionSpec currentSpec, PartitionSpec targetSpec, Schema schema) { + static PartitionSpecChanges evolve(PartitionSpec currentSpec, PartitionSpec targetSpec) { if (currentSpec.compatibleWith(targetSpec)) { - return new PartitionSpecEvolverResult(); + return new PartitionSpecChanges(); } - PartitionSpecEvolverResult result = new PartitionSpecEvolverResult(); + PartitionSpecChanges result = new PartitionSpecChanges(); int maxNumFields = Math.max(currentSpec.fields().size(), targetSpec.fields().size()); for (int i = 0; i < maxNumFields; i++) { PartitionField currentField = Iterables.get(currentSpec.fields(), i, null); PartitionField targetField = Iterables.get(targetSpec.fields(), i, null); - if (!Objects.equals(currentField, targetField)) { + if (!specFieldsAreCompatible( + currentField, currentSpec.schema(), targetField, targetSpec.schema())) { + if (currentField != null) { - result.remove(toTerm(currentField, schema)); + result.remove(toTerm(currentField, currentSpec.schema())); } if (targetField != null) { - result.add(toTerm(targetField, schema)); + result.add(toTerm(targetField, targetSpec.schema())); } } } @@ -87,7 +86,7 @@ public static PartitionSpecEvolverResult evolve( return result; } - public static class PartitionSpecEvolverResult { + static class PartitionSpecChanges { private final List termsToAdd = Lists.newArrayList(); private final List termsToRemove = Lists.newArrayList(); @@ -110,10 +109,31 @@ public List termsToRemove() { public boolean isEmpty() { return termsToAdd.isEmpty() && termsToRemove.isEmpty(); } + + @Override + public String toString() { + return "PartitionSpecChanges{" + + "termsToAdd=" + + termsToAdd + + ", termsToRemove=" + + termsToRemove + + '}'; + } } private static Term toTerm(PartitionField field, Schema schema) { - String sourceName = schema.findField(field.sourceId()).name(); + String sourceName = schema.idToName().get(field.sourceId()); return new UnboundTransform<>(new NamedReference<>(sourceName), field.transform()); } + + private static boolean specFieldsAreCompatible( + PartitionField field1, Schema schemaField1, PartitionField field2, Schema schemaField2) { + if (field1 == null || field2 == null) { + return false; + } + String firstFieldSourceName = schemaField1.idToName().get(field1.sourceId()); + String secondFieldSourceName = schemaField2.idToName().get(field2.sourceId()); + return firstFieldSourceName.equals(secondFieldSourceName) + && field1.transform().toString().equals(field2.transform().toString()); + } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java index a86a4b878405..a1fad155b9e1 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableDataCache.java @@ -146,7 +146,7 @@ private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boole CacheItem cached = cache.getIfPresent(identifier); if (cached != null && cached.tableExists) { for (PartitionSpec tableSpec : cached.specs.values()) { - if (PartitionSpecEvolver.checkCompatibility(tableSpec, spec)) { + if (PartitionSpecEvolution.checkCompatibility(tableSpec, spec)) { return tableSpec; } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java index 7be1d247f5d1..429d232e6440 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -58,9 +58,7 @@ Tuple3 update( findOrCreateBranch(tableIdentifier, branch); Tuple2 newSchema = findOrCreateSchema(tableIdentifier, schema); - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(newSchema.f0, spec); - PartitionSpec newSpec = findOrCreateSpec(tableIdentifier, adjustedSpec, newSchema.f0); + PartitionSpec newSpec = findOrCreateSpec(tableIdentifier, spec); return Tuple3.of(newSchema.f0, newSchema.f1, newSpec); } @@ -156,8 +154,7 @@ private Tuple2 findOrCreateSchema( } } - private PartitionSpec findOrCreateSpec( - TableIdentifier identifier, PartitionSpec targetSpec, Schema newSchema) { + private PartitionSpec findOrCreateSpec(TableIdentifier identifier, PartitionSpec targetSpec) { PartitionSpec currentSpec = cache.spec(identifier, targetSpec); if (currentSpec != null) { return currentSpec; @@ -166,8 +163,8 @@ private PartitionSpec findOrCreateSpec( Table table = catalog.loadTable(identifier); currentSpec = table.spec(); - PartitionSpecEvolver.PartitionSpecEvolverResult result = - PartitionSpecEvolver.evolve(currentSpec, targetSpec, newSchema); + PartitionSpecEvolution.PartitionSpecChanges result = + PartitionSpecEvolution.evolve(currentSpec, targetSpec); if (result.isEmpty()) { LOG.info("Returning equivalent existing spec {} for {}", currentSpec, targetSpec); return currentSpec; @@ -192,8 +189,7 @@ private PartitionSpec findOrCreateSpec( targetSpec, e); PartitionSpec newSpec = cache.spec(identifier, targetSpec); - Schema maybeUpdatedSchema = cache.schema(identifier, newSchema).f0; - result = PartitionSpecEvolver.evolve(targetSpec, newSpec, maybeUpdatedSchema); + result = PartitionSpecEvolution.evolve(targetSpec, newSpec); if (result.isEmpty()) { LOG.info("Table {} partition spec updated concurrently to {}", identifier, newSpec); return newSpec; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java deleted file mode 100644 index b7cd95e00146..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecAdjustment.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestPartitionSpecAdjustment { - - @Test - void testPartitionSpecSourceIdRemappingBasedOnFieldNames() { - Schema specSchema = - new Schema( - // Use zero-based field ids - Types.NestedField.required(0, "id", Types.IntegerType.get()), - Types.NestedField.required( - 1, - "data", - Types.StructType.of(Types.NestedField.required(2, "str", Types.StringType.get())))); - - PartitionSpec spec = PartitionSpec.builderFor(specSchema).bucket("id", 10).build(); - - Schema tableSchema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required( - 2, - "data", - Types.StructType.of(Types.NestedField.required(3, "str", Types.StringType.get())))); - - PartitionSpec adjustedSpec = - PartitionSpecAdjustment.adjustPartitionSpecToTableSchema(tableSchema, spec); - - assertThat(adjustedSpec) - .isEqualTo(PartitionSpec.builderFor(tableSchema).bucket("id", 10).build()); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java new file mode 100644 index 000000000000..3e7025de6f91 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java @@ -0,0 +1,188 @@ +/* + * 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.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionSpecEvolution { + + @Test + void testCompatible() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + PartitionSpec spec2 = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + + // Happy case, source ids and names match + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isTrue(); + } + + @Test + void testNotCompatibleDifferentTransform() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + // Same spec als spec1 but different number of buckets + PartitionSpec spec2 = PartitionSpec.builderFor(schema).bucket("id", 23).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + } + + @Test + void testNotCompatibleMoreFields() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema).bucket("id", 10).build(); + // Additional field + PartitionSpec spec2 = + PartitionSpec.builderFor(schema).bucket("id", 10).truncate("data", 1).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + } + + @Test + void testCompatibleWithNonMatchingSourceIds() { + Schema schema1 = + new Schema( + // Use zero-based field ids + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema1).bucket("id", 10).build(); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + // Same spec als spec1 but bound to a different schema + PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("id", 10).build(); + + // Compatible because the source names match + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isTrue(); + } + + @Test + void testPartitionSpecEvolution() { + Schema schema1 = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema1).bucket("id", 10).build(); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + // Change num buckets + PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("id", 23).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + PartitionSpecEvolution.PartitionSpecChanges result = + PartitionSpecEvolution.evolve(spec1, spec2); + + assertThat(result.termsToAdd().toString()).isEqualTo("[bucket[23](ref(name=\"id\"))]"); + assertThat(result.termsToRemove().toString()).isEqualTo("[bucket[10](ref(name=\"id\"))]"); + } + + @Test + void testPartitionSpecEvolutionAddField() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema).build(); + // Add field + PartitionSpec spec2 = PartitionSpec.builderFor(schema).bucket("id", 23).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + PartitionSpecEvolution.PartitionSpecChanges result = + PartitionSpecEvolution.evolve(spec1, spec2); + + assertThat(result.termsToAdd().toString()).isEqualTo("[bucket[23](ref(name=\"id\"))]"); + assertThat(result.termsToRemove().toString()).isEqualTo("[]"); + } + + @Test + void testPartitionSpecEvolutionRemoveField() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema).bucket("id", 23).build(); + // Remove field + PartitionSpec spec2 = PartitionSpec.builderFor(schema).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + PartitionSpecEvolution.PartitionSpecChanges result = + PartitionSpecEvolution.evolve(spec1, spec2); + + assertThat(result.termsToAdd().toString()).isEqualTo("[]"); + assertThat(result.termsToRemove().toString()).isEqualTo("[bucket[23](ref(name=\"id\"))]"); + } + + @Test + void testPartitionSpecEvolutionWithNestedFields() { + Schema schema1 = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get()), + Types.NestedField.required( + 1, + "data", + Types.StructType.of(Types.NestedField.required(2, "str", Types.StringType.get())))); + + PartitionSpec spec1 = PartitionSpec.builderFor(schema1).bucket("data.str", 10).build(); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required( + 2, + "data", + Types.StructType.of(Types.NestedField.required(3, "str", Types.StringType.get())))); + + // Change num buckets + PartitionSpec spec2 = PartitionSpec.builderFor(schema2).bucket("data.str", 23).build(); + + assertThat(PartitionSpecEvolution.checkCompatibility(spec1, spec2)).isFalse(); + PartitionSpecEvolution.PartitionSpecChanges result = + PartitionSpecEvolution.evolve(spec1, spec2); + + assertThat(result.termsToAdd().toString()).isEqualTo("[bucket[23](ref(name=\"data.str\"))]"); + assertThat(result.termsToRemove().toString()).isEqualTo("[bucket[10](ref(name=\"data.str\"))]"); + } +} From d7e65c833e19169388d4e9173d3cb4f1971b6519 Mon Sep 17 00:00:00 2001 From: Max Michels Date: Tue, 4 Mar 2025 12:16:11 +0100 Subject: [PATCH 6/6] Revert visibility changes in UnboundPartitionSpec --- .../org/apache/iceberg/UnboundPartitionSpec.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java b/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java index b7630110d772..cc8526f9072c 100644 --- a/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UnboundPartitionSpec.java @@ -71,11 +71,11 @@ private PartitionSpec.Builder copyToBuilder(Schema schema) { return builder; } - public static Builder builder() { + static Builder builder() { return new Builder(); } - public static class Builder { + static class Builder { private final List fields; private int specId = 0; @@ -83,12 +83,12 @@ private Builder() { this.fields = Lists.newArrayList(); } - public Builder withSpecId(int newSpecId) { + Builder withSpecId(int newSpecId) { this.specId = newSpecId; return this; } - public Builder addField(String transformAsString, int sourceId, int partitionId, String name) { + Builder addField(String transformAsString, int sourceId, int partitionId, String name) { fields.add(new UnboundPartitionField(transformAsString, sourceId, partitionId, name)); return this; } @@ -98,12 +98,12 @@ Builder addField(String transformAsString, int sourceId, String name) { return this; } - public UnboundPartitionSpec build() { + UnboundPartitionSpec build() { return new UnboundPartitionSpec(specId, fields); } } - public static class UnboundPartitionField { + static class UnboundPartitionField { private final Transform transform; private final int sourceId; private final Integer partitionId;