From 7bb0fa243baa0834fcc3fb6ec71524864a7a501f Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 21 May 2026 15:10:39 -0700 Subject: [PATCH 01/58] Docs: Drop manual deploy step from release instructions (#16495) * Docs: Drop manual deploy step from release instructions The site-ci GitHub Actions workflow runs make deploy automatically on any push to main that touches docs/, site/, or format/. Replace the manual "Release versioned docs and javadoc" step in the release guide with a note describing the auto-trigger so release managers don't push to asf-site by hand. Co-authored-by: Claude Opus 4.7 (1M context) Co-authored-by: Kevin Liu Co-authored-by: Kevin Liu --- site/README.md | 18 ++---------------- site/docs/how-to-release.md | 11 +++++++---- 2 files changed, 9 insertions(+), 20 deletions(-) diff --git a/site/README.md b/site/README.md index e14bb361d512..ee702f030902 100644 --- a/site/README.md +++ b/site/README.md @@ -174,22 +174,8 @@ make build OFFLINE=true ## Release process -Deploying the docs is a two-step process: - -> [!WARNING] -> The `make release` directive is currently unavailable as we wanted to discuss the best way forward on how or if we should automate the release. It involves taking an existing snapshot of the versioned documentation, and potentially automerging the [`docs` branch](https://github.com/apache/iceberg/tree/docs) and the [`javadoc` branch](https://github.com/apache/iceberg/tree/javadoc) which are independent from the `main` branch. Once this is complete, we can create a pull request with an offline build of the documentation to verify everything renders correctly, and then have the release manager merge that PR to finalize the docs release. So the real process would be manually invoking a docs release action, then merging a pull request. - - 1. Release a new version by copying the current `/docs` directory to a new version directory in the `docs` branch and a new javadoc build in the `javadoc` branch. - ```sh - make release ICEBERG_VERSION=${ICEBERG_VERSION} - ``` - 1. Build and push the generated site to the `asf-site` branch of [remote repo](https://github.com/apache/iceberg). This requires committer write permission. - ```sh - # Default remote name is 'origin' - make deploy - # Or specify a different remote - make deploy remote_name=apache - ``` +For release documentation publishing steps, follow the +[Documentation Release](docs/how-to-release.md#documentation-release) section of the release guide. ## Validate Links diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index cb50cce622b2..eca2757c764e 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -356,15 +356,18 @@ cp -R apache-iceberg-1.8.0/site/docs/javadoc/1.8.0 1.8.0 Once this is done, create a PR against the `javadoc` branch, similar to https://github.com/apache/iceberg/pull/12412. -#### Release versioned docs and javadoc - -Please follow the instructions on the GitHub repository in the [`README.md` in the `site/`](https://github.com/apache/iceberg/tree/main/site) directory. - #### Site update Submit a PR, following the approach in https://github.com/apache/iceberg/pull/12242, to update the Iceberg version, the links to the new version's documentation, and the release notes. +Once this PR is merged, the [`site-ci`](https://github.com/apache/iceberg/blob/main/.github/workflows/site-ci.yml) +GitHub Actions workflow is automatically triggered for any push to `main` that touches `docs/`, `site/`, +or `format/`. The workflow runs `make deploy` to build and push the documentation site to the `asf-site` +branch, which publishes the new version's docs and javadoc. + +The site will be updated automatically. Manually running `make deploy` is no longer required as part of the release. + # How to Verify a Release Each Apache Iceberg release is validated by the community by holding a vote. A community release manager From 10ba4eebf1b870627fb20b9a75413e8088cd975f Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Fri, 22 May 2026 12:38:18 +0800 Subject: [PATCH 02/58] Flink: Support writing shredded variant (#15596) --- docs/docs/flink-configuration.md | 2 + .../apache/iceberg/flink/FlinkWriteConf.java | 18 + .../iceberg/flink/FlinkWriteOptions.java | 6 + .../iceberg/flink/data/FlinkFormatModels.java | 8 +- .../data/FlinkVariantShreddingAnalyzer.java | 72 ++ .../apache/iceberg/flink/sink/SinkUtil.java | 6 + .../flink/TestFlinkVariantShreddingType.java | 1008 +++++++++++++++++ .../iceberg/parquet/ParquetFormatModel.java | 42 +- .../parquet/TestParquetDataWriter.java | 8 +- .../spark/source/SparkFormatModels.java | 4 +- .../spark/source/SparkFormatModels.java | 4 +- 11 files changed, 1164 insertions(+), 14 deletions(-) create mode 100644 flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkVariantShreddingAnalyzer.java create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkVariantShreddingType.java diff --git a/docs/docs/flink-configuration.md b/docs/docs/flink-configuration.md index f30b42288896..2f70cbf576d8 100644 --- a/docs/docs/flink-configuration.md +++ b/docs/docs/flink-configuration.md @@ -160,6 +160,8 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ | compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | | write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | | uid-suffix | As per table property | Overrides the uid suffix used in the underlying IcebergSink for this table | +| shred-variants | Table write.parquet.shred-variants | Overrides this table's shred variants for this write | +| variant-inference-buffer-size | Table write.parquet.variant-inference-buffer-size | Overrides this table's variant inference buffer size for this write | #### Range distribution statistics type diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index 990d23f2aaff..fd3fccb224a2 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -262,4 +262,22 @@ public Duration tableRefreshInterval() { .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) .parseOptional(); } + + public boolean parquetShredVariants() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.SHRED_VARIANTS.key()) + .tableProperty(TableProperties.PARQUET_SHRED_VARIANTS) + .defaultValue(TableProperties.PARQUET_SHRED_VARIANTS_DEFAULT) + .parse(); + } + + public int parquetVariantInferenceBufferSize() { + return confParser + .intConf() + .option(FlinkWriteOptions.VARIANT_INFERENCE_BUFFER_SIZE.key()) + .tableProperty(TableProperties.PARQUET_VARIANT_BUFFER_SIZE) + .defaultValue(TableProperties.PARQUET_VARIANT_BUFFER_SIZE_DEFAULT) + .parse(); + } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index ee2aeaa45007..1fdd6df8d753 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -105,4 +105,10 @@ private FlinkWriteOptions() {} // specify the uidSuffix to be used for the underlying IcebergSink public static final ConfigOption UID_SUFFIX = ConfigOptions.key("uid-suffix").stringType().defaultValue(""); + + public static final ConfigOption SHRED_VARIANTS = + ConfigOptions.key("shred-variants").booleanType().defaultValue(false); + + public static final ConfigOption VARIANT_INFERENCE_BUFFER_SIZE = + ConfigOptions.key("variant-inference-buffer-size").intType().noDefaultValue(); } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java index dd713b0dce2a..747a4618682c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java @@ -18,7 +18,10 @@ */ package org.apache.iceberg.flink.data; +import java.util.function.Function; +import java.util.function.UnaryOperator; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.avro.AvroFormatModel; import org.apache.iceberg.formats.FormatModelRegistry; @@ -33,7 +36,10 @@ public static void register() { RowType.class, FlinkParquetWriters::buildWriter, (icebergSchema, fileSchema, engineSchema, idToConstant) -> - FlinkParquetReaders.buildReader(icebergSchema, fileSchema, idToConstant))); + FlinkParquetReaders.buildReader(icebergSchema, fileSchema, idToConstant), + new FlinkVariantShreddingAnalyzer(), + (Function>) + rowType -> new RowDataSerializer(rowType)::copy)); FormatModelRegistry.register( AvroFormatModel.create( diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkVariantShreddingAnalyzer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkVariantShreddingAnalyzer.java new file mode 100644 index 000000000000..cfb4d9a55680 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkVariantShreddingAnalyzer.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.data; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.variant.BinaryVariant; +import org.apache.flink.types.variant.Variant; +import org.apache.iceberg.parquet.VariantShreddingAnalyzer; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantValue; + +/** + * Analyzes Variant fields in Flink {@link RowData} and converts Flink's binary Variant + * representation to Iceberg {@link VariantValue} instances for Variant shredding. + */ +public class FlinkVariantShreddingAnalyzer extends VariantShreddingAnalyzer { + + @Override + protected List extractVariantValues( + List bufferedRows, int variantFieldIndex) { + List values = Lists.newArrayList(); + + for (RowData row : bufferedRows) { + if (!row.isNullAt(variantFieldIndex)) { + Variant flinkVariant = row.getVariant(variantFieldIndex); + if (flinkVariant != null) { + if (flinkVariant instanceof BinaryVariant binaryVariant) { + VariantValue variantValue = + VariantValue.from( + VariantMetadata.from( + ByteBuffer.wrap(binaryVariant.getMetadata()) + .order(ByteOrder.LITTLE_ENDIAN)), + ByteBuffer.wrap(binaryVariant.getValue()).order(ByteOrder.LITTLE_ENDIAN)); + + values.add(variantValue); + } else { + throw new UnsupportedOperationException( + "Not a supported type: " + flinkVariant.getClass()); + } + } + } + } + + return values; + } + + @Override + protected int resolveColumnIndex(RowType flinkSchema, String columnName) { + return flinkSchema.getFieldIndex(columnName); + } +} diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java index b3a9ac6ba2eb..d4c3d3beb80f 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -24,6 +24,8 @@ import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.PARQUET_SHRED_VARIANTS; +import static org.apache.iceberg.TableProperties.PARQUET_VARIANT_BUFFER_SIZE; import java.util.List; import java.util.Map; @@ -128,6 +130,10 @@ public static Map writeProperties( writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); } + writeProperties.put(PARQUET_SHRED_VARIANTS, String.valueOf(conf.parquetShredVariants())); + writeProperties.put( + PARQUET_VARIANT_BUFFER_SIZE, String.valueOf(conf.parquetVariantInferenceBufferSize())); + break; case AVRO: writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkVariantShreddingType.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkVariantShreddingType.java new file mode 100644 index 000000000000..e809a47cd262 --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkVariantShreddingType.java @@ -0,0 +1,1008 @@ +/* + * 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; + +import static org.apache.parquet.schema.Types.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.variants.Variant; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +class TestFlinkVariantShreddingType extends CatalogTestBase { + + private static final String TABLE_NAME = "test_table"; + private Table icebergTable; + + @Parameters(name = "catalogName={0}, baseNamespace={1}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + parameters.add(new Object[] {"testhadoop", Namespace.empty()}); + parameters.add(new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); + return parameters; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + sql( + """ + CREATE TABLE %s ( + id int NOT NULL, + address variant NOT NULL + ) WITH ( + 'write.format.default'='%s', + 'format-version'='3', + 'shred-variants'='true', + 'variant-inference-buffer-size'='10' + ) + """, + TABLE_NAME, FileFormat.PARQUET.name()); + icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + } + + @Override + @AfterEach + public void clean() { + super.clean(); + getTableEnv() + .getConfig() + .getConfiguration() + .setString("table.exec.resource.default-parallelism", "4"); + } + + @TestTemplate + public void testExcludingNullValue() throws IOException { + String values = + """ + (1, parse_json('{"name": "Alice", "age": 30, "dummy": null}')), + (2, parse_json('{"name": "Bob", "age": 25}')), + (3, parse_json('{"name": "Charlie", "age": 35}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType age = + field( + "age", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(age, name)); + MessageType expectedSchema = parquetSchema(address); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testConsistentType() throws IOException { + String values = + """ + (1, parse_json('{"age": "25"}')), + (2, parse_json('{"age": 30}')), + (3, parse_json('{"age": "35"}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType age = + field( + "age", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(age)); + MessageType expectedSchema = parquetSchema(address); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testPrimitiveType() throws IOException { + String values = + """ + (1, parse_json('123')), + (2, parse_json('"abc"')), + (3, parse_json('12')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType address = + variant( + "address", + 2, + Type.Repetition.REQUIRED, + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + MessageType expectedSchema = parquetSchema(address); + + assertThat(SimpleDataUtil.tableRecords(icebergTable)).hasSize(3); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testPrimitiveDecimalType() throws IOException { + String values = + """ + (1, parse_json('123.56')), + (2, parse_json('"abc"')), + (3, parse_json('12.56')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType address = + variant( + "address", + 2, + Type.Repetition.REQUIRED, + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.decimalType(2, 5))); + MessageType expectedSchema = parquetSchema(address); + assertThat(SimpleDataUtil.tableRecords(icebergTable)).hasSize(3); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testBooleanType() throws IOException { + String values = + """ + (1, parse_json('{"active": true}')), + (2, parse_json('{"active": false}')), + (3, parse_json('{"active": true}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType active = field("active", shreddedPrimitive(PrimitiveType.PrimitiveTypeName.BOOLEAN)); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(active)); + MessageType expectedSchema = parquetSchema(address); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testDecimalTypeWithInconsistentScales() throws IOException { + String values = + """ + (1, parse_json('{"price": 123.456789}')), + (2, parse_json('{"price": 678.90}')), + (3, parse_json('{"price": 999.99}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType price = + field( + "price", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.decimalType(6, 9))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(price)); + MessageType expectedSchema = parquetSchema(address); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testDecimalTypeWithConsistentScales() throws IOException { + String values = + """ + (1, parse_json('{"price": 123.45}')), + (2, parse_json('{"price": 678.90}')), + (3, parse_json('{"price": 999.99}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType price = + field( + "price", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.decimalType(2, 5))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(price)); + MessageType expectedSchema = parquetSchema(address); + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testArrayType() throws IOException { + String values = + """ + (1, parse_json('["java", "scala", "python"]')), + (2, parse_json('["rust", "go"]')), + (3, parse_json('["javascript"]')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType arr = + list( + element( + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType()))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, arr); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testNestedArrayType() throws IOException { + + String values = + """ + (1, parse_json('{"tags": ["java", "scala", "python"]}')), + (2, parse_json('{"tags": ["rust", "go"]}')), + (3, parse_json('{"tags": ["javascript"]}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType tags = + field( + "tags", + list( + element( + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, + LogicalTypeAnnotation.stringType())))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(tags)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testNestedObjectType() throws IOException { + + String values = + """ + (1, parse_json('{"location": {"city": "Seattle", "zip": 98101}, "tags": ["java", "scala", "python"]}')), + (2, parse_json('{"location": {"city": "Portland", "zip": 97201}}')), + (3, parse_json('{"location": {"city": "NYC", "zip": 10001}}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType city = + field( + "city", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType zip = + field( + "zip", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(32, true))); + GroupType location = field("location", objectFields(city, zip)); + GroupType tags = + field( + "tags", + list( + element( + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, + LogicalTypeAnnotation.stringType())))); + + GroupType address = + variant("address", 2, Type.Repetition.REQUIRED, objectFields(location, tags)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testLazyInitializationWithBufferedRows() throws IOException { + + String values = + """ + (1, parse_json('{"name": "Alice", "age": 30}')), + (2, parse_json('{"name": "Bob", "age": 25}')), + (3, parse_json('{"name": "Charlie", "age": 35}')), + (4, parse_json('{"name": "David", "age": 28}')), + (5, parse_json('{"name": "Eve", "age": 32}')), + (6, parse_json('{"name": "Frank", "age": 40}')), + (7, parse_json('{"name": "Grace", "age": 27}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType age = + field( + "age", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(age, name)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + assertThat(genericRowData()).hasSize(7); + } + + @TestTemplate + public void testColumnIndexTruncateLength() throws IOException { + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME); + + int customTruncateLength = 10; + sql( + "ALTER TABLE %s SET ('%s'='%d')", + TABLE_NAME, "parquet.columnindex.truncate.length", customTruncateLength); + + StringBuilder valuesBuilder = new StringBuilder(); + for (int i = 1; i <= 10; i++) { + if (i > 1) { + valuesBuilder.append(", "); + } + + String longValue = "A".repeat(20); + valuesBuilder.append( + String.format( + """ + (%d, parse_json('{"description": "%s", "id": %d}')) + """ + .trim(), + i, + longValue, + i)); + } + sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString()); + + GroupType description = + field( + "description", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType id = + field( + "id", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + GroupType address = + variant("address", 2, Type.Repetition.REQUIRED, objectFields(description, id)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + assertThat(genericRowData()).hasSize(10); + } + + @TestTemplate + public void testIntegerFamilyPromotion() throws IOException { + + // Mix of INT8, INT16, INT32, INT64 - should promote to INT64 + String values = + """ + (1, parse_json('{"value": 10}')), + (2, parse_json('{"value": 1000}')), + (3, parse_json('{"value": 100000}')), + (4, parse_json('{"value": 10000000000}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType value = + field( + "value", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, LogicalTypeAnnotation.intType(64, true))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(value)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testDecimalFamilyPromotion() throws IOException { + + // Test that they get promoted to the most capable decimal type observed + String values = + """ + (1, parse_json('{"value": 1.5}')), + (2, parse_json('{"value": 123.456789}')), + (3, parse_json('{"value": 123456789123456.789}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType value = + field( + "value", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + 16, + LogicalTypeAnnotation.decimalType(6, 21))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(value)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testDataRoundTripWithShredding() throws IOException { + String values = + """ + (1, parse_json('{"name": "Alice", "age": 30}')), + (2, parse_json('{"name": "Bob", "age": 25}')), + (3, parse_json('{"name": "Charlie", "age": 35}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType age = + field( + "age", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(age, name)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + + // Verify that we can read the data back correctly + List rows = + sql( + """ + SELECT id, + JSON_VALUE(address, '$.name'), + JSON_VALUE(address, '$.age' RETURNING int) + FROM %s + ORDER BY id + """, + TABLE_NAME); + assertThat(rows).hasSize(3); + assertThat(rows.get(0).getField(0)).isEqualTo(1); + assertThat(rows.get(0).getField(1)).isEqualTo("Alice"); + assertThat(rows.get(0).getField(2)).isEqualTo(30); + assertThat(rows.get(1).getField(0)).isEqualTo(2); + assertThat(rows.get(1).getField(1)).isEqualTo("Bob"); + assertThat(rows.get(1).getField(2)).isEqualTo(25); + assertThat(rows.get(2).getField(0)).isEqualTo(3); + assertThat(rows.get(2).getField(1)).isEqualTo("Charlie"); + assertThat(rows.get(2).getField(2)).isEqualTo(35); + } + + @TestTemplate + public void testVariantWithNullValues() throws IOException { + + String values = + """ + (1, parse_json('null')), + (2, parse_json('null')), + (3, parse_json('null')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, values); + + GroupType address = variant("address", 2, Type.Repetition.REQUIRED); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testArrayOfNullElementsWithShredding() throws IOException { + + sql( + """ + INSERT INTO %s VALUES + (1, parse_json('[null, null, null]')), + (2, parse_json('[null]')) + """, + TABLE_NAME); + + // Array elements are all null, element type is null, falls back to unshredded + GroupType address = variant("address", 2, Type.Repetition.REQUIRED); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testInfrequentFieldPruning() throws IOException { + // This test relies on the current VariantShreddingAnalyzer MIN_FIELD_FREQUENCY threshold of + // 0.10: rare_field appears in 1/11 rows (~0.09), so it should be pruned. + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='11')", TABLE_NAME); + StringBuilder valuesBuilder = new StringBuilder(); + for (int i = 1; i <= 11; i++) { + if (i > 1) { + valuesBuilder.append(", "); + } + + if (i == 1) { + // Only the first row has rare_field + valuesBuilder.append( + String.format( + """ + (%d, parse_json('{"name": "User%d", "rare_field": "rare"}')) + """ + .trim(), + i, + i)); + } else { + valuesBuilder.append( + String.format( + """ + (%d, parse_json('{"name": "User%d"}')) + """ + .trim(), + i, + i)); + } + } + + sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString()); + + // rare_field appears in 1/11 rows, should be pruned + // name appears in 11/11 rows and should be kept + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(name)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testMixedTypeTieBreaking() throws IOException { + StringBuilder valuesBuilder = new StringBuilder(); + for (int i = 1; i <= 10; i++) { + if (i > 1) { + valuesBuilder.append(", "); + } + + if (i <= 5) { + valuesBuilder.append( + String.format( + """ + (%d, parse_json('{"val": %d}')) + """ + .trim(), + i, + i)); + } else { + valuesBuilder.append( + String.format( + """ + (%d, parse_json('{"val": "text%d"}')) + """ + .trim(), + i, + i)); + } + } + + sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString()); + + // 5 ints + 5 strings is a tie so STRING wins (higher TIE_BREAK_PRIORITY) + GroupType val = + field( + "val", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(val)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + } + + @TestTemplate + public void testFieldOnlyAfterBuffer() throws IOException { + getTableEnv() + .getConfig() + .getConfiguration() + .setString("table.exec.resource.default-parallelism", "1"); + + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME); + + sql( + """ + CREATE TEMPORARY VIEW tmp_source AS + SELECT * FROM (VALUES + (1, parse_json('{"name": "Alice"}')), + (2, parse_json('{"name": "Bob"}')), + (3, parse_json('{"name": "Charlie"}')), + (4, parse_json('{"name": "David", "score": 95}')), + (5, parse_json('{"name": "Eve", "score": 88}')), + (6, parse_json('{"name": "Frank", "score": 72}')), + (7, parse_json('{"name": "Grace", "score": 91}')) + ) AS t(id, address) + """); + + sql("INSERT INTO %s SELECT id, address FROM tmp_source ORDER BY id", TABLE_NAME); + + // Schema is determined from buffer (rows 1-3) which only has "name". + // "score" is not shredded + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(name)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + + // Verify all data round-trips despite "score" not being shredded + List rows = + sql( + """ + SELECT id, + JSON_VALUE(address, '$.name'), + JSON_VALUE(address, '$.score' RETURNING int) + FROM %s + ORDER BY id + """, + TABLE_NAME); + assertThat(rows).hasSize(7); + assertThat(rows.get(0).getField(1)).isEqualTo("Alice"); + assertThat(rows.get(0).getField(2)).isNull(); + assertThat(rows.get(3).getField(1)).isEqualTo("David"); + assertThat(rows.get(3).getField(2)).isEqualTo(95); + assertThat(rows.get(6).getField(1)).isEqualTo("Grace"); + assertThat(rows.get(6).getField(2)).isEqualTo(91); + + sql("DROP TEMPORARY VIEW IF EXISTS tmp_source"); + } + + @TestTemplate + public void testCrossFileDifferentShreddedType() throws IOException { + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME); + + // File 1: "score" is always integer → shredded as INT8 + String batch1 = + """ + (1, parse_json('{"score": 95}')), + (2, parse_json('{"score": 88}')), + (3, parse_json('{"score": 72}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, batch1); + + // Verify file 1 schema: score shredded as INT8 + GroupType scoreInt = + field( + "score", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + MessageType expectedSchema1 = + parquetSchema(variant("address", 2, Type.Repetition.REQUIRED, objectFields(scoreInt))); + verifyParquetSchema(icebergTable, expectedSchema1); + + // File 2: "score" is always string → shredded as STRING + String batch2 = + """ + (4, parse_json('{"score": "high"}')), + (5, parse_json('{"score": "medium"}')), + (6, parse_json('{"score": "low"}')) + """; + sql("INSERT INTO %s VALUES %s", TABLE_NAME, batch2); + + // Query across both files, reader must handle different shredded types + List rows = + sql( + """ + SELECT id, + json_value(address, '$.score') + FROM %s + ORDER BY id + """, + TABLE_NAME); + assertThat(rows).hasSize(6); + assertThat(rows.get(0).getField(1)).isEqualTo("95"); + assertThat(rows.get(1).getField(1)).isEqualTo("88"); + assertThat(rows.get(3).getField(1)).isEqualTo("high"); + assertThat(rows.get(5).getField(1)).isEqualTo("low"); + } + + @TestTemplate + public void testAllNullVariantColumn() throws IOException { + + String variantNullAbleTableName = "test_all_null_variant_column"; + sql( + """ + CREATE TABLE %s ( + id int NOT NULL, + address variant + ) WITH ( + 'write.format.default'='%s', + 'format-version'='3', + 'shred-variants'='true', + 'variant-inference-buffer-size'='10' + ) + """, + variantNullAbleTableName, FileFormat.PARQUET.name()); + + sql( + """ + INSERT INTO %s VALUES + (1, CAST(null AS VARIANT)), + (2, CAST(null AS VARIANT)), + (3, CAST(null AS VARIANT)) + """, + variantNullAbleTableName); + + // All variant values are SQL NULL, so no shredding should occur + MessageType expectedSchema = parquetSchema(variant("address", 2, Type.Repetition.OPTIONAL)); + Table variantNullAbleTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, variantNullAbleTableName)); + verifyParquetSchema(variantNullAbleTable, expectedSchema); + + List rows = sql("SELECT id, address FROM %s ORDER BY id", variantNullAbleTableName); + assertThat(rows).hasSize(3); + assertThat(rows.get(0).getField(1)).isNull(); + assertThat(rows.get(1).getField(1)).isNull(); + assertThat(rows.get(2).getField(1)).isNull(); + } + + @TestTemplate + public void testBufferSizeOne() throws IOException { + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='1')", TABLE_NAME); + + sql( + """ + INSERT INTO %s VALUES + (1, parse_json('{"name": "Alice", "age": 30}')), + (2, parse_json('{"name": "Bob", "age": 25}')), + (3, parse_json('{"name": "Charlie", "age": 35}')) + """, + TABLE_NAME); + + // Schema inferred from first row only, should still shred name and age + GroupType age = + field( + "age", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8, true))); + GroupType name = + field( + "name", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, LogicalTypeAnnotation.stringType())); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(age, name)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + + List rows = + sql( + """ + SELECT id, + json_value(address, '$.name') + FROM %s + ORDER BY id + """, + TABLE_NAME); + assertThat(rows).hasSize(3); + assertThat(rows.get(0).getField(1)).isEqualTo("Alice"); + assertThat(rows.get(2).getField(1)).isEqualTo("Charlie"); + } + + @TestTemplate + public void testDecimalFallbackAfterBuffer() throws IOException { + getTableEnv() + .getConfig() + .getConfiguration() + .setString("table.exec.resource.default-parallelism", "1"); + + sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME); + + // Buffer: scale=2, 3 integer digits -> DECIMAL(5,2) + // Row 4: precision overflow -> fallback to value field + // Row 5: scale overflow -> fallback to value field + // Row 6: fits typed column, scale widened from 1 to 2 via setScale + sql( + """ + CREATE TEMPORARY VIEW tmp_source AS + SELECT * FROM (VALUES + (1, parse_json('{"val": 123.45}')), + (2, parse_json('{"val": 678.90}')), + (3, parse_json('{"val": 999.99}')), + (4, parse_json('{"val": 123456.78}')), + (5, parse_json('{"val": 1.2345}')), + (6, parse_json('{"val": 12.3}')) + ) AS t(id, address) + """); + + sql("INSERT INTO %s SELECT id, address FROM tmp_source ORDER BY id", TABLE_NAME); + + GroupType val = + field( + "val", + shreddedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, LogicalTypeAnnotation.decimalType(2, 5))); + GroupType address = variant("address", 2, Type.Repetition.REQUIRED, objectFields(val)); + MessageType expectedSchema = parquetSchema(address); + + verifyParquetSchema(icebergTable, expectedSchema); + + List rows = + sql( + """ + SELECT id, + CAST(json_value(address, '$.val') AS DECIMAL(10, 4)) + FROM %s + ORDER BY id + """, + TABLE_NAME); + assertThat(rows).hasSize(6); + assertThat(rows.get(0).getField(1)).isEqualTo(new BigDecimal("123.4500")); + assertThat(rows.get(3).getField(1)).isEqualTo(new BigDecimal("123456.7800")); + assertThat(rows.get(4).getField(1)).isEqualTo(new BigDecimal("1.2345")); + assertThat(rows.get(5).getField(1)).isEqualTo(new BigDecimal("12.3000")); + + sql("DROP TEMPORARY VIEW IF EXISTS tmp_source"); + } + + private void verifyParquetSchema(Table table, MessageType expectedSchema) throws IOException { + table.refresh(); + try (CloseableIterable tasks = table.newScan().planFiles()) { + assertThat(tasks).isNotEmpty(); + + FileScanTask task = tasks.iterator().next(); + String path = task.file().location(); + + HadoopInputFile inputFile = + HadoopInputFile.fromPath(new org.apache.hadoop.fs.Path(path), new Configuration()); + + try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) { + MessageType actualSchema = reader.getFileMetaData().getSchema(); + assertThat(actualSchema).isEqualTo(expectedSchema); + } + } + } + + private static MessageType parquetSchema(Type variantTypes) { + return org.apache.parquet.schema.Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.INT32) + .id(1) + .named("id") + .addFields(variantTypes) + .named("table"); + } + + private static GroupType variant(String name, int fieldId, Type.Repetition repetition) { + return org.apache.parquet.schema.Types.buildGroup(repetition) + .id(fieldId) + .as(LogicalTypeAnnotation.variantType(Variant.VARIANT_SPEC_VERSION)) + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("metadata") + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("value") + .named(name); + } + + private static GroupType variant( + String name, int fieldId, Type.Repetition repetition, Type shreddedType) { + checkShreddedType(shreddedType); + return org.apache.parquet.schema.Types.buildGroup(repetition) + .id(fieldId) + .as(LogicalTypeAnnotation.variantType(Variant.VARIANT_SPEC_VERSION)) + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("metadata") + .optional(PrimitiveType.PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedType) + .named(name); + } + + private static Type shreddedPrimitive(PrimitiveType.PrimitiveTypeName primitive) { + return optional(primitive).named("typed_value"); + } + + private static Type shreddedPrimitive( + PrimitiveType.PrimitiveTypeName primitive, LogicalTypeAnnotation annotation) { + return optional(primitive).as(annotation).named("typed_value"); + } + + private static Type shreddedPrimitive( + PrimitiveType.PrimitiveTypeName primitive, int length, LogicalTypeAnnotation annotation) { + return optional(primitive).length(length).as(annotation).named("typed_value"); + } + + private static GroupType objectFields(GroupType... fields) { + for (GroupType fieldType : fields) { + checkField(fieldType); + } + + return org.apache.parquet.schema.Types.buildGroup(Type.Repetition.OPTIONAL) + .addFields(fields) + .named("typed_value"); + } + + private static GroupType field(String name, Type shreddedType) { + checkShreddedType(shreddedType); + return org.apache.parquet.schema.Types.buildGroup(Type.Repetition.REQUIRED) + .optional(PrimitiveType.PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedType) + .named(name); + } + + private static GroupType element(Type shreddedType) { + return field("element", shreddedType); + } + + private static GroupType list(GroupType elementType) { + return org.apache.parquet.schema.Types.optionalList().element(elementType).named("typed_value"); + } + + private static void checkShreddedType(Type shreddedType) { + Preconditions.checkArgument( + shreddedType.getName().equals("typed_value"), + "Invalid shredded type name: %s should be typed_value", + shreddedType.getName()); + Preconditions.checkArgument( + shreddedType.isRepetition(Type.Repetition.OPTIONAL), + "Invalid shredded type repetition: %s should be OPTIONAL", + shreddedType.getRepetition()); + } + + private static void checkField(GroupType fieldType) { + Preconditions.checkArgument( + fieldType.isRepetition(Type.Repetition.REQUIRED), + "Invalid field type repetition: %s should be REQUIRED", + fieldType.getRepetition()); + } + + private List genericRowData() throws IOException { + List genericRowData = Lists.newArrayList(); + try (CloseableIterable combinedScanTasks = + icebergTable.newScan().planTasks()) { + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + try (DataIterator dataIterator = + ReaderUtil.createDataIterator( + combinedScanTask, icebergTable.schema(), icebergTable.schema())) { + while (dataIterator.hasNext()) { + GenericRowData rowData = (GenericRowData) dataIterator.next(); + genericRowData.add(rowData); + } + } + } + } + + return genericRowData; + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java index 90dd6e117ba8..9a4a62cae612 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java @@ -51,7 +51,7 @@ public class ParquetFormatModel extends BaseFormatModel, R, MessageType> { private final boolean isBatchReader; private final VariantShreddingAnalyzer variantAnalyzer; - private final UnaryOperator copyFunc; + private final Function> copyFuncFactory; public static ParquetFormatModel, Void, Object> forPositionDeletes() { return new ParquetFormatModel<>( @@ -67,6 +67,11 @@ public static ParquetFormatModel> create( type, schemaType, writerFunction, readerFunction, false, null, null); } + /** + * @deprecated Will be removed in 1.13.0; use {@link #create(Class, Class, WriterFunction, + * ReaderFunction, VariantShreddingAnalyzer, Function)} instead. + */ + @Deprecated public static ParquetFormatModel> create( Class type, Class schemaType, @@ -74,8 +79,24 @@ public static ParquetFormatModel> create( ReaderFunction, S, MessageType> readerFunction, VariantShreddingAnalyzer variantAnalyzer, UnaryOperator copyFunc) { + return create( + type, + schemaType, + writerFunction, + readerFunction, + variantAnalyzer, + (Function>) unused -> copyFunc); + } + + public static ParquetFormatModel> create( + Class type, + Class schemaType, + WriterFunction, S, MessageType> writerFunction, + ReaderFunction, S, MessageType> readerFunction, + VariantShreddingAnalyzer variantAnalyzer, + Function> copyFuncFactory) { return new ParquetFormatModel<>( - type, schemaType, writerFunction, readerFunction, false, variantAnalyzer, copyFunc); + type, schemaType, writerFunction, readerFunction, false, variantAnalyzer, copyFuncFactory); } public static ParquetFormatModel> create( @@ -92,11 +113,11 @@ private ParquetFormatModel( ReaderFunction readerFunction, boolean isBatchReader, VariantShreddingAnalyzer variantAnalyzer, - UnaryOperator copyFunc) { + Function> copyFuncFactory) { super(type, schemaType, writerFunction, readerFunction); this.isBatchReader = isBatchReader; this.variantAnalyzer = variantAnalyzer; - this.copyFunc = copyFunc; + this.copyFuncFactory = copyFuncFactory; } @Override @@ -106,7 +127,8 @@ public FileFormat format() { @Override public ModelWriteBuilder writeBuilder(EncryptedOutputFile outputFile) { - return new WriteBuilderWrapper<>(outputFile, writerFunction(), variantAnalyzer, copyFunc); + return new WriteBuilderWrapper<>( + outputFile, writerFunction(), variantAnalyzer, copyFuncFactory); } @Override @@ -118,7 +140,7 @@ private static class WriteBuilderWrapper implements ModelWriteBuilder, S, MessageType> writerFunction; private final VariantShreddingAnalyzer variantAnalyzer; - private final UnaryOperator copyFunc; + private final Function> copyFuncFactory; private Schema schema; private S engineSchema; private FileContent content; @@ -129,11 +151,11 @@ private WriteBuilderWrapper( EncryptedOutputFile outputFile, WriterFunction, S, MessageType> writerFunction, VariantShreddingAnalyzer variantAnalyzer, - UnaryOperator copyFunc) { + Function> copyFuncFactory) { this.internal = Parquet.write(outputFile); this.writerFunction = writerFunction; this.variantAnalyzer = variantAnalyzer; - this.copyFunc = copyFunc; + this.copyFuncFactory = copyFuncFactory; } @Override @@ -267,6 +289,10 @@ public FileAppender build() throws IOException { * top-level fields. */ private FileAppender buildShreddedAppender() { + Preconditions.checkState(copyFuncFactory != null, "copyFuncFactory must not be null"); + UnaryOperator copyFunc = copyFuncFactory.apply(engineSchema); + Preconditions.checkState(copyFunc != null, "copyFunc must not return null"); + return new BufferedFileAppender<>( bufferSize, bufferedRows -> { diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java index 36e254628a6a..547ada3e5355 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java @@ -26,6 +26,8 @@ import java.nio.file.Path; import java.util.List; import java.util.Optional; +import java.util.function.Function; +import java.util.function.UnaryOperator; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; @@ -370,7 +372,7 @@ protected int resolveColumnIndex(Void engineSchema, String columnName) { (icebergSchema, fileSchema, engineSchema, idToConstant) -> GenericParquetReaders.buildReader(icebergSchema, fileSchema), testAnalyzer, - record -> record); + (Function>) unused -> record -> record); try (FileAppender appender = model @@ -401,7 +403,7 @@ public void testWriteBuilderReturnsDirectAppenderWithNullAnalyzer() throws IOExc (icebergSchema, fileSchema, engineSchema, idToConstant) -> GenericParquetReaders.buildReader(icebergSchema, fileSchema), null, - null); + (Function>) null); try (FileAppender appender = model @@ -471,7 +473,7 @@ protected int resolveColumnIndex(Void engineSchema, String columnName) { (icebergSchema, fileSchema, engineSchema, idToConstant) -> GenericParquetReaders.buildReader(icebergSchema, fileSchema), analyzer, - record1 -> record1); + (Function>) unused -> record1 -> record1); try (FileAppender appender = model diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 5b7862116aea..15c96ff4cd73 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import java.util.function.Function; +import java.util.function.UnaryOperator; import org.apache.iceberg.avro.AvroFormatModel; import org.apache.iceberg.formats.FormatModelRegistry; import org.apache.iceberg.orc.ORCFormatModel; @@ -53,7 +55,7 @@ public static void register() { (icebergSchema, fileSchema, engineSchema, idToConstant) -> SparkParquetReaders.buildReader(icebergSchema, fileSchema, idToConstant), new SparkVariantShreddingAnalyzer(), - InternalRow::copy)); + (Function>) unused -> InternalRow::copy)); FormatModelRegistry.register( ParquetFormatModel.create( diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 5b7862116aea..15c96ff4cd73 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import java.util.function.Function; +import java.util.function.UnaryOperator; import org.apache.iceberg.avro.AvroFormatModel; import org.apache.iceberg.formats.FormatModelRegistry; import org.apache.iceberg.orc.ORCFormatModel; @@ -53,7 +55,7 @@ public static void register() { (icebergSchema, fileSchema, engineSchema, idToConstant) -> SparkParquetReaders.buildReader(icebergSchema, fileSchema, idToConstant), new SparkVariantShreddingAnalyzer(), - InternalRow::copy)); + (Function>) unused -> InternalRow::copy)); FormatModelRegistry.register( ParquetFormatModel.create( From f37a04b532595354fdc85e127491b032f528e2e0 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Fri, 22 May 2026 10:03:42 +0200 Subject: [PATCH 03/58] Core, Orc: Remove deprecated partition stats read functionality (#14998) --- .palantir/revapi.yml | 56 +++ .../org/apache/iceberg/PartitionStats.java | 317 -------------- .../apache/iceberg/PartitionStatsHandler.java | 218 +--------- .../PartitionStatsHandlerTestBase.java | 398 +++++------------- .../apache/iceberg/TestPartitionStats.java | 135 ------ .../orc/TestOrcPartitionStatsHandler.java | 21 - 6 files changed, 154 insertions(+), 991 deletions(-) delete mode 100644 core/src/main/java/org/apache/iceberg/PartitionStats.java delete mode 100644 core/src/test/java/org/apache/iceberg/TestPartitionStats.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 047be57415aa..80fa8f15f168 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -517,6 +517,62 @@ acceptedBreaks: - code: "java.class.removed" old: "class org.apache.iceberg.data.PartitionStatsHandler" justification: "Removing deprecated code for 1.11.0" + "1.11.0": + org.apache.iceberg:iceberg-core: + - code: "java.class.removed" + old: "class org.apache.iceberg.PartitionStats" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.DATA_FILE_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.DATA_RECORD_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.DV_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_FILE_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_RECORD_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_AT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_SNAPSHOT_ID" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_FILE_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_RECORD_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.SPEC_ID" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.TOTAL_DATA_FILE_SIZE_IN_BYTES" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removed" + old: "field org.apache.iceberg.PartitionStatsHandler.TOTAL_RECORD_COUNT" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removedWithConstant" + old: "field org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_ID" + justification: "Removed deprecated functionality for partition stats" + - code: "java.field.removedWithConstant" + old: "field org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_NAME" + justification: "Removed deprecated functionality for partition stats" + - code: "java.method.removed" + old: "method org.apache.iceberg.Schema org.apache.iceberg.PartitionStatsHandler::schema(org.apache.iceberg.types.Types.StructType,\ + \ int)" + justification: "Removed deprecated functionality for partition stats" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable\ + \ org.apache.iceberg.PartitionStatsHandler::readPartitionStatsFile(org.apache.iceberg.Schema,\ + \ org.apache.iceberg.io.InputFile)" + justification: "Removed deprecated functionality for partition stats" "1.2.0": org.apache.iceberg:iceberg-api: - code: "java.field.constantValueChanged" diff --git a/core/src/main/java/org/apache/iceberg/PartitionStats.java b/core/src/main/java/org/apache/iceberg/PartitionStats.java deleted file mode 100644 index e8a4e18916bc..000000000000 --- a/core/src/main/java/org/apache/iceberg/PartitionStats.java +++ /dev/null @@ -1,317 +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; - -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Class to hold partition statistics values. - * - * @deprecated will be removed in 1.12.0. Use {@link BasePartitionStatistics instead} - */ -@Deprecated -public class PartitionStats implements StructLike { - - private static final int STATS_COUNT = 13; - - private StructLike partition; - private int specId; - private long dataRecordCount; - private int dataFileCount; - private long totalDataFileSizeInBytes; - private long positionDeleteRecordCount; // also includes dv record count as per spec - private int positionDeleteFileCount; - private long equalityDeleteRecordCount; - private int equalityDeleteFileCount; - private Long totalRecordCount; // null by default - private Long lastUpdatedAt; // null by default - private Long lastUpdatedSnapshotId; // null by default - private int dvCount; - - public PartitionStats(StructLike partition, int specId) { - this.partition = partition; - this.specId = specId; - } - - public StructLike partition() { - return partition; - } - - public int specId() { - return specId; - } - - public long dataRecordCount() { - return dataRecordCount; - } - - public int dataFileCount() { - return dataFileCount; - } - - public long totalDataFileSizeInBytes() { - return totalDataFileSizeInBytes; - } - - public long positionDeleteRecordCount() { - return positionDeleteRecordCount; - } - - public int positionDeleteFileCount() { - return positionDeleteFileCount; - } - - public long equalityDeleteRecordCount() { - return equalityDeleteRecordCount; - } - - public int equalityDeleteFileCount() { - return equalityDeleteFileCount; - } - - public Long totalRecords() { - return totalRecordCount; - } - - public Long lastUpdatedAt() { - return lastUpdatedAt; - } - - public Long lastUpdatedSnapshotId() { - return lastUpdatedSnapshotId; - } - - public int dvCount() { - return dvCount; - } - - /** - * Updates the partition stats from the data/delete file. - * - * @param file the {@link ContentFile} from the manifest entry. - * @param snapshot the snapshot corresponding to the live entry. - */ - void liveEntry(ContentFile file, Snapshot snapshot) { - Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match"); - - switch (file.content()) { - case DATA: - this.dataRecordCount += file.recordCount(); - this.dataFileCount += 1; - this.totalDataFileSizeInBytes += file.fileSizeInBytes(); - break; - case POSITION_DELETES: - this.positionDeleteRecordCount += file.recordCount(); - if (file.format() == FileFormat.PUFFIN) { - this.dvCount += 1; - } else { - this.positionDeleteFileCount += 1; - } - - break; - case EQUALITY_DELETES: - this.equalityDeleteRecordCount += file.recordCount(); - this.equalityDeleteFileCount += 1; - break; - default: - throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); - } - - if (snapshot != null) { - updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); - } - - // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs scanning the data. - } - - /** - * Updates the modified time and snapshot ID for the deleted manifest entry. - * - * @param snapshot the snapshot corresponding to the deleted manifest entry. - */ - void deletedEntry(Snapshot snapshot) { - if (snapshot != null) { - updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); - } - } - - /** - * Decrement the counters as it was included in the previous stats and updates the modified time - * and snapshot ID for the deleted manifest entry. - * - * @param snapshot the snapshot corresponding to the deleted manifest entry. - */ - void deletedEntryForIncrementalCompute(ContentFile file, Snapshot snapshot) { - Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match"); - - switch (file.content()) { - case DATA: - this.dataRecordCount -= file.recordCount(); - this.dataFileCount -= 1; - this.totalDataFileSizeInBytes -= file.fileSizeInBytes(); - break; - case POSITION_DELETES: - this.positionDeleteRecordCount -= file.recordCount(); - if (file.format() == FileFormat.PUFFIN) { - this.dvCount -= 1; - } else { - this.positionDeleteFileCount -= 1; - } - - break; - case EQUALITY_DELETES: - this.equalityDeleteRecordCount -= file.recordCount(); - this.equalityDeleteFileCount -= 1; - break; - default: - throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); - } - - if (snapshot != null) { - updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); - } - } - - /** - * Appends statistics from given entry to current entry. - * - * @param entry the entry from which statistics will be sourced. - */ - void appendStats(PartitionStats entry) { - Preconditions.checkArgument(specId == entry.specId(), "Spec IDs must match"); - - this.dataRecordCount += entry.dataRecordCount; - this.dataFileCount += entry.dataFileCount; - this.totalDataFileSizeInBytes += entry.totalDataFileSizeInBytes; - this.positionDeleteRecordCount += entry.positionDeleteRecordCount; - this.positionDeleteFileCount += entry.positionDeleteFileCount; - this.equalityDeleteRecordCount += entry.equalityDeleteRecordCount; - this.equalityDeleteFileCount += entry.equalityDeleteFileCount; - this.dvCount += entry.dvCount; - - if (entry.totalRecordCount != null) { - if (totalRecordCount == null) { - this.totalRecordCount = entry.totalRecordCount; - } else { - this.totalRecordCount += entry.totalRecordCount; - } - } - - if (entry.lastUpdatedAt != null) { - updateSnapshotInfo(entry.lastUpdatedSnapshotId, entry.lastUpdatedAt); - } - } - - private void updateSnapshotInfo(long snapshotId, long updatedAt) { - if (lastUpdatedAt == null || lastUpdatedAt < updatedAt) { - this.lastUpdatedAt = updatedAt; - this.lastUpdatedSnapshotId = snapshotId; - } - } - - @Override - public int size() { - return STATS_COUNT; - } - - @Override - public T get(int pos, Class javaClass) { - switch (pos) { - case 0: - return javaClass.cast(partition); - case 1: - return javaClass.cast(specId); - case 2: - return javaClass.cast(dataRecordCount); - case 3: - return javaClass.cast(dataFileCount); - case 4: - return javaClass.cast(totalDataFileSizeInBytes); - case 5: - return javaClass.cast(positionDeleteRecordCount); - case 6: - return javaClass.cast(positionDeleteFileCount); - case 7: - return javaClass.cast(equalityDeleteRecordCount); - case 8: - return javaClass.cast(equalityDeleteFileCount); - case 9: - return javaClass.cast(totalRecordCount); - case 10: - return javaClass.cast(lastUpdatedAt); - case 11: - return javaClass.cast(lastUpdatedSnapshotId); - case 12: - return javaClass.cast(dvCount); - default: - throw new UnsupportedOperationException("Unknown position: " + pos); - } - } - - @Override - public void set(int pos, T value) { - switch (pos) { - case 0: - this.partition = (StructLike) value; - break; - case 1: - this.specId = (int) value; - break; - case 2: - this.dataRecordCount = (long) value; - break; - case 3: - this.dataFileCount = (int) value; - break; - case 4: - this.totalDataFileSizeInBytes = (long) value; - break; - case 5: - // optional field as per spec, implementation initialize to 0 for counters - this.positionDeleteRecordCount = value == null ? 0L : (long) value; - break; - case 6: - // optional field as per spec, implementation initialize to 0 for counters - this.positionDeleteFileCount = value == null ? 0 : (int) value; - break; - case 7: - // optional field as per spec, implementation initialize to 0 for counters - this.equalityDeleteRecordCount = value == null ? 0L : (long) value; - break; - case 8: - // optional field as per spec, implementation initialize to 0 for counters - this.equalityDeleteFileCount = value == null ? 0 : (int) value; - break; - case 9: - this.totalRecordCount = (Long) value; - break; - case 10: - this.lastUpdatedAt = (Long) value; - break; - case 11: - this.lastUpdatedSnapshotId = (Long) value; - break; - case 12: - this.dvCount = value == null ? 0 : (int) value; - break; - default: - throw new UnsupportedOperationException("Unknown position: " + pos); - } - } -} diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java index a3e298d72b83..29f7bcb53ce6 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java @@ -33,20 +33,14 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Queues; import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.types.Types.IntegerType; -import org.apache.iceberg.types.Types.LongType; -import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionMap; @@ -67,180 +61,6 @@ private PartitionStatsHandler() {} private static final Logger LOG = LoggerFactory.getLogger(PartitionStatsHandler.class); - // schema of the partition stats file as per spec - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#EMPTY_PARTITION_FIELD} - */ - @Deprecated public static final int PARTITION_FIELD_ID = 1; - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#EMPTY_PARTITION_FIELD} - */ - @Deprecated public static final String PARTITION_FIELD_NAME = "partition"; - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#SPEC_ID} - */ - @Deprecated - public static final NestedField SPEC_ID = NestedField.required(2, "spec_id", IntegerType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#DATA_RECORD_COUNT} - */ - @Deprecated - public static final NestedField DATA_RECORD_COUNT = - NestedField.required(3, "data_record_count", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#DATA_FILE_COUNT} - */ - @Deprecated - public static final NestedField DATA_FILE_COUNT = - NestedField.required(4, "data_file_count", IntegerType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link - * PartitionStatistics#TOTAL_DATA_FILE_SIZE_IN_BYTES} - */ - @Deprecated - public static final NestedField TOTAL_DATA_FILE_SIZE_IN_BYTES = - NestedField.required(5, "total_data_file_size_in_bytes", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link - * PartitionStatistics#POSITION_DELETE_RECORD_COUNT} - */ - @Deprecated - public static final NestedField POSITION_DELETE_RECORD_COUNT = - NestedField.optional(6, "position_delete_record_count", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link - * PartitionStatistics#POSITION_DELETE_FILE_COUNT} - */ - @Deprecated - public static final NestedField POSITION_DELETE_FILE_COUNT = - NestedField.optional(7, "position_delete_file_count", IntegerType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link - * PartitionStatistics#EQUALITY_DELETE_RECORD_COUNT} - */ - @Deprecated - public static final NestedField EQUALITY_DELETE_RECORD_COUNT = - NestedField.optional(8, "equality_delete_record_count", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link - * PartitionStatistics#EQUALITY_DELETE_FILE_COUNT} - */ - @Deprecated - public static final NestedField EQUALITY_DELETE_FILE_COUNT = - NestedField.optional(9, "equality_delete_file_count", IntegerType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#TOTAL_RECORD_COUNT} - */ - @Deprecated - public static final NestedField TOTAL_RECORD_COUNT = - NestedField.optional(10, "total_record_count", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#LAST_UPDATED_AT} - */ - @Deprecated - public static final NestedField LAST_UPDATED_AT = - NestedField.optional(11, "last_updated_at", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#LAST_UPDATED_SNAPSHOT_ID} - */ - @Deprecated - public static final NestedField LAST_UPDATED_SNAPSHOT_ID = - NestedField.optional(12, "last_updated_snapshot_id", LongType.get()); - - /** - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#DV_COUNT} - */ - @Deprecated - public static final NestedField DV_COUNT = - NestedField.required("dv_count") - .withId(13) - .ofType(Types.IntegerType.get()) - .withInitialDefault(Literal.of(0)) - .withWriteDefault(Literal.of(0)) - .build(); - - /** - * Generates the partition stats file schema for a given format version based on a combined - * partition type which considers all specs in a table. - * - * @param unifiedPartitionType unified partition schema type. Could be calculated by {@link - * Partitioning#partitionType(Table)}. - * @return a schema that corresponds to the provided unified partition type. - * @deprecated will be removed in 1.12.0. Use {@link PartitionStatistics#schema(StructType, int)} - * instead. - */ - @Deprecated - public static Schema schema(StructType unifiedPartitionType, int formatVersion) { - Preconditions.checkState(!unifiedPartitionType.fields().isEmpty(), "Table must be partitioned"); - Preconditions.checkState( - formatVersion > 0 && formatVersion <= TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION, - "Invalid format version: %s", - formatVersion); - - if (formatVersion <= 2) { - return v2Schema(unifiedPartitionType); - } - - return v3Schema(unifiedPartitionType); - } - - private static Schema v2Schema(StructType unifiedPartitionType) { - return new Schema( - NestedField.required(PARTITION_FIELD_ID, PARTITION_FIELD_NAME, unifiedPartitionType), - SPEC_ID, - DATA_RECORD_COUNT, - DATA_FILE_COUNT, - TOTAL_DATA_FILE_SIZE_IN_BYTES, - POSITION_DELETE_RECORD_COUNT, - POSITION_DELETE_FILE_COUNT, - EQUALITY_DELETE_RECORD_COUNT, - EQUALITY_DELETE_FILE_COUNT, - TOTAL_RECORD_COUNT, - LAST_UPDATED_AT, - LAST_UPDATED_SNAPSHOT_ID); - } - - private static Schema v3Schema(StructType unifiedPartitionType) { - return new Schema( - NestedField.required(PARTITION_FIELD_ID, PARTITION_FIELD_NAME, unifiedPartitionType), - SPEC_ID, - DATA_RECORD_COUNT, - DATA_FILE_COUNT, - TOTAL_DATA_FILE_SIZE_IN_BYTES, - NestedField.required( - POSITION_DELETE_RECORD_COUNT.fieldId(), - POSITION_DELETE_RECORD_COUNT.name(), - LongType.get()), - NestedField.required( - POSITION_DELETE_FILE_COUNT.fieldId(), - POSITION_DELETE_FILE_COUNT.name(), - IntegerType.get()), - NestedField.required( - EQUALITY_DELETE_RECORD_COUNT.fieldId(), - EQUALITY_DELETE_RECORD_COUNT.name(), - LongType.get()), - NestedField.required( - EQUALITY_DELETE_FILE_COUNT.fieldId(), - EQUALITY_DELETE_FILE_COUNT.name(), - IntegerType.get()), - TOTAL_RECORD_COUNT, - LAST_UPDATED_AT, - LAST_UPDATED_SNAPSHOT_ID, - DV_COUNT); - } - /** * Computes the stats incrementally after the snapshot that has partition stats file till the * current snapshot and writes the combined result into a {@link PartitionStatisticsFile} after @@ -343,28 +163,6 @@ static PartitionStatisticsFile writePartitionStatsFile( .build(); } - /** - * Reads partition statistics from the specified {@link InputFile} using given schema. - * - * @param schema The {@link Schema} of the partition statistics file. - * @param inputFile An {@link InputFile} pointing to the partition stats file. - * @deprecated will be removed in 1.12.0, use {@link PartitionStatisticsScan} instead - */ - @Deprecated - public static CloseableIterable readPartitionStatsFile( - Schema schema, InputFile inputFile) { - Preconditions.checkArgument(schema != null, "Invalid schema: null"); - Preconditions.checkArgument(inputFile != null, "Invalid input file: null"); - - FileFormat fileFormat = FileFormat.fromFileName(inputFile.location()); - Preconditions.checkArgument( - fileFormat != null, "Unable to determine format of file: %s", inputFile.location()); - - CloseableIterable records = - InternalData.read(fileFormat, inputFile).project(schema).build(); - return CloseableIterable.transform(records, PartitionStatsHandler::recordToPartitionStats); - } - private static OutputFile newPartitionStatsFile( Table table, FileFormat fileFormat, long snapshotId) { Preconditions.checkArgument( @@ -382,19 +180,6 @@ private static OutputFile newPartitionStatsFile( Locale.ROOT, "partition-stats-%d-%s", snapshotId, UUID.randomUUID())))); } - private static PartitionStats recordToPartitionStats(StructLike record) { - int pos = 0; - PartitionStats stats = - new PartitionStats( - record.get(pos++, StructLike.class), // partition - record.get(pos++, Integer.class)); // spec id - for (; pos < record.size(); pos++) { - stats.set(pos, record.get(pos, Object.class)); - } - - return stats; - } - private static Collection computeAndMergeStatsIncremental( Table table, Snapshot snapshot, long lastSnapshotWithStats) { PartitionMap statsMap = PartitionMap.create(table.specs()); @@ -680,7 +465,8 @@ private static void deletedEntryForIncrementalCompute( * @param targetStats partition statistics to be updated. * @param inputStats the partition statistics used as input. */ - private static void appendStats(PartitionStatistics targetStats, PartitionStatistics inputStats) { + @VisibleForTesting + static void appendStats(PartitionStatistics targetStats, PartitionStatistics inputStats) { Preconditions.checkArgument(targetStats.specId() != null, "Invalid spec ID: null"); Preconditions.checkArgument( targetStats.specId().equals(inputStats.specId()), "Spec IDs must match"); diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java index 7b7e85bfffce..54cd9cd4339d 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java @@ -24,7 +24,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Arrays; @@ -32,7 +31,6 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.UUID; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; @@ -41,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.groups.Tuple; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -61,6 +58,10 @@ protected static List formatVersions() { private final Map fileFormatProperty = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format().name()); + private static final PartitionData PARTITION = + new PartitionData( + Types.StructType.of(Types.NestedField.required(1, "foo", Types.IntegerType.get()))); + @Test public void testPartitionStatsOnEmptyTable() throws Exception { Table testTable = @@ -294,185 +295,6 @@ public void testOptionalFieldsWriting() throws Exception { } } - /** - * @deprecated will be removed in 1.12.0 - */ - @SuppressWarnings("checkstyle:MethodLength") - @Test - @Deprecated - public void testPartitionStats() throws Exception { - Table testTable = - TestTables.create( - tempDir("partition_stats_compute"), - "partition_stats_compute", - SCHEMA, - SPEC, - 2, - fileFormatProperty); - - DataFile dataFile1 = - FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); - DataFile dataFile2 = - FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "B")); - DataFile dataFile3 = - FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", "A")); - DataFile dataFile4 = - FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", "B")); - - for (int i = 0; i < 3; i++) { - // insert same set of seven records thrice to have a new manifest files - testTable - .newAppend() - .appendFile(dataFile1) - .appendFile(dataFile2) - .appendFile(dataFile3) - .appendFile(dataFile4) - .commit(); - } - - Snapshot snapshot1 = testTable.currentSnapshot(); - Schema recordSchema = PartitionStatistics.schema(Partitioning.partitionType(testTable), 2); - - Types.StructType partitionType = - recordSchema.findField(EMPTY_PARTITION_FIELD.fieldId()).type().asStructType(); - computeAndValidatePartitionStats( - testTable, - recordSchema, - Tuple.tuple( - partitionRecord(partitionType, "foo", "A"), - 0, - 3 * dataFile1.recordCount(), - 3, - 3 * dataFile1.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0), - Tuple.tuple( - partitionRecord(partitionType, "foo", "B"), - 0, - 3 * dataFile2.recordCount(), - 3, - 3 * dataFile2.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0), - Tuple.tuple( - partitionRecord(partitionType, "bar", "A"), - 0, - 3 * dataFile3.recordCount(), - 3, - 3 * dataFile3.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0), - Tuple.tuple( - partitionRecord(partitionType, "bar", "B"), - 0, - 3 * dataFile4.recordCount(), - 3, - 3 * dataFile4.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0)); - - DeleteFile posDelete = - FileGenerationUtil.generatePositionDeleteFile(testTable, TestHelpers.Row.of("bar", "A")); - testTable.newRowDelta().addDeletes(posDelete).commit(); - // snapshot2 is unused in the result as same partition was updated by snapshot4 - - DeleteFile eqDelete = - FileGenerationUtil.generateEqualityDeleteFile(testTable, TestHelpers.Row.of("foo", "A")); - testTable.newRowDelta().addDeletes(eqDelete).commit(); - Snapshot snapshot3 = testTable.currentSnapshot(); - - testTable.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); - DeleteFile dv = FileGenerationUtil.generateDV(testTable, dataFile3); - testTable.newRowDelta().addDeletes(dv).commit(); - Snapshot snapshot4 = testTable.currentSnapshot(); - - recordSchema = PartitionStatistics.schema(Partitioning.partitionType(testTable), 3); - - computeAndValidatePartitionStats( - testTable, - recordSchema, - Tuple.tuple( - partitionRecord(partitionType, "foo", "A"), - 0, - 3 * dataFile1.recordCount(), - 3, - 3 * dataFile1.fileSizeInBytes(), - 0L, - 0, - eqDelete.recordCount(), - 1, - null, - snapshot3.timestampMillis(), - snapshot3.snapshotId(), - 0), - Tuple.tuple( - partitionRecord(partitionType, "foo", "B"), - 0, - 3 * dataFile2.recordCount(), - 3, - 3 * dataFile2.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0), - Tuple.tuple( - partitionRecord(partitionType, "bar", "A"), - 0, - 3 * dataFile3.recordCount(), - 3, - 3 * dataFile3.fileSizeInBytes(), - posDelete.recordCount() + dv.recordCount(), - 1, - 0L, - 0, - null, - snapshot4.timestampMillis(), - snapshot4.snapshotId(), - 1), // dv count - Tuple.tuple( - partitionRecord(partitionType, "bar", "B"), - 0, - 3 * dataFile4.recordCount(), - 3, - 3 * dataFile4.fileSizeInBytes(), - 0L, - 0, - 0L, - 0, - null, - snapshot1.timestampMillis(), - snapshot1.snapshotId(), - 0)); - } - @Test public void testCopyOnWriteDelete() throws Exception { Table testTable = @@ -591,39 +413,6 @@ public void testLatestStatsFileWithBranch() throws Exception { assertThat(PartitionStatsHandler.latestStatsFile(testTable, snapshotBranchBId)).isNull(); } - /** - * @deprecated will be removed in 1.12.0 - */ - @Test - @Deprecated - public void testReadingStatsWithInvalidSchema() throws Exception { - PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); - Table testTable = - TestTables.create(tempDir("old_schema"), "old_schema", SCHEMA, spec, 2, fileFormatProperty); - Types.StructType partitionType = Partitioning.partitionType(testTable); - Schema newSchema = PartitionStatistics.schema(partitionType, 2); - Schema oldSchema = invalidOldSchema(partitionType); - - PartitionStatisticsFile invalidStatisticsFile = - PartitionStatsHandler.writePartitionStatsFile( - testTable, 42L, oldSchema, Collections.singletonList(randomStats(partitionType))); - - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - newSchema, testTable.io().newInputFile(invalidStatisticsFile.path()))) { - - if (format() == FileFormat.PARQUET) { - assertThatThrownBy(() -> Lists.newArrayList(recordIterator)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Not a primitive type: struct"); - } else if (format() == FileFormat.AVRO) { - assertThatThrownBy(() -> Lists.newArrayList(recordIterator)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("Not an instance of org.apache.iceberg.StructLike"); - } - } - } - @Test public void testFullComputeFallbackWithInvalidStats() throws Exception { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); @@ -661,105 +450,110 @@ public void testFullComputeFallbackWithInvalidStats() throws Exception { assertThat(partitionStats.get(0).dataFileCount()).isEqualTo(2); } - /** - * @deprecated will be removed in 1.12.0 - */ @Test - @Deprecated - public void testV2toV3SchemaEvolution() throws Exception { - Table testTable = - TestTables.create( - tempDir("schema_evolution"), "schema_evolution", SCHEMA, SPEC, 2, fileFormatProperty); + public void testAppendWithAllValues() { + BasePartitionStatistics stats1 = + createStats(100L, 15, 1000L, 2L, 500, 1L, 200, 15L, 1625077800000L, 12345L); + BasePartitionStatistics stats2 = + createStats(200L, 7, 500L, 1L, 100, 0L, 50, 7L, 1625077900000L, 12346L); - // write stats file using v2 schema - DataFile dataFile = - FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); - testTable.newAppend().appendFile(dataFile).commit(); - PartitionStatisticsFile statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile( - testTable, testTable.currentSnapshot().snapshotId()); + PartitionStatsHandler.appendStats(stats1, stats2); - Types.StructType partitionSchema = Partitioning.partitionType(testTable); + validateStats(stats1, 300L, 22, 1500L, 3L, 600, 1L, 250, 22L, 1625077900000L, 12346L); + } - // read with v2 schema - Schema v2Schema = PartitionStatistics.schema(partitionSchema, 2); - List partitionStatsV2; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - v2Schema, testTable.io().newInputFile(statisticsFile.path()))) { - partitionStatsV2 = Lists.newArrayList(recordIterator); - } + @Test + public void testAppendWithThisNullOptionalField() { + BasePartitionStatistics stats1 = + createStats(100L, 15, 1000L, 2L, 500, 1L, 200, null, null, null); + BasePartitionStatistics stats2 = + createStats(100L, 7, 500L, 1L, 100, 0L, 50, 7L, 1625077900000L, 12346L); - // read with v3 schema - Schema v3Schema = PartitionStatistics.schema(partitionSchema, 3); - List partitionStatsV3; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - v3Schema, testTable.io().newInputFile(statisticsFile.path()))) { - partitionStatsV3 = Lists.newArrayList(recordIterator); - } + PartitionStatsHandler.appendStats(stats1, stats2); - assertThat(partitionStatsV2).hasSameSizeAs(partitionStatsV3); - Comparator comparator = Comparators.forType(partitionSchema); - for (int i = 0; i < partitionStatsV2.size(); i++) { - assertThat(isEqual(comparator, partitionStatsV2.get(i), partitionStatsV3.get(i))).isTrue(); - } + validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, 7L, 1625077900000L, 12346L); } - private static void computeAndValidatePartitionStats( - Table testTable, Schema recordSchema, Tuple... expectedValues) throws IOException { - // compute and commit partition stats file - Snapshot currentSnapshot = testTable.currentSnapshot(); - PartitionStatisticsFile result = PartitionStatsHandler.computeAndWriteStatsFile(testTable); - testTable.updatePartitionStatistics().setPartitionStatistics(result).commit(); - assertThat(result.snapshotId()).isEqualTo(currentSnapshot.snapshotId()); + @Test + public void testAppendWithBothNullOptionalFields() { + BasePartitionStatistics stats1 = + createStats(100L, 15, 1000L, 2L, 500, 1L, 200, null, null, null); + BasePartitionStatistics stats2 = createStats(100L, 7, 500L, 1L, 100, 0L, 50, null, null, null); - // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - recordSchema, testTable.io().newInputFile(result.path()))) { - partitionStats = Lists.newArrayList(recordIterator); - } + PartitionStatsHandler.appendStats(stats1, stats2); - assertThat(partitionStats) - .extracting( - PartitionStats::partition, - PartitionStats::specId, - PartitionStats::dataRecordCount, - PartitionStats::dataFileCount, - PartitionStats::totalDataFileSizeInBytes, - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId, - PartitionStats::dvCount) - .containsExactlyInAnyOrder(expectedValues); + validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, null, null, null); } - @SuppressWarnings("checkstyle:CyclomaticComplexity") - private static boolean isEqual( - Comparator partitionComparator, PartitionStats stats1, PartitionStats stats2) { - if (stats1 == stats2) { - return true; - } else if (stats1 == null || stats2 == null) { - return false; - } + @Test + public void testAppendWithOtherNullOptionalFields() { + BasePartitionStatistics stats1 = + createStats(100L, 15, 1000L, 2L, 500, 1L, 200, 15L, 1625077900000L, 12346L); + BasePartitionStatistics stats2 = createStats(100L, 7, 500L, 1L, 100, 0L, 50, null, null, null); - return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 - && stats1.specId() == stats2.specId() - && stats1.dataRecordCount() == stats2.dataRecordCount() - && stats1.dataFileCount() == stats2.dataFileCount() - && stats1.totalDataFileSizeInBytes() == stats2.totalDataFileSizeInBytes() - && stats1.positionDeleteRecordCount() == stats2.positionDeleteRecordCount() - && stats1.positionDeleteFileCount() == stats2.positionDeleteFileCount() - && stats1.equalityDeleteRecordCount() == stats2.equalityDeleteRecordCount() - && stats1.equalityDeleteFileCount() == stats2.equalityDeleteFileCount() - && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) - && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) - && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); + PartitionStatsHandler.appendStats(stats1, stats2); + + validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, 15L, 1625077900000L, 12346L); + } + + @Test + public void testAppendEmptyStats() { + BasePartitionStatistics stats1 = new BasePartitionStatistics(PARTITION, 1); + BasePartitionStatistics stats2 = new BasePartitionStatistics(PARTITION, 1); + + PartitionStatsHandler.appendStats(stats1, stats2); + + validateStats(stats1, 0L, 0, 0L, 0L, 0, 0L, 0, null, null, null); + } + + @Test + public void testAppendWithDifferentSpec() { + BasePartitionStatistics stats1 = new BasePartitionStatistics(PARTITION, 1); + BasePartitionStatistics stats2 = new BasePartitionStatistics(PARTITION, 2); + + assertThatThrownBy(() -> PartitionStatsHandler.appendStats(stats1, stats2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Spec IDs must match"); + } + + private BasePartitionStatistics createStats( + long dataRecordCount, + int dataFileCount, + long totalDataFileSizeInBytes, + long positionDeleteRecordCount, + int positionDeleteFileCount, + long equalityDeleteRecordCount, + int equalityDeleteFileCount, + Long totalRecordCount, + Long lastUpdatedAt, + Long lastUpdatedSnapshotId) { + + BasePartitionStatistics stats = new BasePartitionStatistics(PARTITION, 1); + stats.set(2, dataRecordCount); + stats.set(3, dataFileCount); + stats.set(4, totalDataFileSizeInBytes); + stats.set(5, positionDeleteRecordCount); + stats.set(6, positionDeleteFileCount); + stats.set(7, equalityDeleteRecordCount); + stats.set(8, equalityDeleteFileCount); + stats.set(9, totalRecordCount); + stats.set(10, lastUpdatedAt); + stats.set(11, lastUpdatedSnapshotId); + + return stats; + } + + private void validateStats(PartitionStatistics stats, Object... expectedValues) { + // Spec id and partition data should be unchanged + assertThat(stats.get(0, PartitionData.class)).isEqualTo(PARTITION); + assertThat(stats.get(1, Integer.class)).isEqualTo(1); + + for (int i = 0; i < expectedValues.length; i++) { + if (expectedValues[i] == null) { + assertThat(stats.get(i + 2, Object.class)).isNull(); + } else { + assertThat(stats.get(i + 2, Object.class)).isEqualTo(expectedValues[i]); + } + } } } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionStats.java b/core/src/test/java/org/apache/iceberg/TestPartitionStats.java deleted file mode 100644 index c215fbcb800c..000000000000 --- a/core/src/test/java/org/apache/iceberg/TestPartitionStats.java +++ /dev/null @@ -1,135 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestPartitionStats { - - private static final PartitionData PARTITION = - new PartitionData( - Types.StructType.of(Types.NestedField.required(1, "foo", Types.IntegerType.get()))); - - @Test - public void testAppendWithAllValues() { - PartitionStats stats1 = - createStats(100L, 15, 1000L, 2L, 500, 1L, 200, 15L, 1625077800000L, 12345L); - PartitionStats stats2 = createStats(200L, 7, 500L, 1L, 100, 0L, 50, 7L, 1625077900000L, 12346L); - - stats1.appendStats(stats2); - - validateStats(stats1, 300L, 22, 1500L, 3L, 600, 1L, 250, 22L, 1625077900000L, 12346L); - } - - @Test - public void testAppendWithThisNullOptionalField() { - PartitionStats stats1 = createStats(100L, 15, 1000L, 2L, 500, 1L, 200, null, null, null); - PartitionStats stats2 = createStats(100L, 7, 500L, 1L, 100, 0L, 50, 7L, 1625077900000L, 12346L); - - stats1.appendStats(stats2); - - validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, 7L, 1625077900000L, 12346L); - } - - @Test - public void testAppendWithBothNullOptionalFields() { - PartitionStats stats1 = createStats(100L, 15, 1000L, 2L, 500, 1L, 200, null, null, null); - PartitionStats stats2 = createStats(100L, 7, 500L, 1L, 100, 0L, 50, null, null, null); - - stats1.appendStats(stats2); - - validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, null, null, null); - } - - @Test - public void testAppendWithOtherNullOptionalFields() { - PartitionStats stats1 = - createStats(100L, 15, 1000L, 2L, 500, 1L, 200, 15L, 1625077900000L, 12346L); - PartitionStats stats2 = createStats(100L, 7, 500L, 1L, 100, 0L, 50, null, null, null); - - stats1.appendStats(stats2); - - validateStats(stats1, 200L, 22, 1500L, 3L, 600, 1L, 250, 15L, 1625077900000L, 12346L); - } - - @Test - public void testAppendEmptyStats() { - PartitionStats stats1 = new PartitionStats(PARTITION, 1); - PartitionStats stats2 = new PartitionStats(PARTITION, 1); - - stats1.appendStats(stats2); - - validateStats(stats1, 0L, 0, 0L, 0L, 0, 0L, 0, null, null, null); - } - - @Test - public void testAppendWithDifferentSpec() { - PartitionStats stats1 = new PartitionStats(PARTITION, 1); - PartitionStats stats2 = new PartitionStats(PARTITION, 2); - - assertThatThrownBy(() -> stats1.appendStats(stats2)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Spec IDs must match"); - } - - private PartitionStats createStats( - long dataRecordCount, - int dataFileCount, - long totalDataFileSizeInBytes, - long positionDeleteRecordCount, - int positionDeleteFileCount, - long equalityDeleteRecordCount, - int equalityDeleteFileCount, - Long totalRecordCount, - Long lastUpdatedAt, - Long lastUpdatedSnapshotId) { - - PartitionStats stats = new PartitionStats(PARTITION, 1); - stats.set(2, dataRecordCount); - stats.set(3, dataFileCount); - stats.set(4, totalDataFileSizeInBytes); - stats.set(5, positionDeleteRecordCount); - stats.set(6, positionDeleteFileCount); - stats.set(7, equalityDeleteRecordCount); - stats.set(8, equalityDeleteFileCount); - stats.set(9, totalRecordCount); - stats.set(10, lastUpdatedAt); - stats.set(11, lastUpdatedSnapshotId); - - return stats; - } - - private void validateStats(PartitionStats stats, Object... expectedValues) { - // Spec id and partition data should be unchanged - assertThat(stats.get(0, PartitionData.class)).isEqualTo(PARTITION); - assertThat(stats.get(1, Integer.class)).isEqualTo(1); - - for (int i = 0; i < expectedValues.length; i++) { - if (expectedValues[i] == null) { - assertThat(stats.get(i + 2, Object.class)).isNull(); - } else { - assertThat(stats.get(i + 2, Object.class)).isEqualTo(expectedValues[i]); - } - } - } -} diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java index 117293629f58..04d1d212f77a 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java @@ -43,13 +43,6 @@ public void testOptionalFieldsWriting() throws Exception { .hasMessage("Cannot write using unregistered internal data format: ORC"); } - @Override - public void testPartitionStats() throws Exception { - assertThatThrownBy(super::testPartitionStats) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - @Override public void testLatestStatsFile() throws Exception { assertThatThrownBy(super::testLatestStatsFile) @@ -71,24 +64,10 @@ public void testCopyOnWriteDelete() throws Exception { .hasMessage("Cannot write using unregistered internal data format: ORC"); } - @Override - public void testReadingStatsWithInvalidSchema() { - assertThatThrownBy(super::testReadingStatsWithInvalidSchema) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - @Override public void testFullComputeFallbackWithInvalidStats() { assertThatThrownBy(super::testFullComputeFallbackWithInvalidStats) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot write using unregistered internal data format: ORC"); } - - @Override - public void testV2toV3SchemaEvolution() { - assertThatThrownBy(super::testV2toV3SchemaEvolution) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } } From 26169f73695d75cf08b6ecd69ae3d4d8f82fef8f Mon Sep 17 00:00:00 2001 From: Vova Kolmakov Date: Fri, 22 May 2026 16:39:44 +0700 Subject: [PATCH 04/58] Kafka Connect: Fix ConcurrentModificationException in IcebergSinkConfig.tableConfig (#16438) --- .../main/java/org/apache/iceberg/connect/IcebergSinkConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java index 9650ce16270c..9fcb130e4094 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java @@ -244,7 +244,7 @@ private static ConfigDef newConfigDef() { private final Map kafkaProps; private final Map autoCreateProps; private final Map writeProps; - private final Map tableConfigMap = Maps.newHashMap(); + private final Map tableConfigMap = Maps.newConcurrentMap(); private final JsonConverter jsonConverter; public IcebergSinkConfig(Map originalProps) { From ad9af85139d93487696f59faba743a4a38a558f2 Mon Sep 17 00:00:00 2001 From: Xiening Dai Date: Fri, 22 May 2026 10:46:18 -0700 Subject: [PATCH 05/58] Add `.claude/` to .gitignore (#16533) --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index 848bca2c448b..94bace4f1652 100644 --- a/.gitignore +++ b/.gitignore @@ -79,3 +79,6 @@ derby.log # git hooks like pre-commit .githooks/ + +# claude code +.claude/ From 41a95991aaaa0ec2be5770308925263ff5776e9a Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Fri, 22 May 2026 14:02:24 -0400 Subject: [PATCH 06/58] ci: only run site-ci deploy job for apache/iceberg (#16535) --- .github/workflows/site-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/site-ci.yml b/.github/workflows/site-ci.yml index fbd18caeb6da..ab873c4993bd 100644 --- a/.github/workflows/site-ci.yml +++ b/.github/workflows/site-ci.yml @@ -32,6 +32,7 @@ permissions: jobs: deploy: + if: github.repository == 'apache/iceberg' runs-on: ubuntu-slim permissions: contents: write From fca74a08c6e0a08646647dcbda9a63fe90ee88d5 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Sat, 23 May 2026 02:43:23 +0800 Subject: [PATCH 07/58] Spark: Backport Add _row_id and _last_updated_sequence_number raeder in Orc to support lineage (#16534) --- .../TestRowLevelOperationsWithLineage.java | 12 ++++++++++++ .../apache/iceberg/spark/data/SparkOrcReader.java | 2 +- .../iceberg/spark/data/SparkOrcValueReaders.java | 15 +++++++++++---- .../TestRowLevelOperationsWithLineage.java | 12 ++++++++++++ .../apache/iceberg/spark/data/SparkOrcReader.java | 2 +- .../iceberg/spark/data/SparkOrcValueReaders.java | 15 +++++++++++---- 6 files changed, 48 insertions(+), 10 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java index c1a9a26f849f..63458c64a508 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java @@ -95,6 +95,18 @@ record -> + " fanout = {6}, branch = {7}, planningMode = {8}, formatVersion = {9}") public static Object[][] parameters() { return new Object[][] { + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.ORC, + false, + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 3 + }, { "testhadoop", SparkCatalog.class.getName(), diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index c20be44f6735..c0d3d3efe026 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -77,7 +77,7 @@ public OrcValueReader record( TypeDescription record, List names, List> fields) { - return SparkOrcValueReaders.struct(fields, expected, idToConstant); + return SparkOrcValueReaders.struct(record, fields, expected, idToConstant); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java index 670537fbf872..67664ac6c753 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java @@ -28,6 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.UUIDUtil; +import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; import org.apache.orc.storage.ql.exec.vector.ColumnVector; import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; @@ -70,8 +71,11 @@ public static OrcValueReader decimals(int precision, int scale) { } static OrcValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); + TypeDescription record, + List> readers, + Types.StructType struct, + Map idToConstant) { + return new StructReader(record, readers, struct, idToConstant); } static OrcValueReader array(OrcValueReader elementReader) { @@ -143,8 +147,11 @@ static class StructReader extends OrcValueReaders.StructReader { private final int numFields; protected StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); + TypeDescription record, + List> readers, + Types.StructType struct, + Map idToConstant) { + super(record, readers, struct, idToConstant); this.numFields = struct.fields().size(); } diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java index 77303685235d..f38178a8e883 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java @@ -95,6 +95,18 @@ record -> + " fanout = {6}, branch = {7}, planningMode = {8}, formatVersion = {9}") public static Object[][] parameters() { return new Object[][] { + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.ORC, + false, + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 3 + }, { "testhadoop", SparkCatalog.class.getName(), diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index c20be44f6735..c0d3d3efe026 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -77,7 +77,7 @@ public OrcValueReader record( TypeDescription record, List names, List> fields) { - return SparkOrcValueReaders.struct(fields, expected, idToConstant); + return SparkOrcValueReaders.struct(record, fields, expected, idToConstant); } @Override diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java index 670537fbf872..67664ac6c753 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java @@ -28,6 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.UUIDUtil; +import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; import org.apache.orc.storage.ql.exec.vector.ColumnVector; import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; @@ -70,8 +71,11 @@ public static OrcValueReader decimals(int precision, int scale) { } static OrcValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); + TypeDescription record, + List> readers, + Types.StructType struct, + Map idToConstant) { + return new StructReader(record, readers, struct, idToConstant); } static OrcValueReader array(OrcValueReader elementReader) { @@ -143,8 +147,11 @@ static class StructReader extends OrcValueReaders.StructReader { private final int numFields; protected StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); + TypeDescription record, + List> readers, + Types.StructType struct, + Map idToConstant) { + super(record, readers, struct, idToConstant); this.numFields = struct.fields().size(); } From f23da211bbe108a060df30c1f65684858aea0e8d Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 22 May 2026 15:52:38 -0500 Subject: [PATCH 08/58] Core, Parquet: Allow for Writing Parquet/Avro Manifests in V4 (#15634) --- .../org/apache/iceberg/ManifestBenchmark.java | 171 +++++---------- .../apache/iceberg/ManifestBenchmarkUtil.java | 120 +++++++++++ .../iceberg/ManifestCompressionBenchmark.java | 202 ++++++++++++++++++ .../apache/iceberg/ManifestReadBenchmark.java | 173 --------------- .../iceberg/ManifestWriteBenchmark.java | 173 --------------- .../java/org/apache/iceberg/BaseFile.java | 41 +++- .../org/apache/iceberg/ManifestReader.java | 29 ++- .../org/apache/iceberg/ManifestWriter.java | 62 ++++-- .../org/apache/iceberg/SnapshotProducer.java | 7 +- .../org/apache/iceberg/TableMetadata.java | 1 + .../java/org/apache/iceberg/V4Metadata.java | 74 ++++--- .../java/org/apache/iceberg/TestBase.java | 23 +- .../org/apache/iceberg/TestFastAppend.java | 8 +- .../apache/iceberg/TestManifestReader.java | 4 + .../apache/iceberg/TestManifestWriter.java | 11 +- .../iceberg/TestManifestWriterVersions.java | 125 ++++++++++- .../org/apache/iceberg/TestMergeAppend.java | 18 +- .../apache/iceberg/TestRewriteManifests.java | 70 +++--- .../apache/iceberg/TestSnapshotProducer.java | 13 +- .../org/apache/iceberg/TestTransaction.java | 6 +- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 2 +- .../iceberg/util/TestManifestFileUtil.java | 55 +++-- .../iceberg/parquet/ParquetValueReaders.java | 18 +- .../actions/TestRewriteTablePathsAction.java | 5 +- .../actions/TestRewriteTablePathsAction.java | 5 +- .../actions/TestRewriteTablePathsAction.java | 5 +- 26 files changed, 812 insertions(+), 609 deletions(-) create mode 100644 core/src/jmh/java/org/apache/iceberg/ManifestBenchmarkUtil.java create mode 100644 core/src/jmh/java/org/apache/iceberg/ManifestCompressionBenchmark.java delete mode 100644 core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java delete mode 100644 core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestBenchmark.java index cbd372b7a4ba..b1b3847c5a7a 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestBenchmark.java @@ -18,23 +18,15 @@ */ package org.apache.iceberg; -import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; -import java.nio.ByteBuffer; -import java.nio.file.Files; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Random; import java.util.concurrent.TimeUnit; -import org.apache.commons.io.FileUtils; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.openjdk.jmh.annotations.AuxCounters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -52,7 +44,12 @@ import org.openjdk.jmh.infra.Blackhole; /** - * A benchmark that measures manifest read/write performance across compression codecs. + * A benchmark that measures manifest read/write performance across format versions and file + * formats. + * + *

V1-V3 only support Avro manifests. V4 supports both Avro and Parquet. The {@code + * versionFormat} parameter encodes valid combinations as {@code "_"} (e.g. {@code + * "4_PARQUET"}) so that only meaningful pairings are benchmarked. * *

Entry counts are calibrated per column count via {@link #ENTRY_BASE}. Set to 300_000 for ~8 MB * manifests (matching the default {@code commit.manifest.target-size-bytes}) or 15_000 for ~400 KB. @@ -63,13 +60,25 @@ * # all combinations * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestBenchmark * - * # single codec + * # V4-only (Avro vs Parquet) + * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestBenchmark \ + * -PjmhParams="versionFormat=4_AVRO|4_PARQUET" + * + * # all versions, single column count * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestBenchmark \ - * -PjmhParams="codec=gzip" + * -PjmhParams="numCols=50" + * + * # single version + * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestBenchmark \ + * -PjmhParams="versionFormat=3_AVRO" * } */ @Fork(1) @State(Scope.Benchmark) +// Parquet's columnar write path has a deep call graph (per-column encoders, page assembly, +// dictionary management) that requires more warmup iterations than Avro for the JIT compiler to +// fully optimize. Profiling shows ~650ms of JIT compilation spread across the first 3-4 +// iterations, so 6 warmups ensure measurement begins after JIT has stabilized. @Warmup(iterations = 6) @Measurement(iterations = 10) @BenchmarkMode(Mode.SingleShotTime) @@ -78,19 +87,8 @@ public class ManifestBenchmark { static final int ENTRY_BASE = 300_000; - private static final int FORMAT_VERSION = 4; - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get()), - Types.NestedField.required(3, "customer", Types.StringType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("id").identity("data").identity("customer").build(); - - @Param({"gzip", "snappy", "zstd", "uncompressed"}) - private String codec; + @Param({"1_AVRO", "2_AVRO", "3_AVRO", "4_AVRO", "4_PARQUET"}) + private String versionFormat; @Param({"true", "false"}) private String partitioned; @@ -98,11 +96,11 @@ public class ManifestBenchmark { @Param({"10", "50", "100"}) private int numCols; + private int formatVersion; + private FileFormat fileFormat; private PartitionSpec spec; private Map specsById; - private Map writerProperties; private List dataFiles; - private int numEntries; private String writeBaseDir; private OutputFile writeOutputFile; @@ -112,21 +110,26 @@ public class ManifestBenchmark { @Setup(Level.Trial) public void setupTrial() { - this.spec = Boolean.parseBoolean(partitioned) ? SPEC : PartitionSpec.unpartitioned(); - this.specsById = Map.of(spec.specId(), spec); - this.writerProperties = Map.of(TableProperties.AVRO_COMPRESSION, codec); - // ENTRY_BASE / cols: empirically calibrated — 300_000 → ~8 MB, 15_000 → ~400 KB manifests - this.numEntries = ENTRY_BASE / numCols; - this.dataFiles = generateDataFiles(); + String[] parts = versionFormat.split("_", 2); + this.formatVersion = Integer.parseInt(parts[0]); + this.fileFormat = FileFormat.fromString(parts[1]); + this.spec = + Boolean.parseBoolean(partitioned) + ? ManifestBenchmarkUtil.SPEC + : PartitionSpec.unpartitioned(); + this.specsById = ImmutableMap.of(spec.specId(), spec); + int numEntries = ManifestBenchmarkUtil.entriesForColumnCount(ENTRY_BASE, numCols); + this.dataFiles = ManifestBenchmarkUtil.generateDataFiles(spec, numEntries, numCols); setupReadManifest(); } @Setup(Level.Invocation) public void setupWriteInvocation() throws IOException { - this.writeBaseDir = Files.createTempDirectory("bench-write-").toAbsolutePath().toString(); + this.writeBaseDir = + java.nio.file.Files.createTempDirectory("bench-write-").toAbsolutePath().toString(); this.writeOutputFile = - org.apache.iceberg.Files.localOutput( - String.format(Locale.ROOT, "%s/manifest.avro", writeBaseDir)); + Files.localOutput( + String.format(Locale.ROOT, "%s/%s", writeBaseDir, fileFormat.addExtension("manifest"))); for (DataFile file : dataFiles) { file.path(); @@ -137,7 +140,7 @@ public void setupWriteInvocation() throws IOException { @TearDown(Level.Trial) public void tearDownTrial() { - cleanDir(readBaseDir); + ManifestBenchmarkUtil.cleanDir(readBaseDir); readBaseDir = null; readManifest = null; dataFiles = null; @@ -145,28 +148,15 @@ public void tearDownTrial() { @TearDown(Level.Invocation) public void tearDownInvocation() { - cleanDir(writeBaseDir); + ManifestBenchmarkUtil.cleanDir(writeBaseDir); writeBaseDir = null; writeOutputFile = null; } - @AuxCounters(AuxCounters.Type.EVENTS) - @State(Scope.Thread) - @SuppressWarnings("checkstyle:VisibilityModifier") - public static class FileSizeCounters { - public double manifestSizeMB; - - @Setup(Level.Invocation) - public void reset() { - manifestSizeMB = 0; - } - } - @Benchmark @Threads(1) - public ManifestFile writeManifest(FileSizeCounters counters) throws IOException { - ManifestWriter writer = - ManifestFiles.write(FORMAT_VERSION, spec, writeOutputFile, 1L, writerProperties); + public ManifestFile writeManifest() throws IOException { + ManifestWriter writer = ManifestFiles.write(formatVersion, spec, writeOutputFile, 1L); try (ManifestWriter w = writer) { for (DataFile file : dataFiles) { @@ -174,9 +164,7 @@ public ManifestFile writeManifest(FileSizeCounters counters) throws IOException } } - ManifestFile manifest = writer.toManifestFile(); - counters.manifestSizeMB = manifest.length() / (1024.0 * 1024.0); - return manifest; + return writer.toManifestFile(); } @Benchmark @@ -193,17 +181,17 @@ public void readManifest(Blackhole blackhole) throws IOException { private void setupReadManifest() { try { - this.readBaseDir = Files.createTempDirectory("bench-read-").toAbsolutePath().toString(); + this.readBaseDir = + java.nio.file.Files.createTempDirectory("bench-read-").toAbsolutePath().toString(); } catch (IOException e) { throw new UncheckedIOException(e); } OutputFile manifestFile = - org.apache.iceberg.Files.localOutput( - String.format(Locale.ROOT, "%s/manifest.avro", readBaseDir)); + Files.localOutput( + String.format(Locale.ROOT, "%s/%s", readBaseDir, fileFormat.addExtension("manifest"))); - ManifestWriter writer = - ManifestFiles.write(FORMAT_VERSION, spec, manifestFile, 1L, writerProperties); + ManifestWriter writer = ManifestFiles.write(formatVersion, spec, manifestFile, 1L); try (ManifestWriter w = writer) { for (DataFile file : dataFiles) { @@ -215,65 +203,4 @@ private void setupReadManifest() { this.readManifest = writer.toManifestFile(); } - - private List generateDataFiles() { - Random random = new Random(42); - List files = Lists.newArrayListWithCapacity(numEntries); - for (int i = 0; i < numEntries; i++) { - DataFiles.Builder builder = - DataFiles.builder(spec) - .withFormat(FileFormat.PARQUET) - .withPath(String.format(Locale.ROOT, "/path/to/data-%d.parquet", i)) - .withFileSizeInBytes(1024 + i) - .withRecordCount(1000 + i) - .withMetrics(randomMetrics(random, numCols)); - - if (!spec.isUnpartitioned()) { - builder.withPartitionPath( - String.format( - Locale.ROOT, "id=%d/data=val-%d/customer=cust-%d", i % 100, i % 50, i % 200)); - } - - files.add(builder.build()); - } - - return files; - } - - static Metrics randomMetrics(Random random, int cols) { - long rowCount = 100_000L + random.nextInt(1000); - Map columnSizes = Maps.newHashMap(); - Map valueCounts = Maps.newHashMap(); - Map nullValueCounts = Maps.newHashMap(); - Map nanValueCounts = Maps.newHashMap(); - Map lowerBounds = Maps.newHashMap(); - Map upperBounds = Maps.newHashMap(); - for (int i = 0; i < cols; i++) { - columnSizes.put(i, 1_000_000L + random.nextInt(100_000)); - valueCounts.put(i, 100_000L + random.nextInt(100)); - nullValueCounts.put(i, (long) random.nextInt(5)); - nanValueCounts.put(i, (long) random.nextInt(5)); - byte[] lower = new byte[8]; - random.nextBytes(lower); - lowerBounds.put(i, ByteBuffer.wrap(lower)); - byte[] upper = new byte[8]; - random.nextBytes(upper); - upperBounds.put(i, ByteBuffer.wrap(upper)); - } - - return new Metrics( - rowCount, - columnSizes, - valueCounts, - nullValueCounts, - nanValueCounts, - lowerBounds, - upperBounds); - } - - private static void cleanDir(String dir) { - if (dir != null) { - FileUtils.deleteQuietly(new File(dir)); - } - } } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestBenchmarkUtil.java b/core/src/jmh/java/org/apache/iceberg/ManifestBenchmarkUtil.java new file mode 100644 index 000000000000..d37c48daaba3 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestBenchmarkUtil.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Random; +import org.apache.commons.io.FileUtils; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +/** + * Shared constants and stateless helpers for {@link ManifestBenchmark} and {@link + * ManifestCompressionBenchmark}. + */ +final class ManifestBenchmarkUtil { + + static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.required(3, "customer", Types.StringType.get())); + + static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("id").identity("data").identity("customer").build(); + + private ManifestBenchmarkUtil() {} + + /** + * Returns the number of manifest entries for the given column count. The result is {@code + * entryBase / cols}. + * + *

The linear ratio was determined empirically by writing manifests at various column counts + * and measuring the resulting file sizes. An {@code entryBase} of 300,000 produces ~8 MB + * manifests (matching the default {@code commit.manifest.target-size-bytes}); 15,000 produces + * ~400 KB. + */ + static int entriesForColumnCount(int entryBase, int cols) { + return entryBase / cols; + } + + static List generateDataFiles(PartitionSpec spec, int numEntries, int numCols) { + Random random = new Random(42); + List files = Lists.newArrayListWithCapacity(numEntries); + for (int i = 0; i < numEntries; i++) { + DataFiles.Builder builder = + DataFiles.builder(spec) + .withFormat(FileFormat.PARQUET) + .withPath(String.format(Locale.ROOT, "/path/to/data-%d.parquet", i)) + .withFileSizeInBytes(1024 + i) + .withRecordCount(1000 + i) + .withMetrics(randomMetrics(random, numCols)); + + if (spec.isPartitioned()) { + builder.withPartitionPath( + String.format( + Locale.ROOT, "id=%d/data=val-%d/customer=cust-%d", i % 100, i % 50, i % 200)); + } + + files.add(builder.build()); + } + return files; + } + + static Metrics randomMetrics(Random random, int cols) { + long rowCount = 100_000L + random.nextInt(1000); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + for (int i = 0; i < cols; i++) { + columnSizes.put(i, 1_000_000L + random.nextInt(100_000)); + valueCounts.put(i, 100_000L + random.nextInt(100)); + nullValueCounts.put(i, (long) random.nextInt(5)); + nanValueCounts.put(i, (long) random.nextInt(5)); + byte[] lower = new byte[8]; + random.nextBytes(lower); + lowerBounds.put(i, ByteBuffer.wrap(lower)); + byte[] upper = new byte[8]; + random.nextBytes(upper); + upperBounds.put(i, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } + + static void cleanDir(String dir) { + if (dir != null) { + FileUtils.deleteQuietly(new java.io.File(dir)); + } + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestCompressionBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestCompressionBenchmark.java new file mode 100644 index 000000000000..bf09ae18f91c --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestCompressionBenchmark.java @@ -0,0 +1,202 @@ +/* + * 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; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.OutputFile; +import org.openjdk.jmh.annotations.AuxCounters; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that measures manifest read/write performance across compression codecs. + * + *

Entry counts are calibrated per column count via {@link #ENTRY_BASE}. Set to 300_000 for ~8 MB + * manifests (matching the default {@code commit.manifest.target-size-bytes}) or 15_000 for ~400 KB. + * + *

To run this benchmark: + * + *

{@code
+ * # all combinations
+ * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestCompressionBenchmark
+ *
+ * # single codec
+ * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestCompressionBenchmark \
+ *     -PjmhParams="codec=gzip"
+ * }
+ */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 6) +@Measurement(iterations = 10) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class ManifestCompressionBenchmark { + + static final int ENTRY_BASE = 300_000; + + private static final int FORMAT_VERSION = 4; + + @Param({"gzip", "snappy", "zstd", "uncompressed"}) + private String codec; + + @Param({"true", "false"}) + private String partitioned; + + @Param({"10", "50", "100"}) + private int numCols; + + private PartitionSpec spec; + private Map specsById; + private Map writerProperties; + private List dataFiles; + + private String writeBaseDir; + private OutputFile writeOutputFile; + + private String readBaseDir; + private ManifestFile readManifest; + + @Setup(Level.Trial) + public void setupTrial() { + this.spec = + Boolean.parseBoolean(partitioned) + ? ManifestBenchmarkUtil.SPEC + : PartitionSpec.unpartitioned(); + this.specsById = Map.of(spec.specId(), spec); + this.writerProperties = Map.of(TableProperties.AVRO_COMPRESSION, codec); + int numEntries = ManifestBenchmarkUtil.entriesForColumnCount(ENTRY_BASE, numCols); + this.dataFiles = ManifestBenchmarkUtil.generateDataFiles(spec, numEntries, numCols); + setupReadManifest(); + } + + @Setup(Level.Invocation) + public void setupWriteInvocation() throws IOException { + this.writeBaseDir = + java.nio.file.Files.createTempDirectory("bench-write-").toAbsolutePath().toString(); + this.writeOutputFile = + Files.localOutput(String.format(Locale.ROOT, "%s/manifest.avro", writeBaseDir)); + + for (DataFile file : dataFiles) { + file.path(); + file.fileSizeInBytes(); + file.recordCount(); + } + } + + @TearDown(Level.Trial) + public void tearDownTrial() { + ManifestBenchmarkUtil.cleanDir(readBaseDir); + readBaseDir = null; + readManifest = null; + dataFiles = null; + } + + @TearDown(Level.Invocation) + public void tearDownInvocation() { + ManifestBenchmarkUtil.cleanDir(writeBaseDir); + writeBaseDir = null; + writeOutputFile = null; + } + + @AuxCounters(AuxCounters.Type.EVENTS) + @State(Scope.Thread) + @SuppressWarnings("checkstyle:VisibilityModifier") + public static class FileSizeCounters { + public double manifestSizeMB; + + @Setup(Level.Invocation) + public void reset() { + manifestSizeMB = 0; + } + } + + @Benchmark + @Threads(1) + public ManifestFile writeManifest(FileSizeCounters counters) throws IOException { + ManifestWriter writer = + ManifestFiles.write(FORMAT_VERSION, spec, writeOutputFile, 1L, writerProperties); + + try (ManifestWriter w = writer) { + for (DataFile file : dataFiles) { + w.add(file); + } + } + + ManifestFile manifest = writer.toManifestFile(); + counters.manifestSizeMB = manifest.length() / (1024.0 * 1024.0); + return manifest; + } + + @Benchmark + @Threads(1) + public void readManifest(Blackhole blackhole) throws IOException { + TestTables.LocalFileIO fileIO = new TestTables.LocalFileIO(); + try (CloseableIterator it = + ManifestFiles.read(readManifest, fileIO, specsById).iterator()) { + while (it.hasNext()) { + blackhole.consume(it.next()); + } + } + } + + private void setupReadManifest() { + try { + this.readBaseDir = + java.nio.file.Files.createTempDirectory("bench-read-").toAbsolutePath().toString(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + OutputFile manifestFile = + Files.localOutput(String.format(Locale.ROOT, "%s/manifest.avro", readBaseDir)); + + ManifestWriter writer = + ManifestFiles.write(FORMAT_VERSION, spec, manifestFile, 1L, writerProperties); + + try (ManifestWriter w = writer) { + for (DataFile file : dataFiles) { + w.add(file); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + this.readManifest = writer.toManifestFile(); + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java deleted file mode 100644 index 588b5df1ba97..000000000000 --- a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java +++ /dev/null @@ -1,173 +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; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.ByteBuffer; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Timeout; - -@Fork(1) -@State(Scope.Benchmark) -@Measurement(iterations = 5) -@BenchmarkMode(Mode.SingleShotTime) -@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) -public class ManifestReadBenchmark { - - private static final int NUM_FILES = 10; - private static final int NUM_ROWS = 100000; - private static final int NUM_COLS = 10; - - private String baseDir; - private String manifestListFile; - - @Setup - public void before() { - baseDir = - Paths.get(new File(System.getProperty("java.io.tmpdir")).getAbsolutePath()).toString(); - manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); - - Random random = new Random(System.currentTimeMillis()); - - try (ManifestListWriter listWriter = - ManifestLists.write( - 1, - org.apache.iceberg.Files.localOutput(manifestListFile), - PlaintextEncryptionManager.instance(), - 0, - 1L, - 0, - 0L)) { - for (int i = 0; i < NUM_FILES; i++) { - OutputFile manifestFile = - org.apache.iceberg.Files.localOutput( - String.format("%s/%s.avro", baseDir, UUID.randomUUID())); - - ManifestWriter writer = - ManifestFiles.write(1, PartitionSpec.unpartitioned(), manifestFile, 1L); - try (ManifestWriter finalWriter = writer) { - for (int j = 0; j < NUM_ROWS; j++) { - DataFile dataFile = - DataFiles.builder(PartitionSpec.unpartitioned()) - .withFormat(FileFormat.PARQUET) - .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) - .withFileSizeInBytes(j) - .withRecordCount(j) - .withMetrics(randomMetrics(random)) - .build(); - finalWriter.add(dataFile); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - - listWriter.add(writer.toManifestFile()); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @TearDown - public void after() throws IOException { - if (baseDir != null) { - try (Stream walk = Files.walk(Paths.get(baseDir))) { - walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); - } - baseDir = null; - } - - manifestListFile = null; - } - - @Benchmark - @Threads(1) - public void readManifestFile() throws IOException { - List manifests = - ManifestLists.read(org.apache.iceberg.Files.localInput(manifestListFile)); - TestTables.LocalFileIO fileIO = new TestTables.LocalFileIO(); - Map specs = - ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); - for (ManifestFile manifestFile : manifests) { - ManifestReader reader = ManifestFiles.read(manifestFile, fileIO, specs); - try (CloseableIterator it = reader.iterator()) { - while (it.hasNext()) { - it.next().recordCount(); - } - } - } - } - - private Metrics randomMetrics(Random random) { - long rowCount = 100000L + random.nextInt(1000); - Map columnSizes = Maps.newHashMap(); - Map valueCounts = Maps.newHashMap(); - Map nullValueCounts = Maps.newHashMap(); - Map nanValueCounts = Maps.newHashMap(); - Map lowerBounds = Maps.newHashMap(); - Map upperBounds = Maps.newHashMap(); - for (int i = 0; i < NUM_COLS; i++) { - columnSizes.put(i, 1000000L + random.nextInt(100000)); - valueCounts.put(i, 100000L + random.nextInt(100)); - nullValueCounts.put(i, (long) random.nextInt(5)); - nanValueCounts.put(i, (long) random.nextInt(5)); - byte[] lower = new byte[8]; - random.nextBytes(lower); - lowerBounds.put(i, ByteBuffer.wrap(lower)); - byte[] upper = new byte[8]; - random.nextBytes(upper); - upperBounds.put(i, ByteBuffer.wrap(upper)); - } - - return new Metrics( - rowCount, - columnSizes, - valueCounts, - nullValueCounts, - nanValueCounts, - lowerBounds, - upperBounds); - } -} diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java deleted file mode 100644 index b0dab63dea06..000000000000 --- a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java +++ /dev/null @@ -1,173 +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; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import org.apache.commons.io.FileUtils; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Timeout; - -/** - * A benchmark that evaluates the performance of writing manifest files - * - *

To run this benchmark: - * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestWriteBenchmark - * - */ -@Fork(1) -@State(Scope.Benchmark) -@Measurement(iterations = 5) -@BenchmarkMode(Mode.SingleShotTime) -@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) -public class ManifestWriteBenchmark { - - private static final int NUM_FILES = 10; - private static final int NUM_ROWS = 100000; - private static final int NUM_COLS = 100; - - private String baseDir; - private String manifestListFile; - - private Metrics metrics; - - @Setup - public void before() { - Random random = new Random(System.currentTimeMillis()); - // Pre-create the metrics to avoid doing this in the benchmark itself - metrics = randomMetrics(random); - } - - @TearDown - public void after() { - if (baseDir != null) { - FileUtils.deleteQuietly(new File(baseDir)); - baseDir = null; - } - - manifestListFile = null; - } - - @State(Scope.Benchmark) - public static class BenchmarkState { - @Param({"1", "2"}) - private int formatVersion; - - public int getFormatVersion() { - return formatVersion; - } - } - - @Benchmark - @Threads(1) - public void writeManifestFile(BenchmarkState state) throws IOException { - this.baseDir = - java.nio.file.Files.createTempDirectory("benchmark-").toAbsolutePath().toString(); - this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); - - try (ManifestListWriter listWriter = - ManifestLists.write( - state.getFormatVersion(), - org.apache.iceberg.Files.localOutput(manifestListFile), - PlaintextEncryptionManager.instance(), - 0, - 1L, - 0, - 0L)) { - for (int i = 0; i < NUM_FILES; i++) { - OutputFile manifestFile = - org.apache.iceberg.Files.localOutput( - String.format("%s/%s.avro", baseDir, UUID.randomUUID())); - - ManifestWriter writer = - ManifestFiles.write( - state.formatVersion, PartitionSpec.unpartitioned(), manifestFile, 1L); - try (ManifestWriter finalWriter = writer) { - for (int j = 0; j < NUM_ROWS; j++) { - DataFile dataFile = - DataFiles.builder(PartitionSpec.unpartitioned()) - .withFormat(FileFormat.PARQUET) - .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) - .withFileSizeInBytes(j) - .withRecordCount(j) - .withMetrics(metrics) - .build(); - finalWriter.add(dataFile); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - - listWriter.add(writer.toManifestFile()); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private Metrics randomMetrics(Random random) { - long rowCount = 100000L + random.nextInt(1000); - Map columnSizes = Maps.newHashMap(); - Map valueCounts = Maps.newHashMap(); - Map nullValueCounts = Maps.newHashMap(); - Map nanValueCounts = Maps.newHashMap(); - Map lowerBounds = Maps.newHashMap(); - Map upperBounds = Maps.newHashMap(); - for (int i = 0; i < NUM_COLS; i++) { - columnSizes.put(i, 1000000L + random.nextInt(100000)); - valueCounts.put(i, 100000L + random.nextInt(100)); - nullValueCounts.put(i, (long) random.nextInt(5)); - nanValueCounts.put(i, (long) random.nextInt(5)); - byte[] lower = new byte[8]; - random.nextBytes(lower); - lowerBounds.put(i, ByteBuffer.wrap(lower)); - byte[] upper = new byte[8]; - random.nextBytes(upper); - upperBounds.put(i, ByteBuffer.wrap(upper)); - } - - return new Metrics( - rowCount, - columnSizes, - valueCounts, - nullValueCounts, - nanValueCounts, - lowerBounds, - upperBounds); - } -} diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index 3c31c50f099f..7147ba58787b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -32,6 +32,7 @@ import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.avro.SupportsIndexProjection; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; @@ -329,7 +330,11 @@ protected void internalSet(int pos, T value) { this.partitionSpecId = (value != null) ? (Integer) value : -1; return; case 4: - this.partitionData = (PartitionData) value; + // Preserve the constructor-initialized partitionData when the reader returns null + // (e.g., v4 Parquet manifests for unpartitioned tables omit the partition field). + if (value != null) { + this.partitionData = (PartitionData) value; + } return; case 5: this.recordCount = (Long) value; @@ -581,9 +586,37 @@ private static Map copyMap(Map map, Set keys) { private static Map copyByteBufferMap( Map map, Set keys) { - return SerializableByteBufferMap.wrap(copyMap(map, keys)); + if (map == null) { + return null; + } + + return SerializableByteBufferMap.wrap(deepCopyByteBufferMap(map, keys)); + } + + // Required as long as we have Map in the API since Parquet reuses buffers. + private static Map deepCopyByteBufferMap( + Map map, Set keys) { + Map deepCopy = Maps.newHashMapWithExpectedSize(map.size()); + for (Map.Entry entry : map.entrySet()) { + if (keys == null || keys.contains(entry.getKey())) { + ByteBuffer buf = entry.getValue(); + if (buf != null) { + ByteBuffer copy = ByteBuffer.allocate(buf.remaining()); + copy.put(buf.duplicate()); + copy.flip(); + deepCopy.put(entry.getKey(), copy); + } else { + deepCopy.put(entry.getKey(), null); + } + } + } + + return deepCopy; } + // Returns an unmodifiable view of the map. The SerializableMap check is needed because + // internal maps may be wrapped for serialization after being populated by a format reader + // with container reuse enabled, and immutableMap() provides a stable snapshot. private static Map toReadableMap(Map map) { if (map == null) { return null; @@ -594,6 +627,10 @@ private static Map toReadableMap(Map map) { } } + // Separate from toReadableMap because SerializableByteBufferMap is its own wrapper type + // (not a SerializableMap subclass) to handle ByteBuffer-specific serialization. ByteBuffer + // values are mutable and can be overwritten by Parquet container reuse, so callers that + // retain references must use copyByteBufferMap to get independent copies. private static Map toReadableByteBufferMap(Map map) { if (map == null) { return null; diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index e3c2325ab780..dc34836b6c56 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -60,6 +60,13 @@ public class ManifestReader> extends CloseableGroup static final ImmutableList ALL_COLUMNS = ImmutableList.of("*"); + private static final Types.NestedField UNPARTITIONED_PARTITION_FIELD = + Types.NestedField.optional( + DataFile.PARTITION_ID, + DataFile.PARTITION_NAME, + Types.StructType.of(), + DataFile.PARTITION_DOC); + private static final Set STATS_COLUMNS = ImmutableSet.of( "value_counts", @@ -173,6 +180,12 @@ private > PartitionSpec readPartitionSpec(InputFile inp } private static > Map readMetadata(InputFile inputFile) { + FileFormat manifestFormat = FileFormat.fromFileName(inputFile.location()); + Preconditions.checkArgument( + manifestFormat == FileFormat.AVRO, + "Reading manifest metadata is only supported for Avro manifests: %s", + inputFile.location()); + Map metadata; try { try (CloseableIterable> headerReader = @@ -298,8 +311,22 @@ private CloseableIterable> open(Schema projection) { Preconditions.checkArgument( format != null, "Unable to determine format of manifest: %s", file.location()); + boolean unpartitioned = spec.rawPartitionType().fields().isEmpty(); + + // V4+ manifests omit the partition field when unpartitioned (Parquet cannot represent + // empty structs, and the field is meaningless regardless of format). Mark it optional so + // the reader returns null for the missing field instead of throwing. The field must stay + // in the projection to preserve positional access for callers like StructProjection. + // For older versions where the empty struct is present, making it optional is harmless. List fields = Lists.newArrayList(); - fields.addAll(projection.asStruct().fields()); + for (Types.NestedField field : projection.asStruct().fields()) { + if (unpartitioned && field.fieldId() == DataFile.PARTITION_ID) { + fields.add(UNPARTITIONED_PARTITION_FIELD); + } else { + fields.add(field); + } + } + if (projection.findField(DataFile.RECORD_COUNT.fieldId()) == null) { fields.add(DataFile.RECORD_COUNT); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 7d85f991b080..321bcd89d8b1 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -24,6 +24,7 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptionKeyMetadata; import org.apache.iceberg.encryption.NativeEncryptionKeyMetadata; +import org.apache.iceberg.encryption.NativeEncryptionOutputFile; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; @@ -40,6 +41,7 @@ public abstract class ManifestWriter> implements FileAp // this is replaced when writing a manifest list by the ManifestFile wrapper static final long UNASSIGNED_SEQ = -1L; + private final FileFormat format; private final OutputFile file; private final EncryptionKeyMetadata keyMetadata; private final int specId; @@ -65,7 +67,8 @@ private ManifestWriter( Long snapshotId, Long firstRowId, Map writerProperties) { - this.file = file.encryptingOutputFile(); + this.format = FileFormat.fromFileName(file.encryptingOutputFile().location()); + this.file = outputFile(file); this.specId = spec.specId(); this.writerProperties = writerProperties; this.writer = newAppender(spec, this.file); @@ -82,6 +85,21 @@ private ManifestWriter( protected abstract FileAppender> newAppender( PartitionSpec spec, OutputFile outputFile); + private OutputFile outputFile(EncryptedOutputFile encryptedFile) { + // Casting to NativeEncryptionOutputFile actually makes the file rely on native encryption + // rather than whole-file encryption. + if (format == FileFormat.PARQUET + && encryptedFile instanceof NativeEncryptionOutputFile nativeFile) { + return nativeFile; + } + + return encryptedFile.encryptingOutputFile(); + } + + protected FileFormat format() { + return format; + } + protected Map writerProperties() { return writerProperties; } @@ -206,16 +224,7 @@ public long length() { public ManifestFile toManifestFile() { Preconditions.checkState(closed, "Cannot build ManifestFile, writer is not closed"); - ByteBuffer keyMetadataBuffer; - if (keyMetadata instanceof NativeEncryptionKeyMetadata) { - // File length is required by AES GCM Stream encryption, to prevent file truncation attacks - keyMetadataBuffer = - ((NativeEncryptionKeyMetadata) keyMetadata).copyWithLength(length()).buffer(); - } else if (keyMetadata != null) { - keyMetadataBuffer = keyMetadata.buffer(); - } else { - keyMetadataBuffer = null; - } + ByteBuffer keyMetadataBuffer = keyMetadataBuffer(); // if the minSequenceNumber is null, then no manifests with a sequence number have been written, // so the min data sequence number is the one that will be assigned when this is committed. @@ -240,6 +249,19 @@ public ManifestFile toManifestFile() { firstRowId); } + private ByteBuffer keyMetadataBuffer() { + if (keyMetadata instanceof NativeEncryptionKeyMetadata nativeKeyMetadata + && format == FileFormat.AVRO) { + // Whole-file encryption needs the file length embedded for GCM truncation protection. + // Formats with native encryption (like Parquet) handle this directly and don't need it. + return nativeKeyMetadata.copyWithLength(length()).buffer(); + } else if (keyMetadata != null) { + return keyMetadata.buffer(); + } + + return null; + } + @Override public void close() throws IOException { this.closed = true; @@ -256,7 +278,7 @@ static class V4Writer extends ManifestWriter { Long firstRowId, Map writerProperties) { super(spec, file, snapshotId, firstRowId, writerProperties); - this.entryWrapper = new V4Metadata.ManifestEntryWrapper<>(snapshotId); + this.entryWrapper = new V4Metadata.ManifestEntryWrapper<>(snapshotId, spec.partitionType()); } @Override @@ -269,7 +291,7 @@ protected FileAppender> newAppender( PartitionSpec spec, OutputFile file) { Schema manifestSchema = V4Metadata.entrySchema(spec.partitionType()); try { - return InternalData.write(FileFormat.AVRO, file) + return InternalData.write(format(), file) .schema(manifestSchema) .named("manifest_entry") .meta("schema", SchemaParser.toJson(spec.schema())) @@ -296,7 +318,7 @@ static class V4DeleteWriter extends ManifestWriter { Long snapshotId, Map writerProperties) { super(spec, file, snapshotId, null, writerProperties); - this.entryWrapper = new V4Metadata.ManifestEntryWrapper<>(snapshotId); + this.entryWrapper = new V4Metadata.ManifestEntryWrapper<>(snapshotId, spec.partitionType()); } @Override @@ -309,7 +331,7 @@ protected FileAppender> newAppender( PartitionSpec spec, OutputFile file) { Schema manifestSchema = V4Metadata.entrySchema(spec.partitionType()); try { - return InternalData.write(FileFormat.AVRO, file) + return InternalData.write(format(), file) .schema(manifestSchema) .named("manifest_entry") .meta("schema", SchemaParser.toJson(spec.schema())) @@ -342,6 +364,8 @@ static class V3Writer extends ManifestWriter { Long firstRowId, Map writerProperties) { super(spec, file, snapshotId, firstRowId, writerProperties); + Preconditions.checkArgument( + format() == FileFormat.AVRO, "V3 manifests must use Avro, but got: %s", format()); this.entryWrapper = new V3Metadata.ManifestEntryWrapper<>(snapshotId); } @@ -382,6 +406,8 @@ static class V3DeleteWriter extends ManifestWriter { Long snapshotId, Map writerProperties) { super(spec, file, snapshotId, null, writerProperties); + Preconditions.checkArgument( + format() == FileFormat.AVRO, "V3 manifests must use Avro, but got: %s", format()); this.entryWrapper = new V3Metadata.ManifestEntryWrapper<>(snapshotId); } @@ -427,6 +453,8 @@ static class V2Writer extends ManifestWriter { Long snapshotId, Map writerProperties) { super(spec, file, snapshotId, null, writerProperties); + Preconditions.checkArgument( + format() == FileFormat.AVRO, "V2 manifests must use Avro, but got: %s", format()); this.entryWrapper = new V2Metadata.ManifestEntryWrapper<>(snapshotId); } @@ -467,6 +495,8 @@ static class V2DeleteWriter extends ManifestWriter { Long snapshotId, Map writerProperties) { super(spec, file, snapshotId, null, writerProperties); + Preconditions.checkArgument( + format() == FileFormat.AVRO, "V2 manifests must use Avro, but got: %s", format()); this.entryWrapper = new V2Metadata.ManifestEntryWrapper<>(snapshotId); } @@ -512,6 +542,8 @@ static class V1Writer extends ManifestWriter { Long snapshotId, Map writerProperties) { super(spec, file, snapshotId, null, writerProperties); + Preconditions.checkArgument( + format() == FileFormat.AVRO, "V1 manifests must use Avro, but got: %s", format()); this.entryWrapper = new V1Metadata.ManifestEntryWrapper(); } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 6ba10e8049f6..e351009a9ea6 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -113,6 +113,7 @@ public void accept(String file) { private final AtomicInteger attempt = new AtomicInteger(0); private final List manifestLists = Lists.newArrayList(); private final long targetManifestSizeBytes; + private final FileFormat manifestFormat; private final Map manifestWriterProps; private MetricsReporter reporter = LoggingMetricsReporter.instance(); private volatile Long snapshotId = null; @@ -142,6 +143,10 @@ protected SnapshotProducer(TableOperations ops) { this.targetManifestSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + this.manifestFormat = + ops.current().formatVersion() >= TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS + ? FileFormat.PARQUET + : FileFormat.AVRO; this.manifestWriterProps = manifestWriterProperties(ops.current()); boolean snapshotIdInheritanceEnabled = ops.current() @@ -603,7 +608,7 @@ protected OutputFile manifestListPath() { protected EncryptedOutputFile newManifestOutputFile() { String manifestFileLocation = ops.metadataFileLocation( - FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement())); + manifestFormat.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement())); return EncryptingFileIO.combine(ops.io(), ops.encryption()) .newEncryptingOutputFile(manifestFileLocation); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 43a67dd2bef2..c4a7bfc5c83c 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -57,6 +57,7 @@ public class TableMetadata implements Serializable { static final int DEFAULT_TABLE_FORMAT_VERSION = 2; static final int SUPPORTED_TABLE_FORMAT_VERSION = 4; static final int MIN_FORMAT_VERSION_ROW_LINEAGE = 3; + static final int MIN_FORMAT_VERSION_PARQUET_MANIFESTS = 4; static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; diff --git a/core/src/main/java/org/apache/iceberg/V4Metadata.java b/core/src/main/java/org/apache/iceberg/V4Metadata.java index 67478290aa10..06fc75213df0 100644 --- a/core/src/main/java/org/apache/iceberg/V4Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V4Metadata.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.List; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; class V4Metadata { @@ -278,28 +279,38 @@ static Schema wrapFileSchema(Types.StructType fileSchema) { } static Types.StructType fileType(Types.StructType partitionType) { - return Types.StructType.of( - DataFile.CONTENT.asRequired(), - DataFile.FILE_PATH, - DataFile.FILE_FORMAT, - required( - DataFile.PARTITION_ID, DataFile.PARTITION_NAME, partitionType, DataFile.PARTITION_DOC), - DataFile.RECORD_COUNT, - DataFile.FILE_SIZE, - DataFile.COLUMN_SIZES, - DataFile.VALUE_COUNTS, - DataFile.NULL_VALUE_COUNTS, - DataFile.NAN_VALUE_COUNTS, - DataFile.LOWER_BOUNDS, - DataFile.UPPER_BOUNDS, - DataFile.KEY_METADATA, - DataFile.SPLIT_OFFSETS, - DataFile.EQUALITY_IDS, - DataFile.SORT_ORDER_ID, - DataFile.FIRST_ROW_ID, - DataFile.REFERENCED_DATA_FILE, - DataFile.CONTENT_OFFSET, - DataFile.CONTENT_SIZE); + // Parquet cannot represent empty groups, so the partition field is omitted entirely from + // the file schema for unpartitioned tables. DataFileWrapper adjusts positions to match. + ImmutableList.Builder fields = + ImmutableList.builderWithExpectedSize(partitionType.fields().isEmpty() ? 18 : 19); + fields.add(DataFile.CONTENT.asRequired()); + fields.add(DataFile.FILE_PATH); + fields.add(DataFile.FILE_FORMAT); + if (!partitionType.fields().isEmpty()) { + fields.add( + required( + DataFile.PARTITION_ID, + DataFile.PARTITION_NAME, + partitionType, + DataFile.PARTITION_DOC)); + } + fields.add(DataFile.RECORD_COUNT); + fields.add(DataFile.FILE_SIZE); + fields.add(DataFile.COLUMN_SIZES); + fields.add(DataFile.VALUE_COUNTS); + fields.add(DataFile.NULL_VALUE_COUNTS); + fields.add(DataFile.NAN_VALUE_COUNTS); + fields.add(DataFile.LOWER_BOUNDS); + fields.add(DataFile.UPPER_BOUNDS); + fields.add(DataFile.KEY_METADATA); + fields.add(DataFile.SPLIT_OFFSETS); + fields.add(DataFile.EQUALITY_IDS); + fields.add(DataFile.SORT_ORDER_ID); + fields.add(DataFile.FIRST_ROW_ID); + fields.add(DataFile.REFERENCED_DATA_FILE); + fields.add(DataFile.CONTENT_OFFSET); + fields.add(DataFile.CONTENT_SIZE); + return Types.StructType.of(fields.build()); } static class ManifestEntryWrapper> @@ -309,10 +320,10 @@ static class ManifestEntryWrapper> private final DataFileWrapper fileWrapper; private ManifestEntry wrapped = null; - ManifestEntryWrapper(Long commitSnapshotId) { - this.size = entrySchema(Types.StructType.of()).columns().size(); + ManifestEntryWrapper(Long commitSnapshotId, Types.StructType partitionType) { + this.size = entrySchema(partitionType).columns().size(); this.commitSnapshotId = commitSnapshotId; - this.fileWrapper = new DataFileWrapper<>(); + this.fileWrapper = new DataFileWrapper<>(partitionType); } public ManifestEntryWrapper wrap(ManifestEntry entry) { @@ -423,11 +434,15 @@ public ManifestEntry copyWithoutStats() { /** Wrapper used to write DataFile or DeleteFile to v4 metadata. */ static class DataFileWrapper> extends Delegates.DelegatingContentFile implements ContentFile, StructLike { + private static final int PARTITION_POSITION = 3; + private final int size; + private final boolean hasPartition; - DataFileWrapper() { + DataFileWrapper(Types.StructType partitionType) { super(null); - this.size = fileType(Types.StructType.of()).fields().size(); + this.hasPartition = !partitionType.fields().isEmpty(); + this.size = fileType(partitionType).fields().size(); } @SuppressWarnings("unchecked") @@ -452,7 +467,10 @@ public T get(int pos, Class javaClass) { } private Object get(int pos) { - switch (pos) { + // when the partition field is omitted, positions at or after where it would appear + // shift down by 1, so adjust back to the canonical field ordering + int adjusted = hasPartition ? pos : (pos >= PARTITION_POSITION ? pos + 1 : pos); + switch (adjusted) { case 0: return wrapped.content().id(); case 1: diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 27b8a49d0497..0f649cabeb81 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -263,7 +263,8 @@ List listManifestFiles(File tableDirToList) { .listFiles( (dir, name) -> !name.startsWith("snap") - && Files.getFileExtension(name).equalsIgnoreCase("avro"))); + && (Files.getFileExtension(name).equalsIgnoreCase("avro") + || Files.getFileExtension(name).equalsIgnoreCase("parquet")))); } List listManifestLists(File tableDirToList) { @@ -297,12 +298,22 @@ public TableMetadata readMetadata() { return TestTables.readMetadata("test"); } + static FileFormat manifestFormat(int version) { + return version >= TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS + ? FileFormat.PARQUET + : FileFormat.AVRO; + } + + FileFormat manifestFormat() { + return manifestFormat(formatVersion); + } + ManifestFile writeManifest(DataFile... files) throws IOException { return writeManifest(null, files); } ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOException { - File manifestFile = temp.resolve("input.m0.avro").toFile(); + File manifestFile = temp.resolve(manifestFormat().addExtension("input.m0")).toFile(); assertThat(manifestFile).doesNotExist(); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); @@ -324,7 +335,7 @@ ManifestFile writeManifest(String fileName, ManifestEntry... entries) throws } ManifestFile writeManifest(Long snapshotId, ManifestEntry... entries) throws IOException { - return writeManifest(snapshotId, "input.m0.avro", entries); + return writeManifest(snapshotId, manifestFormat().addExtension("input.m0"), entries); } @SuppressWarnings("unchecked") @@ -360,8 +371,8 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi throws IOException { OutputFile manifestFile = org.apache.iceberg.Files.localOutput( - FileFormat.AVRO.addExtension( - temp.resolve("junit" + System.nanoTime()).toFile().toString())); + manifestFormat(newFormatVersion) + .addExtension(temp.resolve("junit" + System.nanoTime()).toFile().toString())); ManifestWriter writer = ManifestFiles.writeDeleteManifest(newFormatVersion, SPEC, manifestFile, snapshotId); try { @@ -375,7 +386,7 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi } ManifestFile writeManifestWithName(String name, DataFile... files) throws IOException { - File manifestFile = temp.resolve(name + ".avro").toFile(); + File manifestFile = temp.resolve(manifestFormat().addExtension(name)).toFile(); assertThat(manifestFile).doesNotExist(); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 8f427525e214..bc28ecd88022 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -509,14 +509,18 @@ public void testInvalidAppendManifest() throws IOException { assertThat(base.currentSnapshot()).isNull(); ManifestFile manifestWithExistingFiles = - writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); + writeManifest( + manifestFormat().addExtension("manifest-file-1"), + manifestEntry(Status.EXISTING, null, FILE_A)); assertThatThrownBy( () -> table.newFastAppend().appendManifest(manifestWithExistingFiles).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot append manifest with existing files"); ManifestFile manifestWithDeletedFiles = - writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); + writeManifest( + manifestFormat().addExtension("manifest-file-2"), + manifestEntry(Status.DELETED, null, FILE_A)); assertThatThrownBy( () -> table.newFastAppend().appendManifest(manifestWithDeletedFiles).commit()) .isInstanceOf(IllegalArgumentException.class) diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 6690a1483e53..de2b7fd859e6 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -273,6 +273,10 @@ public void testDataFileSplitOffsetsNullWhenInvalid() throws IOException { @SuppressWarnings("deprecation") @TestTemplate public void testDeprecatedReadWithoutSpecsById() throws IOException { + assumeThat(formatVersion) + .as("Deprecated read without specsById requires Avro metadata; V4 uses Parquet") + .isLessThan(TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS); + ManifestFile manifest = writeManifest(1000L, manifestEntry(Status.EXISTING, 1000L, FILE_A)); try (ManifestReader reader = ManifestFiles.read(manifest, FILE_IO)) { ManifestEntry entry = Iterables.getOnlyElement(reader.entries()); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 00e66bdd7d0d..d710d949c52b 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -41,7 +41,7 @@ public class TestManifestWriter extends TestBase { public void testManifestStats() throws IOException { ManifestFile manifest = writeManifest( - "manifest.avro", + manifestFormat().addExtension("manifest"), manifestEntry(Status.ADDED, null, newFile(10)), manifestEntry(Status.ADDED, null, newFile(20)), manifestEntry(Status.ADDED, null, newFile(5)), @@ -67,7 +67,7 @@ public void testManifestStats() throws IOException { public void testManifestPartitionStats() throws IOException { ManifestFile manifest = writeManifest( - "manifest.avro", + manifestFormat().addExtension("manifest"), manifestEntry(Status.ADDED, null, newFile(10, TestHelpers.Row.of(1))), manifestEntry(Status.EXISTING, null, newFile(15, TestHelpers.Row.of(2))), manifestEntry(Status.DELETED, null, newFile(2, TestHelpers.Row.of(3)))); @@ -92,7 +92,8 @@ public void testManifestPartitionStats() throws IOException { @TestTemplate public void testWriteManifestWithSequenceNumber() throws IOException { assumeThat(formatVersion).isGreaterThan(1); - File manifestFile = temp.resolve("manifest" + System.nanoTime() + ".avro").toFile(); + File manifestFile = + temp.resolve(manifestFormat().addExtension("manifest" + System.nanoTime())).toFile(); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L); @@ -119,7 +120,7 @@ public void testCommitManifestWithExplicitDataSequenceNumber() throws IOExceptio ManifestFile manifest = writeManifest( - "manifest.avro", + manifestFormat().addExtension("manifest"), manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file1), manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file2)); @@ -161,7 +162,7 @@ public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() thr ManifestFile newManifest = writeManifest( - "manifest.avro", + manifestFormat().addExtension("manifest"), manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file1), manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file2)); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 5e83827f0c45..966b573bd93b 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -115,6 +115,8 @@ public class TestManifestWriterVersions { null, null); + static final List V4_FORMATS = ImmutableList.of(FileFormat.AVRO, FileFormat.PARQUET); + @TempDir private Path temp; @Test @@ -344,6 +346,100 @@ public void testCustomManifestCompression(int formatVersion) throws IOException assertThat(readAvroCodec(manifestFile)).isEqualTo("snappy"); } + @ParameterizedTest + @FieldSource("V4_FORMATS") + public void testV4WritePartitioned(FileFormat fileFormat) throws IOException { + ManifestFile manifest = writeManifest(4, fileFormat, SPEC, DATA_FILE); + checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); + checkEntry( + readManifest(manifest), + ManifestWriter.UNASSIGNED_SEQ, + ManifestWriter.UNASSIGNED_SEQ, + FileContent.DATA, + FIRST_ROW_ID); + } + + @ParameterizedTest + @FieldSource("V4_FORMATS") + public void testV4WriteUnpartitioned(FileFormat fileFormat) throws IOException { + DataFile unpartitionedFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(PATH) + .withFormat(FORMAT) + .withFileSizeInBytes(150972L) + .withMetrics(METRICS) + .withSplitOffsets(OFFSETS) + .withSortOrderId(SORT_ORDER_ID) + .withFirstRowId(FIRST_ROW_ID) + .build(); + + ManifestFile manifest = + writeManifest(4, fileFormat, PartitionSpec.unpartitioned(), unpartitionedFile); + checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); + + Map unpartitionedSpecs = + ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); + try (CloseableIterable> reader = + ManifestFiles.read(manifest, io, unpartitionedSpecs).entries()) { + ManifestEntry entry = Iterables.getOnlyElement(reader); + assertThat(entry.status()).isEqualTo(ManifestEntry.Status.ADDED); + assertThat(entry.file().location()).isEqualTo(PATH); + assertThat(entry.file().recordCount()).isEqualTo(METRICS.recordCount()); + assertThat(entry.file().firstRowId()).isEqualTo(FIRST_ROW_ID); + } + } + + @ParameterizedTest + @FieldSource("V4_FORMATS") + public void testV4WriteDeletePartitioned(FileFormat fileFormat) throws IOException { + ManifestFile manifest = writeDeleteManifest(4, fileFormat, SPEC); + checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); + assertThat(manifest.content()).isEqualTo(ManifestContent.DELETES); + checkEntry( + readDeleteManifest(manifest), + ManifestWriter.UNASSIGNED_SEQ, + ManifestWriter.UNASSIGNED_SEQ, + FileContent.EQUALITY_DELETES); + } + + @ParameterizedTest + @FieldSource("V4_FORMATS") + public void testV4WriteDeleteUnpartitioned(FileFormat fileFormat) throws IOException { + DeleteFile unpartitionedDelete = + new GenericDeleteFile( + 0, + FileContent.EQUALITY_DELETES, + PATH, + FORMAT, + new PartitionData(PartitionSpec.unpartitioned().partitionType()), + 22905L, + METRICS, + EQUALITY_ID_ARR, + SORT_ORDER_ID, + null, + null, + null, + null, + null); + + ManifestFile manifest = + writeDeleteManifest(4, fileFormat, PartitionSpec.unpartitioned(), unpartitionedDelete); + checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); + assertThat(manifest.content()).isEqualTo(ManifestContent.DELETES); + + Map unpartitionedSpecs = + ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); + try (CloseableIterable> reader = + ManifestFiles.readDeleteManifest(manifest, io, unpartitionedSpecs).entries()) { + ManifestEntry entry = Iterables.getOnlyElement(reader); + assertThat(entry.status()).isEqualTo(ManifestEntry.Status.ADDED); + assertThat(entry.file().content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(entry.file().location()).isEqualTo(PATH); + assertThat(entry.file().recordCount()).isEqualTo(METRICS.recordCount()); + assertThat(entry.file().equalityFieldIds()).isEqualTo(EQUALITY_IDS); + } + } + void checkEntry( ManifestEntry entry, Long expectedDataSequenceNumber, @@ -466,7 +562,7 @@ private ManifestFile writeAndReadManifestList(ManifestFile manifest, int formatV private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) throws IOException { - String filename = FileFormat.AVRO.addExtension("rewrite-manifest"); + String filename = TestBase.manifestFormat(formatVersion).addExtension("rewrite-manifest"); EncryptedOutputFile manifestFile = encryptionManager().encrypt(io.newOutputFile(filename)); ManifestWriter writer = ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); @@ -483,10 +579,16 @@ private ManifestFile writeManifest(int formatVersion) throws IOException { } private ManifestFile writeManifest(int formatVersion, DataFile... files) throws IOException { - String filename = FileFormat.AVRO.addExtension("manifest"); + return writeManifest(formatVersion, TestBase.manifestFormat(formatVersion), SPEC, files); + } + + private ManifestFile writeManifest( + int formatVersion, FileFormat fileFormat, PartitionSpec spec, DataFile... files) + throws IOException { + String filename = fileFormat.addExtension("manifest"); EncryptedOutputFile manifestFile = encryptionManager().encrypt(io.newOutputFile(filename)); ManifestWriter writer = - ManifestFiles.newWriter(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, FIRST_ROW_ID); + ManifestFiles.newWriter(formatVersion, spec, manifestFile, SNAPSHOT_ID, FIRST_ROW_ID); try { for (DataFile file : files) { writer.add(file); @@ -512,12 +614,23 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { - String filename = FileFormat.AVRO.addExtension("manifest"); + return writeDeleteManifest(formatVersion, TestBase.manifestFormat(formatVersion), SPEC); + } + + private ManifestFile writeDeleteManifest( + int formatVersion, FileFormat fileFormat, PartitionSpec spec) throws IOException { + return writeDeleteManifest(formatVersion, fileFormat, spec, DELETE_FILE); + } + + private ManifestFile writeDeleteManifest( + int formatVersion, FileFormat fileFormat, PartitionSpec spec, DeleteFile deleteFile) + throws IOException { + String filename = fileFormat.addExtension("manifest"); EncryptedOutputFile manifestFile = encryptionManager().encrypt(io.newOutputFile(filename)); ManifestWriter writer = - ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); + ManifestFiles.writeDeleteManifest(formatVersion, spec, manifestFile, SNAPSHOT_ID); try { - writer.add(DELETE_FILE); + writer.add(deleteFile); } finally { writer.close(); } diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 3947f16fe159..b7700d7ce719 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -629,7 +629,8 @@ public void testManifestsMergeIntoOne() throws IOException { .newAppend() .appendManifest( writeManifest( - "input-m0.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_C))), + manifestFormat().addExtension("input-m0"), + manifestEntry(ManifestEntry.Status.ADDED, null, FILE_C))), branch); base = readMetadata(); @@ -671,7 +672,8 @@ public void testManifestsMergeIntoOne() throws IOException { .newAppend() .appendManifest( writeManifest( - "input-m1.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_D))), + manifestFormat().addExtension("input-m1"), + manifestEntry(ManifestEntry.Status.ADDED, null, FILE_D))), branch); base = readMetadata(); @@ -1274,7 +1276,7 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); - ManifestFile manifest1 = writeManifestWithName("manifest-file-1.avro", FILE_A, FILE_B); + ManifestFile manifest1 = writeManifestWithName("manifest-file-1", FILE_A, FILE_B); Snapshot snap1 = commit(table, table.newAppend().appendManifest(manifest1), branch); long commitId1 = snap1.snapshotId(); @@ -1290,7 +1292,7 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO statuses(Status.ADDED, Status.ADDED)); assertThat(new File(manifest1.path())).exists(); - ManifestFile manifest2 = writeManifestWithName("manifest-file-2.avro", FILE_C, FILE_D); + ManifestFile manifest2 = writeManifestWithName("manifest-file-2", FILE_C, FILE_D); Snapshot snap2 = commit(table, table.newAppend().appendManifest(manifest2), branch); long commitId2 = snap2.snapshotId(); @@ -1347,7 +1349,9 @@ public void testInvalidAppendManifest() throws IOException { assertThat(base.currentSnapshot()).isNull(); ManifestFile manifestWithExistingFiles = - writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); + writeManifest( + manifestFormat().addExtension("manifest-file-1"), + manifestEntry(Status.EXISTING, null, FILE_A)); assertThatThrownBy( () -> commit(table, table.newAppend().appendManifest(manifestWithExistingFiles), branch)) @@ -1356,7 +1360,9 @@ public void testInvalidAppendManifest() throws IOException { assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); ManifestFile manifestWithDeletedFiles = - writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); + writeManifest( + manifestFormat().addExtension("manifest-file-2"), + manifestEntry(Status.DELETED, null, FILE_A)); assertThatThrownBy( () -> commit(table, table.newAppend().appendManifest(manifestWithDeletedFiles), branch)) .isInstanceOf(IllegalArgumentException.class) diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 19ee156c9eec..dab323743bb1 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -55,7 +55,8 @@ public void testRewriteManifestsAppendedDirectly() throws IOException { ManifestFile newManifest = writeManifest( - "manifest-file-1.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); + manifestFormat().addExtension("manifest-file-1"), + manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); table.newFastAppend().appendManifest(newManifest).commit(); long appendId = table.currentSnapshot().snapshotId(); @@ -79,7 +80,8 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { ManifestFile newManifest = writeManifest( - "manifest-file-1.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); + manifestFormat().addExtension("manifest-file-1"), + manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); table.newFastAppend().appendManifest(newManifest).commit(); @@ -115,7 +117,8 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio ManifestFile newManifest = writeManifest( - "manifest-file-1.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); + manifestFormat().addExtension("manifest-file-1"), + manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); table.newFastAppend().appendManifest(newManifest).commit(); long manifestAppendId = table.currentSnapshot().snapshotId(); @@ -428,11 +431,11 @@ public void testBasicManifestReplacement() throws IOException { ManifestFile firstNewManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); ManifestFile secondNewManifest = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_B)); RewriteManifests rewriteManifests = table.rewriteManifests(); @@ -492,11 +495,11 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc ManifestFile firstNewManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); ManifestFile secondNewManifest = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_B)); RewriteManifests rewriteManifests = table.rewriteManifests(); @@ -679,11 +682,11 @@ public void testManifestReplacementConcurrentAppend() throws IOException { ManifestFile firstNewManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); ManifestFile secondNewManifest = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_B)); RewriteManifests rewriteManifests = table.rewriteManifests(); @@ -741,11 +744,11 @@ public void testManifestReplacementConcurrentDelete() throws IOException { ManifestFile firstNewManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); ManifestFile secondNewManifest = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_B)); RewriteManifests rewriteManifests = table.rewriteManifests(); @@ -796,11 +799,11 @@ public void testManifestReplacementConcurrentConflictingDelete() throws IOExcept ManifestFile firstNewManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); ManifestFile secondNewManifest = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_B)); RewriteManifests rewriteManifests = table.rewriteManifests(); @@ -841,7 +844,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { ManifestFile newManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A)); table @@ -904,7 +907,8 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A); // update the entry's sequence number or else it will be rejected by the writer entry.setDataSequenceNumber(firstSnapshot.sequenceNumber()); - ManifestFile newManifest = writeManifest("manifest-file-1.avro", entry); + ManifestFile newManifest = + writeManifest(manifestFormat().addExtension("manifest-file-1"), entry); RewriteManifests rewriteManifests = table @@ -954,7 +958,8 @@ public void testInvalidUsage() throws IOException { // update the entry's sequence number or else it will be rejected by the writer appendEntry.setDataSequenceNumber(snapshot.sequenceNumber()); - ManifestFile invalidAddedFileManifest = writeManifest("manifest-file-2.avro", appendEntry); + ManifestFile invalidAddedFileManifest = + writeManifest(manifestFormat().addExtension("manifest-file-2"), appendEntry); assertThatThrownBy( () -> @@ -971,7 +976,8 @@ public void testInvalidUsage() throws IOException { // update the entry's sequence number or else it will be rejected by the writer deleteEntry.setDataSequenceNumber(snapshot.sequenceNumber()); - ManifestFile invalidDeletedFileManifest = writeManifest("manifest-file-3.avro", deleteEntry); + ManifestFile invalidDeletedFileManifest = + writeManifest(manifestFormat().addExtension("manifest-file-3"), deleteEntry); assertThatThrownBy( () -> @@ -1009,7 +1015,7 @@ public void testManifestReplacementFailure() throws IOException { ManifestFile newManifest = writeManifest( - "manifest-file.avro", + manifestFormat().addExtension("manifest-file"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A), manifestEntry(ManifestEntry.Status.EXISTING, secondSnapshot.snapshotId(), FILE_B)); @@ -1051,7 +1057,7 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE ManifestFile newManifest = writeManifest( - "manifest-file.avro", + manifestFormat().addExtension("manifest-file"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A), manifestEntry(ManifestEntry.Status.EXISTING, secondSnapshot.snapshotId(), FILE_B)); @@ -1176,7 +1182,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); ManifestFile newDeleteManifest1 = writeManifest( - "delete-manifest-file-1.avro", + manifestFormat().addExtension("delete-manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1185,7 +1191,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( - "delete-manifest-file-2.avro", + manifestFormat().addExtension("delete-manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1262,7 +1268,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { Iterables.getOnlyElement(deleteSnapshot.dataManifests(table.io())); ManifestFile newDataManifest1 = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, appendSnapshotId, @@ -1271,7 +1277,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { FILE_A)); ManifestFile newDataManifest2 = writeManifest( - "manifest-file-2.avro", + manifestFormat().addExtension("manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, appendSnapshotId, @@ -1284,7 +1290,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); ManifestFile newDeleteManifest1 = writeManifest( - "delete-manifest-file-1.avro", + manifestFormat().addExtension("delete-manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1293,7 +1299,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( - "delete-manifest-file-2.avro", + manifestFormat().addExtension("delete-manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1376,7 +1382,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); ManifestFile newDeleteManifest1 = writeManifest( - "delete-manifest-file-1.avro", + manifestFormat().addExtension("delete-manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1385,7 +1391,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( - "delete-manifest-file-2.avro", + manifestFormat().addExtension("delete-manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1486,7 +1492,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO ManifestFile originalDeleteManifest = deleteSnapshot1.deleteManifests(table.io()).get(0); ManifestFile newDeleteManifest1 = writeManifest( - "delete-manifest-file-1.avro", + manifestFormat().addExtension("delete-manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId1, @@ -1495,7 +1501,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( - "delete-manifest-file-2.avro", + manifestFormat().addExtension("delete-manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId1, @@ -1581,7 +1587,7 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I ManifestFile originalDeleteManifest = deleteSnapshot.deleteManifests(table.io()).get(0); ManifestFile newDeleteManifest1 = writeManifest( - "delete-manifest-file-1.avro", + manifestFormat().addExtension("delete-manifest-file-1"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1590,7 +1596,7 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( - "delete-manifest-file-2.avro", + manifestFormat().addExtension("delete-manifest-file-2"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId, @@ -1645,7 +1651,7 @@ public void testDeleteManifestReplacementFailure() throws IOException { // combine the original delete manifests into 1 new delete manifest ManifestFile newDeleteManifest = writeManifest( - "delete-manifest-file.avro", + manifestFormat().addExtension("delete-manifest-file"), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId1, diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java index dd97738759f4..c6092f0238b9 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.avro.AvroTestHelpers.readAvroCodec; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -227,7 +228,11 @@ public TableMetadata refresh() { } @TestTemplate - public void testDefaultManifestCompression() throws IOException { + public void testDefaultAvroManifestCompression() throws IOException { + assumeThat(formatVersion) + .as("V4 uses Parquet manifests by default; Avro codec checks do not apply") + .isLessThan(TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS); + table.newFastAppend().appendFile(FILE_A).commit(); ManifestFile manifest = table.currentSnapshot().dataManifests(table.io()).get(0); @@ -235,7 +240,11 @@ public void testDefaultManifestCompression() throws IOException { } @TestTemplate - public void testManifestCompressionFromTableProperty() throws IOException { + public void testAvroManifestCompressionFromTableProperty() throws IOException { + assumeThat(formatVersion) + .as("V4 uses Parquet manifests by default; Avro codec checks do not apply") + .isLessThan(TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS); + table.updateProperties().set(TableProperties.MANIFEST_COMPRESSION, "snappy").commit(); table.newFastAppend().appendFile(FILE_A).commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 9ec8c47840d9..fe47ac62561d 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -666,7 +666,7 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException ManifestFile newManifest = writeManifest( - "manifest-file-1.avro", + manifestFormat().addExtension("manifest-file-1"), manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshotId, FILE_A), manifestEntry(ManifestEntry.Status.EXISTING, secondSnapshotId, FILE_B)); @@ -811,7 +811,7 @@ public void testRowDeltaWithConcurrentManifestRewrite() throws IOException { .rewriteManifests() .addManifest( writeManifest( - "new_delete_manifest.avro", + manifestFormat().addExtension("new_delete_manifest"), // Specify data sequence number so that the delete files don't get aged out // first manifestEntry( @@ -880,7 +880,7 @@ public void testOverwriteWithConcurrentManifestRewrite() throws IOException { .rewriteManifests() .addManifest( writeManifest( - "new_manifest.avro", + manifestFormat().addExtension("new_manifest"), manifestEntry(Status.EXISTING, first.snapshotId(), FILE_A), manifestEntry(Status.EXISTING, first.snapshotId(), FILE_A2), manifestEntry(Status.EXISTING, second.snapshotId(), FILE_B))) diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java index ff0af5c56306..ce22c8089bc3 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -130,7 +130,7 @@ protected List manifestFiles(String location) { return Stream.of(new File(location).listFiles()) .filter(file -> !file.isDirectory()) .map(File::getName) - .filter(fileName -> fileName.endsWith(".avro")) + .filter(fileName -> !fileName.startsWith(".") && !fileName.endsWith("metadata.json")) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestManifestFileUtil.java b/core/src/test/java/org/apache/iceberg/util/TestManifestFileUtil.java index 8d2416032058..0c7e032bde4e 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestManifestFileUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestManifestFileUtil.java @@ -20,11 +20,14 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; @@ -35,24 +38,32 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.FieldSource; + +class TestManifestFileUtil { + private static final int MIN_FORMAT_VERSION_PARQUET_MANIFESTS = 4; -public class TestManifestFileUtil { private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "unknown", Types.UnknownType.get()), optional(3, "floats", Types.FloatType.get())); + private final AtomicInteger manifestCounter = new AtomicInteger(0); + @TempDir private Path temp; - @Test - public void canContainWithUnknownTypeOnly() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + void canContainWithUnknownTypeOnly(int formatVersion) throws IOException { + // Parquet cannot represent the empty struct produced by an UnknownType-only partition + assumeThat(formatVersion).isLessThan(MIN_FORMAT_VERSION_PARQUET_MANIFESTS); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("unknown").build(); PartitionData partition = new PartitionData(spec.partitionType()); partition.set(0, "someValue"); - ManifestFile manifestFile = writeManifestWithDataFile(spec, partition); + ManifestFile manifestFile = writeManifestWithDataFile(formatVersion, spec, partition); assertThat( ManifestFileUtil.canContainAny( @@ -62,12 +73,13 @@ public void canContainWithUnknownTypeOnly() throws IOException { .isTrue(); } - @Test - public void canContainWithNaNValueOnly() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + void canContainWithNaNValueOnly(int formatVersion) throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("floats").build(); PartitionData partition = new PartitionData(spec.partitionType()); partition.set(0, Float.NaN); - ManifestFile manifestFile = writeManifestWithDataFile(spec, partition); + ManifestFile manifestFile = writeManifestWithDataFile(formatVersion, spec, partition); assertThat( ManifestFileUtil.canContainAny( @@ -77,12 +89,13 @@ public void canContainWithNaNValueOnly() throws IOException { .isTrue(); } - @Test - public void canContainWithNullValueOnly() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + void canContainWithNullValueOnly(int formatVersion) throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("floats").build(); PartitionData partition = new PartitionData(spec.partitionType()); partition.set(0, null); - ManifestFile manifestFile = writeManifestWithDataFile(spec, partition); + ManifestFile manifestFile = writeManifestWithDataFile(formatVersion, spec, partition); assertThat( ManifestFileUtil.canContainAny( @@ -92,14 +105,15 @@ public void canContainWithNullValueOnly() throws IOException { .isTrue(); } - @Test - public void canContainWithUnknownType() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + void canContainWithUnknownType(int formatVersion) throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("floats").identity("unknown").build(); PartitionData partition = new PartitionData(spec.partitionType()); partition.set(0, 1.0f); partition.set(1, "someValue"); - ManifestFile manifestFile = writeManifestWithDataFile(spec, partition); + ManifestFile manifestFile = writeManifestWithDataFile(formatVersion, spec, partition); assertThat( ManifestFileUtil.canContainAny( @@ -109,9 +123,16 @@ public void canContainWithUnknownType() throws IOException { .isTrue(); } - private ManifestFile writeManifestWithDataFile(PartitionSpec spec, PartitionData partition) - throws IOException { - ManifestWriter writer = ManifestFiles.write(spec, Files.localOutput(temp.toFile())); + private ManifestFile writeManifestWithDataFile( + int formatVersion, PartitionSpec spec, PartitionData partition) throws IOException { + FileFormat format = + formatVersion >= MIN_FORMAT_VERSION_PARQUET_MANIFESTS + ? FileFormat.PARQUET + : FileFormat.AVRO; + String filename = format.addExtension("manifest-" + manifestCounter.getAndIncrement()); + ManifestWriter writer = + ManifestFiles.write( + formatVersion, spec, Files.localOutput(temp.resolve(filename).toFile()), null); try (writer) { writer.add( DataFiles.builder(spec) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java index 8aa9aa4779d9..63d6d80d5869 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -25,7 +25,11 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.UUID; @@ -828,6 +832,16 @@ public List> columns() { protected abstract T buildList(I list); } + // Only recycle known growable JDK collections as scratch buffers. Reuse may be an unmodifiable + // view, Guava immutable type, List.of / Map.of, etc.; those are not these concrete classes. + private static boolean canReuseListAsReadBuffer(List list) { + return list instanceof ArrayList || list instanceof LinkedList; + } + + private static boolean canReuseMapAsReadBuffer(Map map) { + return map instanceof LinkedHashMap || map instanceof HashMap; + } + public static class ListReader extends RepeatedReader, List, E> { private List lastList = null; private Iterator elements = null; @@ -847,7 +861,7 @@ protected List newListData(List reuse) { } if (reuse != null) { - this.lastList = reuse; + this.lastList = canReuseListAsReadBuffer(reuse) ? reuse : null; this.elements = reuse.iterator(); } else { this.lastList = null; @@ -973,7 +987,7 @@ protected Map newMapData(Map reuse) { } if (reuse != null) { - this.lastMap = reuse; + this.lastMap = canReuseMapAsReadBuffer(reuse) ? reuse : null; this.pairs = reuse.entrySet().iterator(); } else { this.lastMap = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index dae721b1d73d..c5db04762f21 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -1469,10 +1469,7 @@ protected void checkFileNum( .as(Encoders.STRING()) .collectAsList(); Predicate isManifest = - f -> - (f.contains("optimized-m-") && f.endsWith(".avro")) - || f.endsWith("-m0.avro") - || f.endsWith("-m1.avro"); + f -> f.contains("optimized-m-") || f.contains("-m0.") || f.contains("-m1."); Predicate isManifestList = f -> f.contains("snap-") && f.endsWith(".avro"); Predicate isMetadataJSON = f -> f.endsWith(".metadata.json"); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index dae721b1d73d..c5db04762f21 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -1469,10 +1469,7 @@ protected void checkFileNum( .as(Encoders.STRING()) .collectAsList(); Predicate isManifest = - f -> - (f.contains("optimized-m-") && f.endsWith(".avro")) - || f.endsWith("-m0.avro") - || f.endsWith("-m1.avro"); + f -> f.contains("optimized-m-") || f.contains("-m0.") || f.contains("-m1."); Predicate isManifestList = f -> f.contains("snap-") && f.endsWith(".avro"); Predicate isMetadataJSON = f -> f.endsWith(".metadata.json"); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index dae721b1d73d..c5db04762f21 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -1469,10 +1469,7 @@ protected void checkFileNum( .as(Encoders.STRING()) .collectAsList(); Predicate isManifest = - f -> - (f.contains("optimized-m-") && f.endsWith(".avro")) - || f.endsWith("-m0.avro") - || f.endsWith("-m1.avro"); + f -> f.contains("optimized-m-") || f.contains("-m0.") || f.contains("-m1."); Predicate isManifestList = f -> f.contains("snap-") && f.endsWith(".avro"); Predicate isMetadataJSON = f -> f.endsWith(".metadata.json"); From 4c767c14b02b7b48e0030c7da01b9f87d65dcad5 Mon Sep 17 00:00:00 2001 From: Anupam Yadav Date: Fri, 22 May 2026 14:59:36 -0700 Subject: [PATCH 09/58] Kafka Connect: Add end-to-end test for commit failure propagation (#16432) Fixes #16380 --- .../connect/channel/TestCommitterImpl.java | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCommitterImpl.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCommitterImpl.java index c6b7c86e4c66..f7440dacbe6b 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCommitterImpl.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCommitterImpl.java @@ -19,12 +19,17 @@ package org.apache.iceberg.connect.channel; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.lang.reflect.Field; +import java.util.Collections; import java.util.List; import java.util.Optional; import org.apache.iceberg.connect.IcebergSinkConfig; @@ -114,4 +119,50 @@ public void testHasLeaderPartition() throws NoSuchFieldException, IllegalAccessE assertThat(committer.hasLeaderPartition(nonLeaderAssignments)).isFalse(); } } + + @Test + public void testCommitFailurePropagatesAsNotRunningException() + throws NoSuchFieldException, IllegalAccessException { + Coordinator coordinator = mock(Coordinator.class); + doThrow(new RuntimeException("commit failed")).when(coordinator).process(); + + CoordinatorThread coordinatorThread = new CoordinatorThread(coordinator); + coordinatorThread.start(); + + // wait for the thread to catch the exception, set terminated, and call stop + verify(coordinator, timeout(1000)).stop(); + assertThat(coordinatorThread.isTerminated()).isTrue(); + + CommitterImpl committer = new CommitterImpl(); + Field field = CommitterImpl.class.getDeclaredField("coordinatorThread"); + field.setAccessible(true); + field.set(committer, coordinatorThread); + + assertThatThrownBy(() -> committer.save(Collections.emptyList())) + .isInstanceOf(NotRunningException.class) + .hasMessageContaining("Coordinator unexpectedly terminated"); + } + + @Test + public void testStartFailurePropagatesAsNotRunningException() + throws NoSuchFieldException, IllegalAccessException { + Coordinator coordinator = mock(Coordinator.class); + doThrow(new RuntimeException("start failed")).when(coordinator).start(); + + CoordinatorThread coordinatorThread = new CoordinatorThread(coordinator); + coordinatorThread.start(); + + // wait for the thread to catch the exception, set terminated, and call stop + verify(coordinator, timeout(1000)).stop(); + assertThat(coordinatorThread.isTerminated()).isTrue(); + + CommitterImpl committer = new CommitterImpl(); + Field field = CommitterImpl.class.getDeclaredField("coordinatorThread"); + field.setAccessible(true); + field.set(committer, coordinatorThread); + + assertThatThrownBy(() -> committer.save(Collections.emptyList())) + .isInstanceOf(NotRunningException.class) + .hasMessageContaining("Coordinator unexpectedly terminated"); + } } From 99e451acf38266c5eed9482c7feeaf20914620d5 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sat, 23 May 2026 07:14:10 +0900 Subject: [PATCH 10/58] Core: Skip testAddManyFilesWithConsistentOrdering if WORKER_THREAD_POOL_SIZE < 3 (#16506) --- core/src/test/java/org/apache/iceberg/TestMergeAppend.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index b7700d7ce719..c5f3d88f46d9 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -41,6 +41,7 @@ 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; +import org.apache.iceberg.util.ThreadPools; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,6 +91,11 @@ public void testAddManyFilesWithConsistentOrdering() { assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); int multiplier = 3; + assumeThat(ThreadPools.WORKER_THREAD_POOL_SIZE) + .as( + "Worker thread pool size should be at least 3 to test manifest file ordering with multiple threads") + .isGreaterThanOrEqualTo(multiplier); + int groupSize = SnapshotProducer.MIN_FILE_GROUP_SIZE; List dataFiles = Lists.newArrayList(); From 36ef88722bad12973c081fd6d7bbbaf3e0b06210 Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Fri, 22 May 2026 18:08:30 -0700 Subject: [PATCH 11/58] [SPEC] Add relative paths to v4 spec (#15630) * [SPEC] Add relative paths to v4 spec --------- Co-authored-by: Talat Uyarer Co-authored-by: Daniel Weeks --- format/spec.md | 132 ++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 130 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index 28ff9be3ae27..d5d78ef9713d 100644 --- a/format/spec.md +++ b/format/spec.md @@ -57,6 +57,14 @@ Version 3 of the Iceberg spec extends data types and existing metadata structure The full set of changes are listed in [Appendix E](#version-3). +### Version 4: Metadata Structure and Representation + +Version 4 of the Iceberg spec restructures metadata for improved performance and new capabilities: + +* Support for [relative locations](#file-locations-in-metadata) in metadata fields + +The full set of changes are listed in [Appendix E](#version-4). + ## Goals * **Serializable isolation** -- Reads will be isolated from concurrent writes and always use a committed snapshot of a table’s data. Writes will support removing and adding files in a single operation and are never partially visible. Readers will not acquire locks. @@ -123,9 +131,15 @@ Tables do not require random-access writes. Once written, data and metadata file Tables do not require rename, except for tables that use atomic rename to implement the commit operation for new metadata files. +### File Locations in Metadata + +All location fields in format versions 3 and prior contain fully-qualified paths. + +Version 4 of the Iceberg spec adds support for relative locations in metadata, enabling tables to be relocated without rewriting metadata files. Relative locations are allowed in all metadata tracked location fields and are resolved against the table's base location. The table's location may be fixed in table metadata or inferred, but is intended to be managed and supplied by a catalog. Requirements for relativization and resolution are in [Paths in Metadata](#paths-in-metadata). + ## Specification -#### Terms +### Terms * **Schema** -- Names and types of fields in a table. * **Partition spec** -- A definition of how partition values are derived from data fields. @@ -134,8 +148,10 @@ Tables do not require rename, except for tables that use atomic rename to implem * **Manifest** -- A file that lists data or delete files; a subset of a snapshot. * **Data file** -- A file that contains rows of a table. * **Delete file** -- A file that encodes rows of a table that are deleted by position or data values. +* **Absolute path** -- A path string that includes a [URI](https://datatracker.ietf.org/doc/html/rfc3986#section-3.1) scheme and can be used directly. +* **Relative path** -- A path string without a URI scheme that must be [resolved](#path-resolution) against the table location. -#### Writer requirements +### Writer requirements Some tables in this spec have columns that specify requirements for tables by version. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. @@ -168,6 +184,48 @@ All columns must be written to data files even if they introduce redundancy with Writers are not allowed to commit files with a partition spec that contains a field with an unknown transform. +### Paths in Metadata + +Path strings stored in Iceberg metadata location fields are classified as one of two types: + +* **Absolute path** -- A path string that starts with a [URI scheme](https://datatracker.ietf.org/doc/html/rfc3986#section-3.1) (e.g., `s3:`, `gs:`, `hdfs:`, `file:`). Absolute paths are used as-is without modification. +* **Relative path** -- A path string that does not start with a URI scheme. Relative paths must be resolved against the table's base location before use. + +Prior to v4, all path fields must contain fully-qualified paths. Starting with v4, path fields may contain either absolute or relative paths. [Relative resolution within a URI](https://datatracker.ietf.org/doc/html/rfc3986#section-5.2) (e.g. `.` and `..`) and other file system navigation conventions are not supported in relative paths. + +#### Path Resolution + +Path resolution is the process of producing an absolute path from a relative path by combining it with the table's base location: + +* If the path starts with a URI scheme, it is absolute and is used without modification. +* If the path does not start with a URI scheme, the resolved path is the table location followed by the relative path joined by the URI separator character `/`. + +The relative portion is joined to the prefix (table location) without consideration of any additional separator characters. The recommended convention for table location is to not end in a path separator because the join process would add a second separator character. (See example below.) + +Paths in manifests produced prior to v4 are fully-qualified and must be produced with a URI scheme, if the scheme was omitted, to be consistent with v4 paths. + +Examples of path resolution: + +| | Format Version | Table Location | File Path | Resolved Path | Description | +|---------------------|----------------|-----------------------|-------------------------------------------|--------------------------------------------|-------------------------------------| +| Relative Path | v4 | s3://bucket/db/table | data/00000-0.parquet | s3://bucket/db/table/data/00000-0.parquet | Path parts are joined on `/` | +| Absolute Path | v4 | s3://bucket/db/table | hdfs://wh/db/table/data/00000-0.parquet | hdfs://wh/db/table/data/00000-0.parquet | Absolute path is used | +| Duplicate separator | v4 | s3://bucket/db/table/ | data/00000-0.parquet | s3://bucket/db/table//data/00000-0.parquet | Join results in duplicate `//` | +| Duplicate separator | v4 | s3://bucket/db/table | /data/00000-0.parquet | s3://bucket/db/table//data/00000-0.parquet | Join results in duplicate `//` | +| Fully-qualified | v3 and earlier | s3://bucket/db/table | s3://bucket/db/table/data/00000-0.parquet | s3://bucket/db/table/data/00000-0.parquet | Fully-qualified path is used | +| Missing scheme | v3 and earlier | /wh/db/table | /wh/db/table/data/00000-0.parquet | hdfs://wh/db/table/data/00000-0.parquet | Scheme is prepended for consistency | + +#### Path Relativization + +Path relativization is the process of converting an absolute path to a relative path by removing the table location prefix. This is used when persisting paths to metadata files. + +* If an absolute path starts with the table location immediately followed by a separator character, the relative path is the remainder of the string after the separator character. +* If an absolute path does not start with the table location immediately followed by the separator character, it is stored as an absolute path. + +#### Table Location Specification + +When the `location` field is present in table metadata, it is used directly as the table's base location. When the `location` field is not present (v4 and later), the table location must be provided. How the table location is persisted or determined when not specified in metadata is not a table-level concern; catalogs should provide a table's location. + ### Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. @@ -1101,6 +1159,34 @@ Table metadata consists of the following fields: | _optional_ | _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | | | | _required_ | **`next-row-id`** | A `long` higher than all assigned row IDs; the next snapshot’s `first-row-id`. See [Row Lineage](#row-lineage). | | | | _optional_ | **`encryption-keys`** | A list (optional) of [encryption keys](#encryption-keys) used for table encryption. | +=== "v4" + | v4 | Field | Description | + |------------|-----------------------------|-------------| + | _required_ | **`format-version`** | An integer version number for the format. Implementations must throw an exception if a table's version is higher than the supported version. | + | _required_ | **`table-uuid`** | A UUID that identifies the table, generated when the table is created. Implementations must throw an exception if a table's UUID does not match the expected UUID after refreshing metadata. | + | _optional_ | **`location`** | The table's base location. This is used by writers to determine where to store data files, manifest files, and table metadata files. Must be an absolute path when present. See [Table Locations](#table-location-specification). | + | _required_ | **`last-sequence-number`** | The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | + | _required_ | **`last-updated-ms`** | Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | + | _required_ | **`last-column-id`** | An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | + | | **`schema`** | The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | + | _required_ | **`schemas`** | A list of schemas, stored as objects with `schema-id`. | + | _required_ | **`current-schema-id`** | ID of the table's current schema. | + | | **`partition-spec`** | The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | + | _required_ | **`partition-specs`** | A list of partition specs, stored as full partition spec objects. | + | _required_ | **`default-spec-id`** | ID of the "current" spec that writers should use by default. | + | _required_ | **`last-partition-id`** | An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | + | _optional_ | **`properties`** | A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | + | _optional_ | **`current-snapshot-id`** | `long` ID of the current table snapshot; must be the same as the current ID of the `main` branch in `refs`. | + | _optional_ | **`snapshots`** | A list of valid snapshots. Valid snapshots are snapshots for which all data files exist in the file system. A data file must not be deleted from the file system until the last snapshot in which it was listed is garbage collected. | + | _optional_ | **`snapshot-log`** | A list (optional) of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table. Each time the current-snapshot-id is changed, a new entry should be added with the last-updated-ms and the new current-snapshot-id. When snapshots are expired from the list of valid snapshots, all entries before a snapshot that has expired should be removed. | + | _optional_ | **`metadata-log`** | A list (optional) of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table. Each time a new metadata file is created, a new entry of the previous metadata file location should be added to the list. Tables can be configured to remove oldest metadata log entries and keep a fixed-size log of the most recent entries after a commit. | + | _required_ | **`sort-orders`** | A list of sort orders, stored as full sort order objects. | + | _required_ | **`default-sort-order-id`** | Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | + | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | + | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | + | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | + | _required_ | **`next-row-id`** | A `long` higher than all assigned row IDs; the next snapshot's `first-row-id`. See [Row Lineage](#row-lineage). | + | _optional_ | **`encryption-keys`** | A list (optional) of [encryption keys](#encryption-keys) used for table encryption. | For serialization details, see Appendix C. @@ -1794,6 +1880,30 @@ The binary single-value serialization can be used to store the lower and upper b ## Appendix E: Format version changes +### Version 4 + +Relative path support is added in v4. + +Reading v3 or prior metadata for v4: + +* All location fields are fully-qualified paths and interpreted as absolute paths for v4 +* Any location field without a URI scheme prefix must prepend a scheme component consistent with v4 absolute paths + +Writing v4 metadata: + +* Table metadata JSON: + * `location` is now optional and must be absolute when present + * When not present, the table location must be managed externally and provided when loading the metadata +* Location fields in all metadata structures may contain relative paths +* Writers should produce relative paths by default for files that reside under the table location +* Absolute paths must be used for files that do not share a common prefix with the table location + +Reading v4 metadata: + +* Readers must check whether location fields contain a URI scheme to determine if a path is absolute or relative +* Relative paths must be resolved against the table location before use (see [Path Resolution](#path-resolution)) +* When `location` is omitted, the table location must be provided (see [Table Location Specification](#table-location-specification)) + ### Version 3 Default values are added to struct fields in v3. @@ -1924,6 +2034,24 @@ Note that these requirements apply when writing data to a v2 table. Tables that This section covers topics not required by the specification but recommendations for systems implementing the Iceberg specification to help maintain a uniform experience. +### Path Construction + +Path construction is the process by which new file locations are created for output files referenced by metadata. While the specific construction logic is not strictly required by the spec, the following guidance is provided for reference implementations to encourage consistency. + +The table properties `write.metadata.path` and `write.data.path` control where metadata and data files are written. When not specified, these default to the values `metadata` and `data` respectively. + +For all metadata files: + +* If `write.metadata.path` is an absolute path, it is used directly as the base for new metadata files. +* If `write.metadata.path` is a relative path, the metadata base is the table location joined to the `write.metadata.path` value with a URI separator `/`. + +For data files: + +* If `write.data.path` is an absolute path, it is used directly as the base for new data files. +* If `write.data.path` is a relative path, the base is the table location joined to the `write.data.path` value with a URI separator `/`. + +When persisting paths into metadata, writers should relativize paths against the table location when allowed by the table version (see [Path Relativization](#path-relativization)). If a file's absolute path shares a common prefix with the table location followed by a URI separator `/`, the relative portion should be stored. Otherwise, the absolute path should be stored. + ### Point in Time Reads (Time Travel) Iceberg supports two types of histories for tables. A history of previous "current snapshots" stored in ["snapshot-log" table metadata](#table-metadata-fields) and [parent-child lineage stored in "snapshots"](#table-metadata-fields). These two histories From 29ba0a14dc6667db0683fdfcd520639b3da77774 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 22 May 2026 21:16:44 -0700 Subject: [PATCH 12/58] Core: Add V4 location relativization utilities (#16174) * Core: Add location relativization utilities to RelativePathUtil Add isAbsolute, resolve, and relativize methods for converting between absolute and relative file paths. These will be used by v4 metadata to store locations relative to the table location. --- .../org/apache/iceberg/util/LocationUtil.java | 76 ++++++- .../apache/iceberg/util/TestLocationUtil.java | 208 ++++++++++++++++++ 2 files changed, 282 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java index 4c0d401c74b9..21eacbfbd665 100644 --- a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java @@ -24,14 +24,16 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; public class LocationUtil { + public static final String PATH_SEPARATOR = "/"; + private LocationUtil() {} public static String stripTrailingSlash(String path) { Preconditions.checkArgument(!Strings.isNullOrEmpty(path), "path must not be null or empty"); String result = path; - while (!result.endsWith("://") && result.endsWith("/")) { - result = result.substring(0, result.length() - 1); + while (!result.endsWith("://") && result.endsWith(PATH_SEPARATOR)) { + result = result.substring(0, result.length() - PATH_SEPARATOR.length()); } return result; } @@ -57,4 +59,74 @@ public static String tableLocation(TableIdentifier tableIdentifier, boolean useU return tableIdentifier.name(); } } + + /** + * Returns true if the location contains a URI scheme (e.g. {@code s3:}, {@code hdfs:}, {@code + * file:}), per RFC 3986 + * section 3.1. + */ + private static boolean hasScheme(String location) { + for (int i = 0; i < location.length(); i += 1) { + char ch = location.charAt(i); + if (ch == ':') { + return i > 0; + } + + if (!isSchemeChar(ch, i)) { + return false; + } + } + + return false; + } + + /** + * Returns true if {@code ch} is allowed at {@code position} in a URI scheme, per RFC 3986 section 3.1: + * {@code scheme = ALPHA *( ALPHA / DIGIT / "+" / "-" / "." )}. + */ + private static boolean isSchemeChar(char ch, int position) { + if ((ch >= 'a' && ch <= 'z') || (ch >= 'A' && ch <= 'Z')) { + return true; + } + + return position > 0 && ((ch >= '0' && ch <= '9') || ch == '+' || ch == '-' || ch == '.'); + } + + /** + * Resolves a location against a table location. If the location has a URI scheme, it is returned + * as-is. Otherwise, the table location and the relative location are joined by the URI separator + * character {@code /}. + * + *

The separator is appended unconditionally; {@code tableLocation} is expected not to end with + * {@code /} and {@code location} is expected not to start with {@code /}. Otherwise the result + * will contain a duplicate {@code //}. + */ + public static String resolveLocation(String tableLocation, String location) { + if (hasScheme(location)) { + return location; + } + + return tableLocation + PATH_SEPARATOR + location; + } + + /** + * Relativizes a location against a table location. If the location starts with the table location + * immediately followed by the URI separator character {@code /}, the prefix and separator are + * removed and the remaining relative portion is returned. Otherwise, the location is returned + * as-is. + * + *

{@code tableLocation} is expected not to end with {@code /}. A trailing separator on the + * table location will cause locations that would otherwise match to be returned unchanged. + */ + public static String relativizeLocation(String tableLocation, String location) { + int prefixLength = tableLocation.length(); + if (location.length() > prefixLength + && location.startsWith(PATH_SEPARATOR, prefixLength) + && location.startsWith(tableLocation)) { + return location.substring(prefixLength + PATH_SEPARATOR.length()); + } + + return location; + } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java index 9a7b2768d995..0ce0c9ad3104 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java @@ -84,4 +84,212 @@ void testStripTrailingSlashForRootPathWithTrailingSlashes() { .as("Should be root path") .isEqualTo(rootPath); } + + @Test + public void testResolveRelativeLocations() { + String tableLocation = "s3://bucket/db/table"; + + assertThat(LocationUtil.resolveLocation(tableLocation, "metadata/file.parquet")) + .isEqualTo("s3://bucket/db/table/metadata/file.parquet"); + + assertThat(LocationUtil.resolveLocation(tableLocation, "data/00000-0.parquet")) + .isEqualTo("s3://bucket/db/table/data/00000-0.parquet"); + } + + @Test + public void testResolveLocationsWithColonsInSegments() { + String tableLocation = "s3://bucket/db/table"; + + assertThat(LocationUtil.resolveLocation(tableLocation, "data/partition=key:value/file.parquet")) + .isEqualTo("s3://bucket/db/table/data/partition=key:value/file.parquet"); + + assertThat(LocationUtil.resolveLocation(tableLocation, "metadata/snap-123:456.avro")) + .isEqualTo("s3://bucket/db/table/metadata/snap-123:456.avro"); + } + + @Test + public void testResolveAbsoluteLocationsUnchanged() { + String tableLocation = "s3://bucket/db/table"; + + // different scheme (from the spec example table) + assertThat( + LocationUtil.resolveLocation(tableLocation, "hdfs://wh/db/table/data/00000-0.parquet")) + .isEqualTo("hdfs://wh/db/table/data/00000-0.parquet"); + + // different bucket + assertThat( + LocationUtil.resolveLocation( + tableLocation, "s3://other-bucket/db/table/data/file.parquet")) + .isEqualTo("s3://other-bucket/db/table/data/file.parquet"); + + // same bucket, different path + assertThat( + LocationUtil.resolveLocation( + tableLocation, "s3://bucket/db/other-table/data/file.parquet")) + .isEqualTo("s3://bucket/db/other-table/data/file.parquet"); + } + + @Test + public void testRelativize() { + String tableLocation = "s3://bucket/db/table"; + + assertThat( + LocationUtil.relativizeLocation( + tableLocation, "s3://bucket/db/table/metadata/file.parquet")) + .isEqualTo("metadata/file.parquet"); + + assertThat( + LocationUtil.relativizeLocation( + tableLocation, "s3://bucket/db/table/data/00000-0.parquet")) + .isEqualTo("data/00000-0.parquet"); + } + + @Test + public void testRelativizeLocationNotUnderTableLocation() { + String tableLocation = "s3://bucket/db/table"; + + // different bucket + assertThat( + LocationUtil.relativizeLocation( + tableLocation, "s3://other-bucket/db/table/data/file.parquet")) + .isEqualTo("s3://other-bucket/db/table/data/file.parquet"); + + // same bucket, different path + assertThat( + LocationUtil.relativizeLocation( + tableLocation, "s3://bucket/db/other-table/data/file.parquet")) + .isEqualTo("s3://bucket/db/other-table/data/file.parquet"); + } + + @Test + public void testRelativizeLocationWithSharedPrefix() { + // sibling locations that share a byte prefix with the table location but are not + // children of it must not be relativized (e.g. "table" vs "table_v2") + String tableLocation = "s3://bucket/db/table"; + + assertThat( + LocationUtil.relativizeLocation( + tableLocation, "s3://bucket/db/table_v2/data/00000-0.parquet")) + .isEqualTo("s3://bucket/db/table_v2/data/00000-0.parquet"); + } + + @Test + public void testRelativizeLocationEqualToTableLocation() { + // a location equal to the table location is not followed by a separator, + // so it is not a child of the table location and is returned as-is + String tableLocation = "s3://bucket/db/table"; + + assertThat(LocationUtil.relativizeLocation(tableLocation, tableLocation)) + .isEqualTo(tableLocation); + } + + @Test + public void testRelativizeMismatchedFileSchemeNotRelativized() { + // mixed file: variants are NOT relativized. Consistent URI forms are the caller's + // responsibility + assertThat( + LocationUtil.relativizeLocation( + "file:/tmp/table", "file:///tmp/table/metadata/file.parquet")) + .isEqualTo("file:///tmp/table/metadata/file.parquet"); + + assertThat( + LocationUtil.relativizeLocation( + "file:///tmp/table", "file:/tmp/table/metadata/file.parquet")) + .isEqualTo("file:/tmp/table/metadata/file.parquet"); + } + + @Test + public void testResolveAbsoluteLocationWithNonAlphanumericScheme() { + String tableLocation = "s3://bucket/db/table"; + + assertThat(LocationUtil.resolveLocation(tableLocation, "git+ssh://host/repo")) + .isEqualTo("git+ssh://host/repo"); + } + + @Test + public void testResolveTreatsNonAsciiSchemeAsRelative() { + // RFC 3986 restricts schemes to US-ASCII; a non-ASCII letter such as the Greek alpha + // (U+03B1) is not a valid scheme character and the location is treated as relative + String tableLocation = "s3://bucket/db/table"; + String location = "αscheme://host/path"; + + assertThat(LocationUtil.resolveLocation(tableLocation, location)) + .isEqualTo(tableLocation + "/" + location); + } + + @Test + public void testResolveTreatsNonAlphaLeadingCharAsRelative() { + // RFC 3986 section 3.1 requires the first scheme character to be ALPHA; locations + // beginning with a digit or with '+'/'-'/'.' are treated as relative + String tableLocation = "s3://bucket/db/table"; + + assertThat(LocationUtil.resolveLocation(tableLocation, "3com://host")) + .isEqualTo("s3://bucket/db/table/3com://host"); + + assertThat(LocationUtil.resolveLocation(tableLocation, "+ssh://host")) + .isEqualTo("s3://bucket/db/table/+ssh://host"); + + assertThat(LocationUtil.resolveLocation(tableLocation, "-foo://host")) + .isEqualTo("s3://bucket/db/table/-foo://host"); + + assertThat(LocationUtil.resolveLocation(tableLocation, ".bar://host")) + .isEqualTo("s3://bucket/db/table/.bar://host"); + } + + @Test + public void testRelativizeResolveRoundTrip() { + String tableLocation = "s3://bucket/db/table"; + String absoluteLocation = "s3://bucket/db/table/metadata/root-manifest.parquet"; + + String relativized = LocationUtil.relativizeLocation(tableLocation, absoluteLocation); + assertThat(relativized).isEqualTo("metadata/root-manifest.parquet"); + + String resolved = LocationUtil.resolveLocation(tableLocation, relativized); + assertThat(resolved).isEqualTo(absoluteLocation); + } + + @Test + public void testRelativizeResolveRoundTripWithFileScheme() { + String tableLocation = "file:///tmp/warehouse/table"; + String absoluteLocation = "file:///tmp/warehouse/table/metadata/root-manifest.parquet"; + + String relativized = LocationUtil.relativizeLocation(tableLocation, absoluteLocation); + assertThat(relativized).isEqualTo("metadata/root-manifest.parquet"); + + String resolved = LocationUtil.resolveLocation(tableLocation, relativized); + assertThat(resolved).isEqualTo(absoluteLocation); + } + + @Test + public void testResolveWithTrailingOrLeadingSlashProducesDuplicateSeparator() { + // the spec documents that joining a table location ending with '/' or a relative location + // starting with '/' yields a duplicate '//'; callers are expected to avoid this + assertThat(LocationUtil.resolveLocation("s3://bucket/db/table/", "data/00000-0.parquet")) + .isEqualTo("s3://bucket/db/table//data/00000-0.parquet"); + + assertThat(LocationUtil.resolveLocation("s3://bucket/db/table", "/data/00000-0.parquet")) + .isEqualTo("s3://bucket/db/table//data/00000-0.parquet"); + } + + @Test + public void testRelativizeWithTrailingSlashTableLocationNotRelativized() { + // a trailing '/' on the table location prevents the prefix match because relativization + // expects the separator to follow the prefix; the location is returned as-is + assertThat( + LocationUtil.relativizeLocation( + "s3://bucket/db/table/", "s3://bucket/db/table/data/00000-0.parquet")) + .isEqualTo("s3://bucket/db/table/data/00000-0.parquet"); + } + + @Test + public void testRelativizeResolveRoundTripWithHDFS() { + String tableLocation = "hdfs://namenode/warehouse/table"; + String absoluteLocation = "hdfs://namenode/warehouse/table/data/00000-0.parquet"; + + String relativized = LocationUtil.relativizeLocation(tableLocation, absoluteLocation); + assertThat(relativized).isEqualTo("data/00000-0.parquet"); + + String resolved = LocationUtil.resolveLocation(tableLocation, relativized); + assertThat(resolved).isEqualTo(absoluteLocation); + } } From 988d6cd6f31a58e86d58b2e442a2a0afdf4c5aca Mon Sep 17 00:00:00 2001 From: Yong Zheng Date: Sun, 24 May 2026 08:56:48 -0500 Subject: [PATCH 13/58] Core: Fix SerializableTable.sortOrders() throwing on historical sort orders with dropped fields (#16519) (#16521) --- .../org/apache/iceberg/SerializableTable.java | 5 ++++- .../java/org/apache/iceberg/SortOrderParser.java | 4 ++++ .../iceberg/hadoop/TestTableSerialization.java | 16 ++++++++++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index a26fff1fd565..5b4cd0e55396 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -58,6 +58,7 @@ public class SerializableTable implements Table, HasTableOperations, Serializabl private final int defaultSpecId; private final Map specAsJsonMap; private final String sortOrderAsJson; + private final int defaultSortOrderId; private final Map sortOrderAsJsonMap; private final FileIO io; private final EncryptionManager encryption; @@ -83,6 +84,7 @@ protected SerializableTable(Table table) { Map specs = table.specs(); specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec))); this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder()); + this.defaultSortOrderId = table.sortOrder().orderId(); this.sortOrderAsJsonMap = Maps.newHashMap(); table .sortOrders() @@ -253,7 +255,8 @@ public Map sortOrders() { ImmutableMap.Builder sortOrders = ImmutableMap.builderWithExpectedSize(sortOrderAsJsonMap.size()); sortOrderAsJsonMap.forEach( - (id, json) -> sortOrders.put(id, SortOrderParser.fromJson(schema(), json))); + (id, json) -> + sortOrders.put(id, SortOrderParser.fromJson(schema(), json, defaultSortOrderId))); this.lazySortOrders = sortOrders.build(); } else if (lazySortOrders == null) { this.lazySortOrders = lazyTable.sortOrders(); diff --git a/core/src/main/java/org/apache/iceberg/SortOrderParser.java b/core/src/main/java/org/apache/iceberg/SortOrderParser.java index 31307cf9dc7f..53d7e5090c76 100644 --- a/core/src/main/java/org/apache/iceberg/SortOrderParser.java +++ b/core/src/main/java/org/apache/iceberg/SortOrderParser.java @@ -112,6 +112,10 @@ public static SortOrder fromJson(Schema schema, String json) { return fromJson(json).bind(schema); } + public static SortOrder fromJson(Schema schema, String json, int defaultSortOrderId) { + return JsonUtil.parse(json, node -> fromJson(schema, node, defaultSortOrderId)); + } + public static SortOrder fromJson(Schema schema, JsonNode json, int defaultSortOrderId) { UnboundSortOrder unboundSortOrder = fromJson(json); diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java index 5103e2e9be92..ece9b24af3d1 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -79,6 +79,22 @@ public void testSerializableTable() throws IOException, ClassNotFoundException { .isEqualTo(TableUtil.metadataFileLocation(table)); } + @Test + public void testSerializableTableSortOrdersWithDroppedColumn() + throws IOException, ClassNotFoundException { + table.updateSchema().addColumn("ts", Types.LongType.get()).commit(); + table.replaceSortOrder().asc("id").asc("ts").commit(); + + // historical sort order 1 still references "ts" after the column is dropped + table.replaceSortOrder().asc("id").commit(); + table.updateSchema().deleteColumn("ts").commit(); + + TestHelpers.assertSerializedAndLoadedMetadata(table, TestHelpers.roundTripSerialize(table)); + Table serializableTable = SerializableTable.copyOf(table); + TestHelpers.assertSerializedAndLoadedMetadata( + serializableTable, TestHelpers.KryoHelpers.roundTripSerialize(serializableTable)); + } + @Test public void testSerializableTableWithSnapshot() throws IOException, ClassNotFoundException { table.newAppend().appendFile(FILE_A).commit(); From ffb29d3b1e7ed867afd7ad2f94af071fd7a37982 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:29:36 -0700 Subject: [PATCH 14/58] Build: Bump software.amazon.awssdk:bom from 2.44.4 to 2.44.7 (#16555) Bumps software.amazon.awssdk:bom from 2.44.4 to 2.44.7. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.44.7 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 032fb09dbe14..55e9af94503e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -33,7 +33,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.7" awaitility = "4.3.0" -awssdk-bom = "2.44.4" +awssdk-bom = "2.44.7" azuresdk-bom = "1.3.6" awssdk-s3accessgrants = "2.4.1" bouncycastle = "1.84" From 95c86f8862d6d9b5bdcbe0b5cae039258bab1d1c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:29:50 -0700 Subject: [PATCH 15/58] Build: Bump github/codeql-action from 4.35.4 to 4.35.5 (#16554) Bumps [github/codeql-action](https://github.com/github/codeql-action) from 4.35.4 to 4.35.5. - [Release notes](https://github.com/github/codeql-action/releases) - [Changelog](https://github.com/github/codeql-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/github/codeql-action/compare/68bde559dea0fdcac2102bfdf6230c5f70eb485e...9e0d7b8d25671d64c341c19c0152d693099fb5ba) --- updated-dependencies: - dependency-name: github/codeql-action dependency-version: 4.35.5 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/codeql.yml | 4 ++-- .github/workflows/cve-scan.yml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 6f25d0c9b76f..a811068b135a 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -46,11 +46,11 @@ jobs: persist-credentials: false - name: Initialize CodeQL - uses: github/codeql-action/init@68bde559dea0fdcac2102bfdf6230c5f70eb485e # v4.35.4 + uses: github/codeql-action/init@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 with: languages: actions - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@68bde559dea0fdcac2102bfdf6230c5f70eb485e # v4.35.4 + uses: github/codeql-action/analyze@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 with: category: "/language:actions" diff --git a/.github/workflows/cve-scan.yml b/.github/workflows/cve-scan.yml index 24f2c80015ce..415b6eb3c7de 100644 --- a/.github/workflows/cve-scan.yml +++ b/.github/workflows/cve-scan.yml @@ -166,7 +166,7 @@ jobs: fi - name: Upload Trivy results to GitHub Security tab if: always() && github.event_name == 'push' - uses: github/codeql-action/upload-sarif@68bde559dea0fdcac2102bfdf6230c5f70eb485e # v4.35.4 + uses: github/codeql-action/upload-sarif@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 with: sarif_file: 'trivy-results.sarif' category: ${{ matrix.distribution }} From d9a12fa9c2525058fae9b2773b5480025c3441ef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:30:09 -0700 Subject: [PATCH 16/58] Build: Bump com.diffplug.spotless:spotless-plugin-gradle (#16552) Bumps [com.diffplug.spotless:spotless-plugin-gradle](https://github.com/diffplug/spotless) from 8.4.0 to 8.5.1. - [Release notes](https://github.com/diffplug/spotless/releases) - [Changelog](https://github.com/diffplug/spotless/blob/main/CHANGES.md) - [Commits](https://github.com/diffplug/spotless/compare/gradle/8.4.0...gradle/8.5.1) --- updated-dependencies: - dependency-name: com.diffplug.spotless:spotless-plugin-gradle dependency-version: 8.5.1 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 4aeb1eb93a67..ac5814b570b9 100644 --- a/build.gradle +++ b/build.gradle @@ -28,7 +28,7 @@ buildscript { dependencies { classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.10' classpath 'com.palantir.baseline:gradle-baseline-java:6.90.0' - classpath 'com.diffplug.spotless:spotless-plugin-gradle:8.4.0' + classpath 'com.diffplug.spotless:spotless-plugin-gradle:8.5.1' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.3' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' From 2245a81af168e6b9d2ee51c49ecec8b4c8c0a876 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:30:27 -0700 Subject: [PATCH 17/58] Build: Bump slf4j from 2.0.17 to 2.0.18 (#16553) Bumps `slf4j` from 2.0.17 to 2.0.18. Updates `org.slf4j:slf4j-api` from 2.0.17 to 2.0.18 Updates `org.slf4j:slf4j-simple` from 2.0.17 to 2.0.18 --- updated-dependencies: - dependency-name: org.slf4j:slf4j-api dependency-version: 2.0.18 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.slf4j:slf4j-simple dependency-version: 2.0.18 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 55e9af94503e..92e5d4a831e3 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ orc = "1.9.8" parquet = "1.17.1" roaringbitmap = "1.6.14" scala-collection-compat = "2.14.0" -slf4j = "2.0.17" +slf4j = "2.0.18" snowflake-jdbc = "3.28.0" spark35 = "3.5.8" spark40 = "4.0.2" From 400fff6bd689120b4aa92ad9ff4369933b429cf1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:30:42 -0700 Subject: [PATCH 18/58] Build: Bump zizmorcore/zizmor-action from 0.5.3 to 0.5.6 (#16550) Bumps [zizmorcore/zizmor-action](https://github.com/zizmorcore/zizmor-action) from 0.5.3 to 0.5.6. - [Release notes](https://github.com/zizmorcore/zizmor-action/releases) - [Commits](https://github.com/zizmorcore/zizmor-action/compare/b1d7e1fb5de872772f31590499237e7cce841e8e...5f14fd08f7cf1cb1609c1e344975f152c7ee938d) --- updated-dependencies: - dependency-name: zizmorcore/zizmor-action dependency-version: 0.5.6 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/zizmor.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/zizmor.yml b/.github/workflows/zizmor.yml index 0df5e1362ac5..79b0d293f790 100644 --- a/.github/workflows/zizmor.yml +++ b/.github/workflows/zizmor.yml @@ -39,7 +39,7 @@ jobs: persist-credentials: false - name: Run zizmor 🌈 - uses: zizmorcore/zizmor-action@b1d7e1fb5de872772f31590499237e7cce841e8e # v0.5.3 + uses: zizmorcore/zizmor-action@5f14fd08f7cf1cb1609c1e344975f152c7ee938d # v0.5.6 with: advanced-security: false min-severity: medium From 75fcb8d9e3378237a6b73b1b20d507641c37a392 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 May 2026 10:31:02 -0700 Subject: [PATCH 19/58] Build: Bump com.google.cloud:libraries-bom from 26.81.0 to 26.83.0 (#16551) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.81.0 to 26.83.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.81.0...v26.83.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-version: 26.83.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 92e5d4a831e3..ce2eec19a320 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -51,7 +51,7 @@ findbugs-jsr305 = "3.0.2" flink120 = { strictly = "1.20.1"} flink20 = { strictly = "2.0.0"} flink21 = { strictly = "2.1.0"} -google-libraries-bom = "26.81.0" +google-libraries-bom = "26.83.0" gcs-analytics-core = "1.2.3" guava = "33.6.0-jre" hadoop3 = "3.4.3" From 23c7f231a2e8b8197d4cbecf04a23d66bc033b46 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Sun, 24 May 2026 17:55:06 -0400 Subject: [PATCH 20/58] fix int96 timestamp offset in arrow dictionary decode (#16435) --- ...ectorizedParquetDefinitionLevelReader.java | 2 +- ...ectorizedParquetDefinitionLevelReader.java | 115 ++++++++++++++++++ 2 files changed, 116 insertions(+), 1 deletion(-) create mode 100644 arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/TestVectorizedParquetDefinitionLevelReader.java diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java index 1ca3bfe809c0..d093d4c97989 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java @@ -539,7 +539,7 @@ protected void nextDictEncodedVal( .toByteBuffer() .order(ByteOrder.LITTLE_ENDIAN); long timestampInt96 = ParquetUtil.extractTimestampInt96(buffer); - vector.getDataBuffer().setLong(idx, timestampInt96); + vector.getDataBuffer().setLong((long) idx * typeWidth, timestampInt96); break; default: throw new UnsupportedOperationException( diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/TestVectorizedParquetDefinitionLevelReader.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/TestVectorizedParquetDefinitionLevelReader.java new file mode 100644 index 000000000000..10bf3e64766c --- /dev/null +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/TestVectorizedParquetDefinitionLevelReader.java @@ -0,0 +1,115 @@ +/* + * 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.arrow.vectorized.parquet; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.io.api.Binary; +import org.junit.jupiter.api.Test; + +public class TestVectorizedParquetDefinitionLevelReader { + private static final int UNIX_EPOCH_JULIAN_DAY = 2_440_588; + + @Test + public void timestampInt96ReaderPackedDictionaryDecodeDecodesRowsCorrectly() { + try (RootAllocator allocator = new RootAllocator(); + BigIntVector vector = new BigIntVector("ts", allocator)) { + vector.allocateNew(2); + vector.set(0, -1L); + vector.set(1, -1L); + + VectorizedParquetDefinitionLevelReader definitionReader = + new VectorizedParquetDefinitionLevelReader(1, 1, false); + VectorizedDictionaryEncodedParquetValuesReader dictionaryReader = + new VectorizedDictionaryEncodedParquetValuesReader(1, false); + + dictionaryReader.mode = BaseVectorizedParquetValuesReader.Mode.PACKED; + dictionaryReader.currentCount = 2; + dictionaryReader.packedValuesBuffer[0] = 0; + dictionaryReader.packedValuesBuffer[1] = 1; + + Dictionary dictionary = + new Dictionary(Encoding.PLAIN_DICTIONARY) { + @Override + public int getMaxId() { + return 1; + } + + @Override + public Binary decodeToBinary(int id) { + if (id == 0) { + return int96Binary(111_111L); + } else if (id == 1) { + return int96Binary(222_222L); + } + + throw new IllegalArgumentException("Unexpected dictionary id: " + id); + } + }; + + VectorizedParquetDefinitionLevelReader.TimestampInt96Reader timestampReader = + definitionReader.timestampInt96Reader(); + + timestampReader.nextDictEncodedVal( + vector, + 0, + dictionaryReader, + dictionary, + BaseVectorizedParquetValuesReader.Mode.PACKED, + 1, + null, + Long.BYTES); + timestampReader.nextDictEncodedVal( + vector, + 1, + dictionaryReader, + dictionary, + BaseVectorizedParquetValuesReader.Mode.PACKED, + 1, + null, + Long.BYTES); + + vector.setValueCount(2); + + assertThat(vector.get(0)) + .as("row 0 should receive the first decoded timestamp") + .isEqualTo(111_111L); + assertThat(vector.get(1)) + .as("row 1 should receive the second decoded timestamp") + .isEqualTo(222_222L); + } + } + + private static Binary int96Binary(long micros) { + long timeOfDayNanos = micros * 1_000L; + byte[] bytes = + ByteBuffer.allocate(12) + .order(ByteOrder.LITTLE_ENDIAN) + .putLong(timeOfDayNanos) + .putInt(UNIX_EPOCH_JULIAN_DAY) + .array(); + return Binary.fromConstantByteArray(bytes); + } +} From d3cb9e354485bd5a27bfade31c8a241193b78cfa Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 25 May 2026 14:45:59 +0900 Subject: [PATCH 21/58] Core: Replace deprecated CloseableHttpClient.execute (#16149) --- .../org/apache/iceberg/rest/HTTPClient.java | 101 ++++++++++-------- 1 file changed, 58 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 46d9177b9571..d2ddc22aee97 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -36,7 +36,6 @@ import org.apache.hc.client5.http.config.ConnectionConfig; import org.apache.hc.client5.http.impl.auth.BasicCredentialsProvider; import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; -import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.apache.hc.client5.http.impl.classic.HttpClients; import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; @@ -44,6 +43,7 @@ import org.apache.hc.client5.http.protocol.HttpClientContext; import org.apache.hc.client5.http.ssl.DefaultClientTlsStrategy; import org.apache.hc.client5.http.ssl.HostnameVerificationPolicy; +import org.apache.hc.core5.http.ClassicHttpResponse; import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpHeaders; @@ -164,7 +164,7 @@ public HTTPClient withAuthSession(AuthSession session) { return new HTTPClient(this, session); } - private static String extractResponseBodyAsString(CloseableHttpResponse response) { + private static String extractResponseBodyAsString(ClassicHttpResponse response) { try { if (response.getEntity() == null) { return null; @@ -177,7 +177,7 @@ private static String extractResponseBodyAsString(CloseableHttpResponse response } } - private static boolean isSuccessful(CloseableHttpResponse response) { + private static boolean isSuccessful(ClassicHttpResponse response) { int code = response.getCode(); return code == HttpStatus.SC_OK || code == HttpStatus.SC_ACCEPTED @@ -185,7 +185,7 @@ private static boolean isSuccessful(CloseableHttpResponse response) { || code == HttpStatus.SC_NOT_MODIFIED; } - private static ErrorResponse buildDefaultErrorResponse(CloseableHttpResponse response) { + private static ErrorResponse buildDefaultErrorResponse(ClassicHttpResponse response) { String responseReason = response.getReasonPhrase(); String message = responseReason != null && !responseReason.isEmpty() @@ -202,7 +202,7 @@ private static ErrorResponse buildDefaultErrorResponse(CloseableHttpResponse res // Process a failed response through the provided errorHandler, and throw a RESTException if the // provided error handler doesn't already throw. private static void throwFailure( - CloseableHttpResponse response, String responseBody, Consumer errorHandler) { + ClassicHttpResponse response, String responseBody, Consumer errorHandler) { ErrorResponse errorResponse = null; if (responseBody != null) { @@ -300,7 +300,6 @@ protected T execute( req, responseType, errorHandler, responseHeaders, ParserContext.builder().build()); } - @SuppressWarnings("deprecation") @Override protected T execute( HTTPRequest req, @@ -318,53 +317,69 @@ protected T execute( } HttpContext context = HttpClientContext.create(); - try (CloseableHttpResponse response = httpClient.execute(request, context)) { - Map respHeaders = Maps.newHashMap(); - for (Header header : response.getHeaders()) { - respHeaders.put(header.getName(), header.getValue()); - } - - responseHeaders.accept(respHeaders); - - // Skip parsing the response stream for any successful request not expecting a response body - if (emptyBody(response, responseType)) { - if (response.getCode() == HttpStatus.SC_NOT_MODIFIED - && !req.headers().contains(HttpHeaders.IF_NONE_MATCH)) { - // 304-NOT_MODIFIED is used for freshness-aware loading and requires an ETag sent to the - // server via IF_NONE_MATCH header in the request. If no ETag was sent, we shouldn't - // receive a 304. - throw new RESTException( - "Invalid (NOT_MODIFIED) response for request: method=%s, path=%s", - req.method(), req.path()); - } + try { + return httpClient.execute( + request, + context, + response -> + handleResponse( + req, response, responseType, errorHandler, responseHeaders, parserContext)); + } catch (IOException e) { + throw new RESTException(e, "Error occurred while processing %s request", req.method()); + } + } - return null; - } + private T handleResponse( + HTTPRequest request, + ClassicHttpResponse response, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders, + ParserContext parserContext) + throws IOException { + Map respHeaders = Maps.newHashMap(); + for (Header header : response.getHeaders()) { + respHeaders.put(header.getName(), header.getValue()); + } - if (!isSuccessful(response)) { - // The provided error handler is expected to throw, but a RESTException is thrown if not. - String responseBody = extractResponseBodyAsString(response); - throwFailure(response, responseBody, errorHandler); - } + responseHeaders.accept(respHeaders); - if (response.getEntity() == null) { + // Skip parsing the response stream for any successful request not expecting a response body + if (emptyBody(response, responseType)) { + if (response.getCode() == HttpStatus.SC_NOT_MODIFIED + && !request.headers().contains(HttpHeaders.IF_NONE_MATCH)) { + // 304-NOT_MODIFIED is used for freshness-aware loading and requires an ETag sent to the + // server via IF_NONE_MATCH header in the request. If no ETag was sent, we shouldn't + // receive a 304. throw new RESTException( - "Invalid (null) response body for request (expected %s): method=%s, path=%s, status=%d", - responseType.getSimpleName(), req.method(), req.path(), response.getCode()); + "Invalid (NOT_MODIFIED) response for request: method=%s, path=%s", + request.method(), request.path()); } - ObjectReader reader = objectReaderCache.computeIfAbsent(responseType, mapper::readerFor); - if (parserContext != null && !parserContext.isEmpty()) { - reader = reader.with(parserContext.toInjectableValues()); - } - return reader.readValue(response.getEntity().getContent()); - } catch (IOException e) { - throw new RESTException(e, "Error occurred while processing %s request", req.method()); + return null; + } + + if (!isSuccessful(response)) { + // The provided error handler is expected to throw, but a RESTException is thrown if not. + String responseBody = extractResponseBodyAsString(response); + throwFailure(response, responseBody, errorHandler); + } + + if (response.getEntity() == null) { + throw new RESTException( + "Invalid (null) response body for request (expected %s): method=%s, path=%s, status=%d", + responseType.getSimpleName(), request.method(), request.path(), response.getCode()); + } + + ObjectReader reader = objectReaderCache.computeIfAbsent(responseType, mapper::readerFor); + if (parserContext != null && !parserContext.isEmpty()) { + reader = reader.with(parserContext.toInjectableValues()); } + return reader.readValue(response.getEntity().getContent()); } private boolean emptyBody( - CloseableHttpResponse response, Class responseType) { + ClassicHttpResponse response, Class responseType) { return response.getCode() == HttpStatus.SC_NO_CONTENT || response.getCode() == HttpStatus.SC_NOT_MODIFIED || (responseType == null && isSuccessful(response)); From f12e0cfd07c50ef42f5785c7e8f938009128c02c Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Mon, 25 May 2026 14:37:02 +0800 Subject: [PATCH 22/58] Data: Remove Flag FEATURE_META_ROW_LINEAGE in BaseFormatModelTests (#16529) --- .../org/apache/iceberg/data/BaseFormatModelTests.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/data/BaseFormatModelTests.java b/data/src/test/java/org/apache/iceberg/data/BaseFormatModelTests.java index bdff9f27b2cf..5373749406bc 100644 --- a/data/src/test/java/org/apache/iceberg/data/BaseFormatModelTests.java +++ b/data/src/test/java/org/apache/iceberg/data/BaseFormatModelTests.java @@ -146,7 +146,6 @@ protected boolean supportsBatchReads() { static final String FEATURE_SPLIT = "split"; static final String FEATURE_READER_DEFAULT = "readerDefault"; static final String FEATURE_REUSE_CONTAINERS = "reuseContainers"; - static final String FEATURE_META_ROW_LINEAGE = "metaRowLineage"; static final String FEATURE_COLUMN_LEVEL_METRICS = "columnLevelMetrics"; static final String FEATURE_COLUMN_METRICS_TRUNCATE_BINARY = "columnMetricsTruncateBinary"; @@ -162,10 +161,7 @@ protected boolean supportsBatchReads() { }, FileFormat.ORC, new String[] { - FEATURE_REUSE_CONTAINERS, - FEATURE_COLUMN_METRICS_TRUNCATE_BINARY, - FEATURE_META_ROW_LINEAGE, - FEATURE_READER_DEFAULT + FEATURE_REUSE_CONTAINERS, FEATURE_COLUMN_METRICS_TRUNCATE_BINARY, FEATURE_READER_DEFAULT }); private InMemoryFileIO fileIO; @@ -1121,7 +1117,6 @@ void testReadMetadataColumnIsDeleted(FileFormat fileFormat) throws IOException { @ParameterizedTest @FieldSource("FILE_FORMATS") void testReadMetadataColumnRowLinage(FileFormat fileFormat) throws IOException { - assumeSupports(fileFormat, FEATURE_META_ROW_LINEAGE); DataGenerator dataGenerator = new DataGenerators.DefaultSchema(); Schema schema = dataGenerator.schema(); @@ -1155,7 +1150,6 @@ void testReadMetadataColumnRowLinage(FileFormat fileFormat) throws IOException { @ParameterizedTest @FieldSource("FILE_FORMATS") void testReadMetadataColumnRowLinageExistValue(FileFormat fileFormat) throws IOException { - assumeSupports(fileFormat, FEATURE_META_ROW_LINEAGE); DataGenerator dataGenerator = new DataGenerators.DefaultSchema(); Schema dataSchema = dataGenerator.schema(); From d2cde2950460bcb86a69ee10c98de962b5557b2a Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Tue, 26 May 2026 10:04:15 -0700 Subject: [PATCH 23/58] REST Spec: Add unregister table endpoint (#16400) Co-Authored-By: Claude Code (Opus 4.7, 1M context) --- open-api/rest-catalog-open-api.py | 13 +++++ open-api/rest-catalog-open-api.yaml | 74 +++++++++++++++++++++++++++++ 2 files changed, 87 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 30ae491248fa..393f8bd245dc 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -1575,6 +1575,19 @@ class CreateTableRequest(BaseModel): properties: dict[str, str] | None = None +class UnregisterTableResult(BaseModel): + """ + Last metadata location and the corresponding table metadata for the table that was successfully unregistered and is no longer tracked by the catalog. + """ + + metadata_location: str = Field( + ..., + alias='metadata-location', + description='The last metadata location for the table at the time it was unregistered.', + ) + metadata: TableMetadata + + class CreateViewRequest(BaseModel): name: str location: str | None = None diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 0e5cd9c95e75..ccad6004aede 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1217,6 +1217,56 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/unregister: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + + post: + tags: + - Catalog API + summary: Unregister a table without removing its data or metadata files + operationId: unregisterTable + parameters: + - $ref: '#/components/parameters/idempotency-key' + description: + Unregister a table from the catalog. This is the opposite of + `registerTable`. The table no longer exists in the catalog, but the + underlying data and metadata files are left in place so that the table + can be registered in another catalog. + + + On success, this returns the table's last metadata location and the + corresponding table metadata. This table metadata must include all + commits that happened before the unregister operation. All attempted + commits after the unregister operation in this catalog must fail. + responses: + 200: + $ref: '#/components/responses/UnregisterTableResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchTableException, table to unregister does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableToUnregisterDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/credentials: parameters: - $ref: '#/components/parameters/prefix' @@ -3792,6 +3842,23 @@ components: type: boolean default: false + UnregisterTableResult: + description: + Last metadata location and the corresponding table metadata for the + table that was successfully unregistered and is no longer tracked by + the catalog. + type: object + required: + - metadata-location + - metadata + properties: + metadata-location: + type: string + description: + The last metadata location for the table at the time it was unregistered. + metadata: + $ref: '#/components/schemas/TableMetadata' + CreateViewRequest: type: object required: @@ -5059,6 +5126,13 @@ components: etag: $ref: '#/components/parameters/etag' + UnregisterTableResponse: + description: Response when a table is successfully unregistered. + content: + application/json: + schema: + $ref: '#/components/schemas/UnregisterTableResult' + LoadViewResponse: description: View metadata result when loading a view content: From 8629e7c2de5bf36dcdb5a5a077d5eaaea05e4a2e Mon Sep 17 00:00:00 2001 From: Vova Kolmakov Date: Wed, 27 May 2026 21:20:21 +0700 Subject: [PATCH 24/58] Flink: Fix flaky TestMonitorSource.testStateRestore (#16548) --- .../operator/OperatorTestBase.java | 19 ++++++++++++------- .../operator/OperatorTestBase.java | 15 ++++++++++----- .../operator/OperatorTestBase.java | 15 ++++++++++----- 3 files changed, 32 insertions(+), 17 deletions(-) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 638da61949c9..5d90d262c78f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -27,6 +27,7 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; @@ -373,13 +374,17 @@ protected static Configuration closeJobClient(JobClient jobClient, File savepoin Configuration conf = new Configuration(); if (jobClient != null) { if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - conf.set( - SavepointConfigOptions.SAVEPOINT_PATH, - savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + // Stop with a savepoint; get() blocks until it is fully written and returns its path, so + // that a job restoring from it does not race the savepoint completion + try { + conf.set( + SavepointConfigOptions.SAVEPOINT_PATH, + jobClient + .stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL) + .get()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } } else { jobClient.cancel(); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 4f394be76f4f..56749b552aa5 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -27,6 +27,7 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.configuration.Configuration; @@ -373,11 +374,15 @@ protected TableLoader tableLoader() { protected static String closeJobClient(JobClient jobClient, File savepointDir) { if (jobClient != null) { if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - return savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath(); + // Stop with a savepoint; get() blocks until it is fully written and returns its path, so + // that a job restoring from it does not race the savepoint completion + try { + return jobClient + .stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL) + .get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } } else { jobClient.cancel(); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 4f394be76f4f..56749b552aa5 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -27,6 +27,7 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.configuration.Configuration; @@ -373,11 +374,15 @@ protected TableLoader tableLoader() { protected static String closeJobClient(JobClient jobClient, File savepointDir) { if (jobClient != null) { if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - return savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath(); + // Stop with a savepoint; get() blocks until it is fully written and returns its path, so + // that a job restoring from it does not race the savepoint completion + try { + return jobClient + .stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL) + .get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } } else { jobClient.cancel(); } From 176c6e30f61280c41cc93ea98a73aa3b9403fad5 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 27 May 2026 11:34:19 -0700 Subject: [PATCH 25/58] Spark: Trim row-level test parameter rows from 6 to 3 (#16549) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Spark: Trim row-level test parameter rows from 6 to 3 Reduces SparkRowLevelOperationsTestBase parameter rows from 6 to 3 in v4.0 / v4.1 (and from 7 to 3 in v3.5), shifting from "test every catalog backend" to "test the catalogs that matter for production": - testhive (Hive) — kept as the established Hive metastore baseline - testrest (REST) — added in place of testhadoop, since REST is the OSS-strategic catalog and testhadoop isn't recommended for prod - spark_catalog (REST-backed) — repointed from Hive to REST so the SessionCatalog row exercises the REST commit path instead of Hive formatVersion 2 covered by the testhive and testrest rows; formatVersion 3 covered by the spark_catalog/REST row, which exercises the DV (deletion-vector) path that validateSnapshot checks via formatVersion >= 3. The trim affects 9 concrete subclasses (TestCopyOnWriteMerge/Update/Delete, TestMergeOnReadMerge/Update/Delete, both *MergeMetrics, and TestMergeSchemaEvolution), each cutting test invocations by 50% (~57% on Spark 3.5). Note: TestCopyOnWriteWithLineage and TestMergeOnReadWithLineage are unaffected — TestRowLevelOperationsWithLineage redeclares parameters() with its own row set. Co-Authored-By: Claude Opus 4.7 (1M context) * Replace non-session REST row with second testhive row Drops the testrest row in favor of restoring testhive as the carrier for the HASH/null/DISTRIBUTED axes that previously sat on testhadoop. REST catalog coverage is preserved by the spark_catalog (REST-backed) row. Rationale: for the row-level operation code paths these tests exercise, a non-session SparkCatalog wrapper around REST adds little beyond what the SessionCatalog wrapper already covers. Both return SparkTable; both commit through the same MergingSnapshotProducer; the differences live in table-resolution paths (DDL/aliasing), not in MERGE/UPDATE/DELETE. Other test classes (TestStructuredStreamingRead3 etc.) already exercise REST as a non-session catalog. Eliminates 2 of the 3 known TestBase.move() / metadata-delete fixture failures from the previous version. The remaining failure (testDeleteWithoutScanningTable on spark_catalog/REST) is a pre-existing TestBase.move() limitation with non-URI paths and needs a follow-up fix. Co-Authored-By: Claude Opus 4.7 (1M context) * Spark: Fix TestBase.move() to handle locations without URI schemes Paths.get(URI.create(location)) requires the URI to carry a scheme. HiveCatalog returns manifest paths with file:// schemes, so the existing move() works. RESTCatalog (as configured by RESTServerExtension in tests) returns plain local paths without a scheme, which makes Paths.get(URI) throw IllegalArgumentException: Missing scheme. Add a small toPath() helper that falls back to Paths.get(location) when URI.create(location).getScheme() is null. This unblocks testDeleteWithoutScanningTable and the equivalent MERGE-side test on spark_catalog/REST rows. Co-Authored-By: Claude Opus 4.7 (1M context) * Build: Fix REST catalog test fixture warehouse path scheme Move the URI-scheme handling from TestBase.move() (Spark) to the source of the inconsistency: the REST catalog test fixture's default warehouse path. RESTCatalogServer used getAbsolutePath() which returns plain filesystem paths without a URI scheme; HiveCatalog and HadoopCatalog return file:// paths via Hadoop's Path machinery, so test fixtures that consume catalog paths (e.g. TestBase.move() which calls Paths.get(URI.create(...))) work for those catalogs but break for REST. Switch to toURI().toString() so the REST fixture matches the Hive and Hadoop convention. testDeleteWithoutScanningTable and the equivalent MERGE-side test now pass on the spark_catalog/REST row without needing scheme-handling logic at every consumer. Reverts the TestBase.move() change from the previous commit. Co-Authored-By: Claude Opus 4.7 (1M context) * Spark: Drop REST-only path workaround in TestRewriteTablePathProcedure The test had a special-case branch that called file.getAbsolutePath() (no scheme) instead of file.toURI().toString() when catalogName was testrest, because the old REST fixture returned warehouse paths without a scheme and the test needed the deletes.parquet path to match. With RESTCatalogServer now using toURI().toString() for the default warehouse, table.location() returns a scheme-prefixed path on REST too, so the workaround is no longer needed - and is now incorrect, since RewriteTablePathUtil.newPositionDeleteEntry validates that delete file paths start with the table location prefix (which now includes file://). Removing the special case lets the simple toURI().toString() path apply uniformly across all catalogs. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../iceberg/rest/RESTCatalogServer.java | 2 +- .../SparkRowLevelOperationsTestBase.java | 70 ++----------------- .../TestRewriteTablePathProcedure.java | 7 -- .../SparkRowLevelOperationsTestBase.java | 58 ++------------- .../TestRewriteTablePathProcedure.java | 7 -- .../SparkRowLevelOperationsTestBase.java | 58 ++------------- .../TestRewriteTablePathProcedure.java | 7 -- 7 files changed, 16 insertions(+), 193 deletions(-) diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java index 2e4541b50b33..daed482c74a5 100644 --- a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java @@ -92,7 +92,7 @@ private CatalogContext initializeBackendCatalog() throws IOException { if (warehouseLocation == null) { File tmp = java.nio.file.Files.createTempDirectory("iceberg_warehouse").toFile(); tmp.deleteOnExit(); - warehouseLocation = new File(tmp, "iceberg_data").getAbsolutePath(); + warehouseLocation = new File(tmp, "iceberg_data").toURI().toString(); catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); LOG.info("No warehouse location set. Defaulting to temp location: {}", warehouseLocation); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 893f9931cfa2..72988ae0ed9e 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -46,11 +46,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; @@ -83,8 +82,6 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase { - private static final Random RANDOM = ThreadLocalRandom.current(); - @Parameter(index = 3) protected FileFormat fileFormat; @@ -135,79 +132,22 @@ public static Object[][] parameters() { "default-namespace", "default"), FileFormat.PARQUET, true, - WRITE_DISTRIBUTION_MODE_NONE, - false, - "test", - DISTRIBUTED, - 2 - }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - RANDOM.nextBoolean(), WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 2 - }, - { - "spark_catalog", - SparkSessionCatalog.class.getName(), - ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "clients", "1", - "parquet-enabled", "false", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ), - FileFormat.AVRO, false, - WRITE_DISTRIBUTION_MODE_RANGE, - false, - "test", + null, DISTRIBUTED, 2 }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - true, - WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 3 - }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - false, - WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 3 - }, { "spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of( "type", - "hive", + "rest", + CatalogProperties.URI, + restCatalog.properties().get(CatalogProperties.URI), "default-namespace", "default", - "clients", - "1", - "parquet-enabled", - "false", "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync ), diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index ceb3077c5670..996fb2636ada 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -35,7 +35,6 @@ import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.AnalysisException; import org.junit.jupiter.api.AfterEach; @@ -222,12 +221,6 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); String filePath = file.toURI().toString(); - if (SparkCatalogConfig.REST.catalogName().equals(catalogName)) { - // We applied this special handling because the base path for - // matching the RESTCATALOG's Hive BaseLocation is represented - // in the form of an AbsolutePath. - filePath = file.getAbsolutePath().toString(); - } DeleteFile positionDeletes = FileHelpers.writeDeleteFile(table, table.io().newOutputFile(filePath), rowsToDelete) diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index b5d641576314..72988ae0ed9e 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -46,11 +46,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; @@ -83,8 +82,6 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase { - private static final Random RANDOM = ThreadLocalRandom.current(); - @Parameter(index = 3) protected FileFormat fileFormat; @@ -135,67 +132,22 @@ public static Object[][] parameters() { "default-namespace", "default"), FileFormat.PARQUET, true, - WRITE_DISTRIBUTION_MODE_NONE, - false, - "test", - DISTRIBUTED, - 2 - }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - RANDOM.nextBoolean(), WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 2 - }, - { - "spark_catalog", - SparkSessionCatalog.class.getName(), - ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "clients", "1", - "parquet-enabled", "false", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ), - FileFormat.AVRO, false, - WRITE_DISTRIBUTION_MODE_RANGE, - false, - "test", + null, DISTRIBUTED, 2 }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - RANDOM.nextBoolean(), - WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 3 - }, { "spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of( "type", - "hive", + "rest", + CatalogProperties.URI, + restCatalog.properties().get(CatalogProperties.URI), "default-namespace", "default", - "clients", - "1", - "parquet-enabled", - "false", "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync ), diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index 78f6b80ac948..cab33c8b005c 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -35,7 +35,6 @@ import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.AnalysisException; import org.junit.jupiter.api.AfterEach; @@ -224,12 +223,6 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); String filePath = file.toURI().toString(); - if (SparkCatalogConfig.REST.catalogName().equals(catalogName)) { - // We applied this special handling because the base path for - // matching the RESTCATALOG's Hive BaseLocation is represented - // in the form of an AbsolutePath. - filePath = file.getAbsolutePath().toString(); - } DeleteFile positionDeletes = FileHelpers.writeDeleteFile(table, table.io().newOutputFile(filePath), rowsToDelete) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index b5d641576314..72988ae0ed9e 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -46,11 +46,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; @@ -83,8 +82,6 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase { - private static final Random RANDOM = ThreadLocalRandom.current(); - @Parameter(index = 3) protected FileFormat fileFormat; @@ -135,67 +132,22 @@ public static Object[][] parameters() { "default-namespace", "default"), FileFormat.PARQUET, true, - WRITE_DISTRIBUTION_MODE_NONE, - false, - "test", - DISTRIBUTED, - 2 - }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - RANDOM.nextBoolean(), WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 2 - }, - { - "spark_catalog", - SparkSessionCatalog.class.getName(), - ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "clients", "1", - "parquet-enabled", "false", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ), - FileFormat.AVRO, false, - WRITE_DISTRIBUTION_MODE_RANGE, - false, - "test", + null, DISTRIBUTED, 2 }, - { - "testhadoop", - SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop"), - FileFormat.PARQUET, - RANDOM.nextBoolean(), - WRITE_DISTRIBUTION_MODE_HASH, - true, - null, - LOCAL, - 3 - }, { "spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of( "type", - "hive", + "rest", + CatalogProperties.URI, + restCatalog.properties().get(CatalogProperties.URI), "default-namespace", "default", - "clients", - "1", - "parquet-enabled", - "false", "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync ), diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index 78f6b80ac948..cab33c8b005c 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -35,7 +35,6 @@ import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.AnalysisException; import org.junit.jupiter.api.AfterEach; @@ -224,12 +223,6 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); String filePath = file.toURI().toString(); - if (SparkCatalogConfig.REST.catalogName().equals(catalogName)) { - // We applied this special handling because the base path for - // matching the RESTCATALOG's Hive BaseLocation is represented - // in the form of an AbsolutePath. - filePath = file.getAbsolutePath().toString(); - } DeleteFile positionDeletes = FileHelpers.writeDeleteFile(table, table.io().newOutputFile(filePath), rowsToDelete) From 36d79e74bb19e56b7b0ca3fcc25d9e557b118745 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 27 May 2026 11:35:46 -0700 Subject: [PATCH 26/58] Spark: Trim TestStructuredStreamingRead3 parameter rows from 8 to 2 (#16559) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reduces the parameter set in TestStructuredStreamingRead3 from 8 rows (4 catalogs × async{T,F}) to 2 rows: testhive (async=true) and testrest (async=false). Streaming read semantics aren't catalog-specific in any meaningful way, and async-vs-sync planning is the only axis TestStructuredStreamingRead3 actually exercises beyond basic Spark streaming behavior. Both async values and the strategic catalog backends (Hive metastore + REST) remain covered with one row each. Drops testhadoop (HadoopCatalog isn't recommended for production) and the spark_catalog SessionCatalog rows (the SessionCatalog wrapper differences live in DDL/table-resolution paths, not streaming reads). Each invocation runs 33 streaming tests, so 8→2 rows cuts the class from 264 to 66 invocations - roughly 75% reduction in CPU time for this test (the highest-CPU class in the Spark core CI job at 20.3% of total test CPU). Co-authored-by: Claude Opus 4.7 (1M context) --- .../source/TestStructuredStreamingRead3.java | 39 ------------------- .../source/TestStructuredStreamingRead3.java | 39 ------------------- .../source/TestStructuredStreamingRead3.java | 39 ------------------- 3 files changed, 117 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index d97e6ec00d7f..4efb883b5d6c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -91,24 +91,6 @@ public static Object[][] parameters() { SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), SparkCatalogConfig.HIVE.properties(), - false - }, - { - SparkCatalogConfig.HIVE.catalogName(), - SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties(), - true - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), - false - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), true }, { @@ -119,27 +101,6 @@ public static Object[][] parameters() { .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build(), false - }, - { - SparkCatalogConfig.REST.catalogName(), - SparkCatalogConfig.REST.implementation(), - ImmutableMap.builder() - .putAll(SparkCatalogConfig.REST.properties()) - .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) - .build(), - true - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - false - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - true } }; } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 5f9b460f3707..06189b304299 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -91,24 +91,6 @@ public static Object[][] parameters() { SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), SparkCatalogConfig.HIVE.properties(), - false - }, - { - SparkCatalogConfig.HIVE.catalogName(), - SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties(), - true - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), - false - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), true }, { @@ -119,27 +101,6 @@ public static Object[][] parameters() { .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build(), false - }, - { - SparkCatalogConfig.REST.catalogName(), - SparkCatalogConfig.REST.implementation(), - ImmutableMap.builder() - .putAll(SparkCatalogConfig.REST.properties()) - .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) - .build(), - true - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - false - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - true } }; } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 3957872be721..89947f73ea38 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -92,24 +92,6 @@ public static Object[][] parameters() { SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), SparkCatalogConfig.HIVE.properties(), - false - }, - { - SparkCatalogConfig.HIVE.catalogName(), - SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties(), - true - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), - false - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties(), true }, { @@ -120,27 +102,6 @@ public static Object[][] parameters() { .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build(), false - }, - { - SparkCatalogConfig.REST.catalogName(), - SparkCatalogConfig.REST.implementation(), - ImmutableMap.builder() - .putAll(SparkCatalogConfig.REST.properties()) - .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) - .build(), - true - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - false - }, - { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties(), - true } }; } From ebd0100b398dc9b8f60350866f9f064b4750cfc1 Mon Sep 17 00:00:00 2001 From: Pratham Manja Date: Thu, 28 May 2026 00:53:02 +0530 Subject: [PATCH 27/58] Docs: Document adaptive split sizing configurations --- docs/docs/spark-configuration.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 5972aafc3d39..8d9dd6dd2756 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -207,6 +207,8 @@ val spark = SparkSession.builder() | spark.sql.iceberg.executor-cache.locality.enabled | false | Enables locality-aware executor cache usage | | spark.sql.iceberg.merge-schema | false | Enables modifying the table schema to match the write schema. Only adds columns missing columns | | spark.sql.iceberg.report-column-stats | true | Report Puffin Table Statistics if available to Spark's Cost Based Optimizer. CBO must be enabled for this to be effective | +| spark.sql.iceberg.read.adaptive-split-size.enabled | Table default | Enables adaptive split sizing for read operations. When enabled, split size is automatically adjusted based on scan size and parallelism | +| spark.sql.iceberg.read.adaptive-split-size.parallelism | max(spark.default.parallelism, spark.sql.shuffle.partitions) | Overrides the parallelism used for adaptive split sizing. Must be greater than 0 | | spark.sql.iceberg.async-micro-batch-planning-enabled | false | Enables asynchronous microbatch planning to reduce planning latency by pre-fetching file scan tasks | ### Read options From 141e5be17c56a6c13d560e122f2ca46fab2dea14 Mon Sep 17 00:00:00 2001 From: sanshi <43472713+lilei1128@users.noreply.github.com> Date: Thu, 28 May 2026 11:12:50 +0800 Subject: [PATCH 28/58] Core: Fix flaky test by ensuring generateContentLength returns positive value (#16539) Co-authored-by: shisan --- core/src/test/java/org/apache/iceberg/FileGenerationUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 1d57d96203b9..7f8325eb641f 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -268,7 +268,7 @@ private static long generateValueCount() { } private static long generateFileSize() { - return random().nextInt(50_000); + return 1 + random().nextInt(50_000); } private static long generateContentOffset() { @@ -276,7 +276,7 @@ private static long generateContentOffset() { } private static long generateContentLength() { - return random().nextInt(10_000); + return 1 + random().nextInt(10_000); } private static Pair generateBounds(PrimitiveType type, MetricsMode mode) { From e88aa55687a4767a5fb3152248d3514831da2723 Mon Sep 17 00:00:00 2001 From: Jordan Epstein <32082339+jordepic@users.noreply.github.com> Date: Thu, 28 May 2026 00:07:57 -0500 Subject: [PATCH 29/58] Flink: Honor schema identifier fields in dynamic-sink record routing (#16243) --- docs/docs/flink-writes.md | 4 +- .../sink/dynamic/DynamicRecordProcessor.java | 15 ++- .../flink/sink/dynamic/DynamicSinkUtil.java | 15 +++ .../flink/sink/dynamic/HashKeyGenerator.java | 17 +-- .../dynamic/TestDynamicRecordProcessor.java | 101 ++++++++++++++++++ .../sink/dynamic/TestHashKeyGenerator.java | 97 +++++++++++++++++ 6 files changed, 239 insertions(+), 10 deletions(-) create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index 7be13dae4525..393d5ac563b1 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -487,10 +487,10 @@ We need the following information (DynamicRecord) for every record: | `Schema` | The schema of the record. | | `Spec` | The expected partitioning specification for the record. | | `RowData` | The actual row data to be written. | -| `DistributionMode` | The distribution mode for writing the record (NONE, HASH or `null`). When `null`, the record won't be shuffled at all. | +| `DistributionMode` | The distribution mode for writing the record (NONE, HASH or `null`). When `null`, the record won't be shuffled at all — except when the record resolves to a non-empty equality-field set, in which case the dynamic sink falls back to hash distribution to keep records sharing the same equality key on the same writer. | | `Parallelism` | The maximum number of parallel writers for a given table/branch/schema/spec (WriteTarget). | | `UpsertMode` | Overrides this table's write.upsert.enabled (optional). | -| `EqualityFields` | The equality fields for the table(optional). | +| `EqualityFields` | The equality fields for the table (optional). When unset, the dynamic sink falls back to the schema's identifier fields for both distribution (records sharing a key route to the same writer) and equality-delete inference. Identifier fields do not auto-enable upsert mode — that flag remains opt-in. | ### Schema Evolution diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index c752b8e9b8d9..ba2d816ae9ea 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -129,7 +129,7 @@ public void processElement(T element, Context ctx, Collector getEqualityFieldIds(Set equalityFields, Schema schem return equalityFieldIds; } + /** + * Resolves the effective equality field names. Returns the user-supplied set when non-empty, + * otherwise falls back to the names of {@link Schema#identifierFieldIds()}. Mirrors {@link + * #getEqualityFieldIds} so distribution and write-side equality-field inference stay aligned. + */ + static Set resolveEqualityFieldNames( + @Nullable Set equalityFields, Schema schema) { + if (equalityFields != null && !equalityFields.isEmpty()) { + return equalityFields; + } + + return schema.identifierFieldNames(); + } + static int safeAbs(int input) { if (input >= 0) { return input; diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 61a850212bf4..03541ed596f0 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -20,7 +20,6 @@ import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -79,6 +78,11 @@ int generateKey( @Nullable PartitionSpec tableSpec, @Nullable RowData overrideRowData) { String tableIdent = dynamicRecord.tableIdentifier().toString(); + Schema effectiveSchema = MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()); + Set effectiveEqualityFields = + DynamicSinkUtil.resolveEqualityFieldNames(dynamicRecord.equalityFields(), effectiveSchema); + DistributionMode effectiveDistributionMode = + MoreObjects.firstNonNull(dynamicRecord.distributionMode(), DistributionMode.NONE); SelectorKey cacheKey = new SelectorKey( tableIdent, @@ -87,8 +91,8 @@ int generateKey( tableSpec != null ? tableSpec.specId() : null, dynamicRecord.schema(), dynamicRecord.spec(), - dynamicRecord.equalityFields(), - dynamicRecord.distributionMode(), + effectiveEqualityFields, + effectiveDistributionMode, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism)); KeySelector keySelector = keySelectorCache.computeIfAbsent( @@ -96,11 +100,10 @@ int generateKey( k -> getKeySelector( tableIdent, - MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()), + effectiveSchema, MoreObjects.firstNonNull(tableSpec, dynamicRecord.spec()), - dynamicRecord.distributionMode(), - MoreObjects.firstNonNull( - dynamicRecord.equalityFields(), Collections.emptySet()), + effectiveDistributionMode, + effectiveEqualityFields, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java new file mode 100644 index 000000000000..298a338ea9d0 --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import org.apache.flink.table.data.GenericRowData; +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.SnapshotRef; +import org.apache.iceberg.catalog.TableIdentifier; +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; +import org.junit.jupiter.api.Test; + +class TestDynamicRecordProcessor { + + 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 Schema SCHEMA_WITH_IDENTIFIER = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; + + @Test + void testForwardEligibleWhenNoEqualityFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isTrue(); + } + + @Test + void testNotForwardEligibleWhenDistributionModeSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, DistributionMode.NONE); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenUserEqualityFieldsSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + record.setEqualityFields(Collections.singleton("id")); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenSchemaHasIdentifierFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenEmptyEqualityFieldsButIdentifierFieldsPresent() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + record.setEqualityFields(Collections.emptySet()); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + private static DynamicRecord recordWithDistributionMode( + Schema schema, DistributionMode distributionMode) { + return new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schema, + GenericRowData.of(1, StringData.fromString("foo")), + PartitionSpec.unpartitioned(), + distributionMode, + 2); + } +} diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 9a485fafaf47..4181f00d0a0a 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -38,6 +38,7 @@ import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -294,6 +295,102 @@ void testCapAtMaxWriteParallelism() throws Exception { .isEqualTo(maxWriteParallelism); } + @Test + void testIdentifierFieldsResolvedAsEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + 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")); + + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row1, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row2, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row3, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + + @Test + void testForwardRecordWithIdentifierFieldsRoutesByEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + GenericRowData rowSameKey1 = GenericRowData.of(1, StringData.fromString("foo")); + GenericRowData rowSameKey2 = GenericRowData.of(1, StringData.fromString("bar")); + GenericRowData rowDifferentKey = GenericRowData.of(2, StringData.fromString("baz")); + + // Forward records (null distribution mode) with identifier fields on the schema must still + // route by equality fields, so HashKeyGenerator can handle them when the processor falls back + // off the forward path. + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey1, unpartitioned); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey2, unpartitioned); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowDifferentKey, unpartitioned); + record1.writeParallelism(writeParallelism); + record2.writeParallelism(writeParallelism); + record3.writeParallelism(writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + @Test void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeParallelism = 2; From 1a5b46cd02c7aea50524a36d57b5f3996a3ebacd Mon Sep 17 00:00:00 2001 From: SevenJ <166966490+Wenjun7J@users.noreply.github.com> Date: Thu, 28 May 2026 22:27:57 +0800 Subject: [PATCH 30/58] Core: Cache PartitionData template in PartitionsTable to avoid rebuilding Avro schema per partition (#16208) * avoid partition schema alloc frequently Signed-off-by: SevenJ * Test: fix spotless formatting in metadata table scan test * update the ut code Signed-off-by: SevenJ --------- Signed-off-by: SevenJ --- .../org/apache/iceberg/PartitionsTable.java | 21 ++++++++++++------- .../iceberg/TestMetadataTableScans.java | 17 +++++++++++++++ 2 files changed, 31 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index 10366db5a55d..b5dd5c284ce2 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -193,8 +193,10 @@ private static StaticDataTask.Row convertPartition(Partition partition) { partition.lastUpdatedSnapshotId); } - private static Iterable partitions(Table table, StaticTableScan scan) { + @VisibleForTesting + static Iterable partitions(Table table, StaticTableScan scan) { Types.StructType partitionType = Partitioning.partitionType(table); + PartitionData partitionDataTemplate = new PartitionData(partitionType); StructLikeMap partitions = StructLikeMap.create(partitionType, new PartitionComparator(partitionType)); @@ -207,7 +209,7 @@ private static Iterable partitions(Table table, StaticTableScan scan) PartitionUtil.coercePartition( partitionType, table.specs().get(file.specId()), file.partition()); partitions - .computeIfAbsent(key, () -> new Partition(key, partitionType)) + .computeIfAbsent(key, () -> new Partition(key, partitionDataTemplate)) .update(file, snapshot); } } catch (IOException e) { @@ -309,8 +311,8 @@ static class Partition { private Long lastUpdatedAt; private Long lastUpdatedSnapshotId; - Partition(StructLike key, Types.StructType keyType) { - this.partitionData = toPartitionData(key, keyType); + Partition(StructLike key, PartitionData partitionDataTemplate) { + this.partitionData = toPartitionData(key, partitionDataTemplate); this.specId = 0; this.dataRecordCount = 0L; this.dataFileCount = 0; @@ -321,6 +323,11 @@ static class Partition { this.eqDeleteFileCount = 0; } + @VisibleForTesting + PartitionData partitionData() { + return partitionData; + } + void update(ContentFile file, Snapshot snapshot) { if (snapshot != null) { long snapshotCommitTime = snapshot.timestampMillis() * 1000; @@ -353,9 +360,9 @@ void update(ContentFile file, Snapshot snapshot) { } /** Needed because StructProjection is not serializable */ - private static PartitionData toPartitionData(StructLike key, Types.StructType keyType) { - PartitionData keyTemplate = new PartitionData(keyType); - return keyTemplate.copyFor(key); + private static PartitionData toPartitionData( + StructLike key, PartitionData partitionDataTemplate) { + return partitionDataTemplate.copyFor(key); } } } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 210dfb3683ab..d5a0402a7aa4 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -521,6 +521,23 @@ public void testPartitionsTableScanWithProjection() { validateSingleFieldPartition(entries, 3); } + @TestTemplate + public void testPartitionsTableReusesPartitionDataSchema() { + preparePartitionedTable(); + + Table partitionsTable = new PartitionsTable(table); + StaticTableScan scan = (StaticTableScan) partitionsTable.newScan(); + List partitionSchemas = Lists.newArrayList(); + + for (PartitionsTable.Partition partition : PartitionsTable.partitions(table, scan)) { + partitionSchemas.add(partition.partitionData().getSchema()); + } + + assertThat(partitionSchemas).hasSizeGreaterThanOrEqualTo(2); + org.apache.avro.Schema expectedSchema = partitionSchemas.get(0); + assertThat(partitionSchemas).allSatisfy(schema -> assertThat(schema).isSameAs(expectedSchema)); + } + @TestTemplate public void testPartitionsTableScanNoStats() { table.newFastAppend().appendFile(FILE_WITH_STATS).commit(); From e25cfe72d0e9063b554d9dbcd23eb430b8a6fe46 Mon Sep 17 00:00:00 2001 From: Jordan Epstein <32082339+jordepic@users.noreply.github.com> Date: Thu, 28 May 2026 10:56:32 -0500 Subject: [PATCH 31/58] Flink: Backport honor schema identifier fields in dynamic-sink record routing (#16597) Backports #16243 --- .../sink/dynamic/DynamicRecordProcessor.java | 15 ++- .../flink/sink/dynamic/DynamicSinkUtil.java | 15 +++ .../flink/sink/dynamic/HashKeyGenerator.java | 17 +-- .../dynamic/TestDynamicRecordProcessor.java | 101 ++++++++++++++++++ .../sink/dynamic/TestHashKeyGenerator.java | 97 +++++++++++++++++ .../sink/dynamic/DynamicRecordProcessor.java | 15 ++- .../flink/sink/dynamic/DynamicSinkUtil.java | 15 +++ .../flink/sink/dynamic/HashKeyGenerator.java | 17 +-- .../dynamic/TestDynamicRecordProcessor.java | 101 ++++++++++++++++++ .../sink/dynamic/TestHashKeyGenerator.java | 97 +++++++++++++++++ 10 files changed, 474 insertions(+), 16 deletions(-) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index c752b8e9b8d9..ba2d816ae9ea 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -129,7 +129,7 @@ public void processElement(T element, Context ctx, Collector getEqualityFieldIds(Set equalityFields, Schema schem return equalityFieldIds; } + /** + * Resolves the effective equality field names. Returns the user-supplied set when non-empty, + * otherwise falls back to the names of {@link Schema#identifierFieldIds()}. Mirrors {@link + * #getEqualityFieldIds} so distribution and write-side equality-field inference stay aligned. + */ + static Set resolveEqualityFieldNames( + @Nullable Set equalityFields, Schema schema) { + if (equalityFields != null && !equalityFields.isEmpty()) { + return equalityFields; + } + + return schema.identifierFieldNames(); + } + static int safeAbs(int input) { if (input >= 0) { return input; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 61a850212bf4..03541ed596f0 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -20,7 +20,6 @@ import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -79,6 +78,11 @@ int generateKey( @Nullable PartitionSpec tableSpec, @Nullable RowData overrideRowData) { String tableIdent = dynamicRecord.tableIdentifier().toString(); + Schema effectiveSchema = MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()); + Set effectiveEqualityFields = + DynamicSinkUtil.resolveEqualityFieldNames(dynamicRecord.equalityFields(), effectiveSchema); + DistributionMode effectiveDistributionMode = + MoreObjects.firstNonNull(dynamicRecord.distributionMode(), DistributionMode.NONE); SelectorKey cacheKey = new SelectorKey( tableIdent, @@ -87,8 +91,8 @@ int generateKey( tableSpec != null ? tableSpec.specId() : null, dynamicRecord.schema(), dynamicRecord.spec(), - dynamicRecord.equalityFields(), - dynamicRecord.distributionMode(), + effectiveEqualityFields, + effectiveDistributionMode, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism)); KeySelector keySelector = keySelectorCache.computeIfAbsent( @@ -96,11 +100,10 @@ int generateKey( k -> getKeySelector( tableIdent, - MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()), + effectiveSchema, MoreObjects.firstNonNull(tableSpec, dynamicRecord.spec()), - dynamicRecord.distributionMode(), - MoreObjects.firstNonNull( - dynamicRecord.equalityFields(), Collections.emptySet()), + effectiveDistributionMode, + effectiveEqualityFields, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java new file mode 100644 index 000000000000..298a338ea9d0 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import org.apache.flink.table.data.GenericRowData; +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.SnapshotRef; +import org.apache.iceberg.catalog.TableIdentifier; +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; +import org.junit.jupiter.api.Test; + +class TestDynamicRecordProcessor { + + 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 Schema SCHEMA_WITH_IDENTIFIER = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; + + @Test + void testForwardEligibleWhenNoEqualityFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isTrue(); + } + + @Test + void testNotForwardEligibleWhenDistributionModeSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, DistributionMode.NONE); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenUserEqualityFieldsSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + record.setEqualityFields(Collections.singleton("id")); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenSchemaHasIdentifierFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenEmptyEqualityFieldsButIdentifierFieldsPresent() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + record.setEqualityFields(Collections.emptySet()); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + private static DynamicRecord recordWithDistributionMode( + Schema schema, DistributionMode distributionMode) { + return new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schema, + GenericRowData.of(1, StringData.fromString("foo")), + PartitionSpec.unpartitioned(), + distributionMode, + 2); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 9a485fafaf47..4181f00d0a0a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -38,6 +38,7 @@ import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -294,6 +295,102 @@ void testCapAtMaxWriteParallelism() throws Exception { .isEqualTo(maxWriteParallelism); } + @Test + void testIdentifierFieldsResolvedAsEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + 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")); + + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row1, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row2, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row3, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + + @Test + void testForwardRecordWithIdentifierFieldsRoutesByEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + GenericRowData rowSameKey1 = GenericRowData.of(1, StringData.fromString("foo")); + GenericRowData rowSameKey2 = GenericRowData.of(1, StringData.fromString("bar")); + GenericRowData rowDifferentKey = GenericRowData.of(2, StringData.fromString("baz")); + + // Forward records (null distribution mode) with identifier fields on the schema must still + // route by equality fields, so HashKeyGenerator can handle them when the processor falls back + // off the forward path. + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey1, unpartitioned); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey2, unpartitioned); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowDifferentKey, unpartitioned); + record1.writeParallelism(writeParallelism); + record2.writeParallelism(writeParallelism); + record3.writeParallelism(writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + @Test void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeParallelism = 2; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index c752b8e9b8d9..ba2d816ae9ea 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -129,7 +129,7 @@ public void processElement(T element, Context ctx, Collector getEqualityFieldIds(Set equalityFields, Schema schem return equalityFieldIds; } + /** + * Resolves the effective equality field names. Returns the user-supplied set when non-empty, + * otherwise falls back to the names of {@link Schema#identifierFieldIds()}. Mirrors {@link + * #getEqualityFieldIds} so distribution and write-side equality-field inference stay aligned. + */ + static Set resolveEqualityFieldNames( + @Nullable Set equalityFields, Schema schema) { + if (equalityFields != null && !equalityFields.isEmpty()) { + return equalityFields; + } + + return schema.identifierFieldNames(); + } + static int safeAbs(int input) { if (input >= 0) { 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 61a850212bf4..03541ed596f0 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 @@ -20,7 +20,6 @@ import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -79,6 +78,11 @@ int generateKey( @Nullable PartitionSpec tableSpec, @Nullable RowData overrideRowData) { String tableIdent = dynamicRecord.tableIdentifier().toString(); + Schema effectiveSchema = MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()); + Set effectiveEqualityFields = + DynamicSinkUtil.resolveEqualityFieldNames(dynamicRecord.equalityFields(), effectiveSchema); + DistributionMode effectiveDistributionMode = + MoreObjects.firstNonNull(dynamicRecord.distributionMode(), DistributionMode.NONE); SelectorKey cacheKey = new SelectorKey( tableIdent, @@ -87,8 +91,8 @@ int generateKey( tableSpec != null ? tableSpec.specId() : null, dynamicRecord.schema(), dynamicRecord.spec(), - dynamicRecord.equalityFields(), - dynamicRecord.distributionMode(), + effectiveEqualityFields, + effectiveDistributionMode, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism)); KeySelector keySelector = keySelectorCache.computeIfAbsent( @@ -96,11 +100,10 @@ int generateKey( k -> getKeySelector( tableIdent, - MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()), + effectiveSchema, MoreObjects.firstNonNull(tableSpec, dynamicRecord.spec()), - dynamicRecord.distributionMode(), - MoreObjects.firstNonNull( - dynamicRecord.equalityFields(), Collections.emptySet()), + effectiveDistributionMode, + effectiveEqualityFields, Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java new file mode 100644 index 000000000000..298a338ea9d0 --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordProcessor.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import org.apache.flink.table.data.GenericRowData; +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.SnapshotRef; +import org.apache.iceberg.catalog.TableIdentifier; +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; +import org.junit.jupiter.api.Test; + +class TestDynamicRecordProcessor { + + 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 Schema SCHEMA_WITH_IDENTIFIER = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; + + @Test + void testForwardEligibleWhenNoEqualityFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isTrue(); + } + + @Test + void testNotForwardEligibleWhenDistributionModeSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, DistributionMode.NONE); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenUserEqualityFieldsSet() { + DynamicRecord record = recordWithDistributionMode(SCHEMA, null); + record.setEqualityFields(Collections.singleton("id")); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenSchemaHasIdentifierFields() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + @Test + void testNotForwardEligibleWhenEmptyEqualityFieldsButIdentifierFieldsPresent() { + DynamicRecord record = recordWithDistributionMode(SCHEMA_WITH_IDENTIFIER, null); + record.setEqualityFields(Collections.emptySet()); + + assertThat(DynamicRecordProcessor.isForwardEligible(record)).isFalse(); + } + + private static DynamicRecord recordWithDistributionMode( + Schema schema, DistributionMode distributionMode) { + return new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schema, + GenericRowData.of(1, StringData.fromString("foo")), + PartitionSpec.unpartitioned(), + distributionMode, + 2); + } +} 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 9a485fafaf47..4181f00d0a0a 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 @@ -38,6 +38,7 @@ import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -294,6 +295,102 @@ void testCapAtMaxWriteParallelism() throws Exception { .isEqualTo(maxWriteParallelism); } + @Test + void testIdentifierFieldsResolvedAsEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + 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")); + + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row1, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row2, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, + BRANCH, + schemaWithIdentifier, + row3, + unpartitioned, + DistributionMode.NONE, + writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + + @Test + void testForwardRecordWithIdentifierFieldsRoutesByEqualityFields() throws Exception { + int writeParallelism = 2; + int maxWriteParallelism = 8; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + Schema schemaWithIdentifier = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); + PartitionSpec unpartitioned = PartitionSpec.unpartitioned(); + + GenericRowData rowSameKey1 = GenericRowData.of(1, StringData.fromString("foo")); + GenericRowData rowSameKey2 = GenericRowData.of(1, StringData.fromString("bar")); + GenericRowData rowDifferentKey = GenericRowData.of(2, StringData.fromString("baz")); + + // Forward records (null distribution mode) with identifier fields on the schema must still + // route by equality fields, so HashKeyGenerator can handle them when the processor falls back + // off the forward path. + DynamicRecord record1 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey1, unpartitioned); + DynamicRecord record2 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowSameKey2, unpartitioned); + DynamicRecord record3 = + new DynamicRecord( + TABLE_IDENTIFIER, BRANCH, schemaWithIdentifier, rowDifferentKey, unpartitioned); + record1.writeParallelism(writeParallelism); + record2.writeParallelism(writeParallelism); + record3.writeParallelism(writeParallelism); + + int writeKey1 = generator.generateKey(record1); + int writeKey2 = generator.generateKey(record2); + int writeKey3 = generator.generateKey(record3); + + assertThat(writeKey1).isEqualTo(writeKey2); + assertThat(writeKey3).isNotEqualTo(writeKey1); + } + @Test void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { int writeParallelism = 2; From 60d3fc01b95e51952d07bfd6228f5bc8c855de0b Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Thu, 28 May 2026 14:03:49 -0400 Subject: [PATCH 32/58] Docs: Publish Iceberg security model (#16538) * publish security model for human and agent consumption --- AGENTS.md | 4 + SECURITY-THREAT-MODEL.md | 260 +++++++++++++++++++++++++++++++++++++++ site/docs/security.md | 15 +++ 3 files changed, 279 insertions(+) create mode 100644 SECURITY-THREAT-MODEL.md diff --git a/AGENTS.md b/AGENTS.md index 6378b2a9c534..3b4998065939 100644 --- a/AGENTS.md +++ b/AGENTS.md @@ -21,6 +21,10 @@ Project conventions, architecture, and coding patterns synthesized from 58,000+ review comments across 4,300+ merged PRs. +## Security Model + +When assessing potential vulnerabilities or calibrating automated security findings, use [`SECURITY-THREAT-MODEL.md`](SECURITY-THREAT-MODEL.md) as the authoritative detailed description of Iceberg's security boundaries, trust assumptions, and non-boundaries. + ## Architecture ### Module Boundaries diff --git a/SECURITY-THREAT-MODEL.md b/SECURITY-THREAT-MODEL.md new file mode 100644 index 000000000000..966cfa1ebe9c --- /dev/null +++ b/SECURITY-THREAT-MODEL.md @@ -0,0 +1,260 @@ + + +# Apache Iceberg Security Threat Model + +This document describes Apache Iceberg's detailed security threat model for +maintainers and automated security triage. + +It complements the shorter public-facing security model in +[`site/docs/security.md`](site/docs/security.md) by making Iceberg's trust +assumptions, security boundaries, and recurring non-security bug classes more +explicit. + +## Purpose + +Apache Iceberg is often deployed as a library and integration layer inside +larger systems that provide their own authentication, authorization, and +credential management. Because of that deployment model, many bug classes that +look security-relevant in the abstract are not actually security +vulnerabilities in Iceberg itself. + +This model is intended to answer: + +- what Iceberg generally treats as a security vulnerability +- what Iceberg generally treats as correctness, hardening, or deployment work +- which boundaries are primarily owned by Iceberg versus the surrounding + catalog, engine, or service +- which issue classes should be downgraded by default by scanners + +## Scope + +This model is scoped to the Apache Iceberg project itself: + +- the table format implementation +- client libraries +- engine integrations +- catalog-related components shipped in the Iceberg repository + +It is not a general threat model for every deployment that embeds Iceberg. + +In particular, it does not attempt to define the complete security model for: + +- query engines or applications that embed Iceberg +- storage-level authorization enforced outside Iceberg + +## Security Goals + +Iceberg should: + +- avoid exposing secrets or delegated credentials to principals that were not + already trusted with them +- avoid creating new unauthorized capabilities in Iceberg-owned components or + integrations +- avoid violating trust boundaries that Iceberg itself owns, such as leaking + signer, auth, or credential-bearing state across catalog or session + boundaries in the same process + +Iceberg does not aim to be the primary enforcement point for: + +- user-to-user authorization inside a query engine +- storage-level authorization +- service-side credential scoping performed by an external catalog + +## Roles + +### Operator + +The operator deploys and configures the catalog, metastore, REST service, +engine, and storage integration around Iceberg. This role is trusted to choose +endpoints, warehouses, and storage integrations, configure credentials, and +decide which users may create tables, read tables, or invoke maintenance +actions. + +### Catalog control plane + +The catalog control plane is responsible for resolving tables and supplying +metadata, locations, configuration, and delegated credentials to Iceberg. This +role may be implemented by a REST catalog server, a metastore-backed catalog, +or another catalog implementation. Regardless of how it is implemented, it +should not expose secrets to unintended principals or leak credential-bearing +state across unintended boundaries. + +Iceberg assumes a trusted catalog or metastore, which is outside its primary +security boundary. + +### REST catalog server + +In REST deployments, part of the catalog control plane is implemented by a +server that returns metadata, configuration, and possibly delegated +credentials to the client. This server is generally treated as a trusted +control-plane component. + +### REST catalog client + +In REST deployments, the client-side catalog object consumes server-provided +metadata, configuration, and credentials. Where the client and server are +meaningfully distinct, client-side bugs in routing, caching, or reuse may +still be security-relevant. This is especially true when the Iceberg-owned +client implementation leaks credential-bearing state across catalog, session, +or principal boundaries it is expected to preserve. + +### Engine or embedding application + +Query engines and applications may expose only a subset of Iceberg +capabilities to users. They are responsible for their own user-facing +authorization boundaries unless Iceberg explicitly documents otherwise. + +### Table writer or maintainer + +This role may already have legitimate power to write or replace table +metadata, write or delete files, choose paths under an allowed warehouse or +table location, and invoke destructive maintenance operations. If a report +only shows a new way to achieve the same effect this role can already cause +legitimately, it is usually not a security issue in Iceberg. + +## Trust Boundaries + +### Boundary 1: operator-trusted configuration + +The following are generally treated as trusted operator or deployment inputs: + +- catalog properties +- endpoint configuration +- warehouse and storage roots +- metastore wiring +- REST catalog server configuration + +If a report depends on the attacker controlling those values directly, it is +usually not a vulnerability in Iceberg itself. + +### Boundary 2: catalog-supplied metadata + +Iceberg often accepts metadata locations, table properties, namespace +properties, and related control-plane information from a catalog or +metastore. By default, Iceberg treats those sources as trusted. + +This means a malicious catalog supplying incorrect or malicious metadata is +usually not an Iceberg vulnerability by itself. + +### Boundary 3: REST catalog server-supplied configuration and delegated storage access + +In REST deployments, Iceberg may also accept service endpoints, +configuration, and delegated storage access from the REST catalog server. By +default, those are treated as trusted control-plane inputs unless Iceberg +explicitly documents a stronger guarantee. + +This means a malicious REST catalog server sending dangerous endpoints is +usually not an Iceberg vulnerability by itself. It also means many client-side +credential-selection bugs are often correctness or specification issues rather +than security boundary failures. + +The major exception is secret exposure. If Iceberg surfaces credentials or +secrets to a new audience that was not already trusted with them, that is +security-relevant. + +### Boundary 4: storage-level authorization + +Object store permissions are enforced by the storage provider and the +credentials the surrounding deployment chooses to hand to Iceberg. Iceberg is +not the root authority for bucket- or object-level authorization. + +Reports that depend primarily on over-broad IAM or permissive storage ACLs are +usually deployment-sensitive rather than product-security issues in Iceberg. + +### Boundary 5: engine-level user authorization + +Iceberg integrations may surface data and operations through a query engine or +application, but Iceberg is not a complete user-authorization framework for +those systems. + +## In-Scope Security Vulnerabilities + +The following categories are generally security-relevant in Iceberg when the +report is credible and reproducible. + +### 1. Secret or credential disclosure to a new audience + +Examples include: + +- catalog secrets exposed through a user-visible engine surface +- one catalog's credentials or auth state leaking into another catalog or + session + +### 2. Iceberg-owned trust-boundary violations + +Security issues exist when Iceberg itself is expected to separate catalogs, +principals, or sessions and fails to do so. + +Examples include: + +- process-global signer or auth state crossing catalog instances +- secret-bearing state from one principal reused for another principal within + an Iceberg-owned boundary + +## Usually Out of Scope or Non-Security by Default + +These categories may still be real bugs worth fixing, but they are not usually +security vulnerabilities in Iceberg itself. + +### 1. Correctness bugs + +Examples: + +- wrong byte offsets +- stale or corrupted decoded values +- ambiguous prefix matching +- race conditions or logic bugs that do not create a new trust-boundary + violation + +### 2. Parser hardening and malformed-input robustness + +Malformed-input crashes, raw runtime exceptions, and memory amplification are +usually treated as robustness or hardening work rather than security issues in +Iceberg itself. + +### 3. Malicious catalog, metastore, or external service scenarios + +Reports that require a malicious catalog, metastore, or other external service +are usually outside Iceberg's primary security boundary. + +### 4. Equivalent-harm reports + +If the actor already has a legitimate capability that can cause the same harm, +the new path is usually not a security issue. This often applies to writers or +maintainers who already control metadata layout, file layout, or destructive +maintenance operations. + +## Scanner Calibration Rules + +A scanner targeting Iceberg should treat a finding as higher-confidence only if +it plausibly shows one of the following: + +- exposure of a secret or delegated credential to a new audience +- creation of a new unauthorized capability in an Iceberg-owned component +- violation of an Iceberg-owned trust boundary rather than a surrounding + catalog, engine, service, or operator boundary + +A finding should be downgraded or rejected by default if it instead depends +primarily on: + +- malformed-input robustness or denial-of-service behavior +- a malicious catalog, metastore, or external service +- a principal that already has equivalent power through legitimate write or + maintenance capabilities diff --git a/site/docs/security.md b/site/docs/security.md index 7280888930b6..6936a0db98f2 100644 --- a/site/docs/security.md +++ b/site/docs/security.md @@ -27,6 +27,21 @@ responded. To report a possible security vulnerability, please email security@iceberg.apache.org. +# Security Model + +Apache Iceberg is a table format and a set of libraries and integrations used inside larger systems such as catalogs, query engines, and services. +In most deployments, the primary trust and authorization boundaries are enforced by the surrounding catalog, engine, service, operator configuration, and storage-level authorization rather than by Iceberg alone. + +Iceberg security issues generally include secret or credential disclosure to a newly reachable audience, and other cases where Iceberg itself creates a new unauthorized capability rather than merely reflecting the trust decisions of a catalog, engine, or operator. + +Many other issues may still be valid bugs, but are not normally considered security vulnerabilities in Iceberg. +This includes robustness issues such as malformed-input crashes or memory exhaustion, as well as issues that require a malicious catalog, metastore, or other external service. + +Potential vulnerabilities that fall within this security model should be reported privately using the process above. +Other bugs and hardening issues should be reported through the public issue tracker. + +For a more detailed threat model used for agent-assisted triage and scanner calibration, see the [Apache Iceberg Security Threat Model](https://github.com/apache/iceberg/blob/main/SECURITY-THREAT-MODEL.md). + # Verifying Signed Releases Please refer to the instructions on the [Release Verification](https://www.apache.org/info/verification.html) page. From 0052942699cd7b5e098d54a958827a911d28ac94 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Fri, 29 May 2026 14:22:01 +0200 Subject: [PATCH 33/58] API, Core, Orc: Implement project() for partition statistics scan API (#16569) --- .../iceberg/BasePartitionStatistics.java | 21 ++- .../iceberg/BasePartitionStatisticsScan.java | 10 +- .../apache/iceberg/PartitionStatsHandler.java | 5 +- .../PartitionStatisticsScanTestBase.java | 140 ++++++++++++++++++ .../orc/TestOrcPartitionStatisticsScan.java | 14 ++ 5 files changed, 183 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java index 4b1a3a6dba93..1cdb4a49e341 100644 --- a/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java +++ b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java @@ -23,6 +23,21 @@ public class BasePartitionStatistics extends SupportsIndexProjection implements PartitionStatistics { + private static final Types.StructType BASE_TYPE = + Types.StructType.of( + EMPTY_PARTITION_FIELD, + SPEC_ID, + DATA_RECORD_COUNT, + DATA_FILE_COUNT, + TOTAL_DATA_FILE_SIZE_IN_BYTES, + POSITION_DELETE_RECORD_COUNT, + POSITION_DELETE_FILE_COUNT, + EQUALITY_DELETE_RECORD_COUNT, + EQUALITY_DELETE_FILE_COUNT, + TOTAL_RECORD_COUNT, + LAST_UPDATED_AT, + LAST_UPDATED_SNAPSHOT_ID, + DV_COUNT); private StructLike partition; private Integer specId; @@ -38,10 +53,8 @@ public class BasePartitionStatistics extends SupportsIndexProjection private Long lastUpdatedSnapshotId; private Integer dvCount; - private static final int STATS_COUNT = 13; - BasePartitionStatistics(StructLike partition, int specId) { - super(STATS_COUNT); + super(BASE_TYPE.fields().size()); this.partition = partition; this.specId = specId; @@ -58,7 +71,7 @@ public class BasePartitionStatistics extends SupportsIndexProjection /** Used by internal readers to instantiate this class with a projection schema. */ BasePartitionStatistics(Types.StructType projection) { - super(STATS_COUNT); + super(BASE_TYPE, projection); } @Override diff --git a/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java b/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java index 1316f015adb3..cc1998486fdc 100644 --- a/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java +++ b/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java @@ -22,12 +22,14 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; public class BasePartitionStatisticsScan implements PartitionStatisticsScan { private final Table table; private Long snapshotId; + private Schema projection; public BasePartitionStatisticsScan(Table table) { this.table = table; @@ -49,7 +51,9 @@ public PartitionStatisticsScan filter(Expression newFilter) { @Override public PartitionStatisticsScan project(Schema newSchema) { - throw new UnsupportedOperationException("Projection is not supported"); + Preconditions.checkArgument(newSchema != null, "Invalid projection schema: null"); + this.projection = newSchema; + return this; } @Override @@ -73,13 +77,15 @@ public CloseableIterable scan() { Types.StructType partitionType = Partitioning.partitionType(table); Schema schema = PartitionStatistics.schema(partitionType, TableUtil.formatVersion(table)); + Schema readSchema = + projection == null ? schema : TypeUtil.select(schema, TypeUtil.getProjectedIds(projection)); FileFormat fileFormat = FileFormat.fromFileName(statsFile.get().path()); Preconditions.checkNotNull( fileFormat != null, "Unable to determine format of file: %s", statsFile.get().path()); return InternalData.read(fileFormat, table.io().newInputFile(statsFile.get().path())) - .project(schema) + .project(readSchema) .setRootType(BasePartitionStatistics.class) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java index 29f7bcb53ce6..611bd3cd0783 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java @@ -495,7 +495,10 @@ static void appendStats(PartitionStatistics targetStats, PartitionStatistics inp PartitionStatistics.EQUALITY_DELETE_FILE_COUNT_POSITION, targetStats.equalityDeleteFileCount() + inputStats.equalityDeleteFileCount()); - if (inputStats.dvCount() != null) { + // For backward compatibility, for V2 tables we have to check that the position of dv_count is + // within the schema of the stats file. + if (inputStats.dvCount() != null + && targetStats.size() > PartitionStatistics.DV_COUNT_POSITION) { if (targetStats.dvCount() == null) { targetStats.set(PartitionStatistics.DV_COUNT_POSITION, inputStats.dvCount()); } else { diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java index 321d6fedad5a..04698863eeba 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java @@ -417,6 +417,146 @@ public void testScanPartitionStatsForOlderSnapshot() throws Exception { null)); } + @Test + public void testProjectStatFields() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_project_subset"), + "scan_project_subset", + SCHEMA, + SPEC, + 3, + fileFormatProperty); + + DataFile dataFile1 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); + DataFile dataFile2 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "B")); + testTable.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + long snapshotId = testTable.currentSnapshot().snapshotId(); + PartitionStatisticsFile result = + PartitionStatsHandler.computeAndWriteStatsFile(testTable, snapshotId); + testTable.updatePartitionStatistics().setPartitionStatistics(result).commit(); + + Schema projection = + new Schema( + PartitionStatistics.EMPTY_PARTITION_FIELD, + PartitionStatistics.DATA_RECORD_COUNT, + PartitionStatistics.DATA_FILE_COUNT, + PartitionStatistics.LAST_UPDATED_SNAPSHOT_ID); + + List partitionStats; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().project(projection).scan()) { + partitionStats = Lists.newArrayList(recordIterator); + } + + Types.StructType partitionType = Partitioning.partitionType(testTable); + + assertThat(partitionStats) + .extracting( + PartitionStatistics::partition, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::lastUpdatedSnapshotId) + .containsExactlyInAnyOrder( + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), dataFile1.recordCount(), 1, snapshotId), + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + dataFile2.recordCount(), + 1, + snapshotId)); + + assertThat(partitionStats) + .allSatisfy( + stats -> { + assertThat(stats.specId()).isNull(); + assertThat(stats.totalDataFileSizeInBytes()).isNull(); + assertThat(stats.positionDeleteRecordCount()).isNull(); + assertThat(stats.positionDeleteFileCount()).isNull(); + assertThat(stats.equalityDeleteRecordCount()).isNull(); + assertThat(stats.equalityDeleteFileCount()).isNull(); + assertThat(stats.totalRecords()).isNull(); + assertThat(stats.lastUpdatedAt()).isNull(); + assertThat(stats.dvCount()).isNull(); + }); + } + + @Test + public void testProjectNullSchemaIsRejected() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_project_null"), "scan_project_null", SCHEMA, SPEC, 2, fileFormatProperty); + + assertThatThrownBy(() -> testTable.newPartitionStatisticsScan().project(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid projection schema"); + } + + @Test + public void testProjectIgnoresUnknownField() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_project_unknown"), + "scan_project_unknown", + SCHEMA, + SPEC, + 3, + fileFormatProperty); + + DataFile dataFile1 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); + DataFile dataFile2 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "B")); + testTable.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + long snapshotId = testTable.currentSnapshot().snapshotId(); + testTable + .updatePartitionStatistics() + .setPartitionStatistics( + PartitionStatsHandler.computeAndWriteStatsFile(testTable, snapshotId)) + .commit(); + + Schema projection = + new Schema( + PartitionStatistics.EMPTY_PARTITION_FIELD, + PartitionStatistics.DATA_RECORD_COUNT, + Types.NestedField.optional(9999, "not_a_stats_field", Types.StringType.get())); + + List partitionStats; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().project(projection).scan()) { + partitionStats = Lists.newArrayList(recordIterator); + } + + Types.StructType partitionType = Partitioning.partitionType(testTable); + + // only the valid projected fields are populated + assertThat(partitionStats) + .extracting(PartitionStatistics::partition, PartitionStatistics::dataRecordCount) + .containsExactlyInAnyOrder( + Tuple.tuple(partitionRecord(partitionType, "foo", "A"), dataFile1.recordCount()), + Tuple.tuple(partitionRecord(partitionType, "foo", "B"), dataFile2.recordCount())); + + assertThat(partitionStats) + .allSatisfy( + stats -> { + assertThat(stats.specId()).isNull(); + assertThat(stats.dataFileCount()).isNull(); + assertThat(stats.totalDataFileSizeInBytes()).isNull(); + assertThat(stats.positionDeleteRecordCount()).isNull(); + assertThat(stats.positionDeleteFileCount()).isNull(); + assertThat(stats.equalityDeleteRecordCount()).isNull(); + assertThat(stats.equalityDeleteFileCount()).isNull(); + assertThat(stats.totalRecords()).isNull(); + assertThat(stats.lastUpdatedAt()).isNull(); + assertThat(stats.lastUpdatedSnapshotId()).isNull(); + assertThat(stats.dvCount()).isNull(); + }); + } + private static void computeAndValidatePartitionStats( Table testTable, long snapshotId, Tuple... expectedValues) throws IOException { PartitionStatisticsFile result = diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java index 2040f046ee5d..11c2538662bf 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java @@ -56,4 +56,18 @@ public void testV2toV3SchemaEvolution() throws Exception { .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot write using unregistered internal data format: ORC"); } + + @Override + public void testProjectStatFields() throws Exception { + assertThatThrownBy(super::testProjectStatFields) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } + + @Override + public void testProjectIgnoresUnknownField() throws Exception { + assertThatThrownBy(super::testProjectIgnoresUnknownField) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } } From 8f28a86914d6beaf2615cdf5797da0acb72ea4d9 Mon Sep 17 00:00:00 2001 From: Varun Lakhyani <130844282+varun-lakhyani@users.noreply.github.com> Date: Fri, 29 May 2026 20:06:05 +0530 Subject: [PATCH 34/58] extends compaction base class in sort compaction (#16593) --- .../IcebergSortCompactionBenchmark.java | 98 ++----------------- 1 file changed, 8 insertions(+), 90 deletions(-) diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java index 683f6bb46d05..8900ae81368d 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java @@ -25,42 +25,30 @@ import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; import java.util.Collections; -import java.util.UUID; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.actions.SparkActions; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.DataTypes; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Timeout; @@ -69,7 +57,7 @@ @Measurement(iterations = 10) @BenchmarkMode(Mode.SingleShotTime) @Timeout(time = 1000, timeUnit = TimeUnit.HOURS) -public class IcebergSortCompactionBenchmark { +public class IcebergSortCompactionBenchmark extends IcebergCompactionBenchmark { private static final String[] NAMESPACE = new String[] {"default"}; private static final String NAME = "sortbench"; @@ -78,28 +66,9 @@ public class IcebergSortCompactionBenchmark { private static final long NUM_ROWS = 7500000L; private static final long UNIQUE_VALUES = NUM_ROWS / 4; - private final Configuration hadoopConf = initHadoopConf(); - private SparkSession spark; - - @Setup - public void setupBench() { - setupSpark(); - } - - @TearDown - public void teardownBench() { - tearDownSpark(); - } - - @Setup(Level.Iteration) - public void setupIteration() { - initTable(); - appendData(); - } - - @TearDown(Level.Iteration) - public void cleanUpIteration() throws IOException { - cleanupFiles(); + @Override + protected String tableName() { + return NAME; } @Benchmark @@ -278,11 +247,8 @@ public void zSortSixColumns() { .execute(); } - protected Configuration initHadoopConf() { - return new Configuration(); - } - - protected final void initTable() { + @Override + protected void initTable() { Schema schema = new Schema( required(1, "longCol", Types.LongType.get()), @@ -309,7 +275,8 @@ protected final void initTable() { } } - private void appendData() { + @Override + protected void appendData() { Dataset df = spark() .range(0, NUM_ROWS * NUM_FILES, 1, NUM_FILES) @@ -356,53 +323,4 @@ private void appendData() { .withColumn("stringCol", new RandomGeneratingUDF(UNIQUE_VALUES).randomString().apply()); writeData(df); } - - private void writeData(Dataset df) { - df.write().format("iceberg").mode(SaveMode.Append).save(NAME); - } - - protected final Table table() { - try { - return Spark3Util.loadIcebergTable(spark(), NAME); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - protected final SparkSession spark() { - return spark; - } - - protected String getCatalogWarehouse() { - try { - return Files.createTempDirectory("benchmark-").toAbsolutePath() - + "/" - + UUID.randomUUID() - + "/"; - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - protected void cleanupFiles() throws IOException { - spark.sql("DROP TABLE IF EXISTS " + NAME); - } - - protected void setupSpark() { - SparkSession.Builder builder = - SparkSession.builder() - .config( - "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") - .config("spark.sql.catalog.spark_catalog.type", "hadoop") - .config("spark.sql.catalog.spark_catalog.warehouse", getCatalogWarehouse()) - .config(TestBase.DISABLE_UI) - .master("local[*]"); - spark = builder.getOrCreate(); - Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); - hadoopConf.forEach(entry -> sparkHadoopConf.set(entry.getKey(), entry.getValue())); - } - - protected void tearDownSpark() { - spark.stop(); - } } From f6740364c46afb811fd57c8ca965555600f644bc Mon Sep 17 00:00:00 2001 From: Bharath Krishna Date: Sat, 30 May 2026 15:30:21 -0400 Subject: [PATCH 35/58] Core: Fix optionalOAuthParams dropped during non-exchange token refresh (#16022) (#16023) --- .../apache/iceberg/rest/auth/OAuth2Util.java | 4 +- .../iceberg/rest/auth/TestOAuth2Util.java | 48 +++++++++++++++++++ 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 7a244bff70f6..a9504a7a4e99 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -190,7 +190,7 @@ private static OAuthTokenResponse refreshToken( config.credential(), config.scope(), config.oauth2ServerUri(), - ImmutableMap.of()); + optionalOAuthParams); } } @@ -564,7 +564,7 @@ private OAuthTokenResponse refreshExpiredToken(RESTClient client) { client, config, basicHeaders, token(), tokenType(), optionalOAuthParams()); } else { return fetchToken( - client, Map.of(), credential(), scope(), oauth2ServerUri(), ImmutableMap.of()); + client, Map.of(), credential(), scope(), oauth2ServerUri(), optionalOAuthParams()); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/auth/TestOAuth2Util.java b/core/src/test/java/org/apache/iceberg/rest/auth/TestOAuth2Util.java index 4a6fbf7a1cdc..04f7571eca4e 100644 --- a/core/src/test/java/org/apache/iceberg/rest/auth/TestOAuth2Util.java +++ b/core/src/test/java/org/apache/iceberg/rest/auth/TestOAuth2Util.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.RESTClient; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; import org.apache.iceberg.rest.responses.OAuthTokenResponse; @@ -205,6 +206,53 @@ void refreshUsesRefreshedTokenExpiry() throws IOException { } } + @Test + void refreshExpiredTokenShouldIncludeOptionalOAuthParams() throws IOException { + assertRefreshIncludesOptionalOAuthParams(System.currentTimeMillis() - 10_000); + } + + @Test + void refreshCurrentTokenNonExchangeShouldIncludeOptionalOAuthParams() throws IOException { + assertRefreshIncludesOptionalOAuthParams(System.currentTimeMillis() + 300_000); + } + + private static void assertRefreshIncludesOptionalOAuthParams(long expiresAtMillis) + throws IOException { + String audience = "https://my-catalog.example.com"; + AuthConfig authConfig = + ImmutableAuthConfig.builder() + .keepRefreshed(true) + .exchangeEnabled(false) + .token("token") + .credential("testClientId:testClientSecret") + .oauth2ServerUri("/v1/token") + .expiresAtMillis(expiresAtMillis) + .optionalOAuthParams(ImmutableMap.of("audience", audience, "scope", "catalog")) + .build(); + + OAuthTokenResponse response = + OAuthTokenResponse.builder().withToken("refreshed_token").withTokenType(BEARER).build(); + + try (RESTClient client = Mockito.mock(RESTClient.class); + OAuth2Util.AuthSession session = new OAuth2Util.AuthSession(Map.of(), authConfig)) { + Mockito.when(client.postForm(any(), anyMap(), any(), anyMap(), any())).thenReturn(response); + + session.refresh(client); + + Mockito.verify(client) + .postForm( + any(), + argThat( + formData -> + CLIENT_CREDENTIALS.equals(formData.get(GRANT_TYPE)) + && audience.equals(formData.get("audience")) + && "catalog".equals(formData.get("scope"))), + Mockito.eq(OAuthTokenResponse.class), + anyMap(), + any()); + } + } + private static AuthSession parentSession(long expSeconds) { String parentToken = tokenWithExp(expSeconds); AuthConfig parentConfig = From f6d281122cf901baa8ea60a379eedb8fda2e4136 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Sat, 30 May 2026 15:35:58 -0700 Subject: [PATCH 36/58] Core: Validate non-string elements in JsonUtil.getStringArray (#16586) * Core: Validate non-string elements in JsonUtil.getStringArray Add an isTextual() check on each array element to fail fast with a clear error message when a non-string element is encountered, matching the validation already performed by JsonStringArrayIterator (used by getStringList, getStringSet, getStringListOrNull). Co-Authored-By: Claude Opus 4.7 (1M context) * Core: Add property-aware getStringArray and migrate two callers Add an overload getStringArray(String property, JsonNode node) that delegates to getStringList so its error messages include the field name (e.g. "Cannot parse string from non-text value in default-namespace: 45") matching the convention used by getStringList/getStringSet. Migrate ViewVersionParser and RemoteSignRequestParser, which both have a property name in scope. RESTSerializers.NamespaceDeserializer deserializes a top-level Namespace with no field name in scope and keeps the existing 1-arg overload. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../requests/RemoteSignRequestParser.java | 2 +- .../org/apache/iceberg/util/JsonUtil.java | 9 +++- .../iceberg/view/ViewVersionParser.java | 3 +- .../org/apache/iceberg/util/TestJsonUtil.java | 47 +++++++++++++++++++ .../iceberg/view/TestViewVersionParser.java | 2 +- 5 files changed, 58 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/RemoteSignRequestParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/RemoteSignRequestParser.java index 61b44cc177d1..a29326a76b6f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/requests/RemoteSignRequestParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/requests/RemoteSignRequestParser.java @@ -133,7 +133,7 @@ public static Map> headersFromJson(String property, JsonNod .forEach( entry -> { String key = entry.getKey(); - List values = Arrays.asList(JsonUtil.getStringArray(entry.getValue())); + List values = Arrays.asList(JsonUtil.getStringArray(key, headersNode)); headers.put(key, values); }); return headers; diff --git a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java index d8bb1f919096..58d878b94901 100644 --- a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -272,11 +272,18 @@ public static String[] getStringArray(JsonNode node) { ArrayNode arrayNode = (ArrayNode) node; String[] arr = new String[arrayNode.size()]; for (int i = 0; i < arr.length; i++) { - arr[i] = arrayNode.get(i).asText(); + JsonNode element = arrayNode.get(i); + Preconditions.checkArgument( + element.isTextual(), "Cannot parse string from non-text value: %s", element); + arr[i] = element.asText(); } return arr; } + public static String[] getStringArray(String property, JsonNode node) { + return getStringList(property, node).toArray(new String[0]); + } + public static List getStringList(String property, JsonNode node) { Preconditions.checkArgument(node.has(property), "Cannot parse missing list: %s", property); return ImmutableList.builder() diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java index 06ee3b2648d2..69208ce34062 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java @@ -96,8 +96,7 @@ public static ViewVersion fromJson(JsonNode node) { String defaultCatalog = JsonUtil.getStringOrNull(DEFAULT_CATALOG, node); - Namespace defaultNamespace = - Namespace.of(JsonUtil.getStringArray(JsonUtil.get(DEFAULT_NAMESPACE, node))); + Namespace defaultNamespace = Namespace.of(JsonUtil.getStringArray(DEFAULT_NAMESPACE, node)); return ImmutableViewVersion.builder() .versionId(versionId) diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 91cd96e9088f..27cf6fb2994e 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -430,6 +430,53 @@ public void getLongSetOrNull() throws JsonProcessingException { .containsExactlyElementsOf(Arrays.asList(23L, 45L)); } + @Test + public void getStringArray() throws JsonProcessingException { + assertThatThrownBy(() -> JsonUtil.getStringArray(JsonUtil.mapper().readTree("null"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse string array from non-array: null"); + + assertThatThrownBy(() -> JsonUtil.getStringArray(JsonUtil.mapper().readTree("23"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse string array from non-array: 23"); + + assertThatThrownBy(() -> JsonUtil.getStringArray(JsonUtil.mapper().readTree("[\"23\", 45]"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse string from non-text value: 45"); + + assertThat(JsonUtil.getStringArray(JsonUtil.mapper().readTree("[\"23\", \"45\"]"))) + .containsExactly("23", "45"); + + assertThat(JsonUtil.getStringArray(JsonUtil.mapper().readTree("[]"))).isEmpty(); + } + + @Test + public void getStringArrayWithProperty() throws JsonProcessingException { + assertThatThrownBy(() -> JsonUtil.getStringArray("items", JsonUtil.mapper().readTree("{}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing list: items"); + + assertThatThrownBy( + () -> JsonUtil.getStringArray("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse JSON array from non-array value: items: null"); + + assertThatThrownBy( + () -> + JsonUtil.getStringArray( + "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", 45]}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse string from non-text value in items: 45"); + + assertThat( + JsonUtil.getStringArray( + "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", \"45\"]}"))) + .containsExactly("23", "45"); + + assertThat(JsonUtil.getStringArray("items", JsonUtil.mapper().readTree("{\"items\": []}"))) + .isEmpty(); + } + @Test public void getStringList() throws JsonProcessingException { assertThatThrownBy(() -> JsonUtil.getStringList("items", JsonUtil.mapper().readTree("{}"))) diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java index a68b99a6797b..a46e63401632 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java @@ -127,7 +127,7 @@ public void missingDefaultCatalog() { "{\"version-id\":1,\"timestamp-ms\":12345,\"schema-id\":1," + "\"summary\":{\"operation\":\"create\"},\"representations\":[]}")) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse missing field: default-namespace"); + .hasMessage("Cannot parse missing list: default-namespace"); } @Test From 6a737004506176a902784ec30fc4d545c1ce6997 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 30 May 2026 15:37:46 -0700 Subject: [PATCH 37/58] Spark: Deprecate SparkFilters in favor of SparkV2Filters (#16616) SparkFilters converts the Spark DSv1 Filter[] API into Iceberg Expressions. It is no longer referenced by Iceberg's main source code across all supported Spark versions (3.5, 4.0, 4.1); all production call sites have migrated to SparkV2Filters, which converts the DSv2 Predicate[] API. Only TestSparkFilters still exercises the class. Annotate the class with @Deprecated (target removal: 1.12.0) and add @SuppressWarnings("deprecation") to TestSparkFilters so the existing tests continue to run cleanly until the class is removed. Co-authored-by: Cursor --- .../main/java/org/apache/iceberg/spark/SparkFilters.java | 7 +++++++ .../java/org/apache/iceberg/spark/TestSparkFilters.java | 1 + .../main/java/org/apache/iceberg/spark/SparkFilters.java | 7 +++++++ .../java/org/apache/iceberg/spark/TestSparkFilters.java | 1 + .../main/java/org/apache/iceberg/spark/SparkFilters.java | 7 +++++++ .../java/org/apache/iceberg/spark/TestSparkFilters.java | 1 + 6 files changed, 24 insertions(+) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java index 49b73c7b01af..85f57fdb4ac6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java @@ -70,6 +70,13 @@ import org.apache.spark.sql.sources.Or; import org.apache.spark.sql.sources.StringStartsWith; +/** + * Converts Spark DSv1 filters into Iceberg expressions. + * + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link SparkV2Filters} which converts + * Spark DSv2 {@link org.apache.spark.sql.connector.expressions.filter.Predicate} instead. + */ +@Deprecated public class SparkFilters { private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java index 49c38b34d34a..f6e8d9d61677 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java @@ -44,6 +44,7 @@ import org.apache.spark.sql.sources.Not; import org.junit.jupiter.api.Test; +@SuppressWarnings("deprecation") public class TestSparkFilters { @Test diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java index 49b73c7b01af..85f57fdb4ac6 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java @@ -70,6 +70,13 @@ import org.apache.spark.sql.sources.Or; import org.apache.spark.sql.sources.StringStartsWith; +/** + * Converts Spark DSv1 filters into Iceberg expressions. + * + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link SparkV2Filters} which converts + * Spark DSv2 {@link org.apache.spark.sql.connector.expressions.filter.Predicate} instead. + */ +@Deprecated public class SparkFilters { private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)"); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java index 49c38b34d34a..f6e8d9d61677 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java @@ -44,6 +44,7 @@ import org.apache.spark.sql.sources.Not; import org.junit.jupiter.api.Test; +@SuppressWarnings("deprecation") public class TestSparkFilters { @Test diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java index 49b73c7b01af..85f57fdb4ac6 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java @@ -70,6 +70,13 @@ import org.apache.spark.sql.sources.Or; import org.apache.spark.sql.sources.StringStartsWith; +/** + * Converts Spark DSv1 filters into Iceberg expressions. + * + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link SparkV2Filters} which converts + * Spark DSv2 {@link org.apache.spark.sql.connector.expressions.filter.Predicate} instead. + */ +@Deprecated public class SparkFilters { private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)"); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java index 49c38b34d34a..f6e8d9d61677 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java @@ -44,6 +44,7 @@ import org.apache.spark.sql.sources.Not; import org.junit.jupiter.api.Test; +@SuppressWarnings("deprecation") public class TestSparkFilters { @Test From 05401437cd13c2e0789c0902eb2ce59effb02166 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 30 May 2026 22:52:02 -0700 Subject: [PATCH 38/58] Build: Bump io.netty:netty-buffer from 4.2.13.Final to 4.2.14.Final (#16628) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.2.13.Final to 4.2.14.Final. - [Release notes](https://github.com/netty/netty/releases) - [Commits](https://github.com/netty/netty/compare/netty-4.2.13.Final...netty-4.2.14.Final) --- updated-dependencies: - dependency-name: io.netty:netty-buffer dependency-version: 4.2.14.Final dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ce2eec19a320..f6ab03e5856c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -78,7 +78,7 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.107.5" -netty-buffer = "4.2.13.Final" +netty-buffer = "4.2.14.Final" object-client-bundle = "3.3.2" orc = "1.9.8" parquet = "1.17.1" From fa9b3d58559a32494bc4942eef38cf1a6906d4f9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 30 May 2026 22:53:50 -0700 Subject: [PATCH 39/58] Build: Bump org.immutables:value from 2.12.1 to 2.12.2 (#16636) Bumps [org.immutables:value](https://github.com/immutables/immutables) from 2.12.1 to 2.12.2. - [Release notes](https://github.com/immutables/immutables/releases) - [Commits](https://github.com/immutables/immutables/compare/2.12.1...2.12.2) --- updated-dependencies: - dependency-name: org.immutables:value dependency-version: 2.12.2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f6ab03e5856c..0ca5b2e078fa 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -57,7 +57,7 @@ guava = "33.6.0-jre" hadoop3 = "3.4.3" httpcomponents-httpclient5 = "5.6.1" hive2 = { strictly = "2.3.10"} # see rich version usage explanation above -immutables-value = "2.12.1" +immutables-value = "2.12.2" jackson-annotations = "2.21" jackson-bom = "2.21.3" jackson214 = { strictly = "2.14.2"} From d48629d790610568268e80bc1eec2e41afc5599d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 30 May 2026 22:54:05 -0700 Subject: [PATCH 40/58] Build: Bump github/codeql-action from 4.35.5 to 4.36.0 (#16635) Bumps [github/codeql-action](https://github.com/github/codeql-action) from 4.35.5 to 4.36.0. - [Release notes](https://github.com/github/codeql-action/releases) - [Changelog](https://github.com/github/codeql-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/github/codeql-action/compare/9e0d7b8d25671d64c341c19c0152d693099fb5ba...7211b7c8077ea37d8641b6271f6a365a22a5fbfa) --- updated-dependencies: - dependency-name: github/codeql-action dependency-version: 4.36.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/codeql.yml | 4 ++-- .github/workflows/cve-scan.yml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index a811068b135a..6eb79f0fdbba 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -46,11 +46,11 @@ jobs: persist-credentials: false - name: Initialize CodeQL - uses: github/codeql-action/init@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 + uses: github/codeql-action/init@7211b7c8077ea37d8641b6271f6a365a22a5fbfa # v4.36.0 with: languages: actions - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 + uses: github/codeql-action/analyze@7211b7c8077ea37d8641b6271f6a365a22a5fbfa # v4.36.0 with: category: "/language:actions" diff --git a/.github/workflows/cve-scan.yml b/.github/workflows/cve-scan.yml index 415b6eb3c7de..6bd08de9859b 100644 --- a/.github/workflows/cve-scan.yml +++ b/.github/workflows/cve-scan.yml @@ -166,7 +166,7 @@ jobs: fi - name: Upload Trivy results to GitHub Security tab if: always() && github.event_name == 'push' - uses: github/codeql-action/upload-sarif@9e0d7b8d25671d64c341c19c0152d693099fb5ba # v4.35.5 + uses: github/codeql-action/upload-sarif@7211b7c8077ea37d8641b6271f6a365a22a5fbfa # v4.36.0 with: sarif_file: 'trivy-results.sarif' category: ${{ matrix.distribution }} From 3b7b2d14dea0d82a8855a8ae591137c1c536938c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 May 2026 08:41:20 -0700 Subject: [PATCH 41/58] Build: Bump openapi-spec-validator from 0.8.5 to 0.9.0 (#16629) Bumps [openapi-spec-validator](https://github.com/python-openapi/openapi-spec-validator) from 0.8.5 to 0.9.0. - [Release notes](https://github.com/python-openapi/openapi-spec-validator/releases) - [Commits](https://github.com/python-openapi/openapi-spec-validator/compare/0.8.5...0.9.0) --- updated-dependencies: - dependency-name: openapi-spec-validator dependency-version: 0.9.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index a5e3f6384975..7e798aaeb87a 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -15,6 +15,6 @@ # specific language governing permissions and limitations # under the License. -openapi-spec-validator==0.8.5 +openapi-spec-validator==0.9.0 datamodel-code-generator==0.57.0 yamllint==1.38.0 From dd4d889a1782df9c1a9d3092bacb7463b511e09a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 May 2026 08:41:47 -0700 Subject: [PATCH 42/58] Build: Bump actions/stale from 10.2.0 to 10.3.0 (#16630) Bumps [actions/stale](https://github.com/actions/stale) from 10.2.0 to 10.3.0. - [Release notes](https://github.com/actions/stale/releases) - [Changelog](https://github.com/actions/stale/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/stale/compare/b5d41d4e1d5dceea10e7104786b73624c18a190f...eb5cf3af3ac0a1aa4c9c45633dd1ae542a27a899) --- updated-dependencies: - dependency-name: actions/stale dependency-version: 10.3.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index e3fd0f56dd5d..9d63bb312130 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -32,7 +32,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-slim steps: - - uses: actions/stale@b5d41d4e1d5dceea10e7104786b73624c18a190f # v10.2.0 + - uses: actions/stale@eb5cf3af3ac0a1aa4c9c45633dd1ae542a27a899 # v10.3.0 with: # stale issues stale-issue-label: 'stale' From 80adb7120909543b7ed03ea3e29fbbf0b2d10b5c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 May 2026 08:41:59 -0700 Subject: [PATCH 43/58] Build: Bump docker/build-push-action from 7.1.0 to 7.2.0 (#16631) Bumps [docker/build-push-action](https://github.com/docker/build-push-action) from 7.1.0 to 7.2.0. - [Release notes](https://github.com/docker/build-push-action/releases) - [Commits](https://github.com/docker/build-push-action/compare/bcafcacb16a39f128d818304e6c9c0c18556b85f...f9f3042f7e2789586610d6e8b85c8f03e5195baf) --- updated-dependencies: - dependency-name: docker/build-push-action dependency-version: 7.2.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/publish-iceberg-rest-fixture-docker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml index 0b1d8a2339de..8646acf07c77 100644 --- a/.github/workflows/publish-iceberg-rest-fixture-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -72,7 +72,7 @@ jobs: - name: Set up Docker Buildx uses: docker/setup-buildx-action@4d04d5d9486b7bd6fa91e7baf45bbb4f8b9deedd # v4.0.0 - name: Build and Push - uses: docker/build-push-action@bcafcacb16a39f128d818304e6c9c0c18556b85f # v7.1.0 + uses: docker/build-push-action@f9f3042f7e2789586610d6e8b85c8f03e5195baf # v7.2.0 with: context: ./ file: ./docker/iceberg-rest-fixture/Dockerfile From 16566de39e061cc2b3dcc50f4b94e90e2d56c16b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 May 2026 08:42:12 -0700 Subject: [PATCH 44/58] Build: Bump docker/setup-buildx-action from 4.0.0 to 4.1.0 (#16632) Bumps [docker/setup-buildx-action](https://github.com/docker/setup-buildx-action) from 4.0.0 to 4.1.0. - [Release notes](https://github.com/docker/setup-buildx-action/releases) - [Commits](https://github.com/docker/setup-buildx-action/compare/4d04d5d9486b7bd6fa91e7baf45bbb4f8b9deedd...d7f5e7f509e45cec5c76c4d5afdd7de93d0b3df5) --- updated-dependencies: - dependency-name: docker/setup-buildx-action dependency-version: 4.1.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/publish-iceberg-rest-fixture-docker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml index 8646acf07c77..89756490accc 100644 --- a/.github/workflows/publish-iceberg-rest-fixture-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -70,7 +70,7 @@ jobs: - name: Set up QEMU uses: docker/setup-qemu-action@ce360397dd3f832beb865e1373c09c0e9f86d70a # v4.0.0 - name: Set up Docker Buildx - uses: docker/setup-buildx-action@4d04d5d9486b7bd6fa91e7baf45bbb4f8b9deedd # v4.0.0 + uses: docker/setup-buildx-action@d7f5e7f509e45cec5c76c4d5afdd7de93d0b3df5 # v4.1.0 - name: Build and Push uses: docker/build-push-action@f9f3042f7e2789586610d6e8b85c8f03e5195baf # v7.2.0 with: From 7522b02aaf42d51d4b12e211119d159c9dc03552 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 May 2026 08:42:28 -0700 Subject: [PATCH 45/58] Build: Bump software.amazon.awssdk:bom from 2.44.7 to 2.44.12 (#16634) Bumps software.amazon.awssdk:bom from 2.44.7 to 2.44.12. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.44.12 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 0ca5b2e078fa..5131c9165c22 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -33,7 +33,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.7" awaitility = "4.3.0" -awssdk-bom = "2.44.7" +awssdk-bom = "2.44.12" azuresdk-bom = "1.3.6" awssdk-s3accessgrants = "2.4.1" bouncycastle = "1.84" From 67cbc1aeabbce52d618f8e3c6fc34747f6695a8d Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 1 Jun 2026 10:03:55 +0900 Subject: [PATCH 46/58] Build: Bump com.azure:azure-sdk-bom from 1.3.6 to 1.3.7 (#16637) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.3.6 to 1.3.7. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-identity_1.3.6...azure-identity_1.3.7) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-version: 1.3.7 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- azure-bundle/runtime-deps.txt | 10 +++++----- gradle/libs.versions.toml | 2 +- kafka-connect/kafka-connect-runtime/runtime-deps.txt | 10 +++++----- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/azure-bundle/runtime-deps.txt b/azure-bundle/runtime-deps.txt index 32bc06d03d62..2caed26153e8 100644 --- a/azure-bundle/runtime-deps.txt +++ b/azure-bundle/runtime-deps.txt @@ -1,12 +1,12 @@ com.azure:azure-core-http-netty:1.16 -com.azure:azure-core:1.57 +com.azure:azure-core:1.58 com.azure:azure-identity:1.18 com.azure:azure-json:1.5 com.azure:azure-security-keyvault-keys:4.10 -com.azure:azure-storage-blob:12.33 -com.azure:azure-storage-common:12.32 -com.azure:azure-storage-file-datalake:12.26 -com.azure:azure-storage-internal-avro:12.18 +com.azure:azure-storage-blob:12.34 +com.azure:azure-storage-common:12.33 +com.azure:azure-storage-file-datalake:12.27 +com.azure:azure-storage-internal-avro:12.19 com.azure:azure-xml:1.2 com.fasterxml.jackson.core:jackson-annotations:2.18 com.fasterxml.jackson.core:jackson-core:2.18 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 5131c9165c22..13c98dc984f3 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,7 +34,7 @@ avro = "1.12.1" assertj-core = "3.27.7" awaitility = "4.3.0" awssdk-bom = "2.44.12" -azuresdk-bom = "1.3.6" +azuresdk-bom = "1.3.7" awssdk-s3accessgrants = "2.4.1" bouncycastle = "1.84" bson-ver = "4.11.5" diff --git a/kafka-connect/kafka-connect-runtime/runtime-deps.txt b/kafka-connect/kafka-connect-runtime/runtime-deps.txt index 95ef88a87969..b4eed40327bf 100644 --- a/kafka-connect/kafka-connect-runtime/runtime-deps.txt +++ b/kafka-connect/kafka-connect-runtime/runtime-deps.txt @@ -1,11 +1,11 @@ com.azure:azure-core-http-netty:1.16 -com.azure:azure-core:1.57 +com.azure:azure-core:1.58 com.azure:azure-identity:1.18 com.azure:azure-json:1.5 -com.azure:azure-storage-blob:12.33 -com.azure:azure-storage-common:12.32 -com.azure:azure-storage-file-datalake:12.26 -com.azure:azure-storage-internal-avro:12.18 +com.azure:azure-storage-blob:12.34 +com.azure:azure-storage-common:12.33 +com.azure:azure-storage-file-datalake:12.27 +com.azure:azure-storage-internal-avro:12.19 com.azure:azure-xml:1.2 com.fasterxml.jackson.core:jackson-annotations:2.21 com.fasterxml.jackson.core:jackson-core:2.21 From c11404cb49985a39a7caff0328c5f927a795154f Mon Sep 17 00:00:00 2001 From: lrsb Date: Mon, 1 Jun 2026 14:03:02 +0200 Subject: [PATCH 47/58] Flink: Fix duplicate commits in DynamicCommitter when Flink jobId changes on restart (#16011) --- .../flink/sink/dynamic/DynamicCommitter.java | 85 ++++++--- .../flink/sink/dynamic/JobOperatorKey.java | 57 ++++++ .../iceberg/flink/sink/dynamic/TableKey.java | 5 - .../sink/dynamic/TestDynamicCommitter.java | 179 +++++++++++++++++- .../sink/dynamic/TestDynamicIcebergSink.java | 109 ++++++++++- 5 files changed, 392 insertions(+), 43 deletions(-) create mode 100644 flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 5e824773f4bf..c8b9a097ad92 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -111,6 +112,17 @@ public void commit(Collection> commitRequests) } /* + Group the incoming commit requests into a three-level structure before committing: + + Map>>> + | | | | + | | | +-- commit requests at that checkpoint + | | +-- checkpointId, sorted ascending so older commits go first + | +-- (jobId, operatorId) of the producing aggregator; deduplication against the + | table's snapshot summaries is per (jobId, operatorId), so each group is + | walked against the ancestor chain independently + +-- (tableName, branch); we load the table and its ancestor chain once per TableKey + Each (table, branch, checkpoint) triplet must have only one commit request. There may be commit requests from previous checkpoints which have not been committed yet. @@ -119,43 +131,58 @@ public void commit(Collection> commitRequests) DynamicWriteResultAggregator. Iceberg 1.12 will remove this, and users should upgrade to the 1.11 release first to migrate their state to a single commit request per checkpoint. */ - Map>>> commitRequestMap = - Maps.newHashMap(); + Map>>>> + commitRequestMap = Maps.newHashMap(); for (CommitRequest request : commitRequests) { - NavigableMap>> committables = - commitRequestMap.computeIfAbsent( - new TableKey(request.getCommittable()), unused -> Maps.newTreeMap()); - committables - .computeIfAbsent(request.getCommittable().checkpointId(), unused -> Lists.newArrayList()) + DynamicCommittable committable = request.getCommittable(); + commitRequestMap + .computeIfAbsent(committable.key(), unused -> Maps.newHashMap()) + .computeIfAbsent(new JobOperatorKey(committable), unused -> Maps.newTreeMap()) + .computeIfAbsent(committable.checkpointId(), unused -> Lists.newArrayList()) .add(request); } - for (Map.Entry>>> entry : - commitRequestMap.entrySet()) { - Table table = catalog.loadTable(TableIdentifier.parse(entry.getKey().tableName())); - DynamicCommittable last = entry.getValue().lastEntry().getValue().get(0).getCommittable(); - Snapshot latestSnapshot = table.snapshot(entry.getKey().branch()); + for (Map.Entry< + TableKey, + Map>>>> + tableEntry : commitRequestMap.entrySet()) { + TableKey tableKey = tableEntry.getKey(); + Table table = catalog.loadTable(TableIdentifier.parse(tableKey.tableName())); + Snapshot latestSnapshot = table.snapshot(tableKey.branch()); Iterable ancestors = latestSnapshot != null ? SnapshotUtil.ancestorsOf(latestSnapshot.snapshotId(), table::snapshot) : List.of(); - long maxCommittedCheckpointId = - getMaxCommittedCheckpointId(ancestors, last.jobId(), last.operatorId()); - - NavigableMap>> skippedCommitRequests = - entry.getValue().headMap(maxCommittedCheckpointId, true); - LOG.debug( - "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); - // Mark the already committed FilesCommittable(s) as finished - skippedCommitRequests - .values() - .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); - - NavigableMap>> uncommitted = - entry.getValue().tailMap(maxCommittedCheckpointId, false); - if (!uncommitted.isEmpty()) { - commitPendingRequests( - table, entry.getKey().branch(), uncommitted, last.jobId(), last.operatorId()); + + List>>>> + jobEntries = Lists.newArrayList(tableEntry.getValue().entrySet()); + // Preserve checkpoint order across groups so that older-jobId commits land before newer-jobId + // ones when the batch mixes committables from different jobIds (e.g. state replay after a + // restart). Within a (jobId, operatorId) group, checkpoint order is already guaranteed by + // the inner NavigableMap. + jobEntries.sort(Comparator.comparingLong(entry -> entry.getValue().firstKey())); + + for (Map.Entry>>> + jobEntry : jobEntries) { + JobOperatorKey jobKey = jobEntry.getKey(); + long maxCommittedCheckpointId = + getMaxCommittedCheckpointId(ancestors, jobKey.jobId(), jobKey.operatorId()); + + NavigableMap>> skippedCommitRequests = + jobEntry.getValue().headMap(maxCommittedCheckpointId, true); + LOG.debug( + "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); + // Mark the already committed FilesCommittable(s) as finished + skippedCommitRequests + .values() + .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); + + NavigableMap>> uncommitted = + jobEntry.getValue().tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests( + table, tableKey.branch(), uncommitted, jobKey.jobId(), jobKey.operatorId()); + } } } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java new file mode 100644 index 000000000000..463c09475d97 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.Objects; + +class JobOperatorKey { + private final String jobId; + private final String operatorId; + + JobOperatorKey(DynamicCommittable committable) { + this.jobId = committable.jobId(); + this.operatorId = committable.operatorId(); + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (!(other instanceof JobOperatorKey that)) { + return false; + } + + return Objects.equals(jobId, that.jobId) && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + return Objects.hash(jobId, operatorId); + } +} diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java index 08b755fe14a8..45a2961b62a8 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java @@ -33,11 +33,6 @@ class TableKey { this.branch = branch; } - TableKey(DynamicCommittable committable) { - this.tableName = committable.key().tableName(); - this.branch = committable.key().branch(); - } - String tableName() { return tableName; } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 4cc27151b094..aecad67c1a6b 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -326,6 +326,166 @@ void testSkipsCommitRequestsForPreviousCheckpoints() throws Exception { .build()); } + @Test + void testSkipsAlreadyCommittedDataAfterJobIdChanges() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String uidPrefix = "uidPrefix"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + JobID previousJobId = JobID.generate(); + DynamicCommitterMetrics previousCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter previousCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + previousCommitterMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + // Operator id is stable across Flink job restarts, jobId is not. + final String operatorId = new OperatorID().toHexString(); + final String previousJobIdStr = previousJobId.toHexString(); + final int previousCheckpointId = 10; + + byte[][] previousManifests = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + + DynamicCommittable previousCommittable = + new DynamicCommittable( + tableKey, previousManifests, previousJobIdStr, operatorId, previousCheckpointId); + previousCommitter.commit(Sets.newHashSet(new MockCommitRequest<>(previousCommittable))); + + table.refresh(); + assertThat(table.snapshots()).hasSize(1); + + JobID newJobId = JobID.generate(); + DynamicCommitterMetrics newCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter newCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + newCommitterMetrics); + + final String newJobIdStr = newJobId.toHexString(); + final int newCheckpointId = previousCheckpointId + 1; + + byte[][] previousManifestsNew = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest replayedPreviousCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, + previousManifestsNew, + previousJobIdStr, + operatorId, + previousCheckpointId)); + CommitRequest newCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, newManifests, newJobIdStr, operatorId, newCheckpointId)); + + newCommitter.commit(Sets.newHashSet(replayedPreviousCommitRequest, newCommitRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", previousJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(previousCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + } + + @Test + void testCommitsLandInCheckpointOrderAcrossJobIds() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter committer = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + final String oldJobId = JobID.generate().toHexString(); + final String newJobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int oldCheckpointId = 1; + final int newCheckpointId = 2; + + byte[][] oldManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, oldCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest oldRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, oldManifests, oldJobId, operatorId, oldCheckpointId)); + CommitRequest newRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, newManifests, newJobId, operatorId, newCheckpointId)); + + // Hand the requests in reversed order; the committer must still land them in checkpointId + // order on the snapshot chain. + committer.commit(Lists.newArrayList(newRequest, oldRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", oldJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(oldCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + assertThat(second.parentId()).isEqualTo(first.snapshotId()); + } + @Test void testCommitDeleteInDifferentFormatVersion() throws Exception { Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); @@ -512,8 +672,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { // Two committables, one for each snapshot / table / branch. assertThat(table.snapshots()).hasSize(2); - Snapshot snapshot1 = Iterables.getFirst(table.snapshots(), null); - assertThat(snapshot1.snapshotId()).isEqualTo(table.refs().get("branch1").snapshotId()); + Snapshot snapshot1 = table.snapshot(table.refs().get("branch1").snapshotId()); assertThat(snapshot1.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -531,8 +690,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { .put("total-records", "66") .build()); - Snapshot snapshot2 = Iterables.get(table.snapshots(), 1); - assertThat(snapshot2.snapshotId()).isEqualTo(table.refs().get("branch2").snapshotId()); + Snapshot snapshot2 = table.snapshot(table.refs().get("branch2").snapshotId()); assertThat(snapshot2.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -898,7 +1056,10 @@ void testThrowsValidationExceptionOnDuplicateCommit(boolean overwriteMode) throw } interface CommitHook extends Serializable { - default void beforeCommit(Collection> commitRequests) {} + default Collection> beforeCommit( + Collection> commitRequests) { + return commitRequests; + } default void beforeCommitOperation() {} @@ -919,11 +1080,14 @@ static class FailBeforeAndAfterCommit implements CommitHook { } @Override - public void beforeCommit(Collection> ignored) { + public Collection> beforeCommit( + Collection> requests) { if (!failedBeforeCommit) { failedBeforeCommit = true; throw new RuntimeException("Failing before commit"); } + + return requests; } @Override @@ -977,8 +1141,7 @@ static class CommitHookEnabledDynamicCommitter extends DynamicCommitter { @Override public void commit(Collection> commitRequests) throws IOException, InterruptedException { - commitHook.beforeCommit(commitRequests); - super.commit(commitRequests); + super.commit(commitHook.beforeCommit(commitRequests)); commitHook.afterCommit(); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 3ba579df490b..7c4669ff9826 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -35,11 +35,13 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import javax.annotation.Nullable; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; @@ -1175,6 +1177,48 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + void testSkipsAlreadyCommittedDataAfterJobIdChanges(boolean overwriteMode) throws Exception { + TableIdentifier tableId = TableIdentifier.of(DATABASE, "t1"); + List records = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); + + DataFile seedDataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/seed-data-1.parquet") + .withFileSizeInBytes(0) + .withRecordCount(1) + .build(); + + executeDynamicSink( + records, env, true, 1, new ReplayPreviousJobIdCommittableHook(seedDataFile), overwriteMode); + + Table table = CATALOG_EXTENSION.catalog().loadTable(tableId); + + Snapshot mainSnapshot = + StreamSupport.stream(table.snapshots().spliterator(), false) + .filter( + s -> + !ReplayPreviousJobIdCommittableHook.PREVIOUS_JOB_ID.equals( + s.summary().get("flink.job-id"))) + .filter(s -> s.summary().get("flink.job-id") != null) + .reduce((first, second) -> second) + .orElseThrow(); + assertThat(mainSnapshot.summary()) + .containsEntry("added-data-files", "1") + .containsEntry("added-records", String.valueOf(records.size())); + + long expectedTotalRecords = overwriteMode ? records.size() : records.size() + 1L; + assertThat(Long.parseLong(mainSnapshot.summary().get("total-records"))) + .isEqualTo(expectedTotalRecords); + } + @Test void testCommitsOncePerTableBranchAndCheckpoint() throws Exception { String tableName = "t1"; @@ -1526,10 +1570,13 @@ private static void resetState() { } @Override - public void beforeCommit(Collection> requests) { + public Collection> beforeCommit( + Collection> requests) { if (!hasTriggered) { this.commitRequests.addAll(requests); } + + return requests; } @Override @@ -1547,6 +1594,66 @@ public void beforeCommitOperation() { } } + /** + * Seeds an ancestor snapshot under a synthetic previous jobId and prepends a replay committable + * tagged with that jobId (at an earlier checkpoint, as would happen on restart-replay) to the + * batch. The seed uses {@code table.newAppend()} directly rather than a side committer so the + * real committable's manifest stays intact. + */ + static class ReplayPreviousJobIdCommittableHook implements CommitHook { + static final String PREVIOUS_JOB_ID = JobID.generate().toHexString(); + + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // Static to survive Flink operator serialization. + private static boolean hasTriggered = false; + + private final DataFile seedDataFile; + + ReplayPreviousJobIdCommittableHook(DataFile seedDataFile) { + this.seedDataFile = seedDataFile; + hasTriggered = false; + } + + @Override + public Collection> beforeCommit( + Collection> requests) { + if (hasTriggered || requests.isEmpty()) { + return requests; + } + + hasTriggered = true; + DynamicCommittable original = requests.iterator().next().getCommittable(); + long replayedCheckpointId = original.checkpointId() - 1; + + Table table = + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.parse(original.key().tableName())); + table + .newAppend() + .appendFile(seedDataFile) + .set(FLINK_JOB_ID, PREVIOUS_JOB_ID) + .set(OPERATOR_ID, original.operatorId()) + .set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(replayedCheckpointId)) + .toBranch(original.key().branch()) + .commit(); + + DynamicCommittable replayed = + new DynamicCommittable( + original.key(), + original.manifests(), + PREVIOUS_JOB_ID, + original.operatorId(), + replayedCheckpointId); + List> enriched = + Lists.newArrayListWithCapacity(requests.size() + 1); + enriched.add(new MockCommitRequest<>(replayed)); + enriched.addAll(requests); + return enriched; + } + } + private static class AppendRightBeforeCommit implements CommitHook { final String tableIdentifier; From 696b93d8d0cd01981ebf9871dbd3281bc57cc7a4 Mon Sep 17 00:00:00 2001 From: Vova Kolmakov Date: Mon, 1 Jun 2026 22:25:44 +0700 Subject: [PATCH 48/58] Arrow: Fix truncation of decimals with precision larger than 18 (#16627) --- .../vectorized/ArrowVectorAccessors.java | 4 +- .../arrow/vectorized/TestArrowReader.java | 67 +++++++++++++++++++ 2 files changed, 70 insertions(+), 1 deletion(-) diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java index 24af804b18aa..263c3ec606be 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java @@ -99,7 +99,9 @@ public BigDecimal ofLong(long value, int precision, int scale) { @Override public BigDecimal ofBigDecimal(BigDecimal value, int precision, int scale) { - return BigDecimal.valueOf(value.unscaledValue().longValue(), scale); + // Return the value unchanged: it already has the correct unscaled value and scale. The + // unscaled value can exceed the range of a long, so it must not be narrowed to one. + return value; } } } diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestArrowReader.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestArrowReader.java index e23006c79d18..4d81c52ea243 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestArrowReader.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestArrowReader.java @@ -26,6 +26,7 @@ import java.io.File; import java.io.IOException; import java.math.BigDecimal; +import java.math.BigInteger; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.Instant; @@ -388,6 +389,72 @@ public void testTimestampMillisAreReadCorrectly() throws Exception { assertThat(totalRowsRead).as("Should read all rows").isEqualTo(millisValues.size()); } + /** + * Reads a decimal(38, 0) column whose values exceed Long.MAX_VALUE. Decimals with precision >= + * 19 are stored as a FIXED_LEN_BYTE_ARRAY, and reading them must not narrow the unscaled value + * through {@code BigInteger.longValue()}. This reproduces a bug where the binary-backed decimal + * accessor silently truncated such values. + */ + @Test + public void testHighPrecisionDecimalIsReadCorrectly() throws Exception { + tables = new HadoopTables(); + int precision = 38; + int scale = 0; + Schema schema = + new Schema( + Types.NestedField.required(1, "decimal", Types.DecimalType.of(precision, scale))); + Table table = tables.create(schema, tableLocation); + + // Both values have an unscaled magnitude well beyond Long.MAX_VALUE (~9.2e18). + List values = + Lists.newArrayList( + new BigDecimal(new BigInteger("12345678901234567890"), scale), + new BigDecimal(new BigInteger("99999999999999999999999999999999999999"), scale)); + + List records = Lists.newArrayListWithCapacity(values.size()); + for (BigDecimal value : values) { + GenericRecord record = GenericRecord.create(schema); + record.setField("decimal", value); + records.add(record); + } + + File parquetFile = File.createTempFile("decimal", ".parquet", tempDir); + assertThat(parquetFile.delete()).isTrue(); + FileAppender appender = + Parquet.write(Files.localOutput(parquetFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .build(); + try { + appender.addAll(records); + } finally { + appender.close(); + } + + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(localInput(parquetFile)) + .withMetrics(appender.metrics()) + .withFormat(FileFormat.PARQUET) + .build(); + table.newAppend().appendFile(dataFile).commit(); + + int rowIndex = 0; + try (VectorizedTableScanIterable vectorizedReader = + new VectorizedTableScanIterable(table.newScan(), 1024, false)) { + for (ColumnarBatch batch : vectorizedReader) { + for (int i = 0; i < batch.numRows(); i++) { + assertThat(batch.column(0).getDecimal(i, precision, scale)) + .as("decimal(%d, %d) value at row %d must not be truncated", precision, scale, i) + .isEqualTo(values.get(rowIndex)); + rowIndex++; + } + } + } + + assertThat(rowIndex).isEqualTo(values.size()); + } + @ParameterizedTest @MethodSource("rejectedUnsignedIntegerCases") public void testUnsignedIntegerColumnThrowsException( From 785723853323db8395ffdc5a4c913857b2007c68 Mon Sep 17 00:00:00 2001 From: lrsb Date: Mon, 1 Jun 2026 18:26:46 +0200 Subject: [PATCH 49/58] Flink: Backport DynamicCommitter jobId fix to v1.20 and v2.0 (#16648) Backport commit #16011. --- .../flink/sink/dynamic/DynamicCommitter.java | 85 ++++++--- .../flink/sink/dynamic/JobOperatorKey.java | 57 ++++++ .../iceberg/flink/sink/dynamic/TableKey.java | 5 - .../sink/dynamic/TestDynamicCommitter.java | 179 +++++++++++++++++- .../sink/dynamic/TestDynamicIcebergSink.java | 109 ++++++++++- .../flink/sink/dynamic/DynamicCommitter.java | 85 ++++++--- .../flink/sink/dynamic/JobOperatorKey.java | 57 ++++++ .../iceberg/flink/sink/dynamic/TableKey.java | 5 - .../sink/dynamic/TestDynamicCommitter.java | 179 +++++++++++++++++- .../sink/dynamic/TestDynamicIcebergSink.java | 109 ++++++++++- 10 files changed, 784 insertions(+), 86 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 5e824773f4bf..c8b9a097ad92 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -111,6 +112,17 @@ public void commit(Collection> commitRequests) } /* + Group the incoming commit requests into a three-level structure before committing: + + Map>>> + | | | | + | | | +-- commit requests at that checkpoint + | | +-- checkpointId, sorted ascending so older commits go first + | +-- (jobId, operatorId) of the producing aggregator; deduplication against the + | table's snapshot summaries is per (jobId, operatorId), so each group is + | walked against the ancestor chain independently + +-- (tableName, branch); we load the table and its ancestor chain once per TableKey + Each (table, branch, checkpoint) triplet must have only one commit request. There may be commit requests from previous checkpoints which have not been committed yet. @@ -119,43 +131,58 @@ public void commit(Collection> commitRequests) DynamicWriteResultAggregator. Iceberg 1.12 will remove this, and users should upgrade to the 1.11 release first to migrate their state to a single commit request per checkpoint. */ - Map>>> commitRequestMap = - Maps.newHashMap(); + Map>>>> + commitRequestMap = Maps.newHashMap(); for (CommitRequest request : commitRequests) { - NavigableMap>> committables = - commitRequestMap.computeIfAbsent( - new TableKey(request.getCommittable()), unused -> Maps.newTreeMap()); - committables - .computeIfAbsent(request.getCommittable().checkpointId(), unused -> Lists.newArrayList()) + DynamicCommittable committable = request.getCommittable(); + commitRequestMap + .computeIfAbsent(committable.key(), unused -> Maps.newHashMap()) + .computeIfAbsent(new JobOperatorKey(committable), unused -> Maps.newTreeMap()) + .computeIfAbsent(committable.checkpointId(), unused -> Lists.newArrayList()) .add(request); } - for (Map.Entry>>> entry : - commitRequestMap.entrySet()) { - Table table = catalog.loadTable(TableIdentifier.parse(entry.getKey().tableName())); - DynamicCommittable last = entry.getValue().lastEntry().getValue().get(0).getCommittable(); - Snapshot latestSnapshot = table.snapshot(entry.getKey().branch()); + for (Map.Entry< + TableKey, + Map>>>> + tableEntry : commitRequestMap.entrySet()) { + TableKey tableKey = tableEntry.getKey(); + Table table = catalog.loadTable(TableIdentifier.parse(tableKey.tableName())); + Snapshot latestSnapshot = table.snapshot(tableKey.branch()); Iterable ancestors = latestSnapshot != null ? SnapshotUtil.ancestorsOf(latestSnapshot.snapshotId(), table::snapshot) : List.of(); - long maxCommittedCheckpointId = - getMaxCommittedCheckpointId(ancestors, last.jobId(), last.operatorId()); - - NavigableMap>> skippedCommitRequests = - entry.getValue().headMap(maxCommittedCheckpointId, true); - LOG.debug( - "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); - // Mark the already committed FilesCommittable(s) as finished - skippedCommitRequests - .values() - .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); - - NavigableMap>> uncommitted = - entry.getValue().tailMap(maxCommittedCheckpointId, false); - if (!uncommitted.isEmpty()) { - commitPendingRequests( - table, entry.getKey().branch(), uncommitted, last.jobId(), last.operatorId()); + + List>>>> + jobEntries = Lists.newArrayList(tableEntry.getValue().entrySet()); + // Preserve checkpoint order across groups so that older-jobId commits land before newer-jobId + // ones when the batch mixes committables from different jobIds (e.g. state replay after a + // restart). Within a (jobId, operatorId) group, checkpoint order is already guaranteed by + // the inner NavigableMap. + jobEntries.sort(Comparator.comparingLong(entry -> entry.getValue().firstKey())); + + for (Map.Entry>>> + jobEntry : jobEntries) { + JobOperatorKey jobKey = jobEntry.getKey(); + long maxCommittedCheckpointId = + getMaxCommittedCheckpointId(ancestors, jobKey.jobId(), jobKey.operatorId()); + + NavigableMap>> skippedCommitRequests = + jobEntry.getValue().headMap(maxCommittedCheckpointId, true); + LOG.debug( + "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); + // Mark the already committed FilesCommittable(s) as finished + skippedCommitRequests + .values() + .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); + + NavigableMap>> uncommitted = + jobEntry.getValue().tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests( + table, tableKey.branch(), uncommitted, jobKey.jobId(), jobKey.operatorId()); + } } } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java new file mode 100644 index 000000000000..463c09475d97 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.Objects; + +class JobOperatorKey { + private final String jobId; + private final String operatorId; + + JobOperatorKey(DynamicCommittable committable) { + this.jobId = committable.jobId(); + this.operatorId = committable.operatorId(); + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (!(other instanceof JobOperatorKey that)) { + return false; + } + + return Objects.equals(jobId, that.jobId) && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + return Objects.hash(jobId, operatorId); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java index 08b755fe14a8..45a2961b62a8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java @@ -33,11 +33,6 @@ class TableKey { this.branch = branch; } - TableKey(DynamicCommittable committable) { - this.tableName = committable.key().tableName(); - this.branch = committable.key().branch(); - } - String tableName() { return tableName; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 4cc27151b094..aecad67c1a6b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -326,6 +326,166 @@ void testSkipsCommitRequestsForPreviousCheckpoints() throws Exception { .build()); } + @Test + void testSkipsAlreadyCommittedDataAfterJobIdChanges() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String uidPrefix = "uidPrefix"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + JobID previousJobId = JobID.generate(); + DynamicCommitterMetrics previousCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter previousCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + previousCommitterMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + // Operator id is stable across Flink job restarts, jobId is not. + final String operatorId = new OperatorID().toHexString(); + final String previousJobIdStr = previousJobId.toHexString(); + final int previousCheckpointId = 10; + + byte[][] previousManifests = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + + DynamicCommittable previousCommittable = + new DynamicCommittable( + tableKey, previousManifests, previousJobIdStr, operatorId, previousCheckpointId); + previousCommitter.commit(Sets.newHashSet(new MockCommitRequest<>(previousCommittable))); + + table.refresh(); + assertThat(table.snapshots()).hasSize(1); + + JobID newJobId = JobID.generate(); + DynamicCommitterMetrics newCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter newCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + newCommitterMetrics); + + final String newJobIdStr = newJobId.toHexString(); + final int newCheckpointId = previousCheckpointId + 1; + + byte[][] previousManifestsNew = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest replayedPreviousCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, + previousManifestsNew, + previousJobIdStr, + operatorId, + previousCheckpointId)); + CommitRequest newCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, newManifests, newJobIdStr, operatorId, newCheckpointId)); + + newCommitter.commit(Sets.newHashSet(replayedPreviousCommitRequest, newCommitRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", previousJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(previousCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + } + + @Test + void testCommitsLandInCheckpointOrderAcrossJobIds() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter committer = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + final String oldJobId = JobID.generate().toHexString(); + final String newJobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int oldCheckpointId = 1; + final int newCheckpointId = 2; + + byte[][] oldManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, oldCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest oldRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, oldManifests, oldJobId, operatorId, oldCheckpointId)); + CommitRequest newRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, newManifests, newJobId, operatorId, newCheckpointId)); + + // Hand the requests in reversed order; the committer must still land them in checkpointId + // order on the snapshot chain. + committer.commit(Lists.newArrayList(newRequest, oldRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", oldJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(oldCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + assertThat(second.parentId()).isEqualTo(first.snapshotId()); + } + @Test void testCommitDeleteInDifferentFormatVersion() throws Exception { Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); @@ -512,8 +672,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { // Two committables, one for each snapshot / table / branch. assertThat(table.snapshots()).hasSize(2); - Snapshot snapshot1 = Iterables.getFirst(table.snapshots(), null); - assertThat(snapshot1.snapshotId()).isEqualTo(table.refs().get("branch1").snapshotId()); + Snapshot snapshot1 = table.snapshot(table.refs().get("branch1").snapshotId()); assertThat(snapshot1.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -531,8 +690,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { .put("total-records", "66") .build()); - Snapshot snapshot2 = Iterables.get(table.snapshots(), 1); - assertThat(snapshot2.snapshotId()).isEqualTo(table.refs().get("branch2").snapshotId()); + Snapshot snapshot2 = table.snapshot(table.refs().get("branch2").snapshotId()); assertThat(snapshot2.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -898,7 +1056,10 @@ void testThrowsValidationExceptionOnDuplicateCommit(boolean overwriteMode) throw } interface CommitHook extends Serializable { - default void beforeCommit(Collection> commitRequests) {} + default Collection> beforeCommit( + Collection> commitRequests) { + return commitRequests; + } default void beforeCommitOperation() {} @@ -919,11 +1080,14 @@ static class FailBeforeAndAfterCommit implements CommitHook { } @Override - public void beforeCommit(Collection> ignored) { + public Collection> beforeCommit( + Collection> requests) { if (!failedBeforeCommit) { failedBeforeCommit = true; throw new RuntimeException("Failing before commit"); } + + return requests; } @Override @@ -977,8 +1141,7 @@ static class CommitHookEnabledDynamicCommitter extends DynamicCommitter { @Override public void commit(Collection> commitRequests) throws IOException, InterruptedException { - commitHook.beforeCommit(commitRequests); - super.commit(commitRequests); + super.commit(commitHook.beforeCommit(commitRequests)); commitHook.afterCommit(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 355bc6805f1e..9e05930b883c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -35,11 +35,13 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import javax.annotation.Nullable; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; @@ -1163,6 +1165,48 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + void testSkipsAlreadyCommittedDataAfterJobIdChanges(boolean overwriteMode) throws Exception { + TableIdentifier tableId = TableIdentifier.of(DATABASE, "t1"); + List records = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); + + DataFile seedDataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/seed-data-1.parquet") + .withFileSizeInBytes(0) + .withRecordCount(1) + .build(); + + executeDynamicSink( + records, env, true, 1, new ReplayPreviousJobIdCommittableHook(seedDataFile), overwriteMode); + + Table table = CATALOG_EXTENSION.catalog().loadTable(tableId); + + Snapshot mainSnapshot = + StreamSupport.stream(table.snapshots().spliterator(), false) + .filter( + s -> + !ReplayPreviousJobIdCommittableHook.PREVIOUS_JOB_ID.equals( + s.summary().get("flink.job-id"))) + .filter(s -> s.summary().get("flink.job-id") != null) + .reduce((first, second) -> second) + .orElseThrow(); + assertThat(mainSnapshot.summary()) + .containsEntry("added-data-files", "1") + .containsEntry("added-records", String.valueOf(records.size())); + + long expectedTotalRecords = overwriteMode ? records.size() : records.size() + 1L; + assertThat(Long.parseLong(mainSnapshot.summary().get("total-records"))) + .isEqualTo(expectedTotalRecords); + } + @Test void testCommitsOncePerTableBranchAndCheckpoint() throws Exception { String tableName = "t1"; @@ -1514,10 +1558,13 @@ private static void resetState() { } @Override - public void beforeCommit(Collection> requests) { + public Collection> beforeCommit( + Collection> requests) { if (!hasTriggered) { this.commitRequests.addAll(requests); } + + return requests; } @Override @@ -1535,6 +1582,66 @@ public void beforeCommitOperation() { } } + /** + * Seeds an ancestor snapshot under a synthetic previous jobId and prepends a replay committable + * tagged with that jobId (at an earlier checkpoint, as would happen on restart-replay) to the + * batch. The seed uses {@code table.newAppend()} directly rather than a side committer so the + * real committable's manifest stays intact. + */ + static class ReplayPreviousJobIdCommittableHook implements CommitHook { + static final String PREVIOUS_JOB_ID = JobID.generate().toHexString(); + + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // Static to survive Flink operator serialization. + private static boolean hasTriggered = false; + + private final DataFile seedDataFile; + + ReplayPreviousJobIdCommittableHook(DataFile seedDataFile) { + this.seedDataFile = seedDataFile; + hasTriggered = false; + } + + @Override + public Collection> beforeCommit( + Collection> requests) { + if (hasTriggered || requests.isEmpty()) { + return requests; + } + + hasTriggered = true; + DynamicCommittable original = requests.iterator().next().getCommittable(); + long replayedCheckpointId = original.checkpointId() - 1; + + Table table = + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.parse(original.key().tableName())); + table + .newAppend() + .appendFile(seedDataFile) + .set(FLINK_JOB_ID, PREVIOUS_JOB_ID) + .set(OPERATOR_ID, original.operatorId()) + .set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(replayedCheckpointId)) + .toBranch(original.key().branch()) + .commit(); + + DynamicCommittable replayed = + new DynamicCommittable( + original.key(), + original.manifests(), + PREVIOUS_JOB_ID, + original.operatorId(), + replayedCheckpointId); + List> enriched = + Lists.newArrayListWithCapacity(requests.size() + 1); + enriched.add(new MockCommitRequest<>(replayed)); + enriched.addAll(requests); + return enriched; + } + } + private static class AppendRightBeforeCommit implements CommitHook { final String tableIdentifier; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 5e824773f4bf..c8b9a097ad92 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -111,6 +112,17 @@ public void commit(Collection> commitRequests) } /* + Group the incoming commit requests into a three-level structure before committing: + + Map>>> + | | | | + | | | +-- commit requests at that checkpoint + | | +-- checkpointId, sorted ascending so older commits go first + | +-- (jobId, operatorId) of the producing aggregator; deduplication against the + | table's snapshot summaries is per (jobId, operatorId), so each group is + | walked against the ancestor chain independently + +-- (tableName, branch); we load the table and its ancestor chain once per TableKey + Each (table, branch, checkpoint) triplet must have only one commit request. There may be commit requests from previous checkpoints which have not been committed yet. @@ -119,43 +131,58 @@ public void commit(Collection> commitRequests) DynamicWriteResultAggregator. Iceberg 1.12 will remove this, and users should upgrade to the 1.11 release first to migrate their state to a single commit request per checkpoint. */ - Map>>> commitRequestMap = - Maps.newHashMap(); + Map>>>> + commitRequestMap = Maps.newHashMap(); for (CommitRequest request : commitRequests) { - NavigableMap>> committables = - commitRequestMap.computeIfAbsent( - new TableKey(request.getCommittable()), unused -> Maps.newTreeMap()); - committables - .computeIfAbsent(request.getCommittable().checkpointId(), unused -> Lists.newArrayList()) + DynamicCommittable committable = request.getCommittable(); + commitRequestMap + .computeIfAbsent(committable.key(), unused -> Maps.newHashMap()) + .computeIfAbsent(new JobOperatorKey(committable), unused -> Maps.newTreeMap()) + .computeIfAbsent(committable.checkpointId(), unused -> Lists.newArrayList()) .add(request); } - for (Map.Entry>>> entry : - commitRequestMap.entrySet()) { - Table table = catalog.loadTable(TableIdentifier.parse(entry.getKey().tableName())); - DynamicCommittable last = entry.getValue().lastEntry().getValue().get(0).getCommittable(); - Snapshot latestSnapshot = table.snapshot(entry.getKey().branch()); + for (Map.Entry< + TableKey, + Map>>>> + tableEntry : commitRequestMap.entrySet()) { + TableKey tableKey = tableEntry.getKey(); + Table table = catalog.loadTable(TableIdentifier.parse(tableKey.tableName())); + Snapshot latestSnapshot = table.snapshot(tableKey.branch()); Iterable ancestors = latestSnapshot != null ? SnapshotUtil.ancestorsOf(latestSnapshot.snapshotId(), table::snapshot) : List.of(); - long maxCommittedCheckpointId = - getMaxCommittedCheckpointId(ancestors, last.jobId(), last.operatorId()); - - NavigableMap>> skippedCommitRequests = - entry.getValue().headMap(maxCommittedCheckpointId, true); - LOG.debug( - "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); - // Mark the already committed FilesCommittable(s) as finished - skippedCommitRequests - .values() - .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); - - NavigableMap>> uncommitted = - entry.getValue().tailMap(maxCommittedCheckpointId, false); - if (!uncommitted.isEmpty()) { - commitPendingRequests( - table, entry.getKey().branch(), uncommitted, last.jobId(), last.operatorId()); + + List>>>> + jobEntries = Lists.newArrayList(tableEntry.getValue().entrySet()); + // Preserve checkpoint order across groups so that older-jobId commits land before newer-jobId + // ones when the batch mixes committables from different jobIds (e.g. state replay after a + // restart). Within a (jobId, operatorId) group, checkpoint order is already guaranteed by + // the inner NavigableMap. + jobEntries.sort(Comparator.comparingLong(entry -> entry.getValue().firstKey())); + + for (Map.Entry>>> + jobEntry : jobEntries) { + JobOperatorKey jobKey = jobEntry.getKey(); + long maxCommittedCheckpointId = + getMaxCommittedCheckpointId(ancestors, jobKey.jobId(), jobKey.operatorId()); + + NavigableMap>> skippedCommitRequests = + jobEntry.getValue().headMap(maxCommittedCheckpointId, true); + LOG.debug( + "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); + // Mark the already committed FilesCommittable(s) as finished + skippedCommitRequests + .values() + .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); + + NavigableMap>> uncommitted = + jobEntry.getValue().tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests( + table, tableKey.branch(), uncommitted, jobKey.jobId(), jobKey.operatorId()); + } } } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java new file mode 100644 index 000000000000..463c09475d97 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/JobOperatorKey.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.util.Objects; + +class JobOperatorKey { + private final String jobId; + private final String operatorId; + + JobOperatorKey(DynamicCommittable committable) { + this.jobId = committable.jobId(); + this.operatorId = committable.operatorId(); + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (!(other instanceof JobOperatorKey that)) { + return false; + } + + return Objects.equals(jobId, that.jobId) && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + return Objects.hash(jobId, operatorId); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java index 08b755fe14a8..45a2961b62a8 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java @@ -33,11 +33,6 @@ class TableKey { this.branch = branch; } - TableKey(DynamicCommittable committable) { - this.tableName = committable.key().tableName(); - this.branch = committable.key().branch(); - } - String tableName() { return tableName; } 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 4cc27151b094..aecad67c1a6b 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 @@ -326,6 +326,166 @@ void testSkipsCommitRequestsForPreviousCheckpoints() throws Exception { .build()); } + @Test + void testSkipsAlreadyCommittedDataAfterJobIdChanges() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String uidPrefix = "uidPrefix"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + JobID previousJobId = JobID.generate(); + DynamicCommitterMetrics previousCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter previousCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + previousCommitterMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + // Operator id is stable across Flink job restarts, jobId is not. + final String operatorId = new OperatorID().toHexString(); + final String previousJobIdStr = previousJobId.toHexString(); + final int previousCheckpointId = 10; + + byte[][] previousManifests = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + + DynamicCommittable previousCommittable = + new DynamicCommittable( + tableKey, previousManifests, previousJobIdStr, operatorId, previousCheckpointId); + previousCommitter.commit(Sets.newHashSet(new MockCommitRequest<>(previousCommittable))); + + table.refresh(); + assertThat(table.snapshots()).hasSize(1); + + JobID newJobId = JobID.generate(); + DynamicCommitterMetrics newCommitterMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter newCommitter = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + uidPrefix, + newCommitterMetrics); + + final String newJobIdStr = newJobId.toHexString(); + final int newCheckpointId = previousCheckpointId + 1; + + byte[][] previousManifestsNew = + aggregator.writeToManifests( + tableKey.tableName(), WRITE_RESULT_BY_SPEC, previousCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest replayedPreviousCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, + previousManifestsNew, + previousJobIdStr, + operatorId, + previousCheckpointId)); + CommitRequest newCommitRequest = + new MockCommitRequest<>( + new DynamicCommittable( + tableKey, newManifests, newJobIdStr, operatorId, newCheckpointId)); + + newCommitter.commit(Sets.newHashSet(replayedPreviousCommitRequest, newCommitRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", previousJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(previousCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobIdStr) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + } + + @Test + void testCommitsLandInCheckpointOrderAcrossJobIds() throws Exception { + Table table = catalog.loadTable(TableIdentifier.of(TABLE1)); + assertThat(table.snapshots()).isEmpty(); + + boolean overwriteMode = false; + int workerPoolSize = 1; + String sinkId = "sinkId"; + UnregisteredMetricsGroup metricGroup = new UnregisteredMetricsGroup(); + DynamicCommitterMetrics committerMetrics = new DynamicCommitterMetrics(metricGroup); + DynamicCommitter committer = + new DynamicCommitter( + CATALOG_EXTENSION.catalog(), + Maps.newHashMap(), + overwriteMode, + workerPoolSize, + sinkId, + committerMetrics); + + DynamicWriteResultAggregator aggregator = + new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); + OneInputStreamOperatorTestHarness aggregatorHarness = + new OneInputStreamOperatorTestHarness(aggregator); + aggregatorHarness.open(); + + TableKey tableKey = new TableKey(TABLE1, "branch"); + final String oldJobId = JobID.generate().toHexString(); + final String newJobId = JobID.generate().toHexString(); + final String operatorId = new OperatorID().toHexString(); + final int oldCheckpointId = 1; + final int newCheckpointId = 2; + + byte[][] oldManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, oldCheckpointId); + byte[][] newManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, newCheckpointId); + + CommitRequest oldRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, oldManifests, oldJobId, operatorId, oldCheckpointId)); + CommitRequest newRequest = + new MockCommitRequest<>( + new DynamicCommittable(tableKey, newManifests, newJobId, operatorId, newCheckpointId)); + + // Hand the requests in reversed order; the committer must still land them in checkpointId + // order on the snapshot chain. + committer.commit(Lists.newArrayList(newRequest, oldRequest)); + + table.refresh(); + assertThat(table.snapshots()).hasSize(2); + + Snapshot first = Iterables.get(table.snapshots(), 0); + assertThat(first.summary()) + .containsEntry("flink.job-id", oldJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(oldCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + + Snapshot second = Iterables.get(table.snapshots(), 1); + assertThat(second.summary()) + .containsEntry("flink.job-id", newJobId) + .containsEntry("flink.max-committed-checkpoint-id", String.valueOf(newCheckpointId)) + .containsEntry("flink.operator-id", operatorId); + assertThat(second.parentId()).isEqualTo(first.snapshotId()); + } + @Test void testCommitDeleteInDifferentFormatVersion() throws Exception { Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); @@ -512,8 +672,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { // Two committables, one for each snapshot / table / branch. assertThat(table.snapshots()).hasSize(2); - Snapshot snapshot1 = Iterables.getFirst(table.snapshots(), null); - assertThat(snapshot1.snapshotId()).isEqualTo(table.refs().get("branch1").snapshotId()); + Snapshot snapshot1 = table.snapshot(table.refs().get("branch1").snapshotId()); assertThat(snapshot1.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -531,8 +690,7 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { .put("total-records", "66") .build()); - Snapshot snapshot2 = Iterables.get(table.snapshots(), 1); - assertThat(snapshot2.snapshotId()).isEqualTo(table.refs().get("branch2").snapshotId()); + Snapshot snapshot2 = table.snapshot(table.refs().get("branch2").snapshotId()); assertThat(snapshot2.summary()) .containsAllEntriesOf( ImmutableMap.builder() @@ -898,7 +1056,10 @@ void testThrowsValidationExceptionOnDuplicateCommit(boolean overwriteMode) throw } interface CommitHook extends Serializable { - default void beforeCommit(Collection> commitRequests) {} + default Collection> beforeCommit( + Collection> commitRequests) { + return commitRequests; + } default void beforeCommitOperation() {} @@ -919,11 +1080,14 @@ static class FailBeforeAndAfterCommit implements CommitHook { } @Override - public void beforeCommit(Collection> ignored) { + public Collection> beforeCommit( + Collection> requests) { if (!failedBeforeCommit) { failedBeforeCommit = true; throw new RuntimeException("Failing before commit"); } + + return requests; } @Override @@ -977,8 +1141,7 @@ static class CommitHookEnabledDynamicCommitter extends DynamicCommitter { @Override public void commit(Collection> commitRequests) throws IOException, InterruptedException { - commitHook.beforeCommit(commitRequests); - super.commit(commitRequests); + super.commit(commitHook.beforeCommit(commitRequests)); commitHook.afterCommit(); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 3ba579df490b..7c4669ff9826 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -35,11 +35,13 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import javax.annotation.Nullable; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; @@ -1175,6 +1177,48 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + void testSkipsAlreadyCommittedDataAfterJobIdChanges(boolean overwriteMode) throws Exception { + TableIdentifier tableId = TableIdentifier.of(DATABASE, "t1"); + List records = + Lists.newArrayList( + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); + + DataFile seedDataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/seed-data-1.parquet") + .withFileSizeInBytes(0) + .withRecordCount(1) + .build(); + + executeDynamicSink( + records, env, true, 1, new ReplayPreviousJobIdCommittableHook(seedDataFile), overwriteMode); + + Table table = CATALOG_EXTENSION.catalog().loadTable(tableId); + + Snapshot mainSnapshot = + StreamSupport.stream(table.snapshots().spliterator(), false) + .filter( + s -> + !ReplayPreviousJobIdCommittableHook.PREVIOUS_JOB_ID.equals( + s.summary().get("flink.job-id"))) + .filter(s -> s.summary().get("flink.job-id") != null) + .reduce((first, second) -> second) + .orElseThrow(); + assertThat(mainSnapshot.summary()) + .containsEntry("added-data-files", "1") + .containsEntry("added-records", String.valueOf(records.size())); + + long expectedTotalRecords = overwriteMode ? records.size() : records.size() + 1L; + assertThat(Long.parseLong(mainSnapshot.summary().get("total-records"))) + .isEqualTo(expectedTotalRecords); + } + @Test void testCommitsOncePerTableBranchAndCheckpoint() throws Exception { String tableName = "t1"; @@ -1526,10 +1570,13 @@ private static void resetState() { } @Override - public void beforeCommit(Collection> requests) { + public Collection> beforeCommit( + Collection> requests) { if (!hasTriggered) { this.commitRequests.addAll(requests); } + + return requests; } @Override @@ -1547,6 +1594,66 @@ public void beforeCommitOperation() { } } + /** + * Seeds an ancestor snapshot under a synthetic previous jobId and prepends a replay committable + * tagged with that jobId (at an earlier checkpoint, as would happen on restart-replay) to the + * batch. The seed uses {@code table.newAppend()} directly rather than a side committer so the + * real committable's manifest stays intact. + */ + static class ReplayPreviousJobIdCommittableHook implements CommitHook { + static final String PREVIOUS_JOB_ID = JobID.generate().toHexString(); + + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // Static to survive Flink operator serialization. + private static boolean hasTriggered = false; + + private final DataFile seedDataFile; + + ReplayPreviousJobIdCommittableHook(DataFile seedDataFile) { + this.seedDataFile = seedDataFile; + hasTriggered = false; + } + + @Override + public Collection> beforeCommit( + Collection> requests) { + if (hasTriggered || requests.isEmpty()) { + return requests; + } + + hasTriggered = true; + DynamicCommittable original = requests.iterator().next().getCommittable(); + long replayedCheckpointId = original.checkpointId() - 1; + + Table table = + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.parse(original.key().tableName())); + table + .newAppend() + .appendFile(seedDataFile) + .set(FLINK_JOB_ID, PREVIOUS_JOB_ID) + .set(OPERATOR_ID, original.operatorId()) + .set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(replayedCheckpointId)) + .toBranch(original.key().branch()) + .commit(); + + DynamicCommittable replayed = + new DynamicCommittable( + original.key(), + original.manifests(), + PREVIOUS_JOB_ID, + original.operatorId(), + replayedCheckpointId); + List> enriched = + Lists.newArrayListWithCapacity(requests.size() + 1); + enriched.add(new MockCommitRequest<>(replayed)); + enriched.addAll(requests); + return enriched; + } + } + private static class AppendRightBeforeCommit implements CommitHook { final String tableIdentifier; From 1172b10723b71f76d779b231e8cd57f3a3c66371 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 2 Jun 2026 02:49:53 +0800 Subject: [PATCH 50/58] Spark 4.1: Upgrade to Spark 4.1.2 (#16365) Co-authored-by: Codex --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 13c98dc984f3..c7c59069905f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -88,7 +88,7 @@ slf4j = "2.0.18" snowflake-jdbc = "3.28.0" spark35 = "3.5.8" spark40 = "4.0.2" -spark41 = "4.1.1" +spark41 = "4.1.2" sqlite-jdbc = "3.53.1.0" testcontainers = "2.0.5" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 687c58fc7ca80a595c39c64bfe9bace023fcd39d Mon Sep 17 00:00:00 2001 From: Alex Stephen <1325798+rambleraptor@users.noreply.github.com> Date: Mon, 1 Jun 2026 13:27:40 -0700 Subject: [PATCH 51/58] Core: v4 table metadata location should be optional (#16572) * v4 location should be optional --------- Co-authored-by: Anoop Johnson --- .../main/java/org/apache/iceberg/Files.java | 3 + .../org/apache/iceberg/TableMetadata.java | 11 ++ .../apache/iceberg/TableMetadataParser.java | 10 +- .../org/apache/iceberg/util/LocationUtil.java | 2 +- .../apache/iceberg/TestRemoveSnapshots.java | 17 +-- .../iceberg/TestSequenceNumberForV2Table.java | 11 +- .../org/apache/iceberg/TestTableMetadata.java | 110 +++++++++++++++--- .../java/org/apache/iceberg/TestTables.java | 10 +- .../TestLoadTableResponseParser.java | 4 +- .../apache/iceberg/util/TestHashWriter.java | 2 +- .../flink/sink/TestIcebergCommitter.java | 6 +- .../flink/sink/TestIcebergCommitter.java | 6 +- .../flink/sink/TestIcebergCommitter.java | 6 +- .../apache/iceberg/hive/TestHiveCatalog.java | 2 +- .../nessie/TestNessieIcebergClient.java | 2 +- .../actions/TestRemoveOrphanFilesAction.java | 10 +- .../iceberg/spark/source/TestTables.java | 8 +- .../actions/TestRemoveOrphanFilesAction.java | 10 +- .../iceberg/spark/source/TestTables.java | 8 +- .../actions/TestRemoveOrphanFilesAction.java | 10 +- .../iceberg/spark/source/TestTables.java | 8 +- 21 files changed, 179 insertions(+), 77 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Files.java b/api/src/main/java/org/apache/iceberg/Files.java index 16d3b663ad35..bee5b3f241e5 100644 --- a/api/src/main/java/org/apache/iceberg/Files.java +++ b/api/src/main/java/org/apache/iceberg/Files.java @@ -40,6 +40,9 @@ public static OutputFile localOutput(File file) { } public static OutputFile localOutput(String file) { + if (file.startsWith("file:")) { + return localOutput(new File(file.replaceFirst("file:", ""))); + } return localOutput(Paths.get(file).toAbsolutePath().toFile()); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index c4a7bfc5c83c..5b56ca7e882c 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -58,6 +58,7 @@ public class TableMetadata implements Serializable { static final int SUPPORTED_TABLE_FORMAT_VERSION = 4; static final int MIN_FORMAT_VERSION_ROW_LINEAGE = 3; static final int MIN_FORMAT_VERSION_PARQUET_MANIFESTS = 4; + static final int MIN_FORMAT_VERSION_OPTIONAL_LOCATION = 4; static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; @@ -312,6 +313,16 @@ public String toString() { SUPPORTED_TABLE_FORMAT_VERSION); Preconditions.checkArgument( formatVersion == 1 || uuid != null, "UUID is required in format v%s", formatVersion); + boolean locationOptional = formatVersion >= MIN_FORMAT_VERSION_OPTIONAL_LOCATION; + Preconditions.checkArgument( + locationOptional || location != null, + "Table location is required in format v%s", + formatVersion); + Preconditions.checkArgument( + !locationOptional || location == null || LocationUtil.hasScheme(location), + "Invalid table location in format v%s, must be absolute: %s", + formatVersion, + location); Preconditions.checkArgument( formatVersion > 1 || lastSequenceNumber == 0, "Sequence number must be 0 in v1: %s", diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index eeeeeab8a699..cc4dd5989c57 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -167,7 +167,9 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion()); generator.writeStringField(TABLE_UUID, metadata.uuid()); - generator.writeStringField(LOCATION, metadata.location()); + if (metadata.location() != null) { + generator.writeStringField(LOCATION, metadata.location()); + } if (metadata.formatVersion() > 1) { generator.writeNumberField(LAST_SEQUENCE_NUMBER, metadata.lastSequenceNumber()); } @@ -347,7 +349,11 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { formatVersion); String uuid = JsonUtil.getStringOrNull(TABLE_UUID, node); - String location = JsonUtil.getString(LOCATION, node); + // location is required in v1-v3 but optional in v4 and later + String location = + formatVersion >= TableMetadata.MIN_FORMAT_VERSION_OPTIONAL_LOCATION + ? JsonUtil.getStringOrNull(LOCATION, node) + : JsonUtil.getString(LOCATION, node); long lastSequenceNumber; if (formatVersion > 1) { lastSequenceNumber = JsonUtil.getLong(LAST_SEQUENCE_NUMBER, node); diff --git a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java index 21eacbfbd665..39a3bfdc7d40 100644 --- a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java @@ -65,7 +65,7 @@ public static String tableLocation(TableIdentifier tableIdentifier, boolean useU * file:}), per RFC 3986 * section 3.1. */ - private static boolean hasScheme(String location) { + public static boolean hasScheme(String location) { for (int i = 0; i < location.length(); i += 1) { char ch = location.charAt(i); if (ch == ':') { diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 09e9fdd1f722..eab95f3256ce 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.net.URI; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -413,7 +414,7 @@ public void testRetainNAvailableSnapshotsWithTransaction() { long t3 = waitUntilAfter(table.currentSnapshot().timestampMillis()); - assertThat(listManifestFiles(new File(table.location()))).hasSize(3); + assertThat(listManifestFiles(new File(URI.create(table.location())))).hasSize(3); // Retain last 2 snapshots, which means 1 is deleted. Transaction tx = table.newTransaction(); @@ -422,7 +423,7 @@ public void testRetainNAvailableSnapshotsWithTransaction() { assertThat(table.snapshots()).hasSize(2); assertThat(table.snapshot(firstSnapshotId)).isNull(); - assertThat(listManifestLists(new File(table.location()))).hasSize(2); + assertThat(listManifestLists(new File(URI.create(table.location())))).hasSize(2); } @TestTemplate @@ -1221,8 +1222,8 @@ public void testExpireWithStatisticsFiles() throws IOException { .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); - assertThat(new File(statsFileLocation1)).doesNotExist(); - assertThat(new File(statsFileLocation2)).exists(); + assertThat(new File(URI.create(statsFileLocation1))).doesNotExist(); + assertThat(new File(URI.create(statsFileLocation2))).exists(); } @TestTemplate @@ -1259,7 +1260,7 @@ public void testExpireWithStatisticsFilesWithReuse() throws IOException { .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); // the reused stats file should exist. - assertThat(new File(statsFileLocation1)).exists(); + assertThat(new File(URI.create(statsFileLocation1))).exists(); } @TestTemplate @@ -1290,8 +1291,8 @@ public void testExpireWithPartitionStatisticsFiles() throws IOException { .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); - assertThat(new File(statsFileLocation1)).doesNotExist(); - assertThat(new File(statsFileLocation2)).exists(); + assertThat(new File(URI.create(statsFileLocation1))).doesNotExist(); + assertThat(new File(URI.create(statsFileLocation2))).exists(); } @TestTemplate @@ -1325,7 +1326,7 @@ public void testExpireWithPartitionStatisticsFilesWithReuse() throws IOException .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); // the reused stats file should exist. - assertThat(new File(statsFileLocation1)).exists(); + assertThat(new File(URI.create(statsFileLocation1))).exists(); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index 76178d4ea318..164ee54c898f 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.net.URI; import java.util.Arrays; import java.util.List; import java.util.Set; @@ -316,7 +317,9 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); V2Assert.assertEquals( - "Should be 1 manifest list", 1, listManifestLists(new File(table.location())).size()); + "Should be 1 manifest list", + 1, + listManifestLists(new File(URI.create(table.location()))).size()); table.newAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); @@ -328,7 +331,9 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); V2Assert.assertEquals( - "Should be 2 manifest lists", 2, listManifestLists(new File(table.location())).size()); + "Should be 2 manifest lists", + 2, + listManifestLists(new File(URI.create(table.location()))).size()); Transaction txn = table.newTransaction(); txn.expireSnapshots().expireSnapshotId(commitId1).commit(); @@ -338,7 +343,7 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Should be 1 manifest list as 1 was deleted", 1, - listManifestLists(new File(table.location())).size()); + listManifestLists(new File(URI.create(table.location()))).size()); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index cb1decd2d8dc..8e3f0e638f28 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -36,6 +36,7 @@ import static org.junit.jupiter.params.provider.Arguments.arguments; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.node.ObjectNode; import java.io.File; import java.io.IOException; import java.io.StringWriter; @@ -1133,6 +1134,79 @@ public void testParserVersionValidation() throws Exception { .hasMessageStartingWith("Cannot read unsupported version"); } + @ParameterizedTest + @ValueSource(ints = {1, 2, 3}) + public void testLocationRequiredBeforeV4(int formatVersion) { + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Table location is required in format v%s", formatVersion); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 3}) + public void testParserRequiresLocationBeforeV4(int formatVersion) throws Exception { + TableMetadata metadata = + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + formatVersion); + // drop the location field to simulate a v1-v3 metadata file that omits the required location + ObjectNode node = (ObjectNode) JsonUtil.mapper().readTree(TableMetadataParser.toJson(metadata)); + node.remove(LOCATION); + String withoutLocation = JsonUtil.mapper().writeValueAsString(node); + + assertThatThrownBy(() -> TableMetadataParser.fromJson(withoutLocation)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(LOCATION); + } + + @Test + public void testLocationOptionalInV4() { + TableMetadata metadata = + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + 4); + assertThat(metadata.location()).isNull(); + + // location must be omitted from the serialized metadata, not written as a JSON null + String json = TableMetadataParser.toJson(metadata); + assertThat(json).doesNotContain("\"location\""); + + // and the round trip must succeed with a null location + TableMetadata parsed = TableMetadataParser.fromJson(json); + assertThat(parsed.location()).isNull(); + } + + @Test + public void testV4LocationMustBeAbsolute() { + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + "relative/path", + ImmutableMap.of(), + 4)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table location in format v4, must be absolute: relative/path"); + } + @Test public void testParserV2PartitionSpecsValidation() throws Exception { String unsupportedVersion = @@ -1311,7 +1385,7 @@ public void testSortOrder() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); assertThat(meta.sortOrder().isUnsorted()).isTrue(); assertThat(meta.replaceSortOrder(SortOrder.unsorted())) .as("Should detect identical unsorted order") @@ -1326,7 +1400,7 @@ public void testUpdateSortOrder() { TableMetadata sortedByX = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), order, null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), order, TEST_LOCATION, ImmutableMap.of()); assertThat(sortedByX.sortOrders()).hasSize(1); assertThat(sortedByX.sortOrder().orderId()).isEqualTo(1); assertThat(sortedByX.sortOrder().fields()).hasSize(1); @@ -1363,7 +1437,7 @@ public void testStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); assertThat(meta.statisticsFiles()).as("Should default to no statistics files").isEmpty(); } @@ -1373,7 +1447,7 @@ public void testSetStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); TableMetadata withStatistics = TableMetadata.buildFrom(meta) @@ -1411,7 +1485,7 @@ public void testRemoveStatistics() { TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of())) .setStatistics( new GenericStatisticsFile( 43, "/some/path/to/stats/file", 128, 27, ImmutableList.of())) @@ -1440,7 +1514,7 @@ public void testPartitionStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); assertThat(meta.partitionStatisticsFiles()) .as("Should default to no partition statistics files") .isEmpty(); @@ -1452,7 +1526,7 @@ public void testSetPartitionStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); TableMetadata withPartitionStatistics = TableMetadata.buildFrom(meta) @@ -1502,7 +1576,7 @@ public void testRemovePartitionStatistics() { TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of())) .setPartitionStatistics( ImmutableGenericPartitionStatisticsFile.builder() .snapshotId(43) @@ -1559,7 +1633,7 @@ public void testUpdateSchemaIdentifierFields() { TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); Schema newSchema = new Schema( @@ -1576,7 +1650,7 @@ public void testUpdateSchema() { new Schema(0, Types.NestedField.required(1, "y", Types.LongType.get(), "comment")); TableMetadata freshTable = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), TEST_LOCATION, ImmutableMap.of()); assertThat(freshTable.currentSchemaId()).isEqualTo(TableMetadata.INITIAL_SCHEMA_ID); assertSameSchemaList(ImmutableList.of(schema), freshTable.schemas()); assertThat(freshTable.schema().asStruct()).isEqualTo(schema.asStruct()); @@ -1644,7 +1718,7 @@ public void testCreateV2MetadataThroughTableProperty() { TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), - null, + TEST_LOCATION, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key", "val")); assertThat(meta.formatVersion()).isEqualTo(2); @@ -1730,7 +1804,7 @@ public void testReplaceMetadataThroughTableProperty(int baseFormatVersion, int n TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), - null, + TEST_LOCATION, ImmutableMap.of( TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); @@ -1759,7 +1833,7 @@ public void testUpgradeMetadataThroughTableProperty(int baseFormatVersion, int n TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), - null, + TEST_LOCATION, ImmutableMap.of( TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); @@ -1964,7 +2038,10 @@ public void testMetadataWithRemoveSpecs() { TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - TestBase.SCHEMA, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + TestBase.SCHEMA, + PartitionSpec.unpartitioned(), + TEST_LOCATION, + ImmutableMap.of())) .removeSpecs(Sets.newHashSet()) .build(); @@ -1980,7 +2057,10 @@ public void testMetadataWithRemoveSchemas() { TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - TestBase.SCHEMA, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + TestBase.SCHEMA, + PartitionSpec.unpartitioned(), + TEST_LOCATION, + ImmutableMap.of())) .removeSchemas(Sets.newHashSet()) .build(); diff --git a/core/src/test/java/org/apache/iceberg/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java index 13c859f86065..fdf730a22074 100644 --- a/core/src/test/java/org/apache/iceberg/TestTables.java +++ b/core/src/test/java/org/apache/iceberg/TestTables.java @@ -133,7 +133,8 @@ private static TestTable createTable( } TableMetadata metadata = - newTableMetadata(schema, spec, sortOrder, temp.toString(), properties, formatVersion); + newTableMetadata( + schema, spec, sortOrder, temp.toURI().toString(), properties, formatVersion); if (metaTemp != null) { metadata = @@ -164,7 +165,7 @@ public static Transaction beginCreate( } TableMetadata metadata = - newTableMetadata(schema, spec, sortOrder, temp.toString(), ImmutableMap.of(), 1); + newTableMetadata(schema, spec, sortOrder, temp.toURI().toString(), ImmutableMap.of(), 1); return Transactions.createTableTransaction(name, ops, metadata); } @@ -206,7 +207,7 @@ public static Transaction beginReplace( metadata = current.buildReplacement(schema, spec, sortOrder, current.location(), properties); return Transactions.replaceTableTransaction(name, ops, metadata); } else { - metadata = newTableMetadata(schema, spec, sortOrder, temp.toString(), properties); + metadata = newTableMetadata(schema, spec, sortOrder, temp.toURI().toString(), properties); return Transactions.createTableTransaction(name, ops, metadata); } } @@ -389,7 +390,8 @@ public OutputFile newOutputFile(String path) { @Override public void deleteFile(String path) { - if (!new File(path).delete()) { + String localPath = path.startsWith("file:") ? path.replaceFirst("file:", "") : path; + if (!new File(localPath).delete()) { throw new RuntimeIOException("Failed to delete file: " + path); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java index 80eae1fe5cc5..d0a2be263e23 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -146,7 +146,7 @@ public void roundTripSerdeV3andHigher(int formatVersion) { TableMetadata metadata = TableMetadata.buildFromEmpty(formatVersion) .assignUUID(uuid) - .setLocation("location") + .setLocation("file://location") .setCurrentSchema( new Schema(Types.NestedField.required(1, "x", Types.LongType.get())), 1) .addPartitionSpec(PartitionSpec.unpartitioned()) @@ -164,7 +164,7 @@ public void roundTripSerdeV3andHigher(int formatVersion) { + " \"metadata\" : {\n" + " \"format-version\" : %s,\n" + " \"table-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" - + " \"location\" : \"location\",\n" + + " \"location\" : \"file://location\",\n" + " \"last-sequence-number\" : 0,\n" + " \"last-updated-ms\" : %d,\n" + " \"last-column-id\" : 1,\n" diff --git a/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java b/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java index a1c32e996054..80d9533ef799 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java +++ b/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java @@ -53,7 +53,7 @@ public void testIncrementalHashCalculation() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "col1", Types.StringType.get())); TableMetadata tableMetadata = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, icebergTblProperties); + schema, PartitionSpec.unpartitioned(), "file:///tmp/table", icebergTblProperties); JsonGenerator generator = JsonUtil.factory().createGenerator(hashWriter); TableMetadataParser.toJson(tableMetadata, generator); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index 4617a5fdbe19..a9ad7fba950e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -170,6 +170,8 @@ public void before() throws Exception { String tablePath = warehouse.concat("/test"); assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + // v4 table metadata requires an absolute (scheme-qualified) location + String tableLocation = "file:" + tablePath; Map props = ImmutableMap.of( @@ -179,8 +181,8 @@ public void before() throws Exception { flinkManifestFolder.getAbsolutePath(), IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "1"); - table = SimpleDataUtil.createTable(tablePath, props, false); - tableLoader = TableLoader.fromHadoopTable(tablePath); + table = SimpleDataUtil.createTable(tableLocation, props, false); + tableLoader = TableLoader.fromHadoopTable(tableLocation); } @TestTemplate diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index 2fb7fc10a8a1..bdc363d445cf 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -170,6 +170,8 @@ public void before() throws Exception { String tablePath = warehouse.concat("/test"); assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + // v4 table metadata requires an absolute (scheme-qualified) location + String tableLocation = "file:" + tablePath; Map props = ImmutableMap.of( @@ -179,8 +181,8 @@ public void before() throws Exception { flinkManifestFolder.getAbsolutePath(), IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "1"); - table = SimpleDataUtil.createTable(tablePath, props, false); - tableLoader = TableLoader.fromHadoopTable(tablePath); + table = SimpleDataUtil.createTable(tableLocation, props, false); + tableLoader = TableLoader.fromHadoopTable(tableLocation); } @TestTemplate diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index e77016680ee5..3dca51bd8586 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -170,6 +170,8 @@ public void before() throws Exception { String tablePath = warehouse.concat("/test"); assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + // v4 table metadata requires an absolute (scheme-qualified) location + String tableLocation = "file:" + tablePath; Map props = ImmutableMap.of( @@ -179,8 +181,8 @@ public void before() throws Exception { flinkManifestFolder.getAbsolutePath(), IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "1"); - table = SimpleDataUtil.createTable(tablePath, props, false); - tableLoader = TableLoader.fromHadoopTable(tablePath); + table = SimpleDataUtil.createTable(tableLocation, props, false); + tableLoader = TableLoader.fromHadoopTable(tableLocation); } @TestTemplate diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index f212d307fe9d..0cf2882a79f4 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -1253,7 +1253,7 @@ public void testMetadataHashing(boolean isTableEncrypted) { Schema schema = new Schema(Types.NestedField.required(1, "col1", Types.StringType.get())); TableMetadata tableMetadata = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, PartitionSpec.unpartitioned(), "file:///tmp/table", ImmutableMap.of()); HMSTablePropertyHelper.setMetadataHash(tableMetadata, hiveTblProperties); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index c9fb97e0f9e9..801f8927ebb3 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -593,6 +593,6 @@ private org.projectnessie.model.Namespace fetchNamespace(ContentKey key, String private static TableMetadata newTableMetadata() { Schema schema = new Schema(required(1, "id", Types.LongType.get())); return TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), SortOrder.unsorted(), null, Map.of()); + schema, PartitionSpec.unpartitioned(), SortOrder.unsorted(), "file:///tmp/table", Map.of()); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 0d2a5c0a4daf..34a02a93faf1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -736,20 +736,14 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assumeThat(usePrefixListing) .as("Should not test both prefix listing and Hadoop file listing (redundant)") .isEqualTo(false); - Table table = - TABLES.create( - SCHEMA, PartitionSpec.unpartitioned(), properties, tableDir.getAbsolutePath()); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); - df.select("c1", "c2", "c3") - .write() - .format("iceberg") - .mode("append") - .save(tableDir.getAbsolutePath()); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); List validFiles = spark diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java index b54bb315c543..6fe1e5c55493 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java @@ -53,7 +53,8 @@ static TestTable create( if (ops.current() != null) { throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp); } - ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), properties)); + ops.commit( + null, TableMetadata.newTableMetadata(schema, spec, temp.toURI().toString(), properties)); return new TestTable(ops, name); } @@ -196,12 +197,13 @@ public InputFile newInputFile(String path) { @Override public OutputFile newOutputFile(String path) { - return Files.localOutput(new File(path)); + return Files.localOutput(path); } @Override public void deleteFile(String path) { - if (!new File(path).delete()) { + String localPath = path.startsWith("file:") ? path.replaceFirst("file:", "") : path; + if (!new File(localPath).delete()) { throw new RuntimeIOException("Failed to delete file: " + path); } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 0d2a5c0a4daf..34a02a93faf1 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -736,20 +736,14 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assumeThat(usePrefixListing) .as("Should not test both prefix listing and Hadoop file listing (redundant)") .isEqualTo(false); - Table table = - TABLES.create( - SCHEMA, PartitionSpec.unpartitioned(), properties, tableDir.getAbsolutePath()); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); - df.select("c1", "c2", "c3") - .write() - .format("iceberg") - .mode("append") - .save(tableDir.getAbsolutePath()); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); List validFiles = spark diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java index b54bb315c543..6fe1e5c55493 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java @@ -53,7 +53,8 @@ static TestTable create( if (ops.current() != null) { throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp); } - ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), properties)); + ops.commit( + null, TableMetadata.newTableMetadata(schema, spec, temp.toURI().toString(), properties)); return new TestTable(ops, name); } @@ -196,12 +197,13 @@ public InputFile newInputFile(String path) { @Override public OutputFile newOutputFile(String path) { - return Files.localOutput(new File(path)); + return Files.localOutput(path); } @Override public void deleteFile(String path) { - if (!new File(path).delete()) { + String localPath = path.startsWith("file:") ? path.replaceFirst("file:", "") : path; + if (!new File(localPath).delete()) { throw new RuntimeIOException("Failed to delete file: " + path); } } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 5a33c710b2f6..78e8a0b000a4 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -737,20 +737,14 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assumeThat(usePrefixListing) .as("Should not test both prefix listing and Hadoop file listing (redundant)") .isEqualTo(false); - Table table = - TABLES.create( - SCHEMA, PartitionSpec.unpartitioned(), properties, tableDir.getAbsolutePath()); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); - df.select("c1", "c2", "c3") - .write() - .format("iceberg") - .mode("append") - .save(tableDir.getAbsolutePath()); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); List validFiles = spark diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java index b54bb315c543..6fe1e5c55493 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java @@ -53,7 +53,8 @@ static TestTable create( if (ops.current() != null) { throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp); } - ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), properties)); + ops.commit( + null, TableMetadata.newTableMetadata(schema, spec, temp.toURI().toString(), properties)); return new TestTable(ops, name); } @@ -196,12 +197,13 @@ public InputFile newInputFile(String path) { @Override public OutputFile newOutputFile(String path) { - return Files.localOutput(new File(path)); + return Files.localOutput(path); } @Override public void deleteFile(String path) { - if (!new File(path).delete()) { + String localPath = path.startsWith("file:") ? path.replaceFirst("file:", "") : path; + if (!new File(localPath).delete()) { throw new RuntimeIOException("Failed to delete file: " + path); } } From b88addf85253951ab4898967b62eee287826f7c9 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Mon, 1 Jun 2026 13:54:49 -0700 Subject: [PATCH 52/58] Core: Support pluggable executor service for manifest writing on SnapshotUpdate (#16108) --- .../org/apache/iceberg/SnapshotUpdate.java | 16 +++++ .../org/apache/iceberg/SnapshotProducer.java | 26 ++++++- .../java/org/apache/iceberg/TestBase.java | 26 +++++++ .../org/apache/iceberg/TestMergeAppend.java | 72 +++++++++++++++++-- .../apache/iceberg/TestSnapshotProducer.java | 24 +++++++ 5 files changed, 155 insertions(+), 9 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java b/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java index 73509c15384f..b57607cbf75c 100644 --- a/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java +++ b/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java @@ -60,6 +60,22 @@ public interface SnapshotUpdate extends PendingUpdate { */ ThisT scanManifestsWith(ExecutorService executorService); + /** + * Use a particular executor to write manifests during commit with the specified parallelism. The + * default worker pool will be used by default. + * + *

The parallelism parameter controls how many manifest writers are used, which informs the + * number of manifest files produced. The executor provides the threads for parallel execution. + * + * @param executorService the provided executor + * @param parallelism the number of parallel manifest writers to use + * @return this for method chaining + */ + default ThisT writeManifestsWith(ExecutorService executorService, int parallelism) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not support writeManifestsWith"); + } + /** * Perform operations on a particular branch * diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index e351009a9ea6..f77222d6a2a6 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -124,6 +124,8 @@ public void accept(String file) { SnapshotAncestryValidator.NON_VALIDATING; private ExecutorService workerPool; + private ExecutorService writePool; + private int writePoolParallelism = ThreadPools.WORKER_THREAD_POOL_SIZE; private String targetBranch = SnapshotRef.MAIN_BRANCH; private CommitMetrics commitMetrics; @@ -169,6 +171,16 @@ public ThisT scanManifestsWith(ExecutorService executorService) { return self(); } + @Override + public ThisT writeManifestsWith(ExecutorService executorService, int parallelism) { + Preconditions.checkArgument(executorService != null, "Executor service cannot be null"); + Preconditions.checkArgument( + parallelism > 0, "Parallelism must be greater than 0, but was: %s", parallelism); + this.writePool = executorService; + this.writePoolParallelism = parallelism; + return self(); + } + /** * Set a validator to check snapshot ancestry before committing changes. * @@ -227,6 +239,14 @@ protected ExecutorService workerPool() { return workerPool; } + protected ExecutorService writePool() { + if (writePool == null) { + this.writePool = ThreadPools.getWorkerPool(); + } + + return writePool; + } + @Override public ThisT deleteWith(Consumer deleteCallback) { Preconditions.checkArgument( @@ -780,9 +800,9 @@ private List writeDeleteFileGroup( return writer.toManifestFiles(); } - private static List writeManifests( + private List writeManifests( Collection files, Function, List> writeFunc) { - int parallelism = manifestWriterCount(ThreadPools.WORKER_THREAD_POOL_SIZE, files.size()); + int parallelism = manifestWriterCount(writePoolParallelism, files.size()); List> groups = divide(files, parallelism); // Create a new list pairing each group with its index @@ -796,7 +816,7 @@ private static List writeManifests( Tasks.foreach(groupsWithIndex) .stopOnFailure() .throwFailureWhenFinished() - .executeWith(ThreadPools.getWorkerPool()) + .executeWith(writePool()) .run( indexedGroup -> { int index = indexedGroup.first(); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 0f649cabeb81..6865d1fb1bc4 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -35,6 +35,9 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.io.FileIO; @@ -298,6 +301,29 @@ public TableMetadata readMetadata() { return TestTables.readMetadata("test"); } + protected void assertEmptyTable() { + assertThat(listManifestFiles()).isEmpty(); + TableMetadata base = readMetadata(); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); + } + + protected static ExecutorService newNamedExecutor(String prefix, AtomicInteger counter) { + return newNamedExecutor(prefix, counter, 1); + } + + protected static ExecutorService newNamedExecutor( + String prefix, AtomicInteger counter, int nThreads) { + return Executors.newFixedThreadPool( + nThreads, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName(prefix + "-" + counter.getAndIncrement()); + thread.setDaemon(true); + return thread; + }); + } + static FileFormat manifestFormat(int version) { return version >= TableMetadata.MIN_FORMAT_VERSION_PARQUET_MANIFESTS ? FileFormat.PARQUET diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index c5f3d88f46d9..2654f244f3d1 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -41,7 +41,6 @@ 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; -import org.apache.iceberg.util.ThreadPools; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -91,11 +90,6 @@ public void testAddManyFilesWithConsistentOrdering() { assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); int multiplier = 3; - assumeThat(ThreadPools.WORKER_THREAD_POOL_SIZE) - .as( - "Worker thread pool size should be at least 3 to test manifest file ordering with multiple threads") - .isGreaterThanOrEqualTo(multiplier); - int groupSize = SnapshotProducer.MIN_FILE_GROUP_SIZE; List dataFiles = Lists.newArrayList(); @@ -107,6 +101,7 @@ public void testAddManyFilesWithConsistentOrdering() { AppendFiles append = table.newAppend(); dataFiles.forEach(append::appendFile); + append.writeManifestsWith(Executors.newFixedThreadPool(multiplier), multiplier); append.commit(); Snapshot snapshot = table.currentSnapshot(); @@ -381,6 +376,71 @@ public void testAppendWithManifestScanExecutor() { assertThat(snapshot).isNotNull(); } + @TestTemplate + public void testAppendWithWriteManifestsExecutor() { + assertEmptyTable(); + + AtomicInteger commitThreadsIndex = new AtomicInteger(0); + Snapshot snapshot = + commit( + table, + table + .newAppend() + .appendFile(FILE_A) + .appendFile(FILE_B) + .writeManifestsWith(newNamedExecutor("commit", commitThreadsIndex), 1), + branch); + assertThat(commitThreadsIndex.get()) + .as("Thread should be created in provided commit pool") + .isGreaterThan(0); + + assertThat(snapshot).isNotNull(); + assertThat(snapshot.allManifests(table.io())).hasSize(1); + long snapshotId = snapshot.snapshotId(); + validateManifest( + snapshot.allManifests(table.io()).get(0), + dataSeqs(1L, 1L), + fileSeqs(1L, 1L), + ids(snapshotId, snapshotId), + files(FILE_A, FILE_B), + statuses(Status.ADDED, Status.ADDED)); + } + + @TestTemplate + public void testAppendWithSeparateScanAndWriteExecutors() { + assertEmptyTable(); + + AtomicInteger scanThreadsIndex = new AtomicInteger(0); + AtomicInteger commitThreadsIndex = new AtomicInteger(0); + Snapshot snapshot = + commit( + table, + table + .newAppend() + .appendFile(FILE_A) + .appendFile(FILE_B) + .scanManifestsWith(newNamedExecutor("scan", scanThreadsIndex)) + .writeManifestsWith(newNamedExecutor("commit", commitThreadsIndex), 1), + branch); + assertThat(scanThreadsIndex.get()) + .as("Thread should be created in provided scan pool") + .isGreaterThan(0); + assertThat(commitThreadsIndex.get()) + .as("Thread should be created in provided commit pool") + .isGreaterThan(0); + + assertThat(snapshot).isNotNull(); + assertThat(snapshot.allManifests(table.io())).hasSize(1); + long snapshotId = snapshot.snapshotId(); + validateManifest( + snapshot.allManifests(table.io()).get(0), + dataSeqs(1L, 1L), + fileSeqs(1L, 1L), + ids(snapshotId, snapshotId), + files(FILE_A, FILE_B), + statuses(Status.ADDED, Status.ADDED)); + } + @TestTemplate public void testMergeWithAppendFilesAndManifest() throws IOException { // merge all manifests for this test diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java index c6092f0238b9..1e6021a5c9fc 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java @@ -28,6 +28,8 @@ import java.io.IOException; import java.nio.file.Paths; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.stream.Collectors; import javax.annotation.Nonnull; import org.apache.iceberg.exceptions.ValidationException; @@ -252,4 +254,26 @@ public void testAvroManifestCompressionFromTableProperty() throws IOException { ManifestFile manifest = table.currentSnapshot().dataManifests(table.io()).get(0); assertThat(readAvroCodec(new File(manifest.path()))).isEqualTo("snappy"); } + + @TestTemplate + public void testWriteManifestsWithNullExecutorThrows() { + assertThatThrownBy(() -> table.newAppend().writeManifestsWith(null, 4)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Executor service cannot be null"); + } + + @TestTemplate + public void testWriteManifestsWithInvalidParallelismThrows() { + ExecutorService executor = Executors.newFixedThreadPool(4); + try { + assertThatThrownBy(() -> table.newAppend().writeManifestsWith(executor, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Parallelism must be greater than 0"); + assertThatThrownBy(() -> table.newAppend().writeManifestsWith(executor, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Parallelism must be greater than 0"); + } finally { + executor.shutdownNow(); + } + } } From 26a57711d990c695915afb5ce14f00736325d547 Mon Sep 17 00:00:00 2001 From: Vova Kolmakov Date: Tue, 2 Jun 2026 17:48:51 +0700 Subject: [PATCH 53/58] Parquet: Fix timestamp_ns and timestamptz_ns predicate pushdown (#16619) --- .../iceberg/parquet/MessageTypeToType.java | 11 +- .../iceberg/parquet/ParquetFilters.java | 1 + .../apache/iceberg/parquet/TestParquet.java | 114 ++++++++++++++++++ .../parquet/TestParquetSchemaUtil.java | 32 +++++ 4 files changed, 156 insertions(+), 2 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java index 841777152ee8..98023bafcb8f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java @@ -220,8 +220,15 @@ public Optional visit(LogicalTypeAnnotation.TimeLogicalTypeAnnotation time @Override public Optional visit( LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampType) { - return Optional.of( - timestampType.isAdjustedToUTC() ? TimestampType.withZone() : TimestampType.withoutZone()); + boolean adjustToUtc = timestampType.isAdjustedToUTC(); + if (timestampType.getUnit() == LogicalTypeAnnotation.TimeUnit.NANOS) { + return Optional.of( + adjustToUtc + ? Types.TimestampNanoType.withZone() + : Types.TimestampNanoType.withoutZone()); + } + + return Optional.of(adjustToUtc ? TimestampType.withZone() : TimestampType.withoutZone()); } @Override diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java index fc6febe19438..0c357b3930ac 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java @@ -143,6 +143,7 @@ public FilterPredicate predicate(BoundPredicate pred) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: return pred(op, FilterApi.longColumn(path), getParquetPrimitive(lit)); case FLOAT: return pred(op, FilterApi.floatColumn(path), getParquetPrimitive(lit)); diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index 5f1e0c83cc0f..1a1916895cdc 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -24,6 +24,8 @@ import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write; import static org.apache.iceberg.relocated.com.google.common.collect.Iterables.getOnlyElement; @@ -36,6 +38,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.file.Path; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; import java.util.Collections; import java.util.List; import java.util.function.Function; @@ -48,6 +52,11 @@ import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; @@ -378,4 +387,109 @@ private Pair generateFile( records.toArray(new GenericData.Record[] {})); return Pair.of(file, size); } + + @Test + public void timestampNanoFilterRespectsNanoseconds() throws IOException { + // Predicate pushdown on timestamp_ns must filter at full nanosecond resolution. The five rows + // differ only by sub-microsecond nanoseconds, so a micros-truncating push down could not + // separate id 2 (250 ns) from id 3 (750 ns) and would return the wrong rows. + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "ts", Types.TimestampNanoType.withoutZone())); + + Record template = org.apache.iceberg.data.GenericRecord.create(schema); + List records = + Lists.newArrayList( + template.copy( + ImmutableMap.of( + "id", 1L, "ts", LocalDateTime.parse("2024-01-01T00:00:00.000000000"))), + template.copy( + ImmutableMap.of( + "id", 2L, "ts", LocalDateTime.parse("2024-01-01T00:00:00.000000250"))), + template.copy( + ImmutableMap.of( + "id", 3L, "ts", LocalDateTime.parse("2024-01-01T00:00:00.000000750"))), + template.copy( + ImmutableMap.of( + "id", 4L, "ts", LocalDateTime.parse("2024-01-01T00:00:00.000001500"))), + template.copy( + ImmutableMap.of( + "id", 5L, "ts", LocalDateTime.parse("2024-01-01T00:00:00.000003000")))); + + File file = writeNanoRecords(schema, records); + + // Boundary at 500 ns: only ids 3 (750 ns), 4 (1500 ns), 5 (3000 ns) qualify. + List ids = + filterIds(schema, file, greaterThanOrEqual("ts", "2024-01-01T00:00:00.000000500")); + assertThat(ids).containsExactlyInAnyOrder(3L, 4L, 5L); + } + + @Test + public void timestamptzNanoFilterAcrossTimezones() throws IOException { + // Each row is written in a different zone offset; instants are 0/500/750/1500/3000 ns past the + // same UTC second. id2 lands exactly on the filter boundary but in +05:00, so a strict + // greaterThan must exclude it by instant, not by wall-clock (its 05:00 vs the boundary's + // 04:00). + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "ts", Types.TimestampNanoType.withZone())); + + Record template = org.apache.iceberg.data.GenericRecord.create(schema); + List records = + Lists.newArrayList( + template.copy( + ImmutableMap.of( + "id", 1L, "ts", OffsetDateTime.parse("2024-01-01T00:00:00.000000000+00:00"))), + template.copy( + ImmutableMap.of( + "id", 2L, "ts", OffsetDateTime.parse("2024-01-01T05:00:00.000000500+05:00"))), + template.copy( + ImmutableMap.of( + "id", 3L, "ts", OffsetDateTime.parse("2023-12-31T16:00:00.000000750-08:00"))), + template.copy( + ImmutableMap.of( + "id", 4L, "ts", OffsetDateTime.parse("2024-01-01T05:30:00.000001500+05:30"))), + template.copy( + ImmutableMap.of( + "id", 5L, "ts", OffsetDateTime.parse("2024-01-01T00:00:00.000003000+00:00")))); + + File file = writeNanoRecords(schema, records); + + // Boundary == 2024-01-01T00:00:00.000000500Z, expressed in +04:00. id2 is that same instant + // (written in +05:00); a strict greaterThan excludes it, leaving ids 3/4/5. + List ids = + filterIds(schema, file, greaterThan("ts", "2024-01-01T04:00:00.000000500+04:00")); + assertThat(ids).containsExactlyInAnyOrder(3L, 4L, 5L); + } + + private File writeNanoRecords(Schema schema, List records) throws IOException { + File file = createTempFile(temp); + try (FileAppender appender = + Parquet.write(Files.localOutput(file)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .build()) { + for (Record record : records) { + appender.add(record); + } + } + + return file; + } + + private List filterIds(Schema schema, File file, Expression filter) throws IOException { + List ids = Lists.newArrayList(); + // callInit() drives the parquet-mr ReadSupport path, the only read path that runs + // ParquetFilters. + try (CloseableIterable reader = + Parquet.read(localInput(file)).project(schema).callInit().filter(filter).build()) { + for (GenericData.Record record : reader) { + ids.add((Long) record.get("id")); + } + } + + return ids; + } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java index 1df904f13c7b..51bfc1e811f4 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java @@ -501,6 +501,38 @@ public void testLegacyTwoLevelListGenByParquetThrift1() { .isEqualTo(expectedSchema.asStruct()); } + @Test + public void testTimestampNanoConversionPreservesUnit() { + // INT64 + TIMESTAMP(NANOS) must round-trip back to Iceberg as timestamp_ns, not micros. A + // micros field is included to confirm the unit branch leaves the existing mapping untouched. + MessageType messageType = + org.apache.parquet.schema.Types.buildMessage() + .required(PrimitiveTypeName.INT64) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.NANOS)) + .id(1) + .named("ts_tz_ns") + .optional(PrimitiveTypeName.INT64) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.NANOS)) + .id(2) + .named("ts_ns") + .required(PrimitiveTypeName.INT64) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .id(3) + .named("ts_tz_micros") + .named("table"); + + Schema expectedSchema = + new Schema( + required(1, "ts_tz_ns", Types.TimestampNanoType.withZone()), + optional(2, "ts_ns", Types.TimestampNanoType.withoutZone()), + required(3, "ts_tz_micros", Types.TimestampType.withZone())); + + Schema actualSchema = ParquetSchemaUtil.convert(messageType); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); + } + private Type primitive( Integer id, String name, PrimitiveTypeName typeName, Repetition repetition) { PrimitiveBuilder builder = From f5349db39ea92cee9585f10fee0d5a91d858f22e Mon Sep 17 00:00:00 2001 From: Hao Jiang Date: Tue, 2 Jun 2026 15:26:58 -0400 Subject: [PATCH 54/58] Arrow: Fix vectorized reads of decimal columns with default values (#16501) --- .../vectorized/VectorizedArrowReader.java | 9 +- .../TestVectorizedDefaultValues.java | 140 ++++++++++++++++++ .../parquet/TestParquetVectorizedReads.java | 46 ++++++ 3 files changed, 194 insertions(+), 1 deletion(-) create mode 100644 arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestVectorizedDefaultValues.java diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java index fdaf6d261d7b..11c73aea4a44 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java @@ -252,7 +252,14 @@ private static Types.NestedField getPhysicalType( // Use FixedSizeBinaryVector for binary backed decimal type = Types.FixedType.ofLength(primitive.getTypeLength()); } - physicalType = Types.NestedField.from(logicalType).ofType(type).build(); + // drop initialDefault/writeDefault: they are typed for the logical (decimal) type and + // cannot be cast to the underlying physical type + physicalType = + Types.NestedField.from(logicalType) + .ofType(type) + .withInitialDefault(null) + .withWriteDefault(null) + .build(); } return physicalType; diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestVectorizedDefaultValues.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestVectorizedDefaultValues.java new file mode 100644 index 000000000000..48a9aa1b77e5 --- /dev/null +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/TestVectorizedDefaultValues.java @@ -0,0 +1,140 @@ +/* + * 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.arrow.vectorized; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.math.BigDecimal; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.parquet.hadoop.ParquetOutputFormat; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +/** Vectorized-read tests focused on Iceberg field defaults. */ +public class TestVectorizedDefaultValues { + + @TempDir private File tempDir; + + @Test + public void testDecimalWithDefaultValueNotDictionaryEncoded() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required("id").withId(1).ofType(Types.LongType.get()).build(), + Types.NestedField.optional("int_backed") + .withId(2) + .ofType(Types.DecimalType.of(5, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build(), + Types.NestedField.optional("long_backed") + .withId(3) + .ofType(Types.DecimalType.of(15, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build(), + Types.NestedField.optional("fixed_backed") + .withId(4) + .ofType(Types.DecimalType.of(25, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build()); + + HadoopTables tables = new HadoopTables(); + Table table = + tables.create( + schema, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"), + tempDir.toURI().toString()); + + List records = Lists.newArrayList(); + GenericRecord template = GenericRecord.create(schema); + for (long i = 0; i < 5; i++) { + GenericRecord rec = template.copy(); + rec.setField("id", i); + rec.setField("int_backed", new BigDecimal("12.34")); + rec.setField("long_backed", new BigDecimal("1234567890.12")); + rec.setField("fixed_backed", new BigDecimal("9876543210.99")); + records.add(rec); + } + + File dataFile = new File(tempDir, "decimal-no-dict.parquet"); + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .set(ParquetOutputFormat.ENABLE_DICTIONARY, "false") + .build()) { + writer.addAll(records); + } + + DataFile parquetFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(dataFile.getAbsolutePath()) + .withFileSizeInBytes(dataFile.length()) + .withFormat(FileFormat.PARQUET) + .withRecordCount(records.size()) + .build(); + table.newAppend().appendFile(parquetFile).commit(); + + int rowsRead = 0; + try (VectorizedTableScanIterable reader = + new VectorizedTableScanIterable(table.newScan(), 1024, false)) { + for (ColumnarBatch batch : reader) { + ColumnVector idColumn = batch.column(0); + ColumnVector intBackedColumn = batch.column(1); + ColumnVector longBackedColumn = batch.column(2); + ColumnVector fixedBackedColumn = batch.column(3); + + for (int i = 0; i < batch.numRows(); i++) { + GenericRecord expected = records.get(rowsRead + i); + assertThat(idColumn.getLong(i)).isEqualTo(expected.getField("id")); + assertThat(intBackedColumn.getDecimal(i, 5, 2)) + .isEqualTo(expected.getField("int_backed")); + assertThat(longBackedColumn.getDecimal(i, 15, 2)) + .isEqualTo(expected.getField("long_backed")); + assertThat(fixedBackedColumn.getDecimal(i, 25, 2)) + .isEqualTo(expected.getField("fixed_backed")); + } + + rowsRead += batch.numRows(); + } + } + + assertThat(rowsRead).isEqualTo(records.size()); + } +} diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java index 6011c6dad7d2..4a24a725e0f5 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.math.BigDecimal; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Path; @@ -45,6 +46,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; @@ -65,6 +67,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.hadoop.ParquetOutputFormat; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; @@ -254,6 +257,16 @@ FileAppender getParquetV2Writer(Schema schema, File testFile) throws IOE .build(); } + FileAppender parquetWriterWithoutDictionary(Schema schema, File testFile) + throws IOException { + return Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .named("test") + .set(ParquetOutputFormat.ENABLE_DICTIONARY, "false") + .build(); + } + void assertRecordsMatch( Schema schema, int expectedSize, @@ -460,6 +473,39 @@ public void testUuidReads() throws Exception { assertRecordsMatch(schema, numRows, data, dataFile, false, BATCH_SIZE); } + @Test + public void testDecimalWithDefaultValueNotDictionaryEncoded() throws Exception { + Schema schema = + new Schema( + required(100, "id", Types.LongType.get()), + Types.NestedField.optional("int_backed") + .withId(101) + .ofType(Types.DecimalType.of(5, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build(), + Types.NestedField.optional("long_backed") + .withId(102) + .ofType(Types.DecimalType.of(15, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build(), + Types.NestedField.optional("fixed_backed") + .withId(103) + .ofType(Types.DecimalType.of(25, 2)) + .withInitialDefault(Literal.of(new BigDecimal("0.00"))) + .withWriteDefault(Literal.of(new BigDecimal("0.00"))) + .build()); + + File dataFile = temp.resolve("decimal-no-dict.parquet").toFile(); + Iterable data = generateData(schema, 1000, 0L, 0.0f, IDENTITY); + try (FileAppender writer = parquetWriterWithoutDictionary(schema, dataFile)) { + writer.addAll(data); + } + + assertRecordsMatch(schema, 1000, data, dataFile, false, BATCH_SIZE); + } + private void assertIdenticalFileContents( File actual, File expected, Schema schema, boolean vectorized) throws IOException { try (CloseableIterable expectedIterator = From c958fcf6328d36c78d853acc1774c6c1a75e028e Mon Sep 17 00:00:00 2001 From: Neelesh Salian Date: Tue, 2 Jun 2026 14:24:42 -0700 Subject: [PATCH 55/58] iceberg go 0.6.0 release blog (#16649) --- .../2026-06-01-iceberg-go-0.6.0-release.md | 157 ++++++++++++++++++ 1 file changed, 157 insertions(+) create mode 100644 site/docs/blog/posts/2026-06-01-iceberg-go-0.6.0-release.md diff --git a/site/docs/blog/posts/2026-06-01-iceberg-go-0.6.0-release.md b/site/docs/blog/posts/2026-06-01-iceberg-go-0.6.0-release.md new file mode 100644 index 000000000000..4d5312b05d18 --- /dev/null +++ b/site/docs/blog/posts/2026-06-01-iceberg-go-0.6.0-release.md @@ -0,0 +1,157 @@ +--- +date: 2026-06-01 +title: Apache Iceberg Go 0.6.0 Release +slug: apache-iceberg-go-0.6.0-release +authors: + - iceberg-pmc +categories: + - release +--- + + + +The Apache Iceberg community is pleased to announce version 0.6.0 of [iceberg-go](https://github.com/apache/iceberg-go). + +This release covers approximately three months of development since the 0.5.0 release in March 2026 and is the result of merging nearly **200 PRs** from **40 contributors**, including **26 first-time contributors**. See the [full changelog](https://github.com/apache/iceberg-go/compare/v0.5.0...v0.6.0) for the complete list of changes. + +`iceberg-go` is a native Go implementation of the Apache Iceberg table format, providing libraries for reading, writing, and managing Iceberg tables in Go applications. + + + +## Release Highlights + +### Iceberg V3 Table Spec Support + +This release continues to advance `iceberg-go`'s implementation of the Iceberg V3 table specification: + +- **Variant type**: [Support for the V3 `variant` type](https://github.com/apache/iceberg-go/pull/932) (non-shredding) was added +- **Deletion vectors**: A [deletion vector reader](https://github.com/apache/iceberg-go/pull/866) and [`SerializeDV`/`DVWriter` authoring APIs](https://github.com/apache/iceberg-go/pull/1100), with [blob header validation on read](https://github.com/apache/iceberg-go/pull/1055), [cardinality validation through `ReadDV`](https://github.com/apache/iceberg-go/pull/1056), and [required blob properties enforced on write](https://github.com/apache/iceberg-go/pull/1060) +- **Row lineage**: [Row lineage support in V3](https://github.com/apache/iceberg-go/pull/735), with [reserved row-lineage fields projected as null](https://github.com/apache/iceberg-go/pull/1045) when a file lacks them +- **Nanosecond timestamps**: [Parquet support for the nanosecond timestamp types](https://github.com/apache/iceberg-go/pull/819) and [mapping of nanosecond Arrow timestamps to the V3 `TimestampNs`/`TimestampTzNs` types](https://github.com/apache/iceberg-go/pull/1081) +- **Defaults**: [`write-default` and `initial-default` implemented for V3](https://github.com/apache/iceberg-go/pull/779) and [enforced for required fields](https://github.com/apache/iceberg-go/pull/820) +- **Format-version gating**: [V3-only metadata fields are gated on sub-V3 reads](https://github.com/apache/iceberg-go/pull/1069), [V2-only fields are gated on V1 reads](https://github.com/apache/iceberg-go/pull/1088), and the [remaining V3 metadata-upgrade validation rules](https://github.com/apache/iceberg-go/pull/1070) were ported +- **Format-version upgrade**: A [`Transaction.UpgradeFormatVersion`](https://github.com/apache/iceberg-go/pull/827) API was added + +### Row-Level Deletes + +`iceberg-go` gained a full equality-delete path and an atomic row-level mutation API: + +- **Equality deletes**: [Write path](https://github.com/apache/iceberg-go/pull/809), [read path in the scanner](https://github.com/apache/iceberg-go/pull/818), and [support for partitioned tables](https://github.com/apache/iceberg-go/pull/823) +- **RowDelta API**: An [atomic row-level mutation API](https://github.com/apache/iceberg-go/pull/789) was added +- **Overwrite with deletes**: [Delete file removal in overwrite commits](https://github.com/apache/iceberg-go/pull/851) and [deletion vectors classified in scan planning](https://github.com/apache/iceberg-go/pull/855) + +### Table Maintenance and Compaction + +A new compaction and maintenance suite landed in this release: + +- **Compaction execution**: [`RewriteDataFiles` for compaction](https://github.com/apache/iceberg-go/pull/892) with a [bin-pack strategy](https://github.com/apache/iceberg-go/pull/850) and [`Analyze` for dry-run planning](https://github.com/apache/iceberg-go/pull/893) +- **Dangling deletes**: [Dangling equality deletes are dropped during `RewriteDataFiles`](https://github.com/apache/iceberg-go/pull/947) +- **RewriteFiles op**: A [`RewriteFiles` snapshot-op builder](https://github.com/apache/iceberg-go/pull/1033) was added + +### Concurrency and Conflict Resolution + +Optimistic concurrency control was substantially hardened: + +- **Conflict validation framework**: A [conflict validation framework](https://github.com/apache/iceberg-go/pull/928) was added and [wired into producers](https://github.com/apache/iceberg-go/pull/934) +- **Commit retries**: [`doCommit` retries on commit-conflict errors](https://github.com/apache/iceberg-go/pull/912), with [refresh-and-replay between retries](https://github.com/apache/iceberg-go/pull/945) and [concurrent-write conflicts wrapped with `ErrCommitFailed`](https://github.com/apache/iceberg-go/pull/944) +- **RowDelta conflicts**: [`RowDelta` uses a partition-scoped conflict check](https://github.com/apache/iceberg-go/pull/983) instead of always-true + +### Hadoop Catalog + +A Hadoop catalog implementation was added over the course of the release: + +- **Scaffold and path helpers**: [Initial Hadoop catalog scaffold](https://github.com/apache/iceberg-go/pull/953) +- **Namespace and table operations**: [Namespace operations](https://github.com/apache/iceberg-go/pull/963), [list/drop/rename](https://github.com/apache/iceberg-go/pull/970), and [table and namespace CRUD](https://github.com/apache/iceberg-go/pull/969) +- **CLI integration**: [Hadoop catalog CLI integration](https://github.com/apache/iceberg-go/pull/1034) + +### Catalog Improvements + +- **Multi-table commits**: A [`TransactionalCatalog` interface and REST multi-table commit](https://github.com/apache/iceberg-go/pull/787), plus [`CommitAndReload`](https://github.com/apache/iceberg-go/pull/817) +- **OAuth and credentials**: [OAuth token refresh](https://github.com/apache/iceberg-go/pull/793), [vended credential refresh](https://github.com/apache/iceberg-go/pull/795), [audience and resource OAuth params](https://github.com/apache/iceberg-go/pull/815), and a [`WithOAuthTLSConfig` option](https://github.com/apache/iceberg-go/pull/895) for separate OAuth-server TLS +- **Hive catalog**: [Register table](https://github.com/apache/iceberg-go/pull/816) and [create view](https://github.com/apache/iceberg-go/pull/788) support +- **REST views**: [`RegisterView` was added to the REST catalog](https://github.com/apache/iceberg-go/pull/753) + +### Expanded CLI + +The command-line tool gained a large set of new commands: + +- **Maintenance commands**: [`expire-snapshots`](https://github.com/apache/iceberg-go/pull/1063), [`clean-orphan-files`](https://github.com/apache/iceberg-go/pull/1066), [`partition-stats`](https://github.com/apache/iceberg-go/pull/1065), and [`compact analyze`/`compact run`](https://github.com/apache/iceberg-go/pull/903) +- **Table operations**: [`branch create`/`tag create`](https://github.com/apache/iceberg-go/pull/1068), [`upgrade`/`rollback`](https://github.com/apache/iceberg-go/pull/1071), [`snapshots`/`refs`](https://github.com/apache/iceberg-go/pull/1072), and an [`info` summary command](https://github.com/apache/iceberg-go/pull/1064) +- **Schema flags**: [`--schema-from-file`](https://github.com/apache/iceberg-go/pull/927) to infer a schema from Parquet files and a [`--show-defaults` flag](https://github.com/apache/iceberg-go/pull/1067) +- **Argument parsing**: The CLI [migrated from docopt-go to go-arg](https://github.com/apache/iceberg-go/pull/980) + +### Write Performance and Bloom Filters + +- **File sizing**: [Parquet files are rolled based on actual compressed size](https://github.com/apache/iceberg-go/pull/759) +- **Partitioned writes**: A [clustered partitioned write path](https://github.com/apache/iceberg-go/pull/948) was added +- **Memory pressure**: [Reduced memory pressure by releasing records](https://github.com/apache/iceberg-go/pull/886) and an [optional skip of the duplicate check on `AddDataFiles`](https://github.com/apache/iceberg-go/pull/901) +- **Bloom filters**: [Bloom filter scan pruning for Parquet row groups](https://github.com/apache/iceberg-go/pull/891) and [bloom filter properties wired into the Parquet writer](https://github.com/apache/iceberg-go/pull/878) +- **Metadata compression**: [Support for the zstd metadata compression codec](https://github.com/apache/iceberg-go/pull/1020) + +### IO Improvements + +- **In-memory FS**: A [native in-memory filesystem](https://github.com/apache/iceberg-go/pull/1025) was added +- **Aliyun OSS**: The [`oss` scheme is registered for Aliyun OSS](https://github.com/apache/iceberg-go/pull/1074) S3-compatible storage +- **New interfaces**: A [`ListableIO` interface](https://github.com/apache/iceberg-go/pull/917) replaces reflect-based directory walking, and a [`BulkRemovableIO` interface](https://github.com/apache/iceberg-go/pull/916) was added +- **S3 Table Buckets**: [`PermanentRedirect` is resolved for S3 Table Buckets](https://github.com/apache/iceberg-go/pull/975) + +### Statistics + +- **Statistics updates**: [`SetStatistics`, `RemoveStatistics`, `AddEncryptionKey`, and `RemoveEncryptionKey` updates](https://github.com/apache/iceberg-go/pull/902) were implemented +- **Partition statistics**: [`SetPartitionStatisticsUpdate` and `RemovePartitionStatisticsUpdate`](https://github.com/apache/iceberg-go/pull/1018) were added + +### Bug Fixes + +Notable bug fixes in this release include: + +- [Fix a nil field `Type` panic in `MarshalJSON`](https://github.com/apache/iceberg-go/pull/765) +- [Use floored division in `HourTransform` for pre-epoch timestamps](https://github.com/apache/iceberg-go/pull/926) +- [Handle `LargeString` columns in partition transforms](https://github.com/apache/iceberg-go/pull/780) +- [Fast-append must inherit all parent manifests unconditionally](https://github.com/apache/iceberg-go/pull/869) +- [Fix a goroutine leak in `positionDeleteRecordsToDataFiles`](https://github.com/apache/iceberg-go/pull/825) +- [Fix a deadlock in `MapExec` when workers error](https://github.com/apache/iceberg-go/pull/810) +- [Fix a refcount leak in `enrichRecordsWithPosDeleteFields`](https://github.com/apache/iceberg-go/pull/762) +- [Reject duplicate field IDs at schema construction](https://github.com/apache/iceberg-go/pull/879) +- [Change equality_ids Avro element type from Long to Integer to match the spec](https://github.com/apache/iceberg-go/pull/880) +- [Decode pre-1.4 Java Iceberg legacy manifest list field names](https://github.com/apache/iceberg-go/pull/890) +- [Resolve a data race in `MarshalJSON` on a shared `*Schema`](https://github.com/apache/iceberg-go/pull/967) +- [Emit a proper Avro fixed schema for `FixedType` partition columns](https://github.com/apache/iceberg-go/pull/881) + +### Breaking Changes + +Some of these changes are breaking changes that need to be called out: + +- **Manifest entries iterator** ([#985](https://github.com/apache/iceberg-go/pull/985)): `FetchEntries` was replaced with an iterator-based approach. +- **Multi-arg transforms** ([#824](https://github.com/apache/iceberg-go/pull/824)): partition and sort fields now support multi-argument transforms, changing the relevant field signatures. + +## New Contributors + +Welcome to all 26 first-time contributors: +@starpact, @PranjalChaitanya, @rockwotj, @RSP22, @Hashcode-Ankit, @hcrosse, @Herrtian, @gabrnavarro, @Narwhal-fish, @alliasgher, @abhirathod95, @rohilsurana, @twmb, @cassio-paesleme, @Jeffail, @benbellick, @varun0630, @hectar-glitches, @swjtu-zhanglei, @jacobmarble, @C-Loftus, @nssalian, @tanmayrauth, @fallintoplace, @happydave1, @mzzz-zzm + +## Getting Involved + +The `iceberg-go` project welcomes contributions. We use GitHub [issues](https://github.com/apache/iceberg-go/issues) for tracking work and the [Apache Iceberg Community Slack](https://iceberg.apache.org/community/#slack) for discussions. + +The easiest way to get started is to: + +1. Try `iceberg-go` with your workloads and report any issues you encounter +2. Review the [contributor guide](https://github.com/apache/iceberg-go/blob/main/CONTRIBUTING.md) +3. Look for [good first issues](https://github.com/apache/iceberg-go/contribute) + +For more information, visit the [iceberg-go repository](https://github.com/apache/iceberg-go). From c6464040081ef6c578a59a08f9d668466486dd56 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 3 Jun 2026 11:29:41 -0700 Subject: [PATCH 56/58] Core: Refactor v4 struct builders to improve validation (#16408) --- .../apache/iceberg/DeletionVectorStruct.java | 24 +- .../apache/iceberg/ManifestInfoStruct.java | 107 ++-- .../org/apache/iceberg/TrackingBuilder.java | 179 +++++++ .../org/apache/iceberg/TrackingStruct.java | 97 +--- .../iceberg/TestDeletionVectorStruct.java | 47 +- .../iceberg/TestManifestInfoStruct.java | 257 ++++++++-- .../apache/iceberg/TestTrackedFileStruct.java | 20 +- .../apache/iceberg/TestTrackingStruct.java | 464 +++++++++++++++--- 8 files changed, 934 insertions(+), 261 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/TrackingBuilder.java diff --git a/core/src/main/java/org/apache/iceberg/DeletionVectorStruct.java b/core/src/main/java/org/apache/iceberg/DeletionVectorStruct.java index 0eb7c2fe1eb6..04d23fa33abe 100644 --- a/core/src/main/java/org/apache/iceberg/DeletionVectorStruct.java +++ b/core/src/main/java/org/apache/iceberg/DeletionVectorStruct.java @@ -51,7 +51,7 @@ private DeletionVectorStruct(DeletionVectorStruct toCopy) { } private DeletionVectorStruct(String location, long offset, long sizeInBytes, long cardinality) { - super(BASE_TYPE, BASE_TYPE); + super(BASE_TYPE.fields().size()); this.location = location; this.offset = offset; this.sizeInBytes = sizeInBytes; @@ -140,37 +140,41 @@ public String toString() { static class Builder { private String location = null; - private long offset = -1L; - private long sizeInBytes = -1L; - private long cardinality = -1L; + private Long offset = null; + private Long sizeInBytes = null; + private Long cardinality = null; Builder location(String dvLocation) { + Preconditions.checkArgument(dvLocation != null, "Invalid location: null"); this.location = dvLocation; return this; } Builder offset(long dvOffset) { + Preconditions.checkArgument(dvOffset >= 0, "Invalid offset: %s (must be >= 0)", dvOffset); this.offset = dvOffset; return this; } Builder sizeInBytes(long dvSizeInBytes) { + Preconditions.checkArgument( + dvSizeInBytes >= 0, "Invalid size in bytes: %s (must be >= 0)", dvSizeInBytes); this.sizeInBytes = dvSizeInBytes; return this; } Builder cardinality(long dvCardinality) { + Preconditions.checkArgument( + dvCardinality >= 0, "Invalid cardinality: %s (must be >= 0)", dvCardinality); this.cardinality = dvCardinality; return this; } DeletionVectorStruct build() { - Preconditions.checkArgument(location != null, "Invalid location: null"); - Preconditions.checkArgument(offset >= 0, "Invalid offset: %s (must be >= 0)", offset); - Preconditions.checkArgument( - sizeInBytes >= 0, "Invalid size in bytes: %s (must be >= 0)", sizeInBytes); - Preconditions.checkArgument( - cardinality >= 0, "Invalid cardinality: %s (must be >= 0)", cardinality); + Preconditions.checkArgument(location != null, "Missing required value: location"); + Preconditions.checkArgument(offset != null, "Missing required value: offset"); + Preconditions.checkArgument(sizeInBytes != null, "Missing required value: size in bytes"); + Preconditions.checkArgument(cardinality != null, "Missing required value: cardinality"); return new DeletionVectorStruct(location, offset, sizeInBytes, cardinality); } } diff --git a/core/src/main/java/org/apache/iceberg/ManifestInfoStruct.java b/core/src/main/java/org/apache/iceberg/ManifestInfoStruct.java index 922047bffedd..6a7ccea6b679 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestInfoStruct.java +++ b/core/src/main/java/org/apache/iceberg/ManifestInfoStruct.java @@ -74,7 +74,7 @@ private ManifestInfoStruct(ManifestInfoStruct toCopy) { this.dvCardinality = toCopy.dvCardinality; } - private ManifestInfoStruct( + ManifestInfoStruct( int addedFilesCount, int existingFilesCount, int deletedFilesCount, @@ -86,7 +86,7 @@ private ManifestInfoStruct( long minSequenceNumber, byte[] dv, Long dvCardinality) { - super(BASE_TYPE, BASE_TYPE); + super(BASE_TYPE.fields().size()); this.addedFilesCount = addedFilesCount; this.existingFilesCount = existingFilesCount; this.deletedFilesCount = deletedFilesCount; @@ -252,121 +252,140 @@ public String toString() { .add("replaced_rows_count", replacedRowsCount) .add("min_sequence_number", minSequenceNumber) .add("dv", dv == null ? "null" : "(binary)") - .add("dv_cardinality", dvCardinality == null ? "null" : dvCardinality) + .add("dv_cardinality", dvCardinality) .toString(); } static class Builder { - private int addedFilesCount = -1; - private int existingFilesCount = -1; - private int deletedFilesCount = -1; - private int replacedFilesCount = -1; - private long addedRowsCount = -1L; - private long existingRowsCount = -1L; - private long deletedRowsCount = -1L; - private long replacedRowsCount = -1L; - private long minSequenceNumber = -1L; + private Integer addedFilesCount = null; + private Integer existingFilesCount = null; + private Integer deletedFilesCount = null; + private Integer replacedFilesCount = null; + private Long addedRowsCount = null; + private Long existingRowsCount = null; + private Long deletedRowsCount = null; + private Long replacedRowsCount = null; + private Long minSequenceNumber = null; private byte[] dv = null; private Long dvCardinality = null; Builder addedFilesCount(int count) { + Preconditions.checkArgument( + count >= 0, "Invalid added files count: %s (must be >= 0)", count); this.addedFilesCount = count; return this; } Builder existingFilesCount(int count) { + Preconditions.checkArgument( + count >= 0, "Invalid existing files count: %s (must be >= 0)", count); this.existingFilesCount = count; return this; } Builder deletedFilesCount(int count) { + Preconditions.checkArgument( + count >= 0, "Invalid deleted files count: %s (must be >= 0)", count); this.deletedFilesCount = count; return this; } Builder replacedFilesCount(int count) { + Preconditions.checkArgument( + count >= 0, "Invalid replaced files count: %s (must be >= 0)", count); this.replacedFilesCount = count; return this; } Builder addedRowsCount(long count) { + Preconditions.checkArgument(count >= 0, "Invalid added rows count: %s (must be >= 0)", count); this.addedRowsCount = count; return this; } Builder existingRowsCount(long count) { + Preconditions.checkArgument( + count >= 0, "Invalid existing rows count: %s (must be >= 0)", count); this.existingRowsCount = count; return this; } Builder deletedRowsCount(long count) { + Preconditions.checkArgument( + count >= 0, "Invalid deleted rows count: %s (must be >= 0)", count); this.deletedRowsCount = count; return this; } Builder replacedRowsCount(long count) { + Preconditions.checkArgument( + count >= 0, "Invalid replaced rows count: %s (must be >= 0)", count); this.replacedRowsCount = count; return this; } Builder minSequenceNumber(long sequenceNumber) { + Preconditions.checkArgument( + sequenceNumber >= 0, "Invalid min sequence number: %s (must be >= 0)", sequenceNumber); this.minSequenceNumber = sequenceNumber; return this; } Builder dv(ByteBuffer buffer) { - this.dv = buffer != null ? ByteBuffers.toByteArray(buffer) : null; - return this; - } - - Builder dv(byte[] buffer) { - this.dv = buffer; + Preconditions.checkArgument(buffer != null, "Invalid DV: null"); + this.dv = ByteBuffers.toByteArray(buffer); return this; } - Builder dvCardinality(Long cardinality) { + Builder dvCardinality(long cardinality) { + Preconditions.checkArgument( + cardinality >= 0, "Invalid DV cardinality: %s (must be >= 0)", cardinality); this.dvCardinality = cardinality; return this; } ManifestInfoStruct build() { Preconditions.checkArgument( - addedFilesCount >= 0, "Invalid added files count: %s (must be >= 0)", addedFilesCount); + addedFilesCount != null, "Missing required value: added files count"); Preconditions.checkArgument( - existingFilesCount >= 0, - "Invalid existing files count: %s (must be >= 0)", - existingFilesCount); + existingFilesCount != null, "Missing required value: existing files count"); Preconditions.checkArgument( - deletedFilesCount >= 0, - "Invalid deleted files count: %s (must be >= 0)", - deletedFilesCount); + deletedFilesCount != null, "Missing required value: deleted files count"); Preconditions.checkArgument( - replacedFilesCount >= 0, - "Invalid replaced files count: %s (must be >= 0)", - replacedFilesCount); + replacedFilesCount != null, "Missing required value: replaced files count"); + Preconditions.checkArgument( + addedRowsCount != null, "Missing required value: added rows count"); + Preconditions.checkArgument( + existingRowsCount != null, "Missing required value: existing rows count"); + Preconditions.checkArgument( + deletedRowsCount != null, "Missing required value: deleted rows count"); Preconditions.checkArgument( - addedRowsCount >= 0, "Invalid added rows count: %s (must be >= 0)", addedRowsCount); + replacedRowsCount != null, "Missing required value: replaced rows count"); Preconditions.checkArgument( - existingRowsCount >= 0, - "Invalid existing rows count: %s (must be >= 0)", - existingRowsCount); + minSequenceNumber != null, "Missing required value: min sequence number"); Preconditions.checkArgument( - deletedRowsCount >= 0, "Invalid deleted rows count: %s (must be >= 0)", deletedRowsCount); + addedRowsCount == 0 || addedFilesCount > 0, + "Invalid added counts: %s rows in %s files", + addedRowsCount, + addedFilesCount); + Preconditions.checkArgument( + existingRowsCount == 0 || existingFilesCount > 0, + "Invalid existing counts: %s rows in %s files", + existingRowsCount, + existingFilesCount); Preconditions.checkArgument( - replacedRowsCount >= 0, - "Invalid replaced rows count: %s (must be >= 0)", - replacedRowsCount); + deletedRowsCount == 0 || deletedFilesCount > 0, + "Invalid deleted counts: %s rows in %s files", + deletedRowsCount, + deletedFilesCount); Preconditions.checkArgument( - minSequenceNumber >= 0, - "Invalid min sequence number: %s (must be >= 0)", - minSequenceNumber); + replacedRowsCount == 0 || replacedFilesCount > 0, + "Invalid replaced counts: %s rows in %s files", + replacedRowsCount, + replacedFilesCount); Preconditions.checkArgument( (dv == null) == (dvCardinality == null), "Invalid DV and cardinality: must both be null or non-null"); - Preconditions.checkArgument( - dvCardinality == null || dvCardinality > 0, - "Invalid DV cardinality: %s (must be positive)", - dvCardinality); return new ManifestInfoStruct( addedFilesCount, existingFilesCount, diff --git a/core/src/main/java/org/apache/iceberg/TrackingBuilder.java b/core/src/main/java/org/apache/iceberg/TrackingBuilder.java new file mode 100644 index 000000000000..bf3cb3a0666c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackingBuilder.java @@ -0,0 +1,179 @@ +/* + * 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; + +import java.nio.ByteBuffer; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.ByteBuffers; + +class TrackingBuilder { + private final EntryStatus status; + private final Long snapshotId; + private final Long dataSequenceNumber; + private final Long fileSequenceNumber; + private final Long firstRowId; + // ID of the snapshot in which the new Tracking instance will be committed. + private final long newSnapshotId; + private Long dvSnapshotId; + private byte[] deletedPositions; + private byte[] replacedPositions; + + /** + * Creates a builder for a newly added file. + * + * @param newSnapshotId the snapshot ID in which the new tracking instance will be committed + */ + static TrackingBuilder added(long newSnapshotId) { + return new TrackingBuilder(newSnapshotId); + } + + /** + * Creates a builder for a tracking row derived from {@code source}. + * + * @param source source tracking from a manifest entry + * @param newSnapshotId the snapshot ID in which the new tracking instance will be committed + */ + static TrackingBuilder from(Tracking source, long newSnapshotId) { + return new TrackingBuilder(source, newSnapshotId); + } + + /** + * Returns a DELETED tracking row derived from {@code source}. + * + * @param source source tracking from a manifest entry + * @param newSnapshotId the snapshot ID in which the new tracking instance will be committed + */ + static Tracking deleted(Tracking source, long newSnapshotId) { + return terminal(EntryStatus.DELETED, source, newSnapshotId); + } + + /** + * Returns a REPLACED tracking row derived from {@code source}. + * + * @param source source tracking from a manifest entry + * @param newSnapshotId the snapshot ID in which the new tracking instance will be committed + */ + static Tracking replaced(Tracking source, long newSnapshotId) { + return terminal(EntryStatus.REPLACED, source, newSnapshotId); + } + + private TrackingBuilder(long newSnapshotId) { + this.status = EntryStatus.ADDED; + this.snapshotId = newSnapshotId; + this.newSnapshotId = newSnapshotId; + this.dataSequenceNumber = null; + this.fileSequenceNumber = null; + this.firstRowId = null; + this.dvSnapshotId = null; + this.deletedPositions = null; + this.replacedPositions = null; + } + + private TrackingBuilder(Tracking source, long newSnapshotId) { + validateSource(source); + validateStatusTransition(source.status(), EntryStatus.EXISTING); + this.status = EntryStatus.EXISTING; + this.snapshotId = source.snapshotId(); + this.newSnapshotId = newSnapshotId; + this.dataSequenceNumber = source.dataSequenceNumber(); + this.fileSequenceNumber = source.fileSequenceNumber(); + this.firstRowId = source.firstRowId(); + this.dvSnapshotId = source.dvSnapshotId(); + this.deletedPositions = null; + this.replacedPositions = null; + } + + /** Indicates that the DV has been updated for the new Tracking. */ + TrackingBuilder dvUpdated() { + // DV applies to data files; deleted/replaced positions apply to manifest files + Preconditions.checkState( + deletedPositions == null && replacedPositions == null, + "Cannot mark DV updated on a manifest entry (deleted/replaced positions are set)"); + this.dvSnapshotId = newSnapshotId; + return this; + } + + TrackingBuilder deletedPositions(ByteBuffer positions) { + Preconditions.checkState( + status == EntryStatus.EXISTING, "Cannot set deleted positions on %s entry", status); + // DV applies to data files; deleted positions apply to manifest files + Preconditions.checkState( + dvSnapshotId == null, + "Cannot set deleted positions on a data file entry (DV snapshot ID is set)"); + this.deletedPositions = ByteBuffers.toByteArray(positions); + return this; + } + + TrackingBuilder replacedPositions(ByteBuffer positions) { + Preconditions.checkState( + status == EntryStatus.EXISTING, "Cannot set replaced positions on %s entry", status); + // DV applies to data files; replaced positions apply to manifest files + Preconditions.checkState( + dvSnapshotId == null, + "Cannot set replaced positions on a data file entry (DV snapshot ID is set)"); + this.replacedPositions = ByteBuffers.toByteArray(positions); + return this; + } + + Tracking build() { + return new TrackingStruct( + status, + snapshotId, + dataSequenceNumber, + fileSequenceNumber, + dvSnapshotId, + firstRowId, + deletedPositions, + replacedPositions); + } + + private static Tracking terminal(EntryStatus to, Tracking source, long newSnapshotId) { + validateSource(source); + validateStatusTransition(source.status(), to); + return new TrackingStruct( + to, + newSnapshotId, + source.dataSequenceNumber(), + source.fileSequenceNumber(), + source.dvSnapshotId(), + source.firstRowId(), + null, + null); + } + + private static void validateSource(Tracking source) { + Preconditions.checkArgument(source != null, "Invalid source tracking: null"); + Preconditions.checkArgument( + source.dataSequenceNumber() != null, + "Invalid tracking source: data sequence number is null"); + Preconditions.checkArgument( + source.fileSequenceNumber() != null, + "Invalid tracking source: file sequence number is null"); + } + + private static void validateStatusTransition(EntryStatus from, EntryStatus to) { + Preconditions.checkState(from != null, "Invalid tracking source: status is null"); + Preconditions.checkState( + from != EntryStatus.DELETED && from != EntryStatus.REPLACED, + "Cannot revive non-live entry with status %s", + from); + Preconditions.checkState( + to != EntryStatus.ADDED, "Cannot transition to ADDED: ADDED is the starting status"); + } +} diff --git a/core/src/main/java/org/apache/iceberg/TrackingStruct.java b/core/src/main/java/org/apache/iceberg/TrackingStruct.java index 65513c8d4a7c..8ae4b7e4ce88 100644 --- a/core/src/main/java/org/apache/iceberg/TrackingStruct.java +++ b/core/src/main/java/org/apache/iceberg/TrackingStruct.java @@ -59,6 +59,11 @@ class TrackingStruct extends SupportsIndexProjection implements Tracking, Serial super(BASE_TYPE, type); } + /** Constructor for Java serialization. */ + TrackingStruct() { + super(BASE_TYPE.fields().size()); + } + private TrackingStruct(TrackingStruct toCopy) { super(toCopy); this.status = toCopy.status; @@ -79,7 +84,7 @@ private TrackingStruct(TrackingStruct toCopy) { this.manifestPos = toCopy.manifestPos; } - private TrackingStruct( + TrackingStruct( EntryStatus status, Long snapshotId, Long dataSequenceNumber, @@ -88,7 +93,7 @@ private TrackingStruct( Long firstRowId, byte[] deletedPositions, byte[] replacedPositions) { - super(BASE_TYPE, BASE_TYPE); + super(BASE_TYPE.fields().size()); this.status = status; this.snapshotId = snapshotId; this.dataSequenceNumber = dataSequenceNumber; @@ -249,95 +254,17 @@ protected void internalSet(int pos, T value) { } } - static Builder builder() { - return new Builder(); - } - @Override public String toString() { return MoreObjects.toStringHelper(this) .add("status", status) - .add("snapshot_id", snapshotId == null ? "null" : snapshotId) - .add("data_sequence_number", dataSequenceNumber == null ? "null" : dataSequenceNumber) - .add("file_sequence_number", fileSequenceNumber == null ? "null" : fileSequenceNumber) - .add("dv_snapshot_id", dvSnapshotId == null ? "null" : dvSnapshotId) - .add("first_row_id", firstRowId == null ? "null" : firstRowId) + .add("snapshot_id", snapshotId) + .add("data_sequence_number", dataSequenceNumber) + .add("file_sequence_number", fileSequenceNumber) + .add("dv_snapshot_id", dvSnapshotId) + .add("first_row_id", firstRowId) .add("deleted_positions", deletedPositions == null ? "null" : "(binary)") .add("replaced_positions", replacedPositions == null ? "null" : "(binary)") .toString(); } - - static class Builder { - private EntryStatus status = null; - private Long snapshotId = null; - private Long dataSequenceNumber = null; - private Long fileSequenceNumber = null; - private Long dvSnapshotId = null; - private Long firstRowId = null; - private byte[] deletedPositions = null; - private byte[] replacedPositions = null; - - Builder status(EntryStatus entryStatus) { - this.status = entryStatus; - return this; - } - - Builder snapshotId(Long id) { - this.snapshotId = id; - return this; - } - - Builder dataSequenceNumber(Long sequenceNumber) { - this.dataSequenceNumber = sequenceNumber; - return this; - } - - Builder fileSequenceNumber(Long sequenceNumber) { - this.fileSequenceNumber = sequenceNumber; - return this; - } - - Builder dvSnapshotId(Long id) { - this.dvSnapshotId = id; - return this; - } - - Builder firstRowId(Long rowId) { - this.firstRowId = rowId; - return this; - } - - Builder deletedPositions(ByteBuffer positions) { - this.deletedPositions = positions != null ? ByteBuffers.toByteArray(positions) : null; - return this; - } - - Builder deletedPositions(byte[] positions) { - this.deletedPositions = positions; - return this; - } - - Builder replacedPositions(ByteBuffer positions) { - this.replacedPositions = positions != null ? ByteBuffers.toByteArray(positions) : null; - return this; - } - - Builder replacedPositions(byte[] positions) { - this.replacedPositions = positions; - return this; - } - - TrackingStruct build() { - Preconditions.checkArgument(status != null, "Invalid status: null"); - return new TrackingStruct( - status, - snapshotId, - dataSequenceNumber, - fileSequenceNumber, - dvSnapshotId, - firstRowId, - deletedPositions, - replacedPositions); - } - } } diff --git a/core/src/test/java/org/apache/iceberg/TestDeletionVectorStruct.java b/core/src/test/java/org/apache/iceberg/TestDeletionVectorStruct.java index 325f9afd9ca9..8242be38e94a 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeletionVectorStruct.java +++ b/core/src/test/java/org/apache/iceberg/TestDeletionVectorStruct.java @@ -87,6 +87,26 @@ void testProjectedStructLike() { assertThat(dv.get(1, Long.class)).isEqualTo(42L); } + @Test + void testInternalSetIgnoresUnknownOrdinal() { + DeletionVectorStruct dv = + DeletionVectorStruct.builder() + .location("s3://bucket/data/dv.puffin") + .offset(100L) + .sizeInBytes(512L) + .cardinality(42L) + .build(); + + // unknown ordinals from a newer format version are silently ignored + dv.internalSet(99, "value from a newer format"); + + // every field is unchanged + assertThat(dv.location()).isEqualTo("s3://bucket/data/dv.puffin"); + assertThat(dv.offset()).isEqualTo(100L); + assertThat(dv.sizeInBytes()).isEqualTo(512L); + assertThat(dv.cardinality()).isEqualTo(42L); + } + @Test void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException { DeletionVectorStruct dv = @@ -106,11 +126,11 @@ void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException } @Test - void testBuilderValidation() { + void testBuilderMissingRequiredFields() { assertThatThrownBy( () -> DeletionVectorStruct.builder().offset(0).sizeInBytes(1).cardinality(1).build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid location: null"); + .hasMessage("Missing required value: location"); assertThatThrownBy( () -> @@ -120,7 +140,7 @@ void testBuilderValidation() { .cardinality(1) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid offset: -1 (must be >= 0)"); + .hasMessage("Missing required value: offset"); assertThatThrownBy( () -> @@ -130,7 +150,7 @@ void testBuilderValidation() { .cardinality(1) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid size in bytes: -1 (must be >= 0)"); + .hasMessage("Missing required value: size in bytes"); assertThatThrownBy( () -> @@ -140,6 +160,25 @@ void testBuilderValidation() { .sizeInBytes(1) .build()) .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required value: cardinality"); + } + + @Test + void testBuilderRejectsInvalidValuesAtSetter() { + assertThatThrownBy(() -> DeletionVectorStruct.builder().location(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid location: null"); + + assertThatThrownBy(() -> DeletionVectorStruct.builder().offset(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid offset: -1 (must be >= 0)"); + + assertThatThrownBy(() -> DeletionVectorStruct.builder().sizeInBytes(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid size in bytes: -1 (must be >= 0)"); + + assertThatThrownBy(() -> DeletionVectorStruct.builder().cardinality(-1)) + .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid cardinality: -1 (must be >= 0)"); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestInfoStruct.java b/core/src/test/java/org/apache/iceberg/TestManifestInfoStruct.java index 3a694f1a38f2..9d78dca65f5d 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestInfoStruct.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestInfoStruct.java @@ -30,19 +30,8 @@ class TestManifestInfoStruct { @Test void testFieldAccess() { - ManifestInfoStruct info = new ManifestInfoStruct(ManifestInfo.schema()); - - info.set(0, 10); - info.set(1, 20); - info.set(2, 3); - info.set(3, 2); - info.set(4, 1000L); - info.set(5, 2000L); - info.set(6, 300L); - info.set(7, 200L); - info.set(8, 5L); - info.set(9, ByteBuffer.wrap(new byte[] {0xF})); - info.set(10, 1L); + ManifestInfoStruct info = + new ManifestInfoStruct(10, 20, 3, 2, 1000L, 2000L, 300L, 200L, 5L, new byte[] {0xF}, 1L); assertThat(info.addedFilesCount()).isEqualTo(10); assertThat(info.existingFilesCount()).isEqualTo(20); @@ -70,7 +59,7 @@ void testCopy() { .deletedRowsCount(300L) .replacedRowsCount(200L) .minSequenceNumber(5L) - .dv(new byte[] {0xF}) + .dv(ByteBuffer.wrap(new byte[] {0xF})) .dvCardinality(1L) .build(); @@ -130,6 +119,40 @@ void testProjectedStructLike() { assertThat(info.get(1, Long.class)).isEqualTo(5L); } + @Test + void testInternalSetIgnoresUnknownOrdinal() { + ManifestInfoStruct info = + ManifestInfoStruct.builder() + .addedFilesCount(10) + .existingFilesCount(20) + .deletedFilesCount(3) + .replacedFilesCount(2) + .addedRowsCount(1000L) + .existingRowsCount(2000L) + .deletedRowsCount(300L) + .replacedRowsCount(200L) + .minSequenceNumber(5L) + .dv(ByteBuffer.wrap(new byte[] {0xF})) + .dvCardinality(1L) + .build(); + + // unknown ordinals from a newer format version are silently ignored + info.internalSet(99, "value from a newer format"); + + // every field is unchanged + assertThat(info.addedFilesCount()).isEqualTo(10); + assertThat(info.existingFilesCount()).isEqualTo(20); + assertThat(info.deletedFilesCount()).isEqualTo(3); + assertThat(info.replacedFilesCount()).isEqualTo(2); + assertThat(info.addedRowsCount()).isEqualTo(1000L); + assertThat(info.existingRowsCount()).isEqualTo(2000L); + assertThat(info.deletedRowsCount()).isEqualTo(300L); + assertThat(info.replacedRowsCount()).isEqualTo(200L); + assertThat(info.minSequenceNumber()).isEqualTo(5L); + assertThat(info.dv()).isEqualTo(ByteBuffer.wrap(new byte[] {0xF})); + assertThat(info.dvCardinality()).isEqualTo(1L); + } + @Test void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException { ManifestInfoStruct info = @@ -143,7 +166,7 @@ void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException .deletedRowsCount(300L) .replacedRowsCount(200L) .minSequenceNumber(5L) - .dv(new byte[] {0xF}) + .dv(ByteBuffer.wrap(new byte[] {0xF})) .dvCardinality(1L) .build(); @@ -163,7 +186,7 @@ void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException } @Test - void testBuilderValidation() { + void testBuilderMissingAddedFilesCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -177,8 +200,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid added files count: -1 (must be >= 0)"); + .hasMessage("Missing required value: added files count"); + } + @Test + void testBuilderMissingExistingFilesCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -192,8 +218,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid existing files count: -1 (must be >= 0)"); + .hasMessage("Missing required value: existing files count"); + } + @Test + void testBuilderMissingDeletedFilesCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -207,8 +236,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid deleted files count: -1 (must be >= 0)"); + .hasMessage("Missing required value: deleted files count"); + } + @Test + void testBuilderMissingReplacedFilesCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -222,8 +254,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid replaced files count: -1 (must be >= 0)"); + .hasMessage("Missing required value: replaced files count"); + } + @Test + void testBuilderMissingAddedRowsCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -237,8 +272,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid added rows count: -1 (must be >= 0)"); + .hasMessage("Missing required value: added rows count"); + } + @Test + void testBuilderMissingExistingRowsCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -252,8 +290,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid existing rows count: -1 (must be >= 0)"); + .hasMessage("Missing required value: existing rows count"); + } + @Test + void testBuilderMissingDeletedRowsCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -267,8 +308,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid deleted rows count: -1 (must be >= 0)"); + .hasMessage("Missing required value: deleted rows count"); + } + @Test + void testBuilderMissingReplacedRowsCount() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -282,8 +326,11 @@ void testBuilderValidation() { .minSequenceNumber(0L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid replaced rows count: -1 (must be >= 0)"); + .hasMessage("Missing required value: replaced rows count"); + } + @Test + void testBuilderMissingMinSequenceNumber() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -297,11 +344,81 @@ void testBuilderValidation() { .replacedRowsCount(0L) .build()) .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required value: min sequence number"); + } + + @Test + void testBuilderRejectsNegativeAddedFilesCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().addedFilesCount(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid added files count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeExistingFilesCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().existingFilesCount(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid existing files count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeDeletedFilesCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().deletedFilesCount(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid deleted files count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeReplacedFilesCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().replacedFilesCount(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid replaced files count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeAddedRowsCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().addedRowsCount(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid added rows count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeExistingRowsCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().existingRowsCount(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid existing rows count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeDeletedRowsCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().deletedRowsCount(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid deleted rows count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeReplacedRowsCount() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().replacedRowsCount(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid replaced rows count: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsNegativeMinSequenceNumber() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().minSequenceNumber(-1L)) + .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid min sequence number: -1 (must be >= 0)"); } @Test - void testBuilderDvPairingValidation() { + void testBuilderRejectsNegativeDvCardinality() { + assertThatThrownBy(() -> ManifestInfoStruct.builder().dvCardinality(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid DV cardinality: -1 (must be >= 0)"); + } + + @Test + void testBuilderRejectsRowsWithoutFiles() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -309,16 +426,91 @@ void testBuilderDvPairingValidation() { .existingFilesCount(0) .deletedFilesCount(0) .replacedFilesCount(0) - .addedRowsCount(0L) + .addedRowsCount(10L) .existingRowsCount(0L) .deletedRowsCount(0L) .replacedRowsCount(0L) .minSequenceNumber(0L) - .dv(new byte[] {0xF}) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid DV and cardinality: must both be null or non-null"); + .hasMessage("Invalid added counts: 10 rows in 0 files"); + + assertThatThrownBy( + () -> + ManifestInfoStruct.builder() + .addedFilesCount(0) + .existingFilesCount(0) + .deletedFilesCount(0) + .replacedFilesCount(0) + .addedRowsCount(0L) + .existingRowsCount(5L) + .deletedRowsCount(0L) + .replacedRowsCount(0L) + .minSequenceNumber(0L) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid existing counts: 5 rows in 0 files"); + + assertThatThrownBy( + () -> + ManifestInfoStruct.builder() + .addedFilesCount(0) + .existingFilesCount(0) + .deletedFilesCount(0) + .replacedFilesCount(0) + .addedRowsCount(0L) + .existingRowsCount(0L) + .deletedRowsCount(3L) + .replacedRowsCount(0L) + .minSequenceNumber(0L) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid deleted counts: 3 rows in 0 files"); + assertThatThrownBy( + () -> + ManifestInfoStruct.builder() + .addedFilesCount(0) + .existingFilesCount(0) + .deletedFilesCount(0) + .replacedFilesCount(0) + .addedRowsCount(0L) + .existingRowsCount(0L) + .deletedRowsCount(0L) + .replacedRowsCount(7L) + .minSequenceNumber(0L) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid replaced counts: 7 rows in 0 files"); + } + + @Test + void testBuilderAllowsFilesWithoutRows() { + ManifestInfoStruct info = + ManifestInfoStruct.builder() + .addedFilesCount(5) + .existingFilesCount(5) + .deletedFilesCount(5) + .replacedFilesCount(5) + .addedRowsCount(0L) + .existingRowsCount(0L) + .deletedRowsCount(0L) + .replacedRowsCount(0L) + .minSequenceNumber(0L) + .build(); + + assertThat(info.addedFilesCount()).isEqualTo(5); + assertThat(info.existingFilesCount()).isEqualTo(5); + assertThat(info.deletedFilesCount()).isEqualTo(5); + assertThat(info.replacedFilesCount()).isEqualTo(5); + assertThat(info.addedRowsCount()).isEqualTo(0L); + assertThat(info.existingRowsCount()).isEqualTo(0L); + assertThat(info.deletedRowsCount()).isEqualTo(0L); + assertThat(info.replacedRowsCount()).isEqualTo(0L); + } + + @Test + void testBuilderDvPairingValidation() { assertThatThrownBy( () -> ManifestInfoStruct.builder() @@ -331,7 +523,7 @@ void testBuilderDvPairingValidation() { .deletedRowsCount(0L) .replacedRowsCount(0L) .minSequenceNumber(0L) - .dvCardinality(1L) + .dv(ByteBuffer.wrap(new byte[] {0xF})) .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid DV and cardinality: must both be null or non-null"); @@ -348,11 +540,10 @@ void testBuilderDvPairingValidation() { .deletedRowsCount(0L) .replacedRowsCount(0L) .minSequenceNumber(0L) - .dv(new byte[] {0xF}) - .dvCardinality(0L) + .dvCardinality(1L) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid DV cardinality: 0 (must be positive)"); + .hasMessage("Invalid DV and cardinality: must both be null or non-null"); } @Test @@ -368,7 +559,7 @@ void testKryoSerializationRoundTrip() throws IOException { .deletedRowsCount(300L) .replacedRowsCount(200L) .minSequenceNumber(5L) - .dv(new byte[] {0xF}) + .dv(ByteBuffer.wrap(new byte[] {0xF})) .dvCardinality(1L) .build(); diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileStruct.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileStruct.java index 3abb36aa51ff..8891db408be5 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileStruct.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileStruct.java @@ -35,11 +35,14 @@ class TestTrackedFileStruct { Types.NestedField.optional(1000, "id_bucket", Types.IntegerType.get()), Types.NestedField.optional(1001, "category", Types.StringType.get())); + // Ordinal of MetadataColumns.ROW_POSITION within TrackingStruct's BASE_TYPE, + // which appends ROW_POSITION after the Tracking schema fields. + private static final int MANIFEST_POS_ORDINAL = Tracking.schema().fields().size(); + @Test void testFieldAccess() { TrackedFileStruct file = new TrackedFileStruct(); - TrackingStruct tracking = - TrackingStruct.builder().status(EntryStatus.ADDED).snapshotId(42L).build(); + Tracking tracking = TrackingBuilder.added(42L).build(); DeletionVectorStruct dv = DeletionVectorStruct.builder() .location("s3://bucket/dv.puffin") @@ -98,9 +101,9 @@ void testFieldAccess() { void testReaderSideFields() { TrackedFileStruct file = new TrackedFileStruct(); - TrackingStruct tracking = TrackingStruct.builder().status(EntryStatus.ADDED).build(); + TrackingStruct tracking = new TrackingStruct(); tracking.setManifestLocation("s3://bucket/metadata/manifest.avro"); - tracking.set(8, 7L); + tracking.set(MANIFEST_POS_ORDINAL, 7L); file.set(0, tracking); file.set(1, FileContent.DATA.id()); @@ -328,14 +331,9 @@ void testKryoSerializationRoundTrip() throws IOException { } static TrackedFileStruct createFullTrackedFile() { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(42L) - .dataSequenceNumber(10L) - .build(); + TrackingStruct tracking = (TrackingStruct) TrackingBuilder.added(42L).build(); tracking.setManifestLocation("s3://bucket/manifest.avro"); - tracking.set(8, 3L); + tracking.set(MANIFEST_POS_ORDINAL, 3L); DeletionVectorStruct dv = DeletionVectorStruct.builder() diff --git a/core/src/test/java/org/apache/iceberg/TestTrackingStruct.java b/core/src/test/java/org/apache/iceberg/TestTrackingStruct.java index 98a7eff2af45..0bbd8b348006 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackingStruct.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackingStruct.java @@ -23,23 +23,44 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.Stream; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; class TestTrackingStruct { + // Ordinals looked up from Tracking.schema() so tests don't hard-code positions. + private static final List TRACKING_FIELDS = Tracking.schema().fields(); + private static final int STATUS_ORDINAL = TRACKING_FIELDS.indexOf(Tracking.STATUS); + private static final int SNAPSHOT_ID_ORDINAL = TRACKING_FIELDS.indexOf(Tracking.SNAPSHOT_ID); + private static final int DATA_SEQUENCE_NUMBER_ORDINAL = + TRACKING_FIELDS.indexOf(Tracking.SEQUENCE_NUMBER); + private static final int FILE_SEQUENCE_NUMBER_ORDINAL = + TRACKING_FIELDS.indexOf(Tracking.FILE_SEQUENCE_NUMBER); + private static final int DV_SNAPSHOT_ID_ORDINAL = + TRACKING_FIELDS.indexOf(Tracking.DV_SNAPSHOT_ID); + private static final int FIRST_ROW_ID_ORDINAL = TRACKING_FIELDS.indexOf(Tracking.FIRST_ROW_ID); + private static final int DELETED_POSITIONS_ORDINAL = + TRACKING_FIELDS.indexOf(Tracking.DELETED_POSITIONS); + private static final int REPLACED_POSITIONS_ORDINAL = + TRACKING_FIELDS.indexOf(Tracking.REPLACED_POSITIONS); + @Test void testFieldAccess() { TrackingStruct tracking = new TrackingStruct(Tracking.schema()); - tracking.set(0, EntryStatus.ADDED.id()); - tracking.set(1, 42L); - tracking.set(2, 10L); - tracking.set(3, 11L); - tracking.set(4, 43L); - tracking.set(5, 1000L); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.set(SNAPSHOT_ID_ORDINAL, 42L); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 10L); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 11L); + tracking.set(DV_SNAPSHOT_ID_ORDINAL, 43L); + tracking.set(FIRST_ROW_ID_ORDINAL, 1000L); assertThat(tracking.status()).isEqualTo(EntryStatus.ADDED); assertThat(tracking.snapshotId()).isEqualTo(42L); @@ -53,30 +74,33 @@ void testFieldAccess() { @Test void testCopy() { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(42L) - .dataSequenceNumber(10L) - .deletedPositions(new byte[] {1, 2}) + Tracking tracking = + TrackingBuilder.from(manifestSourceTracking(), 1L) + .deletedPositions(ByteBuffer.wrap(new byte[] {1, 2})) + .replacedPositions(ByteBuffer.wrap(new byte[] {3, 4})) .build(); - TrackingStruct copy = tracking.copy(); + Tracking copy = tracking.copy(); - assertThat(copy.status()).isEqualTo(EntryStatus.ADDED); - assertThat(copy.snapshotId()).isEqualTo(42L); - assertThat(copy.dataSequenceNumber()).isEqualTo(10L); - assertThat(copy.deletedPositions()).isNotNull(); + assertThat(copy.status()).isEqualTo(EntryStatus.EXISTING); + assertThat(copy.snapshotId()).isEqualTo(tracking.snapshotId()); + assertThat(copy.dataSequenceNumber()).isEqualTo(tracking.dataSequenceNumber()); + assertThat(copy.fileSequenceNumber()).isEqualTo(tracking.fileSequenceNumber()); + assertThat(copy.dvSnapshotId()).isNull(); + assertThat(copy.firstRowId()).isEqualTo(tracking.firstRowId()); + assertThat(copy.deletedPositions()).isEqualTo(tracking.deletedPositions()); + assertThat(copy.replacedPositions()).isEqualTo(tracking.replacedPositions()); - // verify deep copy of ByteBuffer - assertThat(copy.deletedPositions()).isNotSameAs(tracking.deletedPositions()); + // verify deep copy of ByteBuffer backing arrays + assertThat(copy.deletedPositions().array()).isNotSameAs(tracking.deletedPositions().array()); + assertThat(copy.replacedPositions().array()).isNotSameAs(tracking.replacedPositions().array()); } @ParameterizedTest @EnumSource(EntryStatus.class) void testAllStatuses(EntryStatus status) { TrackingStruct tracking = new TrackingStruct(Tracking.schema()); - tracking.set(0, status.id()); + tracking.set(STATUS_ORDINAL, status.id()); assertThat(tracking.status()).isEqualTo(status); } @@ -84,22 +108,24 @@ void testAllStatuses(EntryStatus status) { void testIsLive() { TrackingStruct tracking = new TrackingStruct(Tracking.schema()); - tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); assertThat(tracking.isLive()).isTrue(); - tracking.set(0, EntryStatus.EXISTING.id()); + tracking.set(STATUS_ORDINAL, EntryStatus.EXISTING.id()); assertThat(tracking.isLive()).isTrue(); - tracking.set(0, EntryStatus.DELETED.id()); + tracking.set(STATUS_ORDINAL, EntryStatus.DELETED.id()); assertThat(tracking.isLive()).isFalse(); - tracking.set(0, EntryStatus.REPLACED.id()); + tracking.set(STATUS_ORDINAL, EntryStatus.REPLACED.id()); assertThat(tracking.isLive()).isFalse(); } @Test void testInheritSnapshotId() { - TrackingStruct tracking = TrackingStruct.builder().status(EntryStatus.ADDED).build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.inheritFrom(createManifestTracking(100L, 60L)); // snapshotId is null, should inherit from manifest @@ -108,7 +134,9 @@ void testInheritSnapshotId() { @Test void testInheritSequenceNumberForAddedEntries() { - TrackingStruct tracking = TrackingStruct.builder().status(EntryStatus.ADDED).build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.inheritFrom(createManifestTracking(100L, 60L)); // sequence numbers are null and status is ADDED, should inherit @@ -118,12 +146,11 @@ void testInheritSequenceNumberForAddedEntries() { @Test void testDoNotInheritSequenceNumberForExistingEntries() { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.EXISTING) - .dataSequenceNumber(5L) - .fileSequenceNumber(6L) - .build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.EXISTING.id()); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 5L); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 6L); + tracking.inheritFrom(createManifestTracking(100L, 60L)); // sequence numbers are not inherited for EXISTING entries @@ -133,13 +160,12 @@ void testDoNotInheritSequenceNumberForExistingEntries() { @Test void testExplicitValuesOverrideInheritance() { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(200L) - .dataSequenceNumber(75L) - .fileSequenceNumber(76L) - .build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.set(SNAPSHOT_ID_ORDINAL, 200L); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 75L); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 76L); + tracking.inheritFrom(createManifestTracking(100L, 60L)); // explicit values should take precedence @@ -151,13 +177,13 @@ void testExplicitValuesOverrideInheritance() { @Test void testInheritFromRejectsUnequalSequenceNumbers() { TrackingStruct tracking = new TrackingStruct(Tracking.schema()); - tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); TrackingStruct manifestTracking = new TrackingStruct(Tracking.schema()); - manifestTracking.set(0, EntryStatus.ADDED.id()); - manifestTracking.set(1, 100L); - manifestTracking.set(2, 50L); - manifestTracking.set(3, 60L); + manifestTracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + manifestTracking.set(SNAPSHOT_ID_ORDINAL, 100L); + manifestTracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 50L); + manifestTracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 60L); assertThatThrownBy(() -> tracking.inheritFrom(manifestTracking)) .isInstanceOf(IllegalArgumentException.class) @@ -166,7 +192,8 @@ void testInheritFromRejectsUnequalSequenceNumbers() { @Test void testNoDefaultingWithoutInheritance() { - TrackingStruct tracking = TrackingStruct.builder().status(EntryStatus.ADDED).build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); // no inheritance, nulls stay null assertThat(tracking.snapshotId()).isNull(); @@ -174,20 +201,262 @@ void testNoDefaultingWithoutInheritance() { assertThat(tracking.fileSequenceNumber()).isNull(); } + @Test + void testInheritFromNullIsNoOp() { + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + + tracking.inheritFrom(null); + + // null source is a no-op; all unset fields stay null + assertThat(tracking.snapshotId()).isNull(); + assertThat(tracking.dataSequenceNumber()).isNull(); + assertThat(tracking.fileSequenceNumber()).isNull(); + } + private static Tracking createManifestTracking(long snapshotId, long sequenceNumber) { - return TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(snapshotId) - .dataSequenceNumber(sequenceNumber) - .fileSequenceNumber(sequenceNumber) - .build(); + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.set(SNAPSHOT_ID_ORDINAL, snapshotId); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, sequenceNumber); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, sequenceNumber); + return tracking; } @Test - void testBuilderValidation() { - assertThatThrownBy(() -> TrackingStruct.builder().build()) + void testAddedBuilder() { + Tracking tracking = TrackingBuilder.added(42L).dvUpdated().build(); + + assertThat(tracking.status()).isEqualTo(EntryStatus.ADDED); + assertThat(tracking.snapshotId()).isEqualTo(42L); + assertThat(tracking.dvSnapshotId()).isEqualTo(42L); + assertThat(tracking.deletedPositions()).isNull(); + assertThat(tracking.replacedPositions()).isNull(); + // sequence numbers and firstRowId remain null; populated by inheritance + assertThat(tracking.dataSequenceNumber()).isNull(); + assertThat(tracking.fileSequenceNumber()).isNull(); + assertThat(tracking.firstRowId()).isNull(); + } + + @Test + void testExistingBuilderPreservesSourceFields() { + Tracking source = sourceTracking(); + + Tracking existing = TrackingBuilder.from(source, 1L).build(); + + assertThat(existing.status()).isEqualTo(EntryStatus.EXISTING); + assertThat(existing.snapshotId()).isEqualTo(source.snapshotId()); + assertThat(existing.dataSequenceNumber()).isEqualTo(source.dataSequenceNumber()); + assertThat(existing.fileSequenceNumber()).isEqualTo(source.fileSequenceNumber()); + assertThat(existing.dvSnapshotId()).isEqualTo(source.dvSnapshotId()); + assertThat(existing.firstRowId()).isEqualTo(source.firstRowId()); + } + + @Test + void testDeleteUpdatesSnapshotIdAndPreservesRest() { + Tracking source = sourceTracking(); + + Tracking deleted = TrackingBuilder.deleted(source, 999L); + + assertThat(deleted.status()).isEqualTo(EntryStatus.DELETED); + assertThat(deleted.snapshotId()).isEqualTo(999L); + assertThat(deleted.dataSequenceNumber()).isEqualTo(source.dataSequenceNumber()); + assertThat(deleted.fileSequenceNumber()).isEqualTo(source.fileSequenceNumber()); + assertThat(deleted.dvSnapshotId()).isEqualTo(source.dvSnapshotId()); + assertThat(deleted.firstRowId()).isEqualTo(source.firstRowId()); + } + + @Test + void testReplaceUpdatesSnapshotIdAndPreservesRest() { + Tracking source = sourceTracking(); + + Tracking replaced = TrackingBuilder.replaced(source, 999L); + + assertThat(replaced.status()).isEqualTo(EntryStatus.REPLACED); + assertThat(replaced.snapshotId()).isEqualTo(999L); + assertThat(replaced.dataSequenceNumber()).isEqualTo(source.dataSequenceNumber()); + assertThat(replaced.fileSequenceNumber()).isEqualTo(source.fileSequenceNumber()); + assertThat(replaced.dvSnapshotId()).isEqualTo(source.dvSnapshotId()); + assertThat(replaced.firstRowId()).isEqualTo(source.firstRowId()); + } + + @Test + void testSourceDvPositionsAreNotCarriedForward() { + TrackingStruct source = sourceTracking(); + source.set(DELETED_POSITIONS_ORDINAL, ByteBuffer.wrap(new byte[] {1, 2})); + source.set(REPLACED_POSITIONS_ORDINAL, ByteBuffer.wrap(new byte[] {3, 4})); + + Tracking existing = TrackingBuilder.from(source, 1L).build(); + assertThat(existing.deletedPositions()).isNull(); + assertThat(existing.replacedPositions()).isNull(); + + Tracking deleted = TrackingBuilder.deleted(source, 999L); + assertThat(deleted.deletedPositions()).isNull(); + assertThat(deleted.replacedPositions()).isNull(); + + Tracking replaced = TrackingBuilder.replaced(source, 999L); + assertThat(replaced.deletedPositions()).isNull(); + assertThat(replaced.replacedPositions()).isNull(); + } + + @Test + void testExistingBuilderAllowsDvMutation() { + Tracking existing = TrackingBuilder.from(sourceTracking(), 999L).dvUpdated().build(); + assertThat(existing.dvSnapshotId()).isEqualTo(999L); + } + + @Test + void testManifestDVMutatorsRejectedOnAdded() { + assertThatThrownBy( + () -> TrackingBuilder.added(42L).deletedPositions(ByteBuffer.wrap(new byte[] {1}))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot set deleted positions on ADDED entry"); + + assertThatThrownBy( + () -> TrackingBuilder.added(42L).replacedPositions(ByteBuffer.wrap(new byte[] {1}))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot set replaced positions on ADDED entry"); + } + + @Test + void testDvSnapshotIdAndManifestDVPositionsAreMutuallyExclusive() { + // sourceTracking has dvSnapshotId=43, inherited by existing(source) + assertThatThrownBy( + () -> + TrackingBuilder.from(sourceTracking(), 1L) + .deletedPositions(ByteBuffer.wrap(new byte[] {1}))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot set deleted positions on a data file entry (DV snapshot ID is set)"); + + assertThatThrownBy( + () -> + TrackingBuilder.from(sourceTracking(), 1L) + .replacedPositions(ByteBuffer.wrap(new byte[] {1}))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot set replaced positions on a data file entry (DV snapshot ID is set)"); + + // Setting MDV positions first then dvUpdated is also rejected + assertThatThrownBy( + () -> + TrackingBuilder.from(manifestSourceTracking(), 1L) + .deletedPositions(ByteBuffer.wrap(new byte[] {1})) + .dvUpdated()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot mark DV updated on a manifest entry (deleted/replaced positions are set)"); + } + + @Test + void testBuilderRejectsNullSource() { + assertThatThrownBy(() -> TrackingBuilder.from(null, 1L)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid status: null"); + .hasMessage("Invalid source tracking: null"); + } + + @Test + void testSourceBuildersRejectSourceWithoutSequenceNumbers() { + Tracking missingBoth = TrackingBuilder.added(42L).build(); + + assertThatThrownBy(() -> TrackingBuilder.from(missingBoth, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: data sequence number is null"); + + assertThatThrownBy(() -> TrackingBuilder.deleted(missingBoth, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: data sequence number is null"); + + assertThatThrownBy(() -> TrackingBuilder.replaced(missingBoth, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: data sequence number is null"); + + TrackingStruct missingFileSeq = new TrackingStruct(Tracking.schema()); + missingFileSeq.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + missingFileSeq.set(SNAPSHOT_ID_ORDINAL, 42L); + missingFileSeq.set(DATA_SEQUENCE_NUMBER_ORDINAL, 10L); + + assertThatThrownBy(() -> TrackingBuilder.from(missingFileSeq, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: file sequence number is null"); + + assertThatThrownBy(() -> TrackingBuilder.deleted(missingFileSeq, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: file sequence number is null"); + + assertThatThrownBy(() -> TrackingBuilder.replaced(missingFileSeq, 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid tracking source: file sequence number is null"); + } + + private static Stream terminalTransitionCases() { + Consumer builderCall = source -> TrackingBuilder.from(source, 1L); + Consumer deletedCall = source -> TrackingBuilder.deleted(source, 1L); + Consumer replacedCall = source -> TrackingBuilder.replaced(source, 1L); + return Stream.of( + Arguments.of(EntryStatus.DELETED, builderCall), + Arguments.of(EntryStatus.DELETED, deletedCall), + Arguments.of(EntryStatus.DELETED, replacedCall), + Arguments.of(EntryStatus.REPLACED, builderCall), + Arguments.of(EntryStatus.REPLACED, deletedCall), + Arguments.of(EntryStatus.REPLACED, replacedCall)); + } + + @ParameterizedTest + @MethodSource("terminalTransitionCases") + void testRejectsTransitionsFromTerminalStatus( + EntryStatus sourceStatus, Consumer factoryCall) { + Tracking source = sourceTrackingWithStatus(sourceStatus); + assertThatThrownBy(() -> factoryCall.accept(source)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot revive non-live entry with status " + sourceStatus); + } + + @Test + void testExistingToExistingIsAllowed() { + Tracking existingSource = sourceTrackingWithStatus(EntryStatus.EXISTING); + + Tracking existing = TrackingBuilder.from(existingSource, 1L).build(); + + assertThat(existing.status()).isEqualTo(EntryStatus.EXISTING); + assertThat(existing.snapshotId()).isEqualTo(existingSource.snapshotId()); + } + + @Test + void testExistingToTerminalTransitions() { + Tracking existingSource = sourceTrackingWithStatus(EntryStatus.EXISTING); + + Tracking deleted = TrackingBuilder.deleted(existingSource, 999L); + assertThat(deleted.status()).isEqualTo(EntryStatus.DELETED); + assertThat(deleted.snapshotId()).isEqualTo(999L); + + Tracking replaced = TrackingBuilder.replaced(existingSource, 999L); + assertThat(replaced.status()).isEqualTo(EntryStatus.REPLACED); + assertThat(replaced.snapshotId()).isEqualTo(999L); + } + + @Test + void testInternalSetIgnoresUnknownOrdinal() { + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.set(SNAPSHOT_ID_ORDINAL, 42L); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 10L); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 11L); + tracking.set(DV_SNAPSHOT_ID_ORDINAL, 43L); + tracking.set(FIRST_ROW_ID_ORDINAL, 1000L); + tracking.set(DELETED_POSITIONS_ORDINAL, ByteBuffer.wrap(new byte[] {1, 2})); + tracking.set(REPLACED_POSITIONS_ORDINAL, ByteBuffer.wrap(new byte[] {3, 4})); + + // unknown ordinals from a newer format version are silently ignored + tracking.internalSet(99, "value from a newer format"); + + // every field is unchanged + assertThat(tracking.status()).isEqualTo(EntryStatus.ADDED); + assertThat(tracking.snapshotId()).isEqualTo(42L); + assertThat(tracking.dataSequenceNumber()).isEqualTo(10L); + assertThat(tracking.fileSequenceNumber()).isEqualTo(11L); + assertThat(tracking.dvSnapshotId()).isEqualTo(43L); + assertThat(tracking.firstRowId()).isEqualTo(1000L); + assertThat(tracking.deletedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2})); + assertThat(tracking.replacedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {3, 4})); } @Test @@ -210,38 +479,85 @@ void testProjectedStructLike() { } @Test - void testJavaSerializationRoundTrip() throws IOException, ClassNotFoundException { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(42L) - .dataSequenceNumber(10L) - .deletedPositions(new byte[] {1, 2}) - .build(); + void testAddedWithDvSnapshotIdJavaSerializationRoundTrip() + throws IOException, ClassNotFoundException { + Tracking tracking = TrackingBuilder.added(42L).dvUpdated().build(); - TrackingStruct deserialized = TestHelpers.roundTripSerialize(tracking); + Tracking deserialized = TestHelpers.roundTripSerialize(tracking); assertThat(deserialized.status()).isEqualTo(EntryStatus.ADDED); assertThat(deserialized.snapshotId()).isEqualTo(42L); - assertThat(deserialized.dataSequenceNumber()).isEqualTo(10L); - assertThat(deserialized.deletedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2})); + assertThat(deserialized.dvSnapshotId()).isEqualTo(42L); + assertThat(deserialized.deletedPositions()).isNull(); + assertThat(deserialized.replacedPositions()).isNull(); } @Test - void testKryoSerializationRoundTrip() throws IOException { - TrackingStruct tracking = - TrackingStruct.builder() - .status(EntryStatus.ADDED) - .snapshotId(42L) - .dataSequenceNumber(10L) - .deletedPositions(new byte[] {1, 2}) + void testExistingWithManifestDVPositionsJavaSerializationRoundTrip() + throws IOException, ClassNotFoundException { + Tracking tracking = + TrackingBuilder.from(manifestSourceTracking(), 1L) + .deletedPositions(ByteBuffer.wrap(new byte[] {1, 2})) + .replacedPositions(ByteBuffer.wrap(new byte[] {3, 4})) .build(); - TrackingStruct deserialized = TestHelpers.KryoHelpers.roundTripSerialize(tracking); + Tracking deserialized = TestHelpers.roundTripSerialize(tracking); + + assertThat(deserialized.status()).isEqualTo(EntryStatus.EXISTING); + assertThat(deserialized.dvSnapshotId()).isNull(); + assertThat(deserialized.deletedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2})); + assertThat(deserialized.replacedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {3, 4})); + } + + @Test + void testAddedWithDvSnapshotIdKryoSerializationRoundTrip() throws IOException { + Tracking tracking = TrackingBuilder.added(42L).dvUpdated().build(); + + Tracking deserialized = TestHelpers.KryoHelpers.roundTripSerialize(tracking); assertThat(deserialized.status()).isEqualTo(EntryStatus.ADDED); assertThat(deserialized.snapshotId()).isEqualTo(42L); - assertThat(deserialized.dataSequenceNumber()).isEqualTo(10L); + assertThat(deserialized.dvSnapshotId()).isEqualTo(42L); + assertThat(deserialized.deletedPositions()).isNull(); + assertThat(deserialized.replacedPositions()).isNull(); + } + + @Test + void testExistingWithManifestDVPositionsKryoSerializationRoundTrip() throws IOException { + Tracking tracking = + TrackingBuilder.from(manifestSourceTracking(), 1L) + .deletedPositions(ByteBuffer.wrap(new byte[] {1, 2})) + .replacedPositions(ByteBuffer.wrap(new byte[] {3, 4})) + .build(); + + Tracking deserialized = TestHelpers.KryoHelpers.roundTripSerialize(tracking); + + assertThat(deserialized.status()).isEqualTo(EntryStatus.EXISTING); + assertThat(deserialized.dvSnapshotId()).isNull(); assertThat(deserialized.deletedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2})); + assertThat(deserialized.replacedPositions()).isEqualTo(ByteBuffer.wrap(new byte[] {3, 4})); + } + + private static TrackingStruct sourceTracking() { + TrackingStruct tracking = new TrackingStruct(Tracking.schema()); + tracking.set(STATUS_ORDINAL, EntryStatus.ADDED.id()); + tracking.set(SNAPSHOT_ID_ORDINAL, 42L); + tracking.set(DATA_SEQUENCE_NUMBER_ORDINAL, 10L); + tracking.set(FILE_SEQUENCE_NUMBER_ORDINAL, 10L); + tracking.set(DV_SNAPSHOT_ID_ORDINAL, 43L); + tracking.set(FIRST_ROW_ID_ORDINAL, 1000L); + return tracking; + } + + private static TrackingStruct sourceTrackingWithStatus(EntryStatus status) { + TrackingStruct tracking = sourceTracking(); + tracking.set(STATUS_ORDINAL, status.id()); + return tracking; + } + + private static TrackingStruct manifestSourceTracking() { + TrackingStruct tracking = sourceTracking(); + tracking.set(DV_SNAPSHOT_ID_ORDINAL, null); + return tracking; } } From c91025e13d6c5bdcf052957c892574929674cc79 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Wed, 3 Jun 2026 22:48:25 +0200 Subject: [PATCH 57/58] ORC: Remove ORC tests for partition statistics (#16676) Partition statistics readers and writers don't support ORC file format now, however there are test suites for this format too. They don't add any extra covergae just override existing tests expecting failure due to unsupported file format. --- .../orc/TestOrcPartitionStatisticsScan.java | 73 ------------------- .../orc/TestOrcPartitionStatsHandler.java | 73 ------------------- 2 files changed, 146 deletions(-) delete mode 100644 orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java delete mode 100644 orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java deleted file mode 100644 index 11c2538662bf..000000000000 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.orc; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionStatisticsScanTestBase; - -public class TestOrcPartitionStatisticsScan extends PartitionStatisticsScanTestBase { - @Override - public FileFormat format() { - return FileFormat.ORC; - } - - @Override - public void testScanPartitionStatsForCurrentSnapshot() throws Exception { - assertThatThrownBy(super::testScanPartitionStatsForCurrentSnapshot) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testScanPartitionStatsForOlderSnapshot() throws Exception { - assertThatThrownBy(super::testScanPartitionStatsForOlderSnapshot) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testReadingStatsWithInvalidSchema() throws Exception { - assertThatThrownBy(super::testReadingStatsWithInvalidSchema) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testV2toV3SchemaEvolution() throws Exception { - assertThatThrownBy(super::testV2toV3SchemaEvolution) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testProjectStatFields() throws Exception { - assertThatThrownBy(super::testProjectStatFields) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testProjectIgnoresUnknownField() throws Exception { - assertThatThrownBy(super::testProjectIgnoresUnknownField) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } -} diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java deleted file mode 100644 index 04d1d212f77a..000000000000 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatsHandler.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.orc; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionStatsHandlerTestBase; - -public class TestOrcPartitionStatsHandler extends PartitionStatsHandlerTestBase { - - public FileFormat format() { - return FileFormat.ORC; - } - - @Override - public void testAllDatatypePartitionWriting() throws Exception { - assertThatThrownBy(super::testAllDatatypePartitionWriting) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testOptionalFieldsWriting() throws Exception { - assertThatThrownBy(super::testOptionalFieldsWriting) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testLatestStatsFile() throws Exception { - assertThatThrownBy(super::testLatestStatsFile) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testLatestStatsFileWithBranch() throws Exception { - assertThatThrownBy(super::testLatestStatsFileWithBranch) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testCopyOnWriteDelete() throws Exception { - assertThatThrownBy(super::testCopyOnWriteDelete) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } - - @Override - public void testFullComputeFallbackWithInvalidStats() { - assertThatThrownBy(super::testFullComputeFallbackWithInvalidStats) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Cannot write using unregistered internal data format: ORC"); - } -} From 4b4c5b554c829d946d7152dbf70dcd987ff6a9ec Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 3 Jun 2026 15:27:13 -0700 Subject: [PATCH 58/58] Infra: Update collaborators list (#16678) Add anuragmantri and nssalian as collaborators. Remove inactive collaborators chenjunjiedada and jun-he to stay within the 10-person limit. --- .asf.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 13ac27f20db1..7f41ac6bd1fc 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,8 +49,6 @@ github: issues: true projects: true collaborators: # Note: the number of collaborators is limited to 10 - - chenjunjiedada - - jun-he - marton-bod - samarthjain - SreeramGarlapati @@ -59,6 +57,8 @@ github: - ajantha-bhat - jbonofre - manuzhang + - anuragmantri + - nssalian ghp_branch: gh-pages ghp_path: /