From 62e9e33b1b1071170796dda7401b71c8ea9b1425 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 5 Jun 2025 17:01:33 +0200 Subject: [PATCH 01/14] Flink: Dynamic Iceberg Sink: Add HashKeyGenerator / RowDataEvolver / TableUpdateOperator This change adds the following components for the Flink Dynamic Iceberg Sink: *** HashKeyGenerator A hash key generator which will be used in DynamicIcebergSink class (next PR) to implement one of Iceberg's DistributionModes (NONE, HASH, RANGE). The HashKeyGenerator is responsible for creating the appropriate hash key for Flink's keyBy operation. The hash key is generated depending on the user-provided DynamicRecord and the table metadata. Under the hood, we maintain a set of Flink {@link KeySelector}s which implement the appropriate Iceberg {@link DistributionMode}. For every table, we randomly select a consistent subset of writer subtasks which receive data via their associated keys, depending on the chosen DistributionMode. Caching ensures that a new key selector is also created when the table metadata (e.g. schema, spec) or the user-provided metadata changes (e.g. distribution mode, write parallelism). *** RowDataEvolver RowDataEvolver is responsible to change the input RowData to make it compatible with the target schema. This is done when: 1. The input schema has fewer fields than the target schema. 2. The table types are wider than the input type. 3. The field order differs for source and target schema. The resolution is as follows: In the first case, we would add a null values for the missing field (if the field is optional). In the second case, we would convert the data for the input field to a wider type, e.g. int (input type) => long (table type). In the third case, we would rearrange the input data to match the target table. *** DynamicUpdateOperator A dedicated operator to updating the schema / spec for the table associated with a DynamicRecord. --- .../flink/sink/EqualityFieldKeySelector.java | 12 +- .../flink/sink/NonThrowingKeySelector.java | 33 ++ .../flink/sink/PartitionKeySelector.java | 13 +- .../flink/sink/dynamic/DynamicSinkUtil.java | 61 +++ .../dynamic/DynamicTableUpdateOperator.java | 72 ++++ .../flink/sink/dynamic/HashKeyGenerator.java | 371 ++++++++++++++++++ .../flink/sink/dynamic/RowDataEvolver.java | 189 +++++++++ .../TestDynamicTableUpdateOperator.java | 107 +++++ .../sink/dynamic/TestHashKeyGenerator.java | 338 ++++++++++++++++ .../sink/dynamic/TestRowDataEvolver.java | 256 ++++++++++++ 10 files changed, 1441 insertions(+), 11 deletions(-) create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 18b269d6c3e9..0f6d208370bd 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.Schema; @@ -30,10 +30,11 @@ 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 { +@Internal +public class EqualityFieldKeySelector implements NonThrowingKeySelector { private final Schema schema; private final RowType flinkSchema; @@ -43,7 +44,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/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java new file mode 100644 index 000000000000..65a18c1c0047 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java @@ -0,0 +1,33 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.api.java.functions.KeySelector; + +/** + * A non-throwing variant of Flink's {@link KeySelector}. This avoids having to convert checked + * exceptions to runtime exceptions. + */ +@Internal +public interface NonThrowingKeySelector extends KeySelector { + + @Override + K getKey(I value); +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java index df951684b446..a2442198d600 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.PartitionKey; @@ -27,11 +27,12 @@ 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 { +@Internal +public class PartitionKeySelector implements NonThrowingKeySelector { private final Schema schema; private final PartitionKey partitionKey; @@ -39,7 +40,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/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java new file mode 100644 index 000000000000..857644744ddb --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.Collections; +import java.util.List; +import org.apache.iceberg.Schema; +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; + +class DynamicSinkUtil { + + private DynamicSinkUtil() {} + + 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.newArrayListWithCapacity(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; + } + + static int safeAbs(int input) { + if (input >= 0) { + return input; + } + if (input == Integer.MIN_VALUE) { + // -Integer.MIN_VALUE would be Integer.MIN_VALUE due to integer overflow. Map to + // Integer.MAX_VALUE instead! + return Integer.MAX_VALUE; + } + return -input; + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java new file mode 100644 index 000000000000..a217543eae41 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -0,0 +1,72 @@ +/* + * 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 TableMetadataCache(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.DATA_CONVERSION_NEEDED) { + RowData newRowData = RowDataEvolver.convert(data.rowData(), data.schema(), newData.f0); + data.setRowData(newRowData); + } + + return data; + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java new file mode 100644 index 000000000000..044ad3992629 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -0,0 +1,371 @@ +/* + * 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 javax.annotation.Nullable; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.functions.KeySelector; +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; + +/** + * The HashKeyGenerator is responsible for creating the appropriate hash key for Flink's keyBy + * operation. The hash key is generated depending on the user-provided DynamicRecord and the table + * metadata. Under the hood, we maintain a set of Flink {@link KeySelector}s which implement the + * appropriate Iceberg {@link DistributionMode}. For every table, we randomly select a consistent + * subset of writer subtasks which receive data via their associated keys, depending on the chosen + * DistributionMode. + * + *

Caching ensures that a new key selector is also created when the table metadata (e.g. schema, + * spec) or the user-provided metadata changes (e.g. distribution mode, write parallelism). + */ +class HashKeyGenerator { + private static final Logger LOG = LoggerFactory.getLogger(HashKeyGenerator.class); + + private final int maxWriteParallelism; + private final Cache> keySelectorCache; + + HashKeyGenerator(int maxCacheSize, int maxWriteParallelism) { + this.maxWriteParallelism = maxWriteParallelism; + this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); + } + + Integer generateKey(DynamicRecord dynamicRecord) { + return generateKey(dynamicRecord, null, null, null); + } + + Integer generateKey( + DynamicRecord dynamicRecord, + @Nullable Schema tableSchema, + @Nullable PartitionSpec tableSpec, + @Nullable RowData overrideRowData) { + String tableIdent = dynamicRecord.tableIdentifier().toString(); + SelectorKey cacheKey = + new SelectorKey( + tableIdent, + dynamicRecord.branch(), + tableSchema != null ? tableSchema.schemaId() : null, + tableSpec != null ? tableSpec.specId() : null, + dynamicRecord.schema(), + dynamicRecord.spec(), + dynamicRecord.equalityFields()); + return keySelectorCache + .get( + cacheKey, + k -> + getKeySelector( + tableIdent, + MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()), + MoreObjects.firstNonNull(tableSpec, dynamicRecord.spec()), + MoreObjects.firstNonNull( + dynamicRecord.distributionMode(), DistributionMode.NONE), + MoreObjects.firstNonNull( + dynamicRecord.equalityFields(), Collections.emptyList()), + dynamicRecord.writeParallelism())) + .getKey(overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); + } + + private 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), + DynamicSinkUtil.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[ + DynamicSinkUtil.safeAbs(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; + } + } + + /** + * Cache key for the {@link NonThrowingKeySelector}. Only contains the {@link Schema} and the + * {@link PartitionSpec} if their ids are not provided. + */ + 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; + + SelectorKey( + String tableName, + String branch, + @Nullable Integer tableSchemaId, + @Nullable Integer tableSpecId, + Schema schema, + PartitionSpec spec, + List equalityFields) { + this.tableName = tableName; + this.branch = branch; + this.schemaId = tableSchemaId; + this.specId = tableSpecId; + this.schema = tableSchemaId == null ? schema : null; + this.spec = tableSpecId == null ? spec : null; + this.equalityFields = 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("equalityFields", equalityFields) + .toString(); + } + } + + @VisibleForTesting + Cache> getKeySelectorCache() { + return keySelectorCache; + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java new file mode 100644 index 000000000000..10244dc4bbd4 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java @@ -0,0 +1,189 @@ +/* + * 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; + +/** + * A RowDataEvolver is responsible to change the input RowData to make it compatible with the target + * schema. This is done when + * + *

    + *
  1. The input schema has fewer fields than the target schema. + *
  2. The table types are wider than the input type. + *
  3. The field order differs for source and target schema. + *
+ * + *

The resolution is as follows: + * + *

    + *
  1. In the first case, we would add a null values for the missing field (if the field is + * optional). + *
  2. In the second case, we would convert the data for the input field to a wider type, e.g. int + * (input type) => long (table type). + *
  3. In the third case, we would rearrange the input data to match the target table. + *
+ */ +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/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java new file mode 100644 index 000000000000..fd5b5ef6be4f --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import org.apache.flink.table.data.GenericRowData; +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.HadoopCatalogExtension; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +class TestDynamicTableUpdateOperator { + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TABLE); + + private static final Schema SCHEMA1 = + new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + private static final Schema SCHEMA2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + @Test + void testDynamicTableUpdateOperatorNewTable() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + assertThat(catalog.tableExists(table)).isFalse(); + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize, cacheRefreshMs); + operator.open(null); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1, "test"), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptyList()); + DynamicRecordInternal output = operator.map(input); + + assertThat(catalog.tableExists(table)).isTrue(); + assertThat(input).isEqualTo(output); + } + + @Test + void testDynamicTableUpdateOperatorSchemaChange() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize, cacheRefreshMs); + operator.open(null); + + catalog.createTable(table, SCHEMA1); + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA2, + GenericRowData.of(1, "test"), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptyList()); + DynamicRecordInternal output = operator.map(input); + + assertThat(catalog.loadTable(table).schema().sameSchema(SCHEMA2)).isTrue(); + assertThat(input).isEqualTo(output); + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java new file mode 100644 index 000000000000..f64ca5fb286c --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -0,0 +1,338 @@ +/* + * 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 com.github.benmanes.caffeine.cache.Cache; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +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.flink.sink.NonThrowingKeySelector; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +class TestHashKeyGenerator { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + private static final String BRANCH = "main"; + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); + + @Test + void testRoundRobinWithDistributionModeNone() { + int writeParallelism = 10; + int maxWriteParallelism = 2; + HashKeyGenerator generator = new HashKeyGenerator(1, maxWriteParallelism); + PartitionSpec spec = PartitionSpec.unpartitioned(); + + GenericRowData row = GenericRowData.of(1, StringData.fromString("z")); + int writeKey1 = + getWriteKey( + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + int writeKey2 = + getWriteKey( + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + int writeKey3 = + getWriteKey( + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + int writeKey4 = + getWriteKey( + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + + assertThat(writeKey1).isNotEqualTo(writeKey2); + assertThat(writeKey3).isEqualTo(writeKey1); + assertThat(writeKey4).isEqualTo(writeKey2); + } + + @Test + void testBucketingWithDistributionModeHash() { + int writeParallelism = 3; + HashKeyGenerator generator = new HashKeyGenerator(1, 8); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + + GenericRowData row1 = GenericRowData.of(1, StringData.fromString("a")); + GenericRowData row2 = GenericRowData.of(1, StringData.fromString("b")); + GenericRowData row3 = GenericRowData.of(2, StringData.fromString("c")); + GenericRowData row4 = GenericRowData.of(2, StringData.fromString("d")); + + int writeKey1 = + getWriteKey( + generator, + spec, + DistributionMode.HASH, + writeParallelism, + Collections.emptyList(), + row1); + int writeKey2 = + getWriteKey( + generator, + spec, + DistributionMode.HASH, + writeParallelism, + Collections.emptyList(), + row2); + int writeKey3 = + getWriteKey( + generator, + spec, + DistributionMode.HASH, + writeParallelism, + Collections.emptyList(), + row3); + int writeKey4 = + getWriteKey( + generator, + spec, + DistributionMode.HASH, + writeParallelism, + Collections.emptyList(), + row4); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + assertThat(writeKey4).isEqualTo(writeKey3); + } + + @Test + void testEqualityKeys() { + int writeParallelism = 2; + HashKeyGenerator generator = new HashKeyGenerator(16, 8); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + GenericRowData row1 = GenericRowData.of(1, StringData.fromString("foo")); + GenericRowData row2 = GenericRowData.of(1, StringData.fromString("bar")); + GenericRowData row3 = GenericRowData.of(2, StringData.fromString("baz")); + List equalityColumns = Collections.singletonList("id"); + + int writeKey1 = + getWriteKey( + generator, + unpartitioned, + DistributionMode.NONE, + writeParallelism, + equalityColumns, + row1); + int writeKey2 = + getWriteKey( + generator, + unpartitioned, + DistributionMode.NONE, + writeParallelism, + equalityColumns, + row2); + int writeKey3 = + getWriteKey( + generator, + unpartitioned, + DistributionMode.NONE, + writeParallelism, + equalityColumns, + row3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey2).isNotEqualTo(writeKey3); + } + + @Test + void testCapAtMaxWriteParallelism() { + int writeParallelism = 10; + int maxWriteParallelism = 5; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + Set writeKeys = Sets.newHashSet(); + for (int i = 0; i < 20; i++) { + GenericRowData row = GenericRowData.of(i, StringData.fromString("z")); + writeKeys.add( + getWriteKey( + generator, + unpartitioned, + DistributionMode.NONE, + writeParallelism, + Collections.emptyList(), + row)); + } + + assertThat(writeKeys).hasSize(maxWriteParallelism); + } + + @Test + void testHashModeWithoutEqualityFieldsFallsBackToNone() { + int writeParallelism = 2; + HashKeyGenerator generator = new HashKeyGenerator(16, 8); + Schema noIdSchema = new Schema(Types.NestedField.required(1, "x", Types.StringType.get())); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + DynamicRecord record = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + noIdSchema, + GenericRowData.of(StringData.fromString("v")), + unpartitioned, + DistributionMode.HASH, + writeParallelism); + + int writeKey1 = generator.generateKey(record); + int writeKey2 = generator.generateKey(record); + int writeKey3 = generator.generateKey(record); + assertThat(writeKey1).isNotEqualTo(writeKey2); + assertThat(writeKey3).isEqualTo(writeKey1); + } + + @Test + void testOverrides() { + int maxCacheSize = 10; + int writeParallelism = 5; + int maxWriteParallelism = 10; + HashKeyGenerator generator = new HashKeyGenerator(maxCacheSize, maxWriteParallelism); + + DynamicRecord record = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + SCHEMA, + GenericRowData.of(1, StringData.fromString("foo")), + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + writeParallelism); + + int writeKey1 = generator.generateKey(record); + int writeKey2 = generator.generateKey(record); + // Assert that we are bucketing via NONE (round-robin) + assertThat(writeKey1).isNotEqualTo(writeKey2); + + // Schema has different id + Schema overrideSchema = new Schema(42, SCHEMA.columns()); + // Spec has different id + PartitionSpec overrideSpec = PartitionSpec.builderFor(SCHEMA).withSpecId(42).build(); + RowData overrideData = GenericRowData.of(1L, StringData.fromString("foo")); + + // We get a new key selector for the schema which starts off on the same offset + assertThat(generator.generateKey(record, overrideSchema, null, null)).isEqualTo(writeKey1); + // We get a new key selector for the spec which starts off on the same offset + assertThat(generator.generateKey(record, null, overrideSpec, null)).isEqualTo(writeKey1); + // We get the same key selector which yields a different result for the overridden data + assertThat(generator.generateKey(record, null, null, overrideData)).isNotEqualTo(writeKey1); + } + + @Test + void testMultipleTables() { + int maxCacheSize = 10; + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(maxCacheSize, maxWriteParallelism); + + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + GenericRowData rowData = GenericRowData.of(1, StringData.fromString("foo")); + + DynamicRecord record1 = + new DynamicRecord( + TableIdentifier.of("a", "table"), + BRANCH, + SCHEMA, + rowData, + unpartitioned, + DistributionMode.HASH, + writeParallelism); + record1.setEqualityFields(Collections.singletonList("id")); + DynamicRecord record2 = + new DynamicRecord( + TableIdentifier.of("other", "table"), + BRANCH, + SCHEMA, + rowData, + unpartitioned, + DistributionMode.HASH, + writeParallelism); + record2.setEqualityFields(Collections.singletonList("id")); + + // Consistent hashing for the same record due to HASH distribution mode + int writeKeyRecord1 = generator.generateKey(record1); + assertThat(writeKeyRecord1).isEqualTo(generator.generateKey(record1)); + int writeKeyRecord2 = generator.generateKey(record2); + assertThat(writeKeyRecord2).isEqualTo(generator.generateKey(record2)); + + // But the write keys are for different tables and should not be equal + assertThat(writeKeyRecord1).isNotEqualTo(writeKeyRecord2); + } + + @Test + void testCaching() { + int maxCacheSize = 1; + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(maxCacheSize, maxWriteParallelism); + Cache> keySelectorCache = + generator.getKeySelectorCache(); + + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + DynamicRecord record = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + SCHEMA, + GenericRowData.of(1, StringData.fromString("foo")), + unpartitioned, + DistributionMode.NONE, + writeParallelism); + + int writeKey1 = generator.generateKey(record); + assertThat(keySelectorCache.estimatedSize()).isEqualTo(1); + + int writeKey2 = generator.generateKey(record); + assertThat(writeKey2).isNotEqualTo(writeKey1); + // Manually clean up because the cleanup is not always triggered + keySelectorCache.cleanUp(); + assertThat(keySelectorCache.estimatedSize()).isEqualTo(1); + + int writeKey3 = generator.generateKey(record); + // Manually clean up because the cleanup is not always triggered + keySelectorCache.cleanUp(); + assertThat(keySelectorCache.estimatedSize()).isEqualTo(1); + // We create a new key selector which will start off at the same position + assertThat(writeKey1).isEqualTo(writeKey3); + } + + private static int getWriteKey( + HashKeyGenerator generator, + PartitionSpec spec, + DistributionMode mode, + int writeParallelism, + List equalityFields, + GenericRowData row) { + DynamicRecord record = + new DynamicRecord(TABLE_IDENTIFIER, BRANCH, SCHEMA, row, spec, mode, writeParallelism); + record.setEqualityFields(equalityFields); + return generator.generateKey(record); + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java new file mode 100644 index 000000000000..2553575f1893 --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataEvolver.java @@ -0,0 +1,256 @@ +/* + * 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 static org.junit.jupiter.api.Assertions.assertThrows; + +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.jupiter.api.Test; + +class TestRowDataEvolver { + + static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + 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, "onemore", Types.DoubleType.get())); + + @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"), SCHEMA, 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())); + + 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())))); + + 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); + } +} From 871bb098461f065e4fc59bfbdfc435c95d60c156 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 11:40:09 +0200 Subject: [PATCH 02/14] Remove NonThrowingKeySelector --- .../flink/sink/EqualityFieldKeySelector.java | 7 ++-- .../flink/sink/NonThrowingKeySelector.java | 33 ---------------- .../flink/sink/PartitionKeySelector.java | 9 +++-- .../flink/sink/dynamic/DynamicSinkUtil.java | 4 ++ .../flink/sink/dynamic/HashKeyGenerator.java | 39 +++++++++---------- .../sink/dynamic/TestHashKeyGenerator.java | 23 +++++------ 6 files changed, 44 insertions(+), 71 deletions(-) delete mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 0f6d208370bd..41e3e004809c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.flink.annotation.Internal; +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,11 +31,11 @@ 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. */ @Internal -public class EqualityFieldKeySelector implements NonThrowingKeySelector { +public class EqualityFieldKeySelector implements KeySelector { private final Schema schema; private final RowType flinkSchema; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java deleted file mode 100644 index 65a18c1c0047..000000000000 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/NonThrowingKeySelector.java +++ /dev/null @@ -1,33 +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.annotation.Internal; -import org.apache.flink.api.java.functions.KeySelector; - -/** - * A non-throwing variant of Flink's {@link KeySelector}. This avoids having to convert checked - * exceptions to runtime exceptions. - */ -@Internal -public interface NonThrowingKeySelector extends KeySelector { - - @Override - K getKey(I value); -} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java index a2442198d600..17c8233e1f6f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink; import org.apache.flink.annotation.Internal; +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,12 +28,12 @@ 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}. */ @Internal -public class PartitionKeySelector implements NonThrowingKeySelector { +public class PartitionKeySelector implements KeySelector { private final Schema schema; private final PartitionKey partitionKey; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java index 857644744ddb..748130ca8c6e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java @@ -37,6 +37,7 @@ static List getEqualityFieldIds(List equalityFields, Schema sch return Collections.emptyList(); } } + List equalityFieldIds = Lists.newArrayListWithCapacity(equalityFields.size()); for (String equalityField : equalityFields) { Types.NestedField field = schema.findField(equalityField); @@ -44,6 +45,7 @@ static List getEqualityFieldIds(List equalityFields, Schema sch field, "Equality field %s does not exist in schema", equalityField); equalityFieldIds.add(field.fieldId()); } + return equalityFieldIds; } @@ -51,11 +53,13 @@ static int safeAbs(int input) { if (input >= 0) { return input; } + if (input == Integer.MIN_VALUE) { // -Integer.MIN_VALUE would be Integer.MIN_VALUE due to integer overflow. Map to // Integer.MAX_VALUE instead! return Integer.MAX_VALUE; } + return -input; } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 044ad3992629..9bba2182eed4 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -38,7 +38,6 @@ 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; @@ -61,22 +60,23 @@ class HashKeyGenerator { private static final Logger LOG = LoggerFactory.getLogger(HashKeyGenerator.class); private final int maxWriteParallelism; - private final Cache> keySelectorCache; + private final Cache> keySelectorCache; HashKeyGenerator(int maxCacheSize, int maxWriteParallelism) { this.maxWriteParallelism = maxWriteParallelism; this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); } - Integer generateKey(DynamicRecord dynamicRecord) { + int generateKey(DynamicRecord dynamicRecord) throws Exception { return generateKey(dynamicRecord, null, null, null); } - Integer generateKey( + int generateKey( DynamicRecord dynamicRecord, @Nullable Schema tableSchema, @Nullable PartitionSpec tableSpec, - @Nullable RowData overrideRowData) { + @Nullable RowData overrideRowData) + throws Exception { String tableIdent = dynamicRecord.tableIdentifier().toString(); SelectorKey cacheKey = new SelectorKey( @@ -103,7 +103,7 @@ Integer generateKey( .getKey(overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); } - private NonThrowingKeySelector getKeySelector( + private KeySelector getKeySelector( String tableName, Schema schema, PartitionSpec spec, @@ -176,7 +176,7 @@ private NonThrowingKeySelector getKeySelector( } } - private static NonThrowingKeySelector equalityFieldKeySelector( + private static KeySelector equalityFieldKeySelector( String tableName, Schema schema, List equalityFields, @@ -192,13 +192,13 @@ private static NonThrowingKeySelector equalityFieldKeySelector maxWriteParallelism); } - private static NonThrowingKeySelector partitionKeySelector( + private static KeySelector partitionKeySelector( String tableName, Schema schema, PartitionSpec spec, int writeParallelism, int maxWriteParallelism) { - NonThrowingKeySelector inner = + KeySelector inner = new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); return new TargetLimitedKeySelector( in -> inner.getKey(in).hashCode(), @@ -207,7 +207,7 @@ private static NonThrowingKeySelector partitionKeySelector( maxWriteParallelism); } - private static NonThrowingKeySelector tableKeySelector( + private static KeySelector tableKeySelector( String tableName, int writeParallelism, int maxWriteParallelism) { return new TargetLimitedKeySelector( new RoundRobinKeySelector<>(writeParallelism), @@ -218,17 +218,16 @@ private static NonThrowingKeySelector tableKeySelector( /** * 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}. + * #wrapped} {@link KeySelector} to {@link #writeParallelism}. */ - private static class TargetLimitedKeySelector - implements NonThrowingKeySelector { - private final NonThrowingKeySelector wrapped; + private static class TargetLimitedKeySelector implements KeySelector { + private final KeySelector wrapped; private final int writeParallelism; private final int[] distinctKeys; @SuppressWarnings("checkstyle:ParameterAssignment") TargetLimitedKeySelector( - NonThrowingKeySelector wrapped, + KeySelector wrapped, int salt, int writeParallelism, int maxWriteParallelism) { @@ -261,7 +260,7 @@ private static class TargetLimitedKeySelector } @Override - public Integer getKey(RowData value) { + public Integer getKey(RowData value) throws Exception { return distinctKeys[ DynamicSinkUtil.safeAbs(wrapped.getKey(value).hashCode()) % writeParallelism]; } @@ -280,7 +279,7 @@ private static int subtaskId(int key, int writeParallelism, int maxWriteParallel * * @param unused input for key generation */ - private static class RoundRobinKeySelector implements NonThrowingKeySelector { + private static class RoundRobinKeySelector implements KeySelector { private final int maxTarget; private int lastTarget = 0; @@ -296,8 +295,8 @@ public Integer getKey(T value) { } /** - * Cache key for the {@link NonThrowingKeySelector}. Only contains the {@link Schema} and the - * {@link PartitionSpec} if their ids are not provided. + * Cache key for the {@link KeySelector}. Only contains the {@link Schema} and the {@link + * PartitionSpec} if their ids are not provided. */ static class SelectorKey { private final String tableName; @@ -365,7 +364,7 @@ public String toString() { } @VisibleForTesting - Cache> getKeySelectorCache() { + Cache> getKeySelectorCache() { return keySelectorCache; } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index f64ca5fb286c..ad435519b51a 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; import java.util.Set; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -31,7 +32,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.sink.NonThrowingKeySelector; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -47,7 +47,7 @@ class TestHashKeyGenerator { private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); @Test - void testRoundRobinWithDistributionModeNone() { + void testRoundRobinWithDistributionModeNone() throws Exception { int writeParallelism = 10; int maxWriteParallelism = 2; HashKeyGenerator generator = new HashKeyGenerator(1, maxWriteParallelism); @@ -73,7 +73,7 @@ void testRoundRobinWithDistributionModeNone() { } @Test - void testBucketingWithDistributionModeHash() { + void testBucketingWithDistributionModeHash() throws Exception { int writeParallelism = 3; HashKeyGenerator generator = new HashKeyGenerator(1, 8); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); @@ -122,7 +122,7 @@ void testBucketingWithDistributionModeHash() { } @Test - void testEqualityKeys() { + void testEqualityKeys() throws Exception { int writeParallelism = 2; HashKeyGenerator generator = new HashKeyGenerator(16, 8); PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); @@ -162,7 +162,7 @@ void testEqualityKeys() { } @Test - void testCapAtMaxWriteParallelism() { + void testCapAtMaxWriteParallelism() throws Exception { int writeParallelism = 10; int maxWriteParallelism = 5; HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); @@ -185,7 +185,7 @@ void testCapAtMaxWriteParallelism() { } @Test - void testHashModeWithoutEqualityFieldsFallsBackToNone() { + void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeParallelism = 2; HashKeyGenerator generator = new HashKeyGenerator(16, 8); Schema noIdSchema = new Schema(Types.NestedField.required(1, "x", Types.StringType.get())); @@ -209,7 +209,7 @@ void testHashModeWithoutEqualityFieldsFallsBackToNone() { } @Test - void testOverrides() { + void testOverrides() throws Exception { int maxCacheSize = 10; int writeParallelism = 5; int maxWriteParallelism = 10; @@ -245,7 +245,7 @@ void testOverrides() { } @Test - void testMultipleTables() { + void testMultipleTables() throws Exception { int maxCacheSize = 10; int writeParallelism = 2; int maxWriteParallelism = 8; @@ -287,12 +287,12 @@ void testMultipleTables() { } @Test - void testCaching() { + void testCaching() throws Exception { int maxCacheSize = 1; int writeParallelism = 2; int maxWriteParallelism = 8; HashKeyGenerator generator = new HashKeyGenerator(maxCacheSize, maxWriteParallelism); - Cache> keySelectorCache = + Cache> keySelectorCache = generator.getKeySelectorCache(); PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); @@ -329,7 +329,8 @@ private static int getWriteKey( DistributionMode mode, int writeParallelism, List equalityFields, - GenericRowData row) { + GenericRowData row) + throws Exception { DynamicRecord record = new DynamicRecord(TABLE_IDENTIFIER, BRANCH, SCHEMA, row, spec, mode, writeParallelism); record.setEqualityFields(equalityFields); From 1b326a7fd14f31691716623b22cdcc92e825ca80 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 11:40:25 +0200 Subject: [PATCH 03/14] Adjust logging --- .../apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 9bba2182eed4..86a036b8c068 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -110,7 +110,8 @@ private KeySelector getKeySelector( DistributionMode mode, List equalityFields, int writeParallelism) { - LOG.info("Write distribution mode is '{}'", mode.modeName()); + LOG.debug( + "Creating new KeySelector for table '{}' with distribution mode '{}'", tableName, mode); switch (mode) { case NONE: if (equalityFields.isEmpty()) { From dc410e8cfa27464337b0820493da22642846788a Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 11:45:55 +0200 Subject: [PATCH 04/14] JavaDoc for DynamicTableUpdateOperator --- .../flink/sink/dynamic/DynamicTableUpdateOperator.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java index a217543eae41..c37532714d48 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -29,6 +29,12 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.CatalogLoader; +/** + * An optional operator to perform table updates for tables (e.g. schema update) in a non-concurrent + * way. Records must be keyed / routed to this operator by table name to ensure non-concurrent + * updates. The operator itself forwards the record after updating schema / spec of the table. The + * update is also reflected in the record. + */ @Internal class DynamicTableUpdateOperator extends RichMapFunction { From cd2d825e8cb3e448fc72594b049c77f3eb9efc38 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 11:51:42 +0200 Subject: [PATCH 05/14] JavaDoc Deterministic hashing --- .../apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 86a036b8c068..814820551491 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -55,6 +55,9 @@ * *

Caching ensures that a new key selector is also created when the table metadata (e.g. schema, * spec) or the user-provided metadata changes (e.g. distribution mode, write parallelism). + * + *

Note: The hashing must be deterministic given the same parameters of the KeySelector and the + * same provided values. */ class HashKeyGenerator { private static final Logger LOG = LoggerFactory.getLogger(HashKeyGenerator.class); From 36cfeda83f59fa6c47c057c7e8c507d541ae01d8 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 11:54:36 +0200 Subject: [PATCH 06/14] More logging context --- .../flink/sink/dynamic/HashKeyGenerator.java | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 814820551491..41c07867af7e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -120,7 +120,9 @@ private KeySelector getKeySelector( if (equalityFields.isEmpty()) { return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); } else { - LOG.info("Distribute rows by equality fields, because there are equality fields set"); + LOG.info( + "{}Distribute rows by equality fields, because there are equality fields set", + tableName); return equalityFieldKeySelector( tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); } @@ -129,8 +131,9 @@ private KeySelector getKeySelector( 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"); + "{}: Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned", + tableName); return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); } else { return partitionKeySelector( @@ -139,16 +142,18 @@ private KeySelector getKeySelector( } else { if (spec.isUnpartitioned()) { LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and table is unpartitioned"); + "{}: Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned", + tableName); 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' " + "%s: In 'hash' distribution mode with equality fields set, partition field '%s' " + "should be included in equality fields: '%s'", + tableName, partitionField, schema.columns().stream() .filter(c -> equalityFields.contains(c.name())) @@ -162,21 +167,24 @@ private KeySelector getKeySelector( case RANGE: if (schema.identifierFieldIds().isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " + "{}: Fallback to use 'none' distribution mode, because there are no equality fields set " + "and {}=range is not supported yet in flink", + tableName, WRITE_DISTRIBUTION_MODE); return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); } else { LOG.info( - "Distribute rows by equality fields, because there are equality fields set " + "{}: Distribute rows by equality fields, because there are equality fields set " + "and{}=range is not supported yet in flink", + tableName, WRITE_DISTRIBUTION_MODE); return equalityFieldKeySelector( tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); } default: - throw new IllegalArgumentException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + throw new IllegalArgumentException( + tableName + ": Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); } } From bfdbe5afd0251cf2d9349ece2c9ec64c28586c70 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 12:28:52 +0200 Subject: [PATCH 07/14] Assert subtask id (slotid) --- .../sink/dynamic/TestHashKeyGenerator.java | 46 +++++++++++++++++-- 1 file changed, 41 insertions(+), 5 deletions(-) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index ad435519b51a..924b734fd332 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -70,12 +71,18 @@ void testRoundRobinWithDistributionModeNone() throws Exception { assertThat(writeKey1).isNotEqualTo(writeKey2); assertThat(writeKey3).isEqualTo(writeKey1); assertThat(writeKey4).isEqualTo(writeKey2); + + assertThat(getSubTaskId(writeKey1, writeParallelism, maxWriteParallelism)).isEqualTo(0); + assertThat(getSubTaskId(writeKey2, writeParallelism, maxWriteParallelism)).isEqualTo(5); + assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(0); + assertThat(getSubTaskId(writeKey4, writeParallelism, maxWriteParallelism)).isEqualTo(5); } @Test void testBucketingWithDistributionModeHash() throws Exception { int writeParallelism = 3; - HashKeyGenerator generator = new HashKeyGenerator(1, 8); + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(1, maxWriteParallelism); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); GenericRowData row1 = GenericRowData.of(1, StringData.fromString("a")); @@ -119,12 +126,18 @@ void testBucketingWithDistributionModeHash() throws Exception { assertThat(writeKey1).isEqualTo(writeKey2); assertThat(writeKey3).isNotEqualTo(writeKey1); assertThat(writeKey4).isEqualTo(writeKey3); + + assertThat(getSubTaskId(writeKey1, writeParallelism, maxWriteParallelism)).isEqualTo(0); + assertThat(getSubTaskId(writeKey2, writeParallelism, maxWriteParallelism)).isEqualTo(0); + assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(1); + assertThat(getSubTaskId(writeKey4, writeParallelism, maxWriteParallelism)).isEqualTo(1); } @Test void testEqualityKeys() throws Exception { int writeParallelism = 2; - HashKeyGenerator generator = new HashKeyGenerator(16, 8); + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); GenericRowData row1 = GenericRowData.of(1, StringData.fromString("foo")); @@ -159,6 +172,10 @@ void testEqualityKeys() throws Exception { assertThat(writeKey1).isEqualTo(writeKey2); assertThat(writeKey2).isNotEqualTo(writeKey3); + + assertThat(getSubTaskId(writeKey1, writeParallelism, maxWriteParallelism)).isEqualTo(1); + assertThat(getSubTaskId(writeKey2, writeParallelism, maxWriteParallelism)).isEqualTo(1); + assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(0); } @Test @@ -182,12 +199,19 @@ void testCapAtMaxWriteParallelism() throws Exception { } assertThat(writeKeys).hasSize(maxWriteParallelism); + assertThat( + writeKeys.stream() + .map(key -> getSubTaskId(key, writeParallelism, writeParallelism)) + .distinct() + .count()) + .isEqualTo(maxWriteParallelism); } @Test void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeParallelism = 2; - HashKeyGenerator generator = new HashKeyGenerator(16, 8); + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); Schema noIdSchema = new Schema(Types.NestedField.required(1, "x", Types.StringType.get())); PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); @@ -206,10 +230,14 @@ void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeKey3 = generator.generateKey(record); assertThat(writeKey1).isNotEqualTo(writeKey2); assertThat(writeKey3).isEqualTo(writeKey1); + + assertThat(getSubTaskId(writeKey1, writeParallelism, maxWriteParallelism)).isEqualTo(1); + assertThat(getSubTaskId(writeKey2, writeParallelism, maxWriteParallelism)).isEqualTo(0); + assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(1); } @Test - void testOverrides() throws Exception { + void testSchemaSpecOverrides() throws Exception { int maxCacheSize = 10; int writeParallelism = 5; int maxWriteParallelism = 10; @@ -267,7 +295,7 @@ void testMultipleTables() throws Exception { record1.setEqualityFields(Collections.singletonList("id")); DynamicRecord record2 = new DynamicRecord( - TableIdentifier.of("other", "table"), + TableIdentifier.of("my", "other", "table"), BRANCH, SCHEMA, rowData, @@ -284,6 +312,9 @@ void testMultipleTables() throws Exception { // But the write keys are for different tables and should not be equal assertThat(writeKeyRecord1).isNotEqualTo(writeKeyRecord2); + + assertThat(getSubTaskId(writeKeyRecord1, writeParallelism, maxWriteParallelism)).isEqualTo(1); + assertThat(getSubTaskId(writeKeyRecord2, writeParallelism, maxWriteParallelism)).isEqualTo(0); } @Test @@ -336,4 +367,9 @@ private static int getWriteKey( record.setEqualityFields(equalityFields); return generator.generateKey(record); } + + private static int getSubTaskId(int writeKey1, int writeParallelism, int maxWriteParallelism) { + return KeyGroupRangeAssignment.assignKeyToParallelOperator( + writeKey1, maxWriteParallelism, writeParallelism); + } } From 0412697a67ff9f765b3e7b326372e7e9aa234a3b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 12:58:34 +0200 Subject: [PATCH 08/14] Change equalityFields from List to Set --- .../flink/sink/BaseDeltaTaskWriter.java | 4 +- .../flink/sink/EqualityFieldKeySelector.java | 7 ++- .../apache/iceberg/flink/sink/FlinkSink.java | 8 ++-- .../iceberg/flink/sink/IcebergSink.java | 9 ++-- .../flink/sink/PartitionedDeltaWriter.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 14 +++--- .../apache/iceberg/flink/sink/SinkUtil.java | 7 ++- .../flink/sink/UnpartitionedDeltaWriter.java | 4 +- .../flink/sink/dynamic/DynamicRecord.java | 8 ++-- .../sink/dynamic/DynamicRecordInternal.java | 10 ++-- .../DynamicRecordInternalSerializer.java | 16 +++---- .../flink/sink/dynamic/DynamicSinkUtil.java | 12 ++--- .../flink/sink/dynamic/DynamicWriter.java | 8 ++-- .../flink/sink/dynamic/HashKeyGenerator.java | 11 ++--- .../flink/sink/dynamic/WriteTarget.java | 16 +++---- .../flink/sink/TestDeltaTaskWriter.java | 18 +++---- ...namicRecordInternalSerializerTestBase.java | 2 +- .../TestDynamicCommittableSerializer.java | 6 +-- .../sink/dynamic/TestDynamicCommitter.java | 34 +++++++------- .../TestDynamicTableUpdateOperator.java | 4 +- .../TestDynamicWriteResultAggregator.java | 7 ++- .../TestDynamicWriteResultSerializer.java | 6 +-- .../flink/sink/dynamic/TestDynamicWriter.java | 4 +- .../sink/dynamic/TestHashKeyGenerator.java | 47 +++++-------------- .../flink/source/TestProjectMetaColumn.java | 11 +++-- 25 files changed, 128 insertions(+), 149 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index e8a46c5becd7..d845046cd2f6 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.List; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -56,7 +56,7 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.schema = schema; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java index 41e3e004809c..92e47792c13b 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -18,14 +18,13 @@ */ package org.apache.iceberg.flink.sink; -import java.util.List; +import java.util.Set; import org.apache.flink.annotation.Internal; 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; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeWrapper; import org.apache.iceberg.util.StructProjection; @@ -46,10 +45,10 @@ public class EqualityFieldKeySelector implements KeySelector { private transient StructLikeWrapper structLikeWrapper; public EqualityFieldKeySelector( - Schema schema, RowType flinkSchema, List equalityFieldIds) { + Schema schema, RowType flinkSchema, Set equalityFieldIds) { this.schema = schema; this.flinkSchema = flinkSchema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + this.deleteSchema = TypeUtil.select(schema, equalityFieldIds); } /** diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 3f2b265f7b93..c42e4a015ba3 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -419,7 +419,7 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = + Set equalityFieldIds = SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); @@ -524,7 +524,7 @@ private SingleOutputStreamOperator appendCommitter( private SingleOutputStreamOperator appendWriter( DataStream input, RowType flinkRowType, - List equalityFieldIds, + Set equalityFieldIds, int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { @@ -575,7 +575,7 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, - List equalityFieldIds, + Set equalityFieldIds, RowType flinkRowType, int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); @@ -711,7 +711,7 @@ static IcebergStreamWriter createStreamWriter( SerializableSupplier tableSupplier, FlinkWriteConf flinkWriteConf, RowType flinkRowType, - List equalityFieldIds) { + Set equalityFieldIds) { Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); Table initTable = tableSupplier.get(); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index 373568c1b3ea..2db61cc75233 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.function.Function; import org.apache.flink.annotation.Experimental; @@ -144,7 +145,7 @@ public class IcebergSink private final RowType flinkRowType; private final SerializableSupplier
tableSupplier; private final transient FlinkWriteConf flinkWriteConf; - private final List equalityFieldIds; + private final Set equalityFieldIds; private final boolean upsertMode; private final FileFormat dataFileFormat; private final long targetDataFileSize; @@ -153,7 +154,7 @@ public class IcebergSink private final int workerPoolSize; private final Table table; - private final List equalityFieldColumns = null; + private final Set equalityFieldColumns = null; private IcebergSink( TableLoader tableLoader, @@ -164,7 +165,7 @@ private IcebergSink( RowType flinkRowType, SerializableSupplier
tableSupplier, FlinkWriteConf flinkWriteConf, - List equalityFieldIds, + Set equalityFieldIds, String branch, boolean overwriteMode) { this.tableLoader = tableLoader; @@ -561,7 +562,7 @@ IcebergSink build() { boolean overwriteMode = flinkWriteConf.overwriteMode(); // Validate the equality fields and partition fields if we enable the upsert mode. - List equalityFieldIds = + Set equalityFieldIds = SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); if (flinkWriteConf.upsertMode()) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java index 38062dd1a2c4..3eb4dba80281 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -49,7 +49,7 @@ class PartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super( spec, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 8dc8d38869bc..7e8db13d2803 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.flink.sink; -import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -48,7 +48,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final PartitionSpec spec; private final long targetFileSizeBytes; private final FileFormat format; - private final List equalityFieldIds; + private final Set equalityFieldIds; private final boolean upsert; private final FileAppenderFactory appenderFactory; @@ -60,7 +60,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert) { this( () -> table, @@ -78,7 +78,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert) { this( tableSupplier, @@ -98,7 +98,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert, Schema schema, PartitionSpec spec) { @@ -137,7 +137,7 @@ public RowDataTaskWriterFactory( flinkSchema, writeProperties, spec, - ArrayUtil.toIntArray(equalityFieldIds), + ArrayUtil.toPrimitive(equalityFieldIds.toArray(new Integer[0])), TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), null); } else { @@ -148,7 +148,7 @@ public RowDataTaskWriterFactory( flinkSchema, writeProperties, spec, - ArrayUtil.toIntArray(equalityFieldIds), + ArrayUtil.toPrimitive(equalityFieldIds.toArray(new Integer[0])), schema, null); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java index 0a2a7c1b88a7..3f60b45a1f81 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -24,7 +24,6 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; 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.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,8 +41,8 @@ private SinkUtil() {} private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); - static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { - List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + static Set checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + Set equalityFieldIds = Sets.newHashSet(table.schema().identifierFieldIds()); if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); for (String column : equalityFieldColumns) { @@ -63,7 +62,7 @@ static List checkAndGetEqualityFieldIds(Table table, List equal equalityFieldSet, table.schema().identifierFieldIds()); } - equalityFieldIds = Lists.newArrayList(equalityFieldSet); + equalityFieldIds = Sets.newHashSet(equalityFieldSet); } return equalityFieldIds; } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java index 7680fb933b20..b6ad03514bb0 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.List; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -41,7 +41,7 @@ class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - List equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super( spec, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java index 994f7a0865f6..600a4d8b950c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink.dynamic; -import java.util.List; +import java.util.Set; import javax.annotation.Nullable; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DistributionMode; @@ -37,7 +37,7 @@ public class DynamicRecord { private DistributionMode distributionMode; private int writeParallelism; private boolean upsertMode; - @Nullable private List equalityFields; + @Nullable private Set equalityFields; public DynamicRecord( TableIdentifier tableIdentifier, @@ -120,11 +120,11 @@ public void setUpsertMode(boolean upsertMode) { this.upsertMode = upsertMode; } - public List equalityFields() { + public Set equalityFields() { return equalityFields; } - public void setEqualityFields(List equalityFields) { + public void setEqualityFields(Set equalityFields) { this.equalityFields = equalityFields; } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java index 958a1e853983..fe1f4cdac9b7 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.flink.sink.dynamic; -import java.util.List; import java.util.Objects; +import java.util.Set; import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; @@ -37,7 +37,7 @@ class DynamicRecordInternal { private int writerKey; private RowData rowData; private boolean upsertMode; - private List equalityFieldIds; + private Set equalityFieldIds; // Required for serialization instantiation DynamicRecordInternal() {} @@ -50,7 +50,7 @@ class DynamicRecordInternal { PartitionSpec spec, int writerKey, boolean upsertMode, - List equalityFieldsIds) { + Set equalityFieldsIds) { this.tableName = tableName; this.branch = branch; this.schema = schema; @@ -117,11 +117,11 @@ public void setUpsertMode(boolean upsertMode) { this.upsertMode = upsertMode; } - public List equalityFields() { + public Set equalityFields() { return equalityFieldIds; } - public void setEqualityFieldIds(List equalityFieldIds) { + public void setEqualityFieldIds(Set equalityFieldIds) { this.equalityFieldIds = equalityFieldIds; } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java index a6f38f7d6bc9..d0a335b18256 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Objects; +import java.util.Set; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -31,11 +31,11 @@ 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.hadoop.util.Sets; 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 { @@ -126,11 +126,11 @@ public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOE RowData rowData = rowDataSerializer.deserialize(dataInputView); boolean upsertMode = dataInputView.readBoolean(); int numEqualityFields = dataInputView.readInt(); - final List equalityFieldIds; + final Set equalityFieldIds; if (numEqualityFields > 0) { - equalityFieldIds = Lists.newArrayList(); + equalityFieldIds = Sets.newHashSetWithExpectedSize(numEqualityFields); } else { - equalityFieldIds = Collections.emptyList(); + equalityFieldIds = Collections.emptySet(); } for (int i = 0; i < numEqualityFields; i++) { @@ -173,11 +173,11 @@ public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputV RowData rowData = rowDataSerializer.deserialize(dataInputView); boolean upsertMode = dataInputView.readBoolean(); int numEqualityFields = dataInputView.readInt(); - final List equalityFieldIds; + final Set equalityFieldIds; if (numEqualityFields > 0) { - equalityFieldIds = Lists.newArrayList(); + equalityFieldIds = Sets.newHashSetWithExpectedSize(numEqualityFields); } else { - equalityFieldIds = Collections.emptyList(); + equalityFieldIds = Collections.emptySet(); } for (int i = 0; i < numEqualityFields; i++) { equalityFieldIds.add(dataInputView.readInt()); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java index 748130ca8c6e..6ea6dcab867a 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java @@ -19,26 +19,26 @@ package org.apache.iceberg.flink.sink.dynamic; import java.util.Collections; -import java.util.List; +import java.util.Set; +import org.apache.hadoop.util.Sets; import org.apache.iceberg.Schema; 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; class DynamicSinkUtil { private DynamicSinkUtil() {} - static List getEqualityFieldIds(List equalityFields, Schema schema) { + static Set getEqualityFieldIds(Set equalityFields, Schema schema) { if (equalityFields == null || equalityFields.isEmpty()) { if (!schema.identifierFieldIds().isEmpty()) { - return Lists.newArrayList(schema.identifierFieldIds()); + return schema.identifierFieldIds(); } else { - return Collections.emptyList(); + return Collections.emptySet(); } } - List equalityFieldIds = Lists.newArrayListWithCapacity(equalityFields.size()); + Set equalityFieldIds = Sets.newHashSetWithExpectedSize(equalityFields.size()); for (String equalityField : equalityFields) { Types.NestedField field = schema.findField(equalityField); Preconditions.checkNotNull( diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index 3851dbf95603..f6ea3c1a393d 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -114,7 +114,7 @@ public void write(DynamicRecordInternal element, Context context) Maps.newHashMap(commonWriteProperties); tableWriteProperties.putAll(table.properties()); - List equalityFieldIds = + Set equalityFieldIds = getEqualityFields(table, element.equalityFields()); if (element.upsertMode()) { Preconditions.checkState( @@ -199,15 +199,15 @@ public Collection prepareCommit() throws IOException { return result; } - private static List getEqualityFields(Table table, List equalityFieldIds) { + private static Set getEqualityFields(Table table, Set equalityFieldIds) { if (equalityFieldIds != null && !equalityFieldIds.isEmpty()) { return equalityFieldIds; } Set identifierFieldIds = table.schema().identifierFieldIds(); if (identifierFieldIds != null && !identifierFieldIds.isEmpty()) { - return Lists.newArrayList(identifierFieldIds); + return identifierFieldIds; } - return Collections.emptyList(); + return Collections.emptySet(); } @VisibleForTesting diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 41c07867af7e..fb565fc28f48 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -23,7 +23,6 @@ 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; @@ -101,7 +100,7 @@ int generateKey( MoreObjects.firstNonNull( dynamicRecord.distributionMode(), DistributionMode.NONE), MoreObjects.firstNonNull( - dynamicRecord.equalityFields(), Collections.emptyList()), + dynamicRecord.equalityFields(), Collections.emptySet()), dynamicRecord.writeParallelism())) .getKey(overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); } @@ -111,7 +110,7 @@ private KeySelector getKeySelector( Schema schema, PartitionSpec spec, DistributionMode mode, - List equalityFields, + Set equalityFields, int writeParallelism) { LOG.debug( "Creating new KeySelector for table '{}' with distribution mode '{}'", tableName, mode); @@ -191,7 +190,7 @@ private KeySelector getKeySelector( private static KeySelector equalityFieldKeySelector( String tableName, Schema schema, - List equalityFields, + Set equalityFields, int writeParallelism, int maxWriteParallelism) { return new TargetLimitedKeySelector( @@ -317,7 +316,7 @@ static class SelectorKey { private final Integer specId; private final Schema schema; private final PartitionSpec spec; - private final List equalityFields; + private final Set equalityFields; SelectorKey( String tableName, @@ -326,7 +325,7 @@ static class SelectorKey { @Nullable Integer tableSpecId, Schema schema, PartitionSpec spec, - List equalityFields) { + Set equalityFields) { this.tableName = tableName; this.branch = branch; this.schemaId = tableSchemaId; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index 0a43404d13ad..afd5b637e933 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -20,12 +20,12 @@ import java.io.IOException; import java.io.Serializable; -import java.util.List; import java.util.Objects; +import java.util.Set; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.hadoop.util.Sets; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; class WriteTarget implements Serializable { @@ -34,7 +34,7 @@ class WriteTarget implements Serializable { private final Integer schemaId; private final Integer specId; private final boolean upsertMode; - private final List equalityFields; + private final Set equalityFields; WriteTarget( String tableName, @@ -42,7 +42,7 @@ class WriteTarget implements Serializable { Integer schemaId, Integer specId, boolean upsertMode, - List equalityFields) { + Set equalityFields) { this.tableName = tableName; this.branch = branch != null ? branch : "main"; this.schemaId = schemaId; @@ -71,7 +71,7 @@ boolean upsertMode() { return upsertMode; } - List equalityFields() { + Set equalityFields() { return equalityFields; } @@ -94,12 +94,12 @@ static WriteTarget deserializeFrom(DataInputView view) throws IOException { view.readInt(), view.readInt(), view.readBoolean(), - readList(view)); + readSet(view)); } - private static List readList(DataInputView view) throws IOException { + private static Set readSet(DataInputView view) throws IOException { int numFields = view.readInt(); - List equalityFields = Lists.newArrayList(); + Set equalityFields = Sets.newHashSetWithExpectedSize(numFields); for (int i = 0; i < numFields; i++) { equalityFields.add(view.readInt()); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 7df167ec3227..a21c51c378af 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -34,6 +34,7 @@ import java.time.OffsetDateTime; import java.util.Arrays; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -63,6 +64,7 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; @@ -100,7 +102,7 @@ private int dataFieldId() { } private void testCdcEvents(boolean partitioned) throws IOException { - List equalityFieldIds = Lists.newArrayList(idFieldId()); + Set equalityFieldIds = Sets.newHashSet(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -178,7 +180,7 @@ public void testPartitioned() throws IOException { private void testWritePureEqDeletes(boolean partitioned) throws IOException { createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); + Set equalityFieldIds = Sets.newHashSet(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -207,7 +209,7 @@ public void testPartitionedPureEqDeletes() throws IOException { private void testAbort(boolean partitioned) throws IOException { createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); + Set equalityFieldIds = Sets.newHashSet(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -247,7 +249,7 @@ public void testPartitionedAbort() throws IOException { @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId()); + Set equalityFieldIds = Sets.newHashSet(dataFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -290,7 +292,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); + Set equalityFieldIds = Sets.newHashSet(dataFieldId(), idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -325,7 +327,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { this.table = create(tableSchema, PartitionSpec.unpartitioned()); initTable(table); - List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + Set equalityIds = ImmutableSet.of(table.schema().findField("ts").fieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); taskWriterFactory.initialize(1, 1); @@ -383,7 +385,7 @@ private StructLikeSet actualRowSet(String... columns) throws IOException { return SimpleDataUtil.actualRowSet(table, columns); } - private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { + private TaskWriterFactory createTaskWriterFactory(Set equalityFieldIds) { return new RowDataTaskWriterFactory( SerializableTable.copyOf(table), FlinkSchemaUtil.convert(table.schema()), @@ -395,7 +397,7 @@ private TaskWriterFactory createTaskWriterFactory(List equalit } private TaskWriterFactory createTaskWriterFactory( - RowType flinkType, List equalityFieldIds) { + RowType flinkType, Set equalityFieldIds) { return new RowDataTaskWriterFactory( SerializableTable.copyOf(table), flinkType, diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java index 243b7f90959b..30782e8d4170 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -80,7 +80,7 @@ protected DynamicRecordInternal[] getTestData() { return new DynamicRecordInternal[] { new DynamicRecordInternal( - TABLE, BRANCH, SCHEMA, rowData, SPEC, 42, false, Collections.emptyList()) + TABLE, BRANCH, SCHEMA, rowData, SPEC, 42, false, Collections.emptySet()) }; } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java index f4109a6476aa..13a06d362717 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java @@ -24,7 +24,7 @@ import java.io.IOException; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.api.Test; class TestDynamicCommittableSerializer { @@ -33,7 +33,7 @@ class TestDynamicCommittableSerializer { void testRoundtrip() throws IOException { DynamicCommittable committable = new DynamicCommittable( - new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), new byte[] {3, 4}, JobID.generate().toHexString(), new OperatorID().toHexString(), @@ -48,7 +48,7 @@ void testRoundtrip() throws IOException { void testUnsupportedVersion() throws IOException { DynamicCommittable committable = new DynamicCommittable( - new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), new byte[] {3, 4}, JobID.generate().toHexString(), new OperatorID().toHexString(), diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index d9129d6eacf6..ed1961d8f9b3 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; +import com.google.common.collect.Sets; import java.nio.ByteBuffer; import java.util.Map; import org.apache.flink.api.common.JobID; @@ -41,7 +42,6 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -105,11 +105,11 @@ void testCommit() throws Exception { committerMetrics); WriteTarget writeTarget1 = - new WriteTarget(TABLE1, "branch", 42, 0, true, Lists.newArrayList(1, 2)); + new WriteTarget(TABLE1, "branch", 42, 0, true, Sets.newHashSet(1, 2)); WriteTarget writeTarget2 = - new WriteTarget(TABLE1, "branch2", 43, 0, true, Lists.newArrayList(1, 2)); + new WriteTarget(TABLE1, "branch2", 43, 0, true, Sets.newHashSet(1, 2)); WriteTarget writeTarget3 = - new WriteTarget(TABLE2, "branch2", 43, 0, true, Lists.newArrayList(1, 2)); + new WriteTarget(TABLE2, "branch2", 43, 0, true, Sets.newHashSet(1, 2)); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); @@ -120,21 +120,21 @@ void testCommit() throws Exception { byte[] deltaManifest1 = aggregator.writeToManifest( writeTarget1, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget1, WriteResult.builder().addDataFiles(DATA_FILE).build())), 0); byte[] deltaManifest2 = aggregator.writeToManifest( writeTarget2, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget2, WriteResult.builder().addDataFiles(DATA_FILE).build())), 0); byte[] deltaManifest3 = aggregator.writeToManifest( writeTarget3, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget3, WriteResult.builder().addDataFiles(DATA_FILE).build())), 0); @@ -155,7 +155,7 @@ void testCommit() throws Exception { new MockCommitRequest<>( new DynamicCommittable(writeTarget3, deltaManifest3, jobId, operatorId, checkpointId)); - dynamicCommitter.commit(Lists.newArrayList(commitRequest1, commitRequest2, commitRequest3)); + dynamicCommitter.commit(Sets.newHashSet(commitRequest1, commitRequest2, commitRequest3)); table1.refresh(); assertThat(table1.snapshots()).hasSize(2); @@ -238,7 +238,7 @@ void testAlreadyCommitted() throws Exception { committerMetrics); WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Lists.newArrayList(1, 2)); + new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); @@ -253,7 +253,7 @@ void testAlreadyCommitted() throws Exception { byte[] deltaManifest = aggregator.writeToManifest( writeTarget, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), checkpointId); @@ -262,7 +262,7 @@ void testAlreadyCommitted() throws Exception { new MockCommitRequest<>( new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); - dynamicCommitter.commit(Lists.newArrayList(commitRequest)); + dynamicCommitter.commit(Sets.newHashSet(commitRequest)); CommitRequest oldCommitRequest = new MockCommitRequest<>( @@ -270,7 +270,7 @@ void testAlreadyCommitted() throws Exception { writeTarget, deltaManifest, jobId, operatorId, checkpointId - 1)); // Old commits requests shouldn't affect the result - dynamicCommitter.commit(Lists.newArrayList(oldCommitRequest)); + dynamicCommitter.commit(Sets.newHashSet(oldCommitRequest)); table1.refresh(); assertThat(table1.snapshots()).hasSize(1); @@ -315,7 +315,7 @@ void testReplacePartitions() throws Exception { committerMetrics); WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Lists.newArrayList(1, 2)); + new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader()); @@ -330,7 +330,7 @@ void testReplacePartitions() throws Exception { byte[] deltaManifest = aggregator.writeToManifest( writeTarget, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), checkpointId); @@ -339,12 +339,12 @@ void testReplacePartitions() throws Exception { new MockCommitRequest<>( new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); - dynamicCommitter.commit(Lists.newArrayList(commitRequest)); + dynamicCommitter.commit(Sets.newHashSet(commitRequest)); byte[] overwriteManifest = aggregator.writeToManifest( writeTarget, - Lists.newArrayList( + Sets.newHashSet( new DynamicWriteResult( writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), checkpointId + 1); @@ -354,7 +354,7 @@ void testReplacePartitions() throws Exception { new DynamicCommittable( writeTarget, overwriteManifest, jobId, operatorId, checkpointId + 1)); - dynamicCommitter.commit(Lists.newArrayList(overwriteRequest)); + dynamicCommitter.commit(Sets.newHashSet(overwriteRequest)); table1.refresh(); assertThat(table1.snapshots()).hasSize(2); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index fd5b5ef6be4f..94a2b5a90e72 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -69,7 +69,7 @@ void testDynamicTableUpdateOperatorNewTable() throws Exception { PartitionSpec.unpartitioned(), 42, false, - Collections.emptyList()); + Collections.emptySet()); DynamicRecordInternal output = operator.map(input); assertThat(catalog.tableExists(table)).isTrue(); @@ -98,7 +98,7 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { PartitionSpec.unpartitioned(), 42, false, - Collections.emptyList()); + Collections.emptySet()); DynamicRecordInternal output = operator.map(input); assertThat(catalog.loadTable(table).schema().sameSchema(SCHEMA2)).isTrue(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java index 137b87bb171d..713c67da170a 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java @@ -24,11 +24,11 @@ import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.hadoop.util.Sets; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -49,12 +49,11 @@ void testAggregator() throws Exception { testHarness = new OneInputStreamOperatorTestHarness<>(aggregator)) { testHarness.open(); - WriteTarget writeTarget1 = - new WriteTarget("table", "branch", 42, 0, true, Lists.newArrayList()); + WriteTarget writeTarget1 = new WriteTarget("table", "branch", 42, 0, true, Sets.newHashSet()); DynamicWriteResult dynamicWriteResult1 = new DynamicWriteResult(writeTarget1, WriteResult.builder().build()); WriteTarget writeTarget2 = - new WriteTarget("table2", "branch", 42, 0, true, Lists.newArrayList(1, 2)); + new WriteTarget("table2", "branch", 42, 0, true, Sets.newHashSet(1, 2)); DynamicWriteResult dynamicWriteResult2 = new DynamicWriteResult(writeTarget2, WriteResult.builder().build()); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java index bede5d42b9f4..a3a9691107eb 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java @@ -23,13 +23,13 @@ import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.hadoop.util.Sets; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.Test; class TestDynamicWriteResultSerializer { @@ -53,7 +53,7 @@ class TestDynamicWriteResultSerializer { void testRoundtrip() throws IOException { DynamicWriteResult dynamicWriteResult = new DynamicWriteResult( - new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), WriteResult.builder().addDataFiles(DATA_FILE).build()); DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); @@ -71,7 +71,7 @@ void testRoundtrip() throws IOException { void testUnsupportedVersion() throws IOException { DynamicWriteResult dynamicWriteResult = new DynamicWriteResult( - new WriteTarget("table", "branch", 42, 23, false, Lists.newArrayList(1, 2)), + new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), WriteResult.builder().addDataFiles(DATA_FILE).build()); DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 0a723c9d5700..4aa3f9d67b45 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import com.google.common.collect.Sets; import java.io.File; import java.net.URI; import java.util.Collection; @@ -33,7 +34,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Test; @@ -118,7 +118,7 @@ void testDynamicWriterUpsert() throws Exception { DynamicRecordInternal record = getDynamicRecordInternal(table1); record.setUpsertMode(true); - record.setEqualityFieldIds(Lists.newArrayList(1)); + record.setEqualityFieldIds(Sets.newHashSet(1)); dyamicWriter.write(record, null); dyamicWriter.prepareCommit(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 924b734fd332..b8d86feb99cf 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -22,7 +22,6 @@ import com.github.benmanes.caffeine.cache.Cache; import java.util.Collections; -import java.util.List; import java.util.Set; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; @@ -57,16 +56,16 @@ void testRoundRobinWithDistributionModeNone() throws Exception { GenericRowData row = GenericRowData.of(1, StringData.fromString("z")); int writeKey1 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); int writeKey2 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); int writeKey3 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); int writeKey4 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptyList(), row); + generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); assertThat(writeKey1).isNotEqualTo(writeKey2); assertThat(writeKey3).isEqualTo(writeKey1); @@ -92,36 +91,16 @@ void testBucketingWithDistributionModeHash() throws Exception { int writeKey1 = getWriteKey( - generator, - spec, - DistributionMode.HASH, - writeParallelism, - Collections.emptyList(), - row1); + generator, spec, DistributionMode.HASH, writeParallelism, Collections.emptySet(), row1); int writeKey2 = getWriteKey( - generator, - spec, - DistributionMode.HASH, - writeParallelism, - Collections.emptyList(), - row2); + generator, spec, DistributionMode.HASH, writeParallelism, Collections.emptySet(), row2); int writeKey3 = getWriteKey( - generator, - spec, - DistributionMode.HASH, - writeParallelism, - Collections.emptyList(), - row3); + generator, spec, DistributionMode.HASH, writeParallelism, Collections.emptySet(), row3); int writeKey4 = getWriteKey( - generator, - spec, - DistributionMode.HASH, - writeParallelism, - Collections.emptyList(), - row4); + generator, spec, DistributionMode.HASH, writeParallelism, Collections.emptySet(), row4); assertThat(writeKey1).isEqualTo(writeKey2); assertThat(writeKey3).isNotEqualTo(writeKey1); @@ -143,7 +122,7 @@ void testEqualityKeys() throws Exception { GenericRowData row1 = GenericRowData.of(1, StringData.fromString("foo")); GenericRowData row2 = GenericRowData.of(1, StringData.fromString("bar")); GenericRowData row3 = GenericRowData.of(2, StringData.fromString("baz")); - List equalityColumns = Collections.singletonList("id"); + Set equalityColumns = Collections.singleton("id"); int writeKey1 = getWriteKey( @@ -194,7 +173,7 @@ void testCapAtMaxWriteParallelism() throws Exception { unpartitioned, DistributionMode.NONE, writeParallelism, - Collections.emptyList(), + Collections.emptySet(), row)); } @@ -292,7 +271,7 @@ void testMultipleTables() throws Exception { unpartitioned, DistributionMode.HASH, writeParallelism); - record1.setEqualityFields(Collections.singletonList("id")); + record1.setEqualityFields(Collections.singleton("id")); DynamicRecord record2 = new DynamicRecord( TableIdentifier.of("my", "other", "table"), @@ -302,7 +281,7 @@ void testMultipleTables() throws Exception { unpartitioned, DistributionMode.HASH, writeParallelism); - record2.setEqualityFields(Collections.singletonList("id")); + record2.setEqualityFields(Collections.singleton("id")); // Consistent hashing for the same record due to HASH distribution mode int writeKeyRecord1 = generator.generateKey(record1); @@ -359,7 +338,7 @@ private static int getWriteKey( PartitionSpec spec, DistributionMode mode, int writeParallelism, - List equalityFields, + Set equalityFields, GenericRowData row) throws Exception { DynamicRecord record = diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index ce9054ad49b6..ef8380c21613 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -24,6 +24,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; +import java.util.Set; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; @@ -46,6 +47,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -81,7 +83,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { SimpleDataUtil.createInsert(1, "AAA"), SimpleDataUtil.createInsert(2, "BBB"), SimpleDataUtil.createInsert(3, "CCC")); - writeAndCommit(table, ImmutableList.of(), false, rows); + writeAndCommit(table, ImmutableSet.of(), false, rows); FlinkInputFormat input = FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); @@ -124,7 +126,7 @@ public void testV2RemoveMetaColumn() throws Exception { SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")); int eqFieldId = table.schema().findField("data").fieldId(); - writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); + writeAndCommit(table, ImmutableSet.of(eqFieldId), true, rows); FlinkInputFormat input = FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); @@ -147,8 +149,7 @@ public void testV2RemoveMetaColumn() throws Exception { } private void writeAndCommit( - Table table, List eqFieldIds, boolean upsert, List rows) - throws IOException { + Table table, Set eqFieldIds, boolean upsert, List rows) throws IOException { TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); try (TaskWriter io = writer) { for (RowData row : rows) { @@ -171,7 +172,7 @@ private void writeAndCommit( } private TaskWriter createTaskWriter( - Table table, List equalityFieldIds, boolean upsert) { + Table table, Set equalityFieldIds, boolean upsert) { TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory( SerializableTable.copyOf(table), From 3d4ccd7b435fb632499f73a34f1f099ea8783d3c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 12:59:15 +0200 Subject: [PATCH 09/14] newline --- .../org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java index 10244dc4bbd4..fe670c54ebd2 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataEvolver.java @@ -165,6 +165,7 @@ private static ArrayData convertArray( convertedArray[i] = convert(elementGetter.getElementOrNull(sourceData, i), fromElementType, toElementType); } + return new GenericArrayData(convertedArray); } From 78fe6828710bedd9168f92eb7b669f247e959e6b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 Jun 2025 13:04:56 +0200 Subject: [PATCH 10/14] verify duplicate updates --- .../flink/sink/dynamic/TestDynamicTableUpdateOperator.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index 94a2b5a90e72..aa483b3e5338 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -103,5 +103,10 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { assertThat(catalog.loadTable(table).schema().sameSchema(SCHEMA2)).isTrue(); assertThat(input).isEqualTo(output); + + // Process the same input again + DynamicRecordInternal output2 = operator.map(input); + assertThat(output2).isEqualTo(output); + assertThat(catalog.loadTable(table).schema().schemaId()).isEqualTo(output.schema().schemaId()); } } From 05fd580e91d5ee9ceefa1de00a4d5653e19186d2 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 11 Jun 2025 10:05:05 +0200 Subject: [PATCH 11/14] Revert public API change --- .../apache/iceberg/flink/sink/BaseDeltaTaskWriter.java | 4 ++-- .../iceberg/flink/sink/PartitionedDeltaWriter.java | 4 ++-- .../iceberg/flink/sink/RowDataTaskWriterFactory.java | 10 +++++----- .../iceberg/flink/sink/UnpartitionedDeltaWriter.java | 4 ++-- .../iceberg/flink/sink/dynamic/DynamicWriter.java | 2 +- .../flink/sink/dynamic/TestDynamicCommitter.java | 2 +- .../iceberg/flink/sink/dynamic/TestDynamicWriter.java | 2 +- 7 files changed, 14 insertions(+), 14 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index d845046cd2f6..0059ff197628 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.Set; +import java.util.Collection; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -56,7 +56,7 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.schema = schema; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java index 3eb4dba80281..141979025711 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Collection; import java.util.Map; -import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -49,7 +49,7 @@ class PartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert) { super( spec, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 7e8db13d2803..d213c6d1b007 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Collection; import java.util.Map; -import java.util.Set; import java.util.function.Supplier; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -48,7 +48,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final PartitionSpec spec; private final long targetFileSizeBytes; private final FileFormat format; - private final Set equalityFieldIds; + private final Collection equalityFieldIds; private final boolean upsert; private final FileAppenderFactory appenderFactory; @@ -60,7 +60,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert) { this( () -> table, @@ -78,7 +78,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert) { this( tableSupplier, @@ -98,7 +98,7 @@ public RowDataTaskWriterFactory( long targetFileSizeBytes, FileFormat format, Map writeProperties, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert, Schema schema, PartitionSpec spec) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java index b6ad03514bb0..c0b30de3db3e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.Set; +import java.util.Collection; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -41,7 +41,7 @@ class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Set equalityFieldIds, + Collection equalityFieldIds, boolean upsert) { super( spec, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index f6ea3c1a393d..c4c4e61de151 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -138,7 +138,7 @@ public void write(DynamicRecordInternal element, Context context) targetDataFileSize, dataFileFormat, tableWriteProperties, - equalityFieldIds, + Lists.newArrayList(equalityFieldIds), element.upsertMode(), element.schema(), element.spec()); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index ed1961d8f9b3..99a546536208 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import com.google.common.collect.Sets; import java.nio.ByteBuffer; import java.util.Map; import org.apache.flink.api.common.JobID; @@ -43,6 +42,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 4aa3f9d67b45..0c223f32370d 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; -import com.google.common.collect.Sets; import java.io.File; import java.net.URI; import java.util.Collection; @@ -34,6 +33,7 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Test; From f9d2b67c4e29ffcdedadd1578f15d3532fb094b8 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 11 Jun 2025 10:09:35 +0200 Subject: [PATCH 12/14] Fix logging --- .../apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index fb565fc28f48..d7e281334b27 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -120,7 +120,7 @@ private KeySelector getKeySelector( return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); } else { LOG.info( - "{}Distribute rows by equality fields, because there are equality fields set", + "{}: Distribute rows by equality fields, because there are equality fields set", tableName); return equalityFieldKeySelector( tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); @@ -167,14 +167,14 @@ private KeySelector getKeySelector( 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", + + "and {}='range' is not supported yet in flink", tableName, 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", + + "and {}='range' is not supported yet in flink", tableName, WRITE_DISTRIBUTION_MODE); return equalityFieldKeySelector( From bcbaa71548dba5e50450c3dd781824cbb63d9273 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 11 Jun 2025 10:09:42 +0200 Subject: [PATCH 13/14] Log table name in TargetLimitedKeySelector --- .../flink/sink/dynamic/HashKeyGenerator.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index d7e281334b27..6cb1f4608983 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -198,7 +198,7 @@ private static KeySelector equalityFieldKeySelector( schema, FlinkSchemaUtil.convert(schema), DynamicSinkUtil.getEqualityFieldIds(equalityFields, schema)), - tableName.hashCode(), + tableName, writeParallelism, maxWriteParallelism); } @@ -212,17 +212,14 @@ private static KeySelector partitionKeySelector( KeySelector inner = new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); return new TargetLimitedKeySelector( - in -> inner.getKey(in).hashCode(), - tableName.hashCode(), - writeParallelism, - maxWriteParallelism); + in -> inner.getKey(in).hashCode(), tableName, writeParallelism, maxWriteParallelism); } private static KeySelector tableKeySelector( String tableName, int writeParallelism, int maxWriteParallelism) { return new TargetLimitedKeySelector( new RoundRobinKeySelector<>(writeParallelism), - tableName.hashCode(), + tableName, writeParallelism, maxWriteParallelism); } @@ -239,12 +236,13 @@ private static class TargetLimitedKeySelector implements KeySelector wrapped, - int salt, + String tableName, int writeParallelism, int maxWriteParallelism) { if (writeParallelism > maxWriteParallelism) { LOG.warn( - "writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", + "{}: writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", + tableName, writeParallelism, maxWriteParallelism, maxWriteParallelism); @@ -256,7 +254,7 @@ private static class TargetLimitedKeySelector implements KeySelector targetSlots = Sets.newHashSetWithExpectedSize(writeParallelism); - int nextKey = salt; + int nextKey = tableName.hashCode(); for (int i = 0; i < writeParallelism; ++i) { int subtaskId = subtaskId(nextKey, writeParallelism, maxWriteParallelism); while (targetSlots.contains(subtaskId)) { From 303d3b4c16bd736973f2ff602d4422fc68806aab Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 11 Jun 2025 16:32:05 +0200 Subject: [PATCH 14/14] Use Set for equalityFields in BaseDeltaTaskWriter / PartitionedDeltaWriter --- .../org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java | 4 ++-- .../apache/iceberg/flink/sink/PartitionedDeltaWriter.java | 4 ++-- .../apache/iceberg/flink/sink/RowDataTaskWriterFactory.java | 5 +++-- .../apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java | 4 ++-- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index 0059ff197628..d845046cd2f6 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.Collection; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -56,7 +56,7 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Collection equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.schema = schema; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java index 141979025711..3eb4dba80281 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Collection; import java.util.Map; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -49,7 +49,7 @@ class PartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Collection equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super( spec, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index d213c6d1b007..7c11b20c449d 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -48,7 +49,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final PartitionSpec spec; private final long targetFileSizeBytes; private final FileFormat format; - private final Collection equalityFieldIds; + private final Set equalityFieldIds; private final boolean upsert; private final FileAppenderFactory appenderFactory; @@ -117,7 +118,7 @@ public RowDataTaskWriterFactory( this.spec = spec; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; - this.equalityFieldIds = equalityFieldIds; + this.equalityFieldIds = equalityFieldIds != null ? Sets.newHashSet(equalityFieldIds) : null; this.upsert = upsert; if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java index c0b30de3db3e..b6ad03514bb0 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; -import java.util.Collection; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; @@ -41,7 +41,7 @@ class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { long targetFileSize, Schema schema, RowType flinkSchema, - Collection equalityFieldIds, + Set equalityFieldIds, boolean upsert) { super( spec,