From a13265b35009747d0dcf66b553c4956a31d97a7d Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 22 Apr 2026 08:21:34 -0700 Subject: [PATCH 01/18] [core] v4: Add TrackedFileAdapters: bridge TrackedFile to DataFile/DeleteFile APIs This adapter would allow to minimize the v4 related code changes during scan planning and commits. --- .../apache/iceberg/TrackedFileAdapters.java | 461 +++++++++++++ .../iceberg/TestTrackedFileAdapters.java | 629 ++++++++++++++++++ 2 files changed, 1090 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java create mode 100644 core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java new file mode 100644 index 000000000000..295f90a67e9a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -0,0 +1,461 @@ +/* + * 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.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; + +/** + * Adapts {@link TrackedFile} entries to the {@link DataFile} and {@link DeleteFile} APIs. + * + *

Note: V4 colocates deletion vectors with data file entries in {@link TrackedFile}. This + * adapter does not carry over {@link TrackedFile#deletionVector()} because {@link DataFile} has no + * way to represent it. Once {@link DataFile} is extended with deletion vector support, this adapter + * should be updated to include it. + */ +class TrackedFileAdapters { + + private TrackedFileAdapters() {} + + static DataFile asDataFile(TrackedFile file, PartitionSpec spec) { + Preconditions.checkState( + file.contentType() == FileContent.DATA, + "Cannot convert tracked file to DataFile: content type is %s, not DATA", + file.contentType()); + return new TrackedDataFile(file, spec); + } + + static DeleteFile asDeleteFile(TrackedFile file, PartitionSpec spec) { + Preconditions.checkState( + file.contentType() == FileContent.EQUALITY_DELETES, + "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", + file.contentType()); + return new TrackedDeleteFile(file, spec); + } + + // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition + // schema), replacing this transform-based derivation. Once that lands, this method should be + // removed and the adapter should read the tuple directly. + @SuppressWarnings({"unchecked", "rawtypes"}) + static StructLike extractPartition(TrackedFile file, PartitionSpec spec) { + if (spec == null || spec.isUnpartitioned()) { + return null; + } + + ContentStats stats = file.contentStats(); + if (stats == null) { + return null; + } + + PartitionData partition = new PartitionData(spec.partitionType()); + + for (int i = 0; i < spec.fields().size(); i += 1) { + PartitionField field = spec.fields().get(i); + + if (field.transform().isVoid()) { + partition.set(i, null); + continue; + } + + FieldStats fieldStats = stats.statsFor(field.sourceId()); + if (fieldStats == null || fieldStats.lowerBound() == null) { + partition.set(i, null); + continue; + } + + Type sourceType = spec.schema().findType(field.sourceId()); + Function boundTransform = field.transform().bind(sourceType); + partition.set(i, boundTransform.apply(fieldStats.lowerBound())); + } + + return partition; + } + + static Map valueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.valueCount() != null) { + result.put(fs.fieldId(), fs.valueCount()); + } + } + + return result.isEmpty() ? null : result; + } + + static Map nullValueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.nullValueCount() != null) { + result.put(fs.fieldId(), fs.nullValueCount()); + } + } + + return result.isEmpty() ? null : result; + } + + static Map nanValueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.nanValueCount() != null) { + result.put(fs.fieldId(), fs.nanValueCount()); + } + } + + return result.isEmpty() ? null : result; + } + + static Map lowerBounds(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.lowerBound() != null && fs.type() != null) { + result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.lowerBound())); + } + } + + return result.isEmpty() ? null : result; + } + + static Map upperBounds(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.upperBound() != null && fs.type() != null) { + result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.upperBound())); + } + } + + return result.isEmpty() ? null : result; + } + + /** Adapts a TrackedFile DATA entry to the {@link DataFile} interface. */ + private static class TrackedDataFile implements DataFile { + private final TrackedFile file; + private final Tracking tracking; + private final PartitionSpec spec; + + private TrackedDataFile(TrackedFile file, PartitionSpec spec) { + this.file = file; + this.tracking = file.tracking(); + this.spec = spec; + } + + @Override + public Long pos() { + return tracking != null ? tracking.manifestPos() : null; + } + + @Override + public int specId() { + // null specId in v4 means unpartitioned; default to 0 to match PartitionSpec.unpartitioned() + return file.specId() != null ? file.specId() : 0; + } + + @Override + public FileContent content() { + return FileContent.DATA; + } + + @SuppressWarnings("deprecation") + @Override + public CharSequence path() { + return file.location(); + } + + @Override + public FileFormat format() { + return file.fileFormat(); + } + + @Override + public StructLike partition() { + return extractPartition(file, spec); + } + + @Override + public long recordCount() { + return file.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return file.fileSizeInBytes(); + } + + @Override + public Integer sortOrderId() { + return file.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + return tracking != null ? tracking.dataSequenceNumber() : null; + } + + @Override + public Long fileSequenceNumber() { + return tracking != null ? tracking.fileSequenceNumber() : null; + } + + @Override + public Long firstRowId() { + return tracking != null ? tracking.firstRowId() : null; + } + + @Override + public ByteBuffer keyMetadata() { + return file.keyMetadata(); + } + + @Override + public List splitOffsets() { + return file.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return null; + } + + @Override + public String manifestLocation() { + return tracking != null ? tracking.manifestLocation() : null; + } + + @Override + public Map columnSizes() { + return null; + } + + @Override + public Map valueCounts() { + return TrackedFileAdapters.valueCounts(file.contentStats()); + } + + @Override + public Map nullValueCounts() { + return TrackedFileAdapters.nullValueCounts(file.contentStats()); + } + + @Override + public Map nanValueCounts() { + return TrackedFileAdapters.nanValueCounts(file.contentStats()); + } + + @Override + public Map lowerBounds() { + return TrackedFileAdapters.lowerBounds(file.contentStats()); + } + + @Override + public Map upperBounds() { + return TrackedFileAdapters.upperBounds(file.contentStats()); + } + + @Override + public DataFile copy() { + return this; + } + + @Override + public DataFile copy(boolean withStats) { + return this; + } + + @Override + public DataFile copyWithoutStats() { + return this; + } + + @Override + public DataFile copyWithStats(Set requestedColumnIds) { + return this; + } + } + + /** Adapts a TrackedFile EQUALITY_DELETES entry to the {@link DeleteFile} interface. */ + private static class TrackedDeleteFile implements DeleteFile { + private final TrackedFile file; + private final Tracking tracking; + private final PartitionSpec spec; + + private TrackedDeleteFile(TrackedFile file, PartitionSpec spec) { + this.file = file; + this.tracking = file.tracking(); + this.spec = spec; + } + + @Override + public Long pos() { + return tracking != null ? tracking.manifestPos() : null; + } + + @Override + public int specId() { + // null specId in v4 means unpartitioned; default to 0 to match PartitionSpec.unpartitioned() + return file.specId() != null ? file.specId() : 0; + } + + @Override + public FileContent content() { + return FileContent.EQUALITY_DELETES; + } + + @SuppressWarnings("deprecation") + @Override + public CharSequence path() { + return file.location(); + } + + @Override + public FileFormat format() { + return file.fileFormat(); + } + + @Override + public StructLike partition() { + return extractPartition(file, spec); + } + + @Override + public long recordCount() { + return file.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return file.fileSizeInBytes(); + } + + @Override + public Integer sortOrderId() { + return file.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + return tracking != null ? tracking.dataSequenceNumber() : null; + } + + @Override + public Long fileSequenceNumber() { + return tracking != null ? tracking.fileSequenceNumber() : null; + } + + @Override + public Long firstRowId() { + return tracking != null ? tracking.firstRowId() : null; + } + + @Override + public ByteBuffer keyMetadata() { + return file.keyMetadata(); + } + + @Override + public List splitOffsets() { + return file.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return file.equalityIds(); + } + + @Override + public String manifestLocation() { + return tracking != null ? tracking.manifestLocation() : null; + } + + @Override + public Map columnSizes() { + return null; + } + + @Override + public Map valueCounts() { + return TrackedFileAdapters.valueCounts(file.contentStats()); + } + + @Override + public Map nullValueCounts() { + return TrackedFileAdapters.nullValueCounts(file.contentStats()); + } + + @Override + public Map nanValueCounts() { + return TrackedFileAdapters.nanValueCounts(file.contentStats()); + } + + @Override + public Map lowerBounds() { + return TrackedFileAdapters.lowerBounds(file.contentStats()); + } + + @Override + public Map upperBounds() { + return TrackedFileAdapters.upperBounds(file.contentStats()); + } + + @Override + public DeleteFile copy() { + return this; + } + + @Override + public DeleteFile copy(boolean withStats) { + return this; + } + + @Override + public DeleteFile copyWithoutStats() { + return this; + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return this; + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java new file mode 100644 index 000000000000..33c41dfb8dde --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -0,0 +1,629 @@ +/* + * 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 java.nio.ByteBuffer; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +class TestTrackedFileAdapters { + + @Test + void testAsDataFileValidatesContentType() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + assertThat(dataFile).isNotNull(); + assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + assertThat(dataFile.location()).isEqualTo("s3://bucket/data.parquet"); + } + + @Test + void testAsDataFileRejectsNonData() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.EQUALITY_DELETES.id()); + file.set(2, "s3://bucket/delete.avro"); + file.set(3, "avro"); + file.set(4, 50L); + file.set(5, 512L); + file.set(6, 0); + + assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned())) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot convert tracked file to DataFile: content type is %s, not DATA", + FileContent.EQUALITY_DELETES); + } + + @Test + void testAsDeleteFileValidatesContentType() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.EQUALITY_DELETES.id()); + file.set(2, "s3://bucket/eq-delete.avro"); + file.set(3, "avro"); + file.set(4, 50L); + file.set(5, 512L); + file.set(6, 0); + file.set(13, ImmutableList.of(1, 2)); + + DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + assertThat(deleteFile).isNotNull(); + assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2); + } + + @Test + void testAsDeleteFileRejectsNonEqualityDeletes() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + + assertThatThrownBy(() -> TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned())) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", + FileContent.DATA); + } + + @Test + void testDataFileAdapterDelegatesAllFields() { + TrackedFileStruct file = new TrackedFileStruct(); + Types.StructType trackingWithPos = + Types.StructType.of( + ImmutableList.builder() + .addAll(Tracking.schema().fields()) + .add(MetadataColumns.ROW_POSITION) + .build()); + TrackingStruct tracking = new TrackingStruct(trackingWithPos); + + tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(1, 42L); + tracking.set(2, 10L); + tracking.set(3, 11L); + tracking.set(5, 1000L); + tracking.setManifestLocation("s3://bucket/manifest.avro"); + tracking.set(8, 3L); + + file.set(0, tracking); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data/file.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + file.set(8, 3); + file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); + file.set(12, ImmutableList.of(50L, 100L)); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + + assertThat(dataFile.pos()).isEqualTo(3L); + assertThat(dataFile.specId()).isEqualTo(0); + assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + assertThat(dataFile.location()).isEqualTo("s3://bucket/data/file.parquet"); + assertThat(dataFile.format()).isEqualTo(FileFormat.PARQUET); + assertThat(dataFile.recordCount()).isEqualTo(100L); + assertThat(dataFile.fileSizeInBytes()).isEqualTo(1024L); + assertThat(dataFile.sortOrderId()).isEqualTo(3); + assertThat(dataFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(dataFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(dataFile.firstRowId()).isEqualTo(1000L); + assertThat(dataFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2, 3})); + assertThat(dataFile.splitOffsets()).containsExactly(50L, 100L); + assertThat(dataFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + assertThat(dataFile.equalityFieldIds()).isNull(); + assertThat(dataFile.columnSizes()).isNull(); + } + + @Test + void testDeleteFileAdapterDelegatesAllFields() { + TrackedFileStruct file = new TrackedFileStruct(); + Types.StructType trackingWithPos = + Types.StructType.of( + ImmutableList.builder() + .addAll(Tracking.schema().fields()) + .add(MetadataColumns.ROW_POSITION) + .build()); + TrackingStruct tracking = new TrackingStruct(trackingWithPos); + + tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(1, 42L); + tracking.set(2, 10L); + tracking.set(3, 11L); + tracking.set(5, 1000L); + tracking.setManifestLocation("s3://bucket/manifest.avro"); + tracking.set(8, 5L); + + file.set(0, tracking); + file.set(1, FileContent.EQUALITY_DELETES.id()); + file.set(2, "s3://bucket/eq-delete.avro"); + file.set(3, "avro"); + file.set(4, 50L); + file.set(5, 512L); + file.set(6, 1); + file.set(8, 5); + file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); + file.set(12, ImmutableList.of(200L)); + file.set(13, ImmutableList.of(1, 2, 3)); + + DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + + assertThat(deleteFile.pos()).isEqualTo(5L); + assertThat(deleteFile.specId()).isEqualTo(1); + assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(deleteFile.location()).isEqualTo("s3://bucket/eq-delete.avro"); + assertThat(deleteFile.format()).isEqualTo(FileFormat.AVRO); + assertThat(deleteFile.recordCount()).isEqualTo(50L); + assertThat(deleteFile.fileSizeInBytes()).isEqualTo(512L); + assertThat(deleteFile.sortOrderId()).isEqualTo(5); + assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(deleteFile.firstRowId()).isEqualTo(1000L); + assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); + assertThat(deleteFile.splitOffsets()).containsExactly(200L); + assertThat(deleteFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2, 3); + assertThat(deleteFile.columnSizes()).isNull(); + } + + @Test + void testAdapterDelegatesNullTracking() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + + assertThat(dataFile.dataSequenceNumber()).isNull(); + assertThat(dataFile.fileSequenceNumber()).isNull(); + assertThat(dataFile.firstRowId()).isNull(); + assertThat(dataFile.manifestLocation()).isNull(); + assertThat(dataFile.pos()).isNull(); + } + + @Test + void testDataFileAdapterStatsFromContentStats() { + TrackedFileStruct file = createTrackedFileWithStats(); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + + assertThat(dataFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); + assertThat(dataFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); + assertThat(dataFile.nanValueCounts()).containsOnly(entry(2, 3L)); + assertThat(dataFile.lowerBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); + assertThat(dataFile.upperBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); + assertThat(dataFile.columnSizes()).isNull(); + } + + @Test + void testDeleteFileAdapterStatsFromContentStats() { + TrackedFileStruct file = createTrackedFileWithStats(); + file.set(1, FileContent.EQUALITY_DELETES.id()); + file.set(13, ImmutableList.of(1)); + + DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + + assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); + assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); + assertThat(deleteFile.nanValueCounts()).containsOnly(entry(2, 3L)); + assertThat(deleteFile.lowerBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); + assertThat(deleteFile.upperBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); + assertThat(deleteFile.columnSizes()).isNull(); + } + + @Test + void testDataFileAdapterStatsNullWhenNoContentStats() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + + assertThat(dataFile.valueCounts()).isNull(); + assertThat(dataFile.nullValueCounts()).isNull(); + assertThat(dataFile.nanValueCounts()).isNull(); + assertThat(dataFile.lowerBounds()).isNull(); + assertThat(dataFile.upperBounds()).isNull(); + } + + @Test + void testPartitionExtractedFromContentStatsWithIdentityTransform() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "category", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); + + TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + + StructLike partition = dataFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); + } + + @Test + void testPartitionExtractedWithYearTransform() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "ts", Types.DateType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).year("ts").build(); + + // date value 18628 = 2021-01-01 (days since epoch) + TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.DateType.get(), 18628); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + + StructLike partition = dataFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, Integer.class)).isEqualTo(51); + } + + @Test + void testPartitionExtractedWithBucketTransform() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "value", Types.IntegerType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 16).build(); + + TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.IntegerType.get(), 42); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + + StructLike partition = dataFile.partition(); + assertThat(partition).isNotNull(); + + // verify the bucket value is a valid bucket (0-15) + int bucket = partition.get(0, Integer.class); + assertThat(bucket).isBetween(0, 15); + } + + @Test + void testPartitionNullWhenNoContentStats() { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); + + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, spec.specId()); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + assertThat(dataFile.partition()).isNull(); + } + + @Test + void testPartitionNullWhenNullSpec() { + TrackedFileStruct file = createTrackedFileWithStats(); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, null); + assertThat(dataFile.partition()).isNull(); + } + + @Test + void testPartitionNullForUnpartitioned() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + + TrackedFileStruct file = createTrackedFileWithStats(); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + assertThat(dataFile.partition()).isNull(); + } + + @Test + void testPartitionWithMultipleFields() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "category", Types.StringType.get())); + + PartitionSpec spec = + PartitionSpec.builderFor(schema).identity("id").identity("category").build(); + + Types.StructType statsStruct = + Types.StructType.of( + Types.NestedField.optional( + 10000, + "1", + Types.StructType.of( + Types.NestedField.optional(10006, "lower_bound", Types.IntegerType.get()), + Types.NestedField.optional(10007, "upper_bound", Types.IntegerType.get()))), + Types.NestedField.optional( + 20000, + "2", + Types.StructType.of( + Types.NestedField.optional(20006, "lower_bound", Types.StringType.get()), + Types.NestedField.optional(20007, "upper_bound", Types.StringType.get())))); + + @SuppressWarnings("unchecked") + List> fieldStatsList = + ImmutableList.of( + (FieldStats) + BaseFieldStats.builder() + .fieldId(1) + .type(Types.IntegerType.get()) + .lowerBound(42) + .upperBound(42) + .build(), + (FieldStats) + BaseFieldStats.builder() + .fieldId(2) + .type(Types.StringType.get()) + .lowerBound("electronics") + .upperBound("electronics") + .build()); + + BaseContentStats stats = + BaseContentStats.builder() + .withStatsStruct(statsStruct) + .withFieldStats(fieldStatsList) + .build(); + + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, spec.specId()); + file.set(7, stats); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + + StructLike partition = dataFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, Integer.class)).isEqualTo(42); + assertThat(partition.get(1, CharSequence.class).toString()).isEqualTo("electronics"); + } + + @Test + void testPartitionWithVoidTransform() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").alwaysNull("data").build(); + + TrackedFileStruct file = createTrackedFileWithFieldStats(1, Types.IntegerType.get(), 42); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + + StructLike partition = dataFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, Integer.class)).isEqualTo(42); + assertThat(partition.get(1, CharSequence.class)).isNull(); + } + + @Test + void testDeleteFilePartitionExtracted() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "category", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); + + TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); + file.set(1, FileContent.EQUALITY_DELETES.id()); + file.set(13, ImmutableList.of(1)); + + DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, spec); + + StructLike partition = deleteFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); + } + + @Test + void testSpecIdDefaultsToZeroWhenNull() { + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + + DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + assertThat(dataFile.specId()).isEqualTo(0); + } + + private static java.util.Map.Entry entry(int key, long value) { + return java.util.Map.entry(key, value); + } + + @SuppressWarnings("unchecked") + private static TrackedFileStruct createTrackedFileWithPartitionStats(PartitionSpec spec) { + Types.StructType statsStruct = + Types.StructType.of( + Types.NestedField.optional( + 20000, + "2", + Types.StructType.of( + Types.NestedField.optional(20006, "lower_bound", Types.StringType.get()), + Types.NestedField.optional(20007, "upper_bound", Types.StringType.get())))); + + List> fieldStatsList = + ImmutableList.of( + (FieldStats) + BaseFieldStats.builder() + .fieldId(2) + .type(Types.StringType.get()) + .lowerBound("electronics") + .upperBound("electronics") + .build()); + + BaseContentStats stats = + BaseContentStats.builder() + .withStatsStruct(statsStruct) + .withFieldStats(fieldStatsList) + .build(); + + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data/file.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, spec.specId()); + file.set(7, stats); + + return file; + } + + @SuppressWarnings("unchecked") + private static TrackedFileStruct createTrackedFileWithFieldStats( + int fieldId, Type type, T value) { + int statsFieldId = fieldId * 10000; + Types.StructType statsStruct = + Types.StructType.of( + Types.NestedField.optional( + statsFieldId, + Integer.toString(fieldId), + Types.StructType.of( + Types.NestedField.optional(statsFieldId + 6, "lower_bound", type), + Types.NestedField.optional(statsFieldId + 7, "upper_bound", type)))); + + List> fieldStatsList = + ImmutableList.of( + (FieldStats) + BaseFieldStats.builder() + .fieldId(fieldId) + .type(type) + .lowerBound(value) + .upperBound(value) + .build()); + + BaseContentStats stats = + BaseContentStats.builder() + .withStatsStruct(statsStruct) + .withFieldStats(fieldStatsList) + .build(); + + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data/file.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + file.set(7, stats); + + return file; + } + + @SuppressWarnings("unchecked") + private static TrackedFileStruct createTrackedFileWithStats() { + Types.StructType statsStruct = + Types.StructType.of( + Types.NestedField.optional( + 10000, + "1", + Types.StructType.of( + Types.NestedField.optional(10001, "value_count", Types.LongType.get()), + Types.NestedField.optional(10002, "null_value_count", Types.LongType.get()), + Types.NestedField.optional(10003, "nan_value_count", Types.LongType.get()), + Types.NestedField.optional(10006, "lower_bound", Types.IntegerType.get()), + Types.NestedField.optional(10007, "upper_bound", Types.IntegerType.get()))), + Types.NestedField.optional( + 20000, + "2", + Types.StructType.of( + Types.NestedField.optional(20001, "value_count", Types.LongType.get()), + Types.NestedField.optional(20002, "null_value_count", Types.LongType.get()), + Types.NestedField.optional(20003, "nan_value_count", Types.LongType.get()), + Types.NestedField.optional(20006, "lower_bound", Types.FloatType.get()), + Types.NestedField.optional(20007, "upper_bound", Types.FloatType.get())))); + + List> fieldStatsList = + ImmutableList.of( + (FieldStats) + BaseFieldStats.builder() + .fieldId(1) + .type(Types.IntegerType.get()) + .valueCount(100L) + .nullValueCount(5L) + .lowerBound(1) + .upperBound(1000) + .build(), + (FieldStats) + BaseFieldStats.builder() + .fieldId(2) + .type(Types.FloatType.get()) + .valueCount(200L) + .nullValueCount(10L) + .nanValueCount(3L) + .lowerBound(1.0f) + .upperBound(100.0f) + .build()); + + BaseContentStats stats = + BaseContentStats.builder() + .withStatsStruct(statsStruct) + .withFieldStats(fieldStatsList) + .build(); + + TrackedFileStruct file = new TrackedFileStruct(); + file.set(1, FileContent.DATA.id()); + file.set(2, "s3://bucket/data/file.parquet"); + file.set(3, "parquet"); + file.set(4, 100L); + file.set(5, 1024L); + file.set(6, 0); + file.set(7, stats); + + return file; + } +} From cc787c8c9ccf8895a1708338e15a3698c4f165a0 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 24 Apr 2026 07:38:12 -0700 Subject: [PATCH 02/18] Clean up tests --- .../iceberg/TestTrackedFileAdapters.java | 169 +++++++++--------- 1 file changed, 80 insertions(+), 89 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index 33c41dfb8dde..37441db3dbda 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -33,12 +33,9 @@ class TestTrackedFileAdapters { @Test void testAsDataFileValidatesContentType() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); @@ -49,12 +46,14 @@ void testAsDataFileValidatesContentType() { @Test void testAsDataFileRejectsNonData() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.EQUALITY_DELETES.id()); - file.set(2, "s3://bucket/delete.avro"); - file.set(3, "avro"); - file.set(4, 50L); - file.set(5, 512L); + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.EQUALITY_DELETES, + "s3://bucket/delete.avro", + FileFormat.AVRO, + 50L, + 512L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned())) @@ -66,12 +65,14 @@ void testAsDataFileRejectsNonData() { @Test void testAsDeleteFileValidatesContentType() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.EQUALITY_DELETES.id()); - file.set(2, "s3://bucket/eq-delete.avro"); - file.set(3, "avro"); - file.set(4, 50L); - file.set(5, 512L); + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.EQUALITY_DELETES, + "s3://bucket/eq-delete.avro", + FileFormat.AVRO, + 50L, + 512L); file.set(6, 0); file.set(13, ImmutableList.of(1, 2)); @@ -83,12 +84,9 @@ void testAsDeleteFileValidatesContentType() { @Test void testAsDeleteFileRejectsNonEqualityDeletes() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned())) @@ -100,7 +98,6 @@ void testAsDeleteFileRejectsNonEqualityDeletes() { @Test void testDataFileAdapterDelegatesAllFields() { - TrackedFileStruct file = new TrackedFileStruct(); Types.StructType trackingWithPos = Types.StructType.of( ImmutableList.builder() @@ -117,12 +114,14 @@ void testDataFileAdapterDelegatesAllFields() { tracking.setManifestLocation("s3://bucket/manifest.avro"); tracking.set(8, 3L); - file.set(0, tracking); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data/file.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); file.set(6, 0); file.set(8, 3); file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); @@ -150,7 +149,6 @@ void testDataFileAdapterDelegatesAllFields() { @Test void testDeleteFileAdapterDelegatesAllFields() { - TrackedFileStruct file = new TrackedFileStruct(); Types.StructType trackingWithPos = Types.StructType.of( ImmutableList.builder() @@ -167,12 +165,14 @@ void testDeleteFileAdapterDelegatesAllFields() { tracking.setManifestLocation("s3://bucket/manifest.avro"); tracking.set(8, 5L); - file.set(0, tracking); - file.set(1, FileContent.EQUALITY_DELETES.id()); - file.set(2, "s3://bucket/eq-delete.avro"); - file.set(3, "avro"); - file.set(4, 50L); - file.set(5, 512L); + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.EQUALITY_DELETES, + "s3://bucket/eq-delete.avro", + FileFormat.AVRO, + 50L, + 512L); file.set(6, 1); file.set(8, 5); file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); @@ -201,12 +201,9 @@ void testDeleteFileAdapterDelegatesAllFields() { @Test void testAdapterDelegatesNullTracking() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); @@ -257,12 +254,9 @@ void testDeleteFileAdapterStatsFromContentStats() { @Test void testDataFileAdapterStatsNullWhenNoContentStats() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); @@ -300,13 +294,13 @@ void testPartitionExtractedWithYearTransform() { PartitionSpec spec = PartitionSpec.builderFor(schema).year("ts").build(); - // date value 18628 = 2021-01-01 (days since epoch) - TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.DateType.get(), 18628); + // date value 20546 = 2026-04-03 (days since epoch) + TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.DateType.get(), 20546); DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); - assertThat(partition.get(0, Integer.class)).isEqualTo(51); + assertThat(partition.get(0, Integer.class)).isEqualTo(56); } @Test @@ -335,12 +329,9 @@ void testPartitionNullWhenNoContentStats() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, spec.specId()); DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); @@ -412,12 +403,9 @@ void testPartitionWithMultipleFields() { .withFieldStats(fieldStatsList) .build(); - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, spec.specId()); file.set(7, stats); @@ -469,12 +457,9 @@ void testDeleteFilePartitionExtracted() { @Test void testSpecIdDefaultsToZeroWhenNull() { - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); assertThat(dataFile.specId()).isEqualTo(0); @@ -511,12 +496,14 @@ private static TrackedFileStruct createTrackedFileWithPartitionStats(PartitionSp .withFieldStats(fieldStatsList) .build(); - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data/file.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); file.set(6, spec.specId()); file.set(7, stats); @@ -552,12 +539,14 @@ private static TrackedFileStruct createTrackedFileWithFieldStats( .withFieldStats(fieldStatsList) .build(); - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data/file.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); file.set(6, 0); file.set(7, stats); @@ -615,12 +604,14 @@ private static TrackedFileStruct createTrackedFileWithStats() { .withFieldStats(fieldStatsList) .build(); - TrackedFileStruct file = new TrackedFileStruct(); - file.set(1, FileContent.DATA.id()); - file.set(2, "s3://bucket/data/file.parquet"); - file.set(3, "parquet"); - file.set(4, 100L); - file.set(5, 1024L); + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); file.set(6, 0); file.set(7, stats); From a5c13fbf40bcb6a369083ae46a313785cd465b46 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 28 Apr 2026 09:47:58 -0700 Subject: [PATCH 03/18] Change design such that a DV adapted to DeleteFile --- .../apache/iceberg/TrackedFileAdapters.java | 187 +++++++++++++++++- .../iceberg/TestTrackedFileAdapters.java | 154 +++++++++++++++ 2 files changed, 337 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 295f90a67e9a..98a3426b1db5 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -31,10 +31,11 @@ /** * Adapts {@link TrackedFile} entries to the {@link DataFile} and {@link DeleteFile} APIs. * - *

Note: V4 colocates deletion vectors with data file entries in {@link TrackedFile}. This - * adapter does not carry over {@link TrackedFile#deletionVector()} because {@link DataFile} has no - * way to represent it. Once {@link DataFile} is extended with deletion vector support, this adapter - * should be updated to include it. + *

V4 colocates deletion vectors with data file entries in {@link TrackedFile}. Rather than + * extending {@link DataFile} with deletion vector fields, DVs are extracted as separate {@link + * DeleteFile} objects via {@link #asDVDeleteFile(TrackedFile, PartitionSpec)}. This matches the v3 + * convention where DVs are tracked as {@link DeleteFile} entries in delete manifests and keeps the + * existing {@link FileScanTask} contract ({@code file()} + {@code deletes()}) unchanged. */ class TrackedFileAdapters { @@ -56,6 +57,16 @@ static DeleteFile asDeleteFile(TrackedFile file, PartitionSpec spec) { return new TrackedDeleteFile(file, spec); } + static DeleteFile asDVDeleteFile(TrackedFile file, PartitionSpec spec) { + Preconditions.checkState( + file.contentType() == FileContent.DATA, + "Cannot extract DV from tracked file: content type is %s, not DATA", + file.contentType()); + Preconditions.checkState( + file.deletionVector() != null, "Cannot extract DV from tracked file: no deletion vector"); + return new TrackedDVDeleteFile(file, spec); + } + // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition // schema), replacing this transform-based derivation. Once that lands, this method should be // removed and the adapter should read the tuple directly. @@ -458,4 +469,172 @@ public DeleteFile copyWithStats(Set requestedColumnIds) { return this; } } + + /** + * Adapts the deletion vector from a TrackedFile DATA entry to the {@link DeleteFile} interface. + * + *

The DV blob metadata is mapped to the DeleteFile DV fields: {@link + * DeleteFile#referencedDataFile()} is the data file location, and {@link + * DeleteFile#contentOffset()} / {@link DeleteFile#contentSizeInBytes()} point to the blob within + * the Puffin file. + */ + private static class TrackedDVDeleteFile implements DeleteFile { + private final TrackedFile file; + private final DeletionVector dv; + private final Tracking tracking; + private final PartitionSpec spec; + + private TrackedDVDeleteFile(TrackedFile file, PartitionSpec spec) { + this.file = file; + this.dv = file.deletionVector(); + this.tracking = file.tracking(); + this.spec = spec; + } + + @Override + public Long pos() { + return tracking != null ? tracking.manifestPos() : null; + } + + @Override + public int specId() { + return file.specId() != null ? file.specId() : 0; + } + + @Override + public FileContent content() { + return FileContent.POSITION_DELETES; + } + + @SuppressWarnings("deprecation") + @Override + public CharSequence path() { + return dv.location(); + } + + @Override + public FileFormat format() { + return FileFormat.PUFFIN; + } + + @Override + public StructLike partition() { + return extractPartition(file, spec); + } + + @Override + public long recordCount() { + return dv.cardinality(); + } + + @Override + public long fileSizeInBytes() { + return dv.sizeInBytes(); + } + + @Override + public Integer sortOrderId() { + return null; + } + + @Override + public Long dataSequenceNumber() { + return tracking != null ? tracking.dataSequenceNumber() : null; + } + + @Override + public Long fileSequenceNumber() { + return tracking != null ? tracking.fileSequenceNumber() : null; + } + + @Override + public Long firstRowId() { + return null; + } + + @Override + public ByteBuffer keyMetadata() { + return null; + } + + @Override + public List splitOffsets() { + return null; + } + + @Override + public List equalityFieldIds() { + return null; + } + + @Override + public String referencedDataFile() { + return file.location(); + } + + @Override + public Long contentOffset() { + return dv.offset(); + } + + @Override + public Long contentSizeInBytes() { + return dv.sizeInBytes(); + } + + @Override + public String manifestLocation() { + return tracking != null ? tracking.manifestLocation() : null; + } + + @Override + public Map columnSizes() { + return null; + } + + @Override + public Map valueCounts() { + return null; + } + + @Override + public Map nullValueCounts() { + return null; + } + + @Override + public Map nanValueCounts() { + return null; + } + + @Override + public Map lowerBounds() { + return null; + } + + @Override + public Map upperBounds() { + return null; + } + + @Override + public DeleteFile copy() { + return this; + } + + @Override + public DeleteFile copy(boolean withStats) { + return this; + } + + @Override + public DeleteFile copyWithoutStats() { + return this; + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return this; + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index 37441db3dbda..1a33de77276a 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -96,6 +96,151 @@ void testAsDeleteFileRejectsNonEqualityDeletes() { FileContent.DATA); } + @Test + void testAsDVDeleteFileValidatesContentType() { + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + file.set(6, 0); + file.set(9, createDeletionVector()); + + DeleteFile dv = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + assertThat(dv).isNotNull(); + assertThat(dv.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(dv.format()).isEqualTo(FileFormat.PUFFIN); + } + + @Test + void testAsDVDeleteFileRejectsNonData() { + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.EQUALITY_DELETES, + "s3://bucket/eq-delete.avro", + FileFormat.AVRO, + 50L, + 512L); + file.set(6, 0); + file.set(9, createDeletionVector()); + + assertThatThrownBy( + () -> TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned())) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot extract DV from tracked file: content type is %s, not DATA", + FileContent.EQUALITY_DELETES); + } + + @Test + void testAsDVDeleteFileRejectsNullDV() { + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + file.set(6, 0); + + assertThatThrownBy( + () -> TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned())) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot extract DV from tracked file: no deletion vector"); + } + + @Test + void testDVDeleteFileAdapterDelegatesAllFields() { + Types.StructType trackingWithPos = + Types.StructType.of( + ImmutableList.builder() + .addAll(Tracking.schema().fields()) + .add(MetadataColumns.ROW_POSITION) + .build()); + TrackingStruct tracking = new TrackingStruct(trackingWithPos); + + tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(1, 42L); + tracking.set(2, 10L); + tracking.set(3, 11L); + tracking.set(5, 1000L); + tracking.setManifestLocation("s3://bucket/manifest.avro"); + tracking.set(8, 7L); + + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); + file.set(6, 2); + file.set(9, createDeletionVector()); + + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + + // DV-specific fields from DeletionVector + assertThat(dvFile.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(dvFile.location()).isEqualTo("s3://bucket/puffin/dv-file.bin"); + assertThat(dvFile.format()).isEqualTo(FileFormat.PUFFIN); + assertThat(dvFile.recordCount()).isEqualTo(10L); + assertThat(dvFile.fileSizeInBytes()).isEqualTo(256L); + assertThat(dvFile.referencedDataFile()).isEqualTo("s3://bucket/data/file.parquet"); + assertThat(dvFile.contentOffset()).isEqualTo(128L); + assertThat(dvFile.contentSizeInBytes()).isEqualTo(256L); + + // fields delegated from TrackedFile / Tracking + assertThat(dvFile.pos()).isEqualTo(7L); + assertThat(dvFile.specId()).isEqualTo(2); + assertThat(dvFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(dvFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(dvFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + + // fields that should be null for DVs + assertThat(dvFile.sortOrderId()).isNull(); + assertThat(dvFile.firstRowId()).isNull(); + assertThat(dvFile.keyMetadata()).isNull(); + assertThat(dvFile.splitOffsets()).isNull(); + assertThat(dvFile.equalityFieldIds()).isNull(); + assertThat(dvFile.columnSizes()).isNull(); + assertThat(dvFile.valueCounts()).isNull(); + assertThat(dvFile.nullValueCounts()).isNull(); + assertThat(dvFile.nanValueCounts()).isNull(); + assertThat(dvFile.lowerBounds()).isNull(); + assertThat(dvFile.upperBounds()).isNull(); + } + + @Test + void testDVDeleteFileAdapterDelegatesNullTracking() { + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + file.set(6, 0); + file.set(9, createDeletionVector()); + + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + + assertThat(dvFile.dataSequenceNumber()).isNull(); + assertThat(dvFile.fileSequenceNumber()).isNull(); + assertThat(dvFile.manifestLocation()).isNull(); + assertThat(dvFile.pos()).isNull(); + } + + @Test + void testDVDeleteFilePartitionExtracted() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "category", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); + + TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); + file.set(9, createDeletionVector()); + + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, spec); + + StructLike partition = dvFile.partition(); + assertThat(partition).isNotNull(); + assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); + } + @Test void testDataFileAdapterDelegatesAllFields() { Types.StructType trackingWithPos = @@ -465,6 +610,15 @@ void testSpecIdDefaultsToZeroWhenNull() { assertThat(dataFile.specId()).isEqualTo(0); } + private static DeletionVectorStruct createDeletionVector() { + DeletionVectorStruct dv = new DeletionVectorStruct(DeletionVector.schema()); + dv.set(0, "s3://bucket/puffin/dv-file.bin"); + dv.set(1, 128L); + dv.set(2, 256L); + dv.set(3, 10L); + return dv; + } + private static java.util.Map.Entry entry(int key, long value) { return java.util.Map.entry(key, value); } From c5dadc262cbb9834989b96538b5ac37a6bb1329e Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 28 Apr 2026 10:07:08 -0700 Subject: [PATCH 04/18] Make copy safe --- .../apache/iceberg/TrackedFileAdapters.java | 37 ++++++++++++------- .../iceberg/TestTrackedFileAdapters.java | 16 +++++--- 2 files changed, 33 insertions(+), 20 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 98a3426b1db5..d9a7addf1288 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -70,15 +70,20 @@ static DeleteFile asDVDeleteFile(TrackedFile file, PartitionSpec spec) { // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition // schema), replacing this transform-based derivation. Once that lands, this method should be // removed and the adapter should read the tuple directly. + // + // This derives partition values by applying the partition transform to the lower bound of the + // source column stats. This is correct because each data file belongs to exactly one partition, + // so lower == upper for partition source columns. For non-identity transforms (bucket, truncate), + // the transform of the lower bound produces the correct partition value under this invariant. @SuppressWarnings({"unchecked", "rawtypes"}) static StructLike extractPartition(TrackedFile file, PartitionSpec spec) { if (spec == null || spec.isUnpartitioned()) { - return null; + return BaseFile.EMPTY_PARTITION_DATA; } ContentStats stats = file.contentStats(); if (stats == null) { - return null; + return new PartitionData(spec.partitionType()); } PartitionData partition = new PartitionData(spec.partitionType()); @@ -306,22 +311,22 @@ public Map upperBounds() { @Override public DataFile copy() { - return this; + return new TrackedDataFile(file.copy(), spec); } @Override public DataFile copy(boolean withStats) { - return this; + return withStats ? copy() : copyWithoutStats(); } @Override public DataFile copyWithoutStats() { - return this; + return new TrackedDataFile(file.copyWithoutStats(), spec); } @Override public DataFile copyWithStats(Set requestedColumnIds) { - return this; + return new TrackedDataFile(file.copyWithStats(requestedColumnIds), spec); } } @@ -451,22 +456,22 @@ public Map upperBounds() { @Override public DeleteFile copy() { - return this; + return new TrackedDeleteFile(file.copy(), spec); } @Override public DeleteFile copy(boolean withStats) { - return this; + return withStats ? copy() : copyWithoutStats(); } @Override public DeleteFile copyWithoutStats() { - return this; + return new TrackedDeleteFile(file.copyWithoutStats(), spec); } @Override public DeleteFile copyWithStats(Set requestedColumnIds) { - return this; + return new TrackedDeleteFile(file.copyWithStats(requestedColumnIds), spec); } } @@ -527,6 +532,10 @@ public long recordCount() { return dv.cardinality(); } + // Returns the DV blob size, not the full Puffin file size. The DeletionVector metadata does not + // include the Puffin file size, so this is the best approximation available. Space accounting + // that sums fileSizeInBytes() was already imprecise in v3 (multiple DVs sharing a Puffin file + // each reported the full file size). @Override public long fileSizeInBytes() { return dv.sizeInBytes(); @@ -619,22 +628,22 @@ public Map upperBounds() { @Override public DeleteFile copy() { - return this; + return new TrackedDVDeleteFile(file.copy(), spec); } @Override public DeleteFile copy(boolean withStats) { - return this; + return copy(); } @Override public DeleteFile copyWithoutStats() { - return this; + return copy(); } @Override public DeleteFile copyWithStats(Set requestedColumnIds) { - return this; + return copy(); } } } diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index 1a33de77276a..e432ce2b9d5d 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -469,7 +469,7 @@ void testPartitionExtractedWithBucketTransform() { } @Test - void testPartitionNullWhenNoContentStats() { + void testPartitionEmptyWhenNoContentStats() { Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); @@ -480,23 +480,27 @@ void testPartitionNullWhenNoContentStats() { file.set(6, spec.specId()); DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); - assertThat(dataFile.partition()).isNull(); + assertThat(dataFile.partition()).isNotNull(); + assertThat(dataFile.partition().size()).isEqualTo(1); + assertThat(dataFile.partition().get(0, Integer.class)).isNull(); } @Test - void testPartitionNullWhenNullSpec() { + void testPartitionEmptyWhenNullSpec() { TrackedFileStruct file = createTrackedFileWithStats(); DataFile dataFile = TrackedFileAdapters.asDataFile(file, null); - assertThat(dataFile.partition()).isNull(); + assertThat(dataFile.partition()).isNotNull(); + assertThat(dataFile.partition().size()).isEqualTo(0); } @Test - void testPartitionNullForUnpartitioned() { + void testPartitionEmptyForUnpartitioned() { PartitionSpec spec = PartitionSpec.unpartitioned(); TrackedFileStruct file = createTrackedFileWithStats(); DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); - assertThat(dataFile.partition()).isNull(); + assertThat(dataFile.partition()).isNotNull(); + assertThat(dataFile.partition().size()).isEqualTo(0); } @Test From 06f0cb10e981e023934232ddb328323808fcd498 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 28 Apr 2026 10:13:12 -0700 Subject: [PATCH 05/18] Reorder --- .../apache/iceberg/TrackedFileAdapters.java | 16 ++++++------- .../iceberg/TestTrackedFileAdapters.java | 24 +++++++++++-------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index d9a7addf1288..a28dd872a713 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -49,14 +49,6 @@ static DataFile asDataFile(TrackedFile file, PartitionSpec spec) { return new TrackedDataFile(file, spec); } - static DeleteFile asDeleteFile(TrackedFile file, PartitionSpec spec) { - Preconditions.checkState( - file.contentType() == FileContent.EQUALITY_DELETES, - "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", - file.contentType()); - return new TrackedDeleteFile(file, spec); - } - static DeleteFile asDVDeleteFile(TrackedFile file, PartitionSpec spec) { Preconditions.checkState( file.contentType() == FileContent.DATA, @@ -67,6 +59,14 @@ static DeleteFile asDVDeleteFile(TrackedFile file, PartitionSpec spec) { return new TrackedDVDeleteFile(file, spec); } + static DeleteFile asEqualityDeleteFile(TrackedFile file, PartitionSpec spec) { + Preconditions.checkState( + file.contentType() == FileContent.EQUALITY_DELETES, + "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", + file.contentType()); + return new TrackedDeleteFile(file, spec); + } + // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition // schema), replacing this transform-based derivation. Once that lands, this method should be // removed and the adapter should read the tuple directly. diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index e432ce2b9d5d..b5f19dbbe920 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -64,7 +64,7 @@ void testAsDataFileRejectsNonData() { } @Test - void testAsDeleteFileValidatesContentType() { + void testAsEqualityDeleteFileValidatesContentType() { TrackedFileStruct file = new TrackedFileStruct( null, @@ -76,20 +76,22 @@ void testAsDeleteFileValidatesContentType() { file.set(6, 0); file.set(13, ImmutableList.of(1, 2)); - DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = + TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); assertThat(deleteFile).isNotNull(); assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2); } @Test - void testAsDeleteFileRejectsNonEqualityDeletes() { + void testAsEqualityDeleteFileRejectsNonEqualityDeletes() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - assertThatThrownBy(() -> TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned())) + assertThatThrownBy( + () -> TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned())) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", @@ -293,7 +295,7 @@ void testDataFileAdapterDelegatesAllFields() { } @Test - void testDeleteFileAdapterDelegatesAllFields() { + void testEqualityDeleteFileAdapterDelegatesAllFields() { Types.StructType trackingWithPos = Types.StructType.of( ImmutableList.builder() @@ -324,7 +326,8 @@ void testDeleteFileAdapterDelegatesAllFields() { file.set(12, ImmutableList.of(200L)); file.set(13, ImmutableList.of(1, 2, 3)); - DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = + TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); assertThat(deleteFile.pos()).isEqualTo(5L); assertThat(deleteFile.specId()).isEqualTo(1); @@ -378,12 +381,13 @@ void testDataFileAdapterStatsFromContentStats() { } @Test - void testDeleteFileAdapterStatsFromContentStats() { + void testEqualityDeleteFileAdapterStatsFromContentStats() { TrackedFileStruct file = createTrackedFileWithStats(); file.set(1, FileContent.EQUALITY_DELETES.id()); file.set(13, ImmutableList.of(1)); - DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = + TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); @@ -585,7 +589,7 @@ void testPartitionWithVoidTransform() { } @Test - void testDeleteFilePartitionExtracted() { + void testEqualityDeleteFilePartitionExtracted() { Schema schema = new Schema( Types.NestedField.required(1, "id", Types.IntegerType.get()), @@ -597,7 +601,7 @@ void testDeleteFilePartitionExtracted() { file.set(1, FileContent.EQUALITY_DELETES.id()); file.set(13, ImmutableList.of(1)); - DeleteFile deleteFile = TrackedFileAdapters.asDeleteFile(file, spec); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, spec); StructLike partition = deleteFile.partition(); assertThat(partition).isNotNull(); From dffc00316b4321d45050c345d6699d6636fd02f0 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 5 May 2026 11:22:26 -0700 Subject: [PATCH 06/18] Feedback from Ryan --- .../apache/iceberg/TrackedFileAdapters.java | 227 +++++++----------- .../iceberg/TestTrackedFileAdapters.java | 79 +++--- 2 files changed, 126 insertions(+), 180 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index a28dd872a713..ad9c852afdae 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -33,38 +34,51 @@ * *

V4 colocates deletion vectors with data file entries in {@link TrackedFile}. Rather than * extending {@link DataFile} with deletion vector fields, DVs are extracted as separate {@link - * DeleteFile} objects via {@link #asDVDeleteFile(TrackedFile, PartitionSpec)}. This matches the v3 - * convention where DVs are tracked as {@link DeleteFile} entries in delete manifests and keeps the - * existing {@link FileScanTask} contract ({@code file()} + {@code deletes()}) unchanged. + * DeleteFile} objects via {@link #asDVDeleteFile(TrackedFile, Map)}. This matches the v3 convention + * where DVs are tracked as {@link DeleteFile} entries in delete manifests and keeps the existing + * {@link FileScanTask} contract ({@code file()} + {@code deletes()}) unchanged. */ class TrackedFileAdapters { private TrackedFileAdapters() {} - static DataFile asDataFile(TrackedFile file, PartitionSpec spec) { + static DataFile asDataFile(TrackedFile file, Map specsById) { Preconditions.checkState( file.contentType() == FileContent.DATA, "Cannot convert tracked file to DataFile: content type is %s, not DATA", file.contentType()); - return new TrackedDataFile(file, spec); + return new TrackedDataFile(file, resolveSpec(file, specsById)); } - static DeleteFile asDVDeleteFile(TrackedFile file, PartitionSpec spec) { + static DeleteFile asDVDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkState( file.contentType() == FileContent.DATA, "Cannot extract DV from tracked file: content type is %s, not DATA", file.contentType()); Preconditions.checkState( file.deletionVector() != null, "Cannot extract DV from tracked file: no deletion vector"); - return new TrackedDVDeleteFile(file, spec); + return new TrackedDVDeleteFile(file, resolveSpec(file, specsById)); } - static DeleteFile asEqualityDeleteFile(TrackedFile file, PartitionSpec spec) { + static DeleteFile asEqualityDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkState( file.contentType() == FileContent.EQUALITY_DELETES, "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", file.contentType()); - return new TrackedDeleteFile(file, spec); + return new TrackedDeleteFile(file, resolveSpec(file, specsById)); + } + + private static PartitionSpec resolveSpec( + TrackedFile file, Map specsById) { + Integer specId = file.specId(); + if (specId != null) { + PartitionSpec spec = specsById.get(specId); + Preconditions.checkArgument( + spec != null, "Cannot find partition spec for spec ID: %s", specId); + return spec; + } + + return PartitionSpec.unpartitioned(); } // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition @@ -77,7 +91,7 @@ static DeleteFile asEqualityDeleteFile(TrackedFile file, PartitionSpec spec) { // the transform of the lower bound produces the correct partition value under this invariant. @SuppressWarnings({"unchecked", "rawtypes"}) static StructLike extractPartition(TrackedFile file, PartitionSpec spec) { - if (spec == null || spec.isUnpartitioned()) { + if (spec.isUnpartitioned()) { return BaseFile.EMPTY_PARTITION_DATA; } @@ -122,7 +136,7 @@ static Map valueCounts(ContentStats stats) { } } - return result.isEmpty() ? null : result; + return result.isEmpty() ? null : Collections.unmodifiableMap(result); } static Map nullValueCounts(ContentStats stats) { @@ -137,7 +151,7 @@ static Map nullValueCounts(ContentStats stats) { } } - return result.isEmpty() ? null : result; + return result.isEmpty() ? null : Collections.unmodifiableMap(result); } static Map nanValueCounts(ContentStats stats) { @@ -152,7 +166,7 @@ static Map nanValueCounts(ContentStats stats) { } } - return result.isEmpty() ? null : result; + return result.isEmpty() ? null : Collections.unmodifiableMap(result); } static Map lowerBounds(ContentStats stats) { @@ -167,7 +181,7 @@ static Map lowerBounds(ContentStats stats) { } } - return result.isEmpty() ? null : result; + return result.isEmpty() ? null : Collections.unmodifiableMap(result); } static Map upperBounds(ContentStats stats) { @@ -182,16 +196,22 @@ static Map upperBounds(ContentStats stats) { } } - return result.isEmpty() ? null : result; + return result.isEmpty() ? null : Collections.unmodifiableMap(result); } - /** Adapts a TrackedFile DATA entry to the {@link DataFile} interface. */ - private static class TrackedDataFile implements DataFile { - private final TrackedFile file; - private final Tracking tracking; - private final PartitionSpec spec; + /** + * Shared base for adapters that delegate to a {@link TrackedFile} for content file fields. + * + *

Subclasses provide {@code content()}, {@code firstRowId()}, {@code equalityFieldIds()}, and + * the copy methods. + */ + private abstract static class AbstractTrackedContentFile> + implements ContentFile { + protected final TrackedFile file; + protected final Tracking tracking; + protected final PartitionSpec spec; - private TrackedDataFile(TrackedFile file, PartitionSpec spec) { + private AbstractTrackedContentFile(TrackedFile file, PartitionSpec spec) { this.file = file; this.tracking = file.tracking(); this.spec = spec; @@ -204,18 +224,17 @@ public Long pos() { @Override public int specId() { - // null specId in v4 means unpartitioned; default to 0 to match PartitionSpec.unpartitioned() - return file.specId() != null ? file.specId() : 0; + return spec.specId(); } + @SuppressWarnings("deprecation") @Override - public FileContent content() { - return FileContent.DATA; + public CharSequence path() { + return file.location(); } - @SuppressWarnings("deprecation") @Override - public CharSequence path() { + public String location() { return file.location(); } @@ -254,11 +273,6 @@ public Long fileSequenceNumber() { return tracking != null ? tracking.fileSequenceNumber() : null; } - @Override - public Long firstRowId() { - return tracking != null ? tracking.firstRowId() : null; - } - @Override public ByteBuffer keyMetadata() { return file.keyMetadata(); @@ -269,11 +283,6 @@ public List splitOffsets() { return file.splitOffsets(); } - @Override - public List equalityFieldIds() { - return null; - } - @Override public String manifestLocation() { return tracking != null ? tracking.manifestLocation() : null; @@ -308,110 +317,66 @@ public Map lowerBounds() { public Map upperBounds() { return TrackedFileAdapters.upperBounds(file.contentStats()); } - - @Override - public DataFile copy() { - return new TrackedDataFile(file.copy(), spec); - } - - @Override - public DataFile copy(boolean withStats) { - return withStats ? copy() : copyWithoutStats(); - } - - @Override - public DataFile copyWithoutStats() { - return new TrackedDataFile(file.copyWithoutStats(), spec); - } - - @Override - public DataFile copyWithStats(Set requestedColumnIds) { - return new TrackedDataFile(file.copyWithStats(requestedColumnIds), spec); - } } - /** Adapts a TrackedFile EQUALITY_DELETES entry to the {@link DeleteFile} interface. */ - private static class TrackedDeleteFile implements DeleteFile { - private final TrackedFile file; - private final Tracking tracking; - private final PartitionSpec spec; - - private TrackedDeleteFile(TrackedFile file, PartitionSpec spec) { - this.file = file; - this.tracking = file.tracking(); - this.spec = spec; - } - - @Override - public Long pos() { - return tracking != null ? tracking.manifestPos() : null; - } - - @Override - public int specId() { - // null specId in v4 means unpartitioned; default to 0 to match PartitionSpec.unpartitioned() - return file.specId() != null ? file.specId() : 0; + /** Adapts a TrackedFile DATA entry to the {@link DataFile} interface. */ + private static class TrackedDataFile extends AbstractTrackedContentFile + implements DataFile { + private TrackedDataFile(TrackedFile file, PartitionSpec spec) { + super(file, spec); } @Override public FileContent content() { - return FileContent.EQUALITY_DELETES; + return FileContent.DATA; } - @SuppressWarnings("deprecation") @Override - public CharSequence path() { - return file.location(); + public Long firstRowId() { + return tracking != null ? tracking.firstRowId() : null; } @Override - public FileFormat format() { - return file.fileFormat(); + public List equalityFieldIds() { + return null; } @Override - public StructLike partition() { - return extractPartition(file, spec); + public DataFile copy() { + return new TrackedDataFile(file.copy(), spec); } @Override - public long recordCount() { - return file.recordCount(); + public DataFile copy(boolean withStats) { + return withStats ? copy() : copyWithoutStats(); } @Override - public long fileSizeInBytes() { - return file.fileSizeInBytes(); + public DataFile copyWithoutStats() { + return new TrackedDataFile(file.copyWithoutStats(), spec); } @Override - public Integer sortOrderId() { - return file.sortOrderId(); + public DataFile copyWithStats(Set requestedColumnIds) { + return new TrackedDataFile(file.copyWithStats(requestedColumnIds), spec); } + } - @Override - public Long dataSequenceNumber() { - return tracking != null ? tracking.dataSequenceNumber() : null; + /** Adapts a TrackedFile EQUALITY_DELETES entry to the {@link DeleteFile} interface. */ + private static class TrackedDeleteFile extends AbstractTrackedContentFile + implements DeleteFile { + private TrackedDeleteFile(TrackedFile file, PartitionSpec spec) { + super(file, spec); } @Override - public Long fileSequenceNumber() { - return tracking != null ? tracking.fileSequenceNumber() : null; + public FileContent content() { + return FileContent.EQUALITY_DELETES; } @Override public Long firstRowId() { - return tracking != null ? tracking.firstRowId() : null; - } - - @Override - public ByteBuffer keyMetadata() { - return file.keyMetadata(); - } - - @Override - public List splitOffsets() { - return file.splitOffsets(); + return null; } @Override @@ -419,41 +384,6 @@ public List equalityFieldIds() { return file.equalityIds(); } - @Override - public String manifestLocation() { - return tracking != null ? tracking.manifestLocation() : null; - } - - @Override - public Map columnSizes() { - return null; - } - - @Override - public Map valueCounts() { - return TrackedFileAdapters.valueCounts(file.contentStats()); - } - - @Override - public Map nullValueCounts() { - return TrackedFileAdapters.nullValueCounts(file.contentStats()); - } - - @Override - public Map nanValueCounts() { - return TrackedFileAdapters.nanValueCounts(file.contentStats()); - } - - @Override - public Map lowerBounds() { - return TrackedFileAdapters.lowerBounds(file.contentStats()); - } - - @Override - public Map upperBounds() { - return TrackedFileAdapters.upperBounds(file.contentStats()); - } - @Override public DeleteFile copy() { return new TrackedDeleteFile(file.copy(), spec); @@ -490,6 +420,8 @@ private static class TrackedDVDeleteFile implements DeleteFile { private final PartitionSpec spec; private TrackedDVDeleteFile(TrackedFile file, PartitionSpec spec) { + Preconditions.checkArgument( + file.deletionVector() != null, "Cannot create DV delete file: no deletion vector"); this.file = file; this.dv = file.deletionVector(); this.tracking = file.tracking(); @@ -503,7 +435,7 @@ public Long pos() { @Override public int specId() { - return file.specId() != null ? file.specId() : 0; + return spec.specId(); } @Override @@ -517,6 +449,11 @@ public CharSequence path() { return dv.location(); } + @Override + public String location() { + return dv.location(); + } + @Override public FileFormat format() { return FileFormat.PUFFIN; @@ -541,6 +478,8 @@ public long fileSizeInBytes() { return dv.sizeInBytes(); } + // Position deletes are required to be sorted by file and position, not a table order, and + // should set sort order id to null @Override public Integer sortOrderId() { return null; diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index b5f19dbbe920..8fe542875b98 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -23,7 +23,9 @@ import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; 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.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,6 +33,13 @@ class TestTrackedFileAdapters { + private static final Map UNPARTITIONED = + ImmutableMap.of(0, PartitionSpec.unpartitioned()); + + private static Map specsById(PartitionSpec spec) { + return ImmutableMap.of(spec.specId(), spec); + } + @Test void testAsDataFileValidatesContentType() { TrackedFileStruct file = @@ -38,7 +47,7 @@ void testAsDataFileValidatesContentType() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile).isNotNull(); assertThat(dataFile.content()).isEqualTo(FileContent.DATA); assertThat(dataFile.location()).isEqualTo("s3://bucket/data.parquet"); @@ -56,7 +65,7 @@ void testAsDataFileRejectsNonData() { 512L); file.set(6, 0); - assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, UNPARTITIONED)) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot convert tracked file to DataFile: content type is %s, not DATA", @@ -76,8 +85,7 @@ void testAsEqualityDeleteFileValidatesContentType() { file.set(6, 0); file.set(13, ImmutableList.of(1, 2)); - DeleteFile deleteFile = - TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED); assertThat(deleteFile).isNotNull(); assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2); @@ -90,8 +98,7 @@ void testAsEqualityDeleteFileRejectsNonEqualityDeletes() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - assertThatThrownBy( - () -> TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", @@ -106,7 +113,7 @@ void testAsDVDeleteFileValidatesContentType() { file.set(6, 0); file.set(9, createDeletionVector()); - DeleteFile dv = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile dv = TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED); assertThat(dv).isNotNull(); assertThat(dv.content()).isEqualTo(FileContent.POSITION_DELETES); assertThat(dv.format()).isEqualTo(FileFormat.PUFFIN); @@ -125,8 +132,7 @@ void testAsDVDeleteFileRejectsNonData() { file.set(6, 0); file.set(9, createDeletionVector()); - assertThatThrownBy( - () -> TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot extract DV from tracked file: content type is %s, not DATA", @@ -140,8 +146,7 @@ void testAsDVDeleteFileRejectsNullDV() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - assertThatThrownBy( - () -> TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot extract DV from tracked file: no deletion vector"); } @@ -164,6 +169,8 @@ void testDVDeleteFileAdapterDelegatesAllFields() { tracking.setManifestLocation("s3://bucket/manifest.avro"); tracking.set(8, 7L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(2).build(); + TrackedFileStruct file = new TrackedFileStruct( tracking, @@ -175,7 +182,7 @@ void testDVDeleteFileAdapterDelegatesAllFields() { file.set(6, 2); file.set(9, createDeletionVector()); - DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, specsById(spec)); // DV-specific fields from DeletionVector assertThat(dvFile.content()).isEqualTo(FileContent.POSITION_DELETES); @@ -216,7 +223,7 @@ void testDVDeleteFileAdapterDelegatesNullTracking() { file.set(6, 0); file.set(9, createDeletionVector()); - DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED); assertThat(dvFile.dataSequenceNumber()).isNull(); assertThat(dvFile.fileSequenceNumber()).isNull(); @@ -236,7 +243,7 @@ void testDVDeleteFilePartitionExtracted() { TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); file.set(9, createDeletionVector()); - DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, spec); + DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, specsById(spec)); StructLike partition = dvFile.partition(); assertThat(partition).isNotNull(); @@ -274,7 +281,7 @@ void testDataFileAdapterDelegatesAllFields() { file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); file.set(12, ImmutableList.of(50L, 100L)); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile.pos()).isEqualTo(3L); assertThat(dataFile.specId()).isEqualTo(0); @@ -312,6 +319,8 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { tracking.setManifestLocation("s3://bucket/manifest.avro"); tracking.set(8, 5L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + TrackedFileStruct file = new TrackedFileStruct( tracking, @@ -326,8 +335,7 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { file.set(12, ImmutableList.of(200L)); file.set(13, ImmutableList.of(1, 2, 3)); - DeleteFile deleteFile = - TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); assertThat(deleteFile.pos()).isEqualTo(5L); assertThat(deleteFile.specId()).isEqualTo(1); @@ -339,7 +347,7 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { assertThat(deleteFile.sortOrderId()).isEqualTo(5); assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); - assertThat(deleteFile.firstRowId()).isEqualTo(1000L); + assertThat(deleteFile.firstRowId()).isNull(); assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); assertThat(deleteFile.splitOffsets()).containsExactly(200L); assertThat(deleteFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); @@ -354,7 +362,7 @@ void testAdapterDelegatesNullTracking() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile.dataSequenceNumber()).isNull(); assertThat(dataFile.fileSequenceNumber()).isNull(); @@ -366,7 +374,7 @@ void testAdapterDelegatesNullTracking() { @Test void testDataFileAdapterStatsFromContentStats() { TrackedFileStruct file = createTrackedFileWithStats(); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); assertThat(dataFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); @@ -386,8 +394,7 @@ void testEqualityDeleteFileAdapterStatsFromContentStats() { file.set(1, FileContent.EQUALITY_DELETES.id()); file.set(13, ImmutableList.of(1)); - DeleteFile deleteFile = - TrackedFileAdapters.asEqualityDeleteFile(file, PartitionSpec.unpartitioned()); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED); assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); @@ -408,7 +415,7 @@ void testDataFileAdapterStatsNullWhenNoContentStats() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile.valueCounts()).isNull(); assertThat(dataFile.nullValueCounts()).isNull(); @@ -427,7 +434,7 @@ void testPartitionExtractedFromContentStatsWithIdentityTransform() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); @@ -445,7 +452,7 @@ void testPartitionExtractedWithYearTransform() { // date value 20546 = 2026-04-03 (days since epoch) TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.DateType.get(), 20546); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); @@ -462,7 +469,7 @@ void testPartitionExtractedWithBucketTransform() { PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 16).build(); TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.IntegerType.get(), 42); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); @@ -483,18 +490,18 @@ void testPartitionEmptyWhenNoContentStats() { null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, spec.specId()); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); assertThat(dataFile.partition()).isNotNull(); assertThat(dataFile.partition().size()).isEqualTo(1); assertThat(dataFile.partition().get(0, Integer.class)).isNull(); } @Test - void testPartitionEmptyWhenNullSpec() { + void testAsDataFileRejectsUnknownSpecId() { TrackedFileStruct file = createTrackedFileWithStats(); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, null); - assertThat(dataFile.partition()).isNotNull(); - assertThat(dataFile.partition().size()).isEqualTo(0); + assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot find partition spec for spec ID"); } @Test @@ -502,7 +509,7 @@ void testPartitionEmptyForUnpartitioned() { PartitionSpec spec = PartitionSpec.unpartitioned(); TrackedFileStruct file = createTrackedFileWithStats(); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); assertThat(dataFile.partition()).isNotNull(); assertThat(dataFile.partition().size()).isEqualTo(0); } @@ -562,7 +569,7 @@ void testPartitionWithMultipleFields() { file.set(6, spec.specId()); file.set(7, stats); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); @@ -580,7 +587,7 @@ void testPartitionWithVoidTransform() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").alwaysNull("data").build(); TrackedFileStruct file = createTrackedFileWithFieldStats(1, Types.IntegerType.get(), 42); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, spec); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); StructLike partition = dataFile.partition(); assertThat(partition).isNotNull(); @@ -601,7 +608,7 @@ void testEqualityDeleteFilePartitionExtracted() { file.set(1, FileContent.EQUALITY_DELETES.id()); file.set(13, ImmutableList.of(1)); - DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, spec); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); StructLike partition = deleteFile.partition(); assertThat(partition).isNotNull(); @@ -614,7 +621,7 @@ void testSpecIdDefaultsToZeroWhenNull() { new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, PartitionSpec.unpartitioned()); + DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); assertThat(dataFile.specId()).isEqualTo(0); } From 3cb9df1e96876843d6424c3537571a865a15af17 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 5 May 2026 14:44:58 -0700 Subject: [PATCH 07/18] fix checksytle error --- .../apache/iceberg/TrackedFileAdapters.java | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index ad9c852afdae..8f6db8c60e6e 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -207,9 +207,9 @@ static Map upperBounds(ContentStats stats) { */ private abstract static class AbstractTrackedContentFile> implements ContentFile { - protected final TrackedFile file; - protected final Tracking tracking; - protected final PartitionSpec spec; + private final TrackedFile file; + private final Tracking tracking; + private final PartitionSpec spec; private AbstractTrackedContentFile(TrackedFile file, PartitionSpec spec) { this.file = file; @@ -217,6 +217,18 @@ private AbstractTrackedContentFile(TrackedFile file, PartitionSpec spec) { this.spec = spec; } + TrackedFile file() { + return file; + } + + Tracking tracking() { + return tracking; + } + + PartitionSpec spec() { + return spec; + } + @Override public Long pos() { return tracking != null ? tracking.manifestPos() : null; @@ -333,7 +345,7 @@ public FileContent content() { @Override public Long firstRowId() { - return tracking != null ? tracking.firstRowId() : null; + return tracking() != null ? tracking().firstRowId() : null; } @Override @@ -343,7 +355,7 @@ public List equalityFieldIds() { @Override public DataFile copy() { - return new TrackedDataFile(file.copy(), spec); + return new TrackedDataFile(file().copy(), spec()); } @Override @@ -353,12 +365,12 @@ public DataFile copy(boolean withStats) { @Override public DataFile copyWithoutStats() { - return new TrackedDataFile(file.copyWithoutStats(), spec); + return new TrackedDataFile(file().copyWithoutStats(), spec()); } @Override public DataFile copyWithStats(Set requestedColumnIds) { - return new TrackedDataFile(file.copyWithStats(requestedColumnIds), spec); + return new TrackedDataFile(file().copyWithStats(requestedColumnIds), spec()); } } @@ -381,12 +393,12 @@ public Long firstRowId() { @Override public List equalityFieldIds() { - return file.equalityIds(); + return file().equalityIds(); } @Override public DeleteFile copy() { - return new TrackedDeleteFile(file.copy(), spec); + return new TrackedDeleteFile(file().copy(), spec()); } @Override @@ -396,12 +408,12 @@ public DeleteFile copy(boolean withStats) { @Override public DeleteFile copyWithoutStats() { - return new TrackedDeleteFile(file.copyWithoutStats(), spec); + return new TrackedDeleteFile(file().copyWithoutStats(), spec()); } @Override public DeleteFile copyWithStats(Set requestedColumnIds) { - return new TrackedDeleteFile(file.copyWithStats(requestedColumnIds), spec); + return new TrackedDeleteFile(file().copyWithStats(requestedColumnIds), spec()); } } From 5cf6b0baf2dde5db62a5a92db4282cd621298726 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 6 May 2026 11:54:14 -0700 Subject: [PATCH 08/18] Feedback from Eduard, Steven, Ryan --- .../java/org/apache/iceberg/MetricsUtil.java | 76 +++++ .../apache/iceberg/TrackedFileAdapters.java | 293 +++++++----------- .../iceberg/TestTrackedFileAdapters.java | 163 ++++++---- 3 files changed, 301 insertions(+), 231 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MetricsUtil.java b/core/src/main/java/org/apache/iceberg/MetricsUtil.java index 72c57a8bebcf..052a13bb88f4 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsUtil.java +++ b/core/src/main/java/org/apache/iceberg/MetricsUtil.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -479,6 +480,81 @@ public void set(int pos, T value) { } } + static Map valueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.valueCount() != null) { + result.put(fs.fieldId(), fs.valueCount()); + } + } + + return result.isEmpty() ? null : Collections.unmodifiableMap(result); + } + + static Map nullValueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.nullValueCount() != null) { + result.put(fs.fieldId(), fs.nullValueCount()); + } + } + + return result.isEmpty() ? null : Collections.unmodifiableMap(result); + } + + static Map nanValueCounts(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.nanValueCount() != null) { + result.put(fs.fieldId(), fs.nanValueCount()); + } + } + + return result.isEmpty() ? null : Collections.unmodifiableMap(result); + } + + static Map lowerBounds(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.lowerBound() != null && fs.type() != null) { + result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.lowerBound())); + } + } + + return result.isEmpty() ? null : Collections.unmodifiableMap(result); + } + + static Map upperBounds(ContentStats stats) { + if (stats == null) { + return null; + } + + Map result = Maps.newHashMap(); + for (FieldStats fs : stats.fieldStats()) { + if (fs != null && fs.upperBound() != null && fs.type() != null) { + result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.upperBound())); + } + } + + return result.isEmpty() ? null : Collections.unmodifiableMap(result); + } + static ContentStats fromMetrics(Schema schema, Metrics metrics) { if (null == metrics) { return null; diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 8f6db8c60e6e..51f641c45077 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -19,14 +19,11 @@ package org.apache.iceberg; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; /** @@ -43,7 +40,7 @@ class TrackedFileAdapters { private TrackedFileAdapters() {} static DataFile asDataFile(TrackedFile file, Map specsById) { - Preconditions.checkState( + Preconditions.checkArgument( file.contentType() == FileContent.DATA, "Cannot convert tracked file to DataFile: content type is %s, not DATA", file.contentType()); @@ -51,21 +48,27 @@ static DataFile asDataFile(TrackedFile file, Map specsBy } static DeleteFile asDVDeleteFile(TrackedFile file, Map specsById) { - Preconditions.checkState( + Preconditions.checkArgument( file.contentType() == FileContent.DATA, "Cannot extract DV from tracked file: content type is %s, not DATA", file.contentType()); - Preconditions.checkState( - file.deletionVector() != null, "Cannot extract DV from tracked file: no deletion vector"); return new TrackedDVDeleteFile(file, resolveSpec(file, specsById)); } + static DeleteFile asPositionDeleteFile(TrackedFile file, Map specsById) { + Preconditions.checkArgument( + file.contentType() == FileContent.POSITION_DELETES, + "Cannot convert tracked file to DeleteFile: content type is %s, not POSITION_DELETES", + file.contentType()); + return new TrackedPositionDeleteFile(file, resolveSpec(file, specsById)); + } + static DeleteFile asEqualityDeleteFile(TrackedFile file, Map specsById) { - Preconditions.checkState( + Preconditions.checkArgument( file.contentType() == FileContent.EQUALITY_DELETES, "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", file.contentType()); - return new TrackedDeleteFile(file, resolveSpec(file, specsById)); + return new TrackedEqualityDeleteFile(file, resolveSpec(file, specsById)); } private static PartitionSpec resolveSpec( @@ -124,108 +127,31 @@ static StructLike extractPartition(TrackedFile file, PartitionSpec spec) { return partition; } - static Map valueCounts(ContentStats stats) { - if (stats == null) { - return null; - } - - Map result = Maps.newHashMap(); - for (FieldStats fs : stats.fieldStats()) { - if (fs != null && fs.valueCount() != null) { - result.put(fs.fieldId(), fs.valueCount()); - } - } - - return result.isEmpty() ? null : Collections.unmodifiableMap(result); - } - - static Map nullValueCounts(ContentStats stats) { - if (stats == null) { - return null; - } - - Map result = Maps.newHashMap(); - for (FieldStats fs : stats.fieldStats()) { - if (fs != null && fs.nullValueCount() != null) { - result.put(fs.fieldId(), fs.nullValueCount()); - } - } - - return result.isEmpty() ? null : Collections.unmodifiableMap(result); - } - - static Map nanValueCounts(ContentStats stats) { - if (stats == null) { - return null; - } - - Map result = Maps.newHashMap(); - for (FieldStats fs : stats.fieldStats()) { - if (fs != null && fs.nanValueCount() != null) { - result.put(fs.fieldId(), fs.nanValueCount()); - } - } - - return result.isEmpty() ? null : Collections.unmodifiableMap(result); - } - - static Map lowerBounds(ContentStats stats) { - if (stats == null) { - return null; - } - - Map result = Maps.newHashMap(); - for (FieldStats fs : stats.fieldStats()) { - if (fs != null && fs.lowerBound() != null && fs.type() != null) { - result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.lowerBound())); - } - } - - return result.isEmpty() ? null : Collections.unmodifiableMap(result); - } - - static Map upperBounds(ContentStats stats) { - if (stats == null) { - return null; - } - - Map result = Maps.newHashMap(); - for (FieldStats fs : stats.fieldStats()) { - if (fs != null && fs.upperBound() != null && fs.type() != null) { - result.put(fs.fieldId(), Conversions.toByteBuffer(fs.type(), fs.upperBound())); - } - } - - return result.isEmpty() ? null : Collections.unmodifiableMap(result); - } - /** - * Shared base for adapters that delegate to a {@link TrackedFile} for content file fields. - * - *

Subclasses provide {@code content()}, {@code firstRowId()}, {@code equalityFieldIds()}, and - * the copy methods. + * Shared base for all tracked file adapters. Holds the common fields and implements the methods + * that delegate to {@link Tracking} and {@link PartitionSpec}. */ - private abstract static class AbstractTrackedContentFile> + private abstract static class TrackedFileAdapter> implements ContentFile { private final TrackedFile file; private final Tracking tracking; private final PartitionSpec spec; - private AbstractTrackedContentFile(TrackedFile file, PartitionSpec spec) { + private TrackedFileAdapter(TrackedFile file, PartitionSpec spec) { this.file = file; this.tracking = file.tracking(); this.spec = spec; } - TrackedFile file() { + protected TrackedFile file() { return file; } - Tracking tracking() { + protected Tracking tracking() { return tracking; } - PartitionSpec spec() { + protected PartitionSpec spec() { return spec; } @@ -235,69 +161,82 @@ public Long pos() { } @Override - public int specId() { - return spec.specId(); + public String manifestLocation() { + return tracking != null ? tracking.manifestLocation() : null; } - @SuppressWarnings("deprecation") @Override - public CharSequence path() { - return file.location(); + public int specId() { + return file.specId() != null ? file.specId() : spec.specId(); } @Override - public String location() { - return file.location(); + public StructLike partition() { + return extractPartition(file, spec); } @Override - public FileFormat format() { - return file.fileFormat(); + public Long dataSequenceNumber() { + return tracking != null ? tracking.dataSequenceNumber() : null; } @Override - public StructLike partition() { - return extractPartition(file, spec); + public Long fileSequenceNumber() { + return tracking != null ? tracking.fileSequenceNumber() : null; } + } + /** + * Shared base for adapters that delegate to a {@link TrackedFile} for content file fields. + * + *

Subclasses provide {@code content()}, {@code firstRowId()}, {@code equalityFieldIds()}, and + * the copy methods. + */ + private abstract static class TrackedContentFile> + extends TrackedFileAdapter { + private TrackedContentFile(TrackedFile file, PartitionSpec spec) { + super(file, spec); + } + + @SuppressWarnings("deprecation") @Override - public long recordCount() { - return file.recordCount(); + public CharSequence path() { + return file().location(); } @Override - public long fileSizeInBytes() { - return file.fileSizeInBytes(); + public String location() { + return file().location(); } @Override - public Integer sortOrderId() { - return file.sortOrderId(); + public FileFormat format() { + return file().fileFormat(); } @Override - public Long dataSequenceNumber() { - return tracking != null ? tracking.dataSequenceNumber() : null; + public long recordCount() { + return file().recordCount(); } @Override - public Long fileSequenceNumber() { - return tracking != null ? tracking.fileSequenceNumber() : null; + public long fileSizeInBytes() { + return file().fileSizeInBytes(); } @Override - public ByteBuffer keyMetadata() { - return file.keyMetadata(); + public Integer sortOrderId() { + return file().sortOrderId(); } @Override - public List splitOffsets() { - return file.splitOffsets(); + public ByteBuffer keyMetadata() { + return file().keyMetadata(); } @Override - public String manifestLocation() { - return tracking != null ? tracking.manifestLocation() : null; + public List splitOffsets() { + return file().splitOffsets(); } @Override @@ -307,33 +246,32 @@ public Map columnSizes() { @Override public Map valueCounts() { - return TrackedFileAdapters.valueCounts(file.contentStats()); + return MetricsUtil.valueCounts(file().contentStats()); } @Override public Map nullValueCounts() { - return TrackedFileAdapters.nullValueCounts(file.contentStats()); + return MetricsUtil.nullValueCounts(file().contentStats()); } @Override public Map nanValueCounts() { - return TrackedFileAdapters.nanValueCounts(file.contentStats()); + return MetricsUtil.nanValueCounts(file().contentStats()); } @Override public Map lowerBounds() { - return TrackedFileAdapters.lowerBounds(file.contentStats()); + return MetricsUtil.lowerBounds(file().contentStats()); } @Override public Map upperBounds() { - return TrackedFileAdapters.upperBounds(file.contentStats()); + return MetricsUtil.upperBounds(file().contentStats()); } } /** Adapts a TrackedFile DATA entry to the {@link DataFile} interface. */ - private static class TrackedDataFile extends AbstractTrackedContentFile - implements DataFile { + private static class TrackedDataFile extends TrackedContentFile implements DataFile { private TrackedDataFile(TrackedFile file, PartitionSpec spec) { super(file, spec); } @@ -375,9 +313,9 @@ public DataFile copyWithStats(Set requestedColumnIds) { } /** Adapts a TrackedFile EQUALITY_DELETES entry to the {@link DeleteFile} interface. */ - private static class TrackedDeleteFile extends AbstractTrackedContentFile + private static class TrackedEqualityDeleteFile extends TrackedContentFile implements DeleteFile { - private TrackedDeleteFile(TrackedFile file, PartitionSpec spec) { + private TrackedEqualityDeleteFile(TrackedFile file, PartitionSpec spec) { super(file, spec); } @@ -398,7 +336,50 @@ public List equalityFieldIds() { @Override public DeleteFile copy() { - return new TrackedDeleteFile(file().copy(), spec()); + return new TrackedEqualityDeleteFile(file().copy(), spec()); + } + + @Override + public DeleteFile copy(boolean withStats) { + return withStats ? copy() : copyWithoutStats(); + } + + @Override + public DeleteFile copyWithoutStats() { + return new TrackedEqualityDeleteFile(file().copyWithoutStats(), spec()); + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return new TrackedEqualityDeleteFile(file().copyWithStats(requestedColumnIds), spec()); + } + } + + /** Adapts a TrackedFile POSITION_DELETES entry to the {@link DeleteFile} interface. */ + private static class TrackedPositionDeleteFile extends TrackedContentFile + implements DeleteFile { + private TrackedPositionDeleteFile(TrackedFile file, PartitionSpec spec) { + super(file, spec); + } + + @Override + public FileContent content() { + return FileContent.POSITION_DELETES; + } + + @Override + public Long firstRowId() { + return null; + } + + @Override + public List equalityFieldIds() { + return null; + } + + @Override + public DeleteFile copy() { + return new TrackedPositionDeleteFile(file().copy(), spec()); } @Override @@ -408,12 +389,12 @@ public DeleteFile copy(boolean withStats) { @Override public DeleteFile copyWithoutStats() { - return new TrackedDeleteFile(file().copyWithoutStats(), spec()); + return new TrackedPositionDeleteFile(file().copyWithoutStats(), spec()); } @Override public DeleteFile copyWithStats(Set requestedColumnIds) { - return new TrackedDeleteFile(file().copyWithStats(requestedColumnIds), spec()); + return new TrackedPositionDeleteFile(file().copyWithStats(requestedColumnIds), spec()); } } @@ -425,29 +406,15 @@ public DeleteFile copyWithStats(Set requestedColumnIds) { * DeleteFile#contentOffset()} / {@link DeleteFile#contentSizeInBytes()} point to the blob within * the Puffin file. */ - private static class TrackedDVDeleteFile implements DeleteFile { - private final TrackedFile file; + private static class TrackedDVDeleteFile extends TrackedFileAdapter + implements DeleteFile { private final DeletionVector dv; - private final Tracking tracking; - private final PartitionSpec spec; private TrackedDVDeleteFile(TrackedFile file, PartitionSpec spec) { + super(file, spec); Preconditions.checkArgument( file.deletionVector() != null, "Cannot create DV delete file: no deletion vector"); - this.file = file; this.dv = file.deletionVector(); - this.tracking = file.tracking(); - this.spec = spec; - } - - @Override - public Long pos() { - return tracking != null ? tracking.manifestPos() : null; - } - - @Override - public int specId() { - return spec.specId(); } @Override @@ -471,11 +438,6 @@ public FileFormat format() { return FileFormat.PUFFIN; } - @Override - public StructLike partition() { - return extractPartition(file, spec); - } - @Override public long recordCount() { return dv.cardinality(); @@ -490,23 +452,13 @@ public long fileSizeInBytes() { return dv.sizeInBytes(); } - // Position deletes are required to be sorted by file and position, not a table order, and - // should set sort order id to null + // From the spec: position deletes are required to be sorted by file and position, not a table + // order, and should set sort order id to null @Override public Integer sortOrderId() { return null; } - @Override - public Long dataSequenceNumber() { - return tracking != null ? tracking.dataSequenceNumber() : null; - } - - @Override - public Long fileSequenceNumber() { - return tracking != null ? tracking.fileSequenceNumber() : null; - } - @Override public Long firstRowId() { return null; @@ -529,7 +481,7 @@ public List equalityFieldIds() { @Override public String referencedDataFile() { - return file.location(); + return file().location(); } @Override @@ -542,11 +494,6 @@ public Long contentSizeInBytes() { return dv.sizeInBytes(); } - @Override - public String manifestLocation() { - return tracking != null ? tracking.manifestLocation() : null; - } - @Override public Map columnSizes() { return null; @@ -579,7 +526,7 @@ public Map upperBounds() { @Override public DeleteFile copy() { - return new TrackedDVDeleteFile(file.copy(), spec); + return new TrackedDVDeleteFile(file().copyWithoutStats(), spec()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index 8fe542875b98..ce1af8eed82f 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -66,7 +66,7 @@ void testAsDataFileRejectsNonData() { file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalStateException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot convert tracked file to DataFile: content type is %s, not DATA", FileContent.EQUALITY_DELETES); @@ -99,12 +99,82 @@ void testAsEqualityDeleteFileRejectsNonEqualityDeletes() { file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalStateException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", FileContent.DATA); } + @Test + void testAsPositionDeleteFileValidatesContentType() { + TrackedFileStruct file = + new TrackedFileStruct( + null, + FileContent.POSITION_DELETES, + "s3://bucket/pos-delete.parquet", + FileFormat.PARQUET, + 50L, + 512L); + file.set(6, 0); + + DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED); + assertThat(deleteFile).isNotNull(); + assertThat(deleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(deleteFile.location()).isEqualTo("s3://bucket/pos-delete.parquet"); + } + + @Test + void testAsPositionDeleteFileRejectsNonPositionDeletes() { + TrackedFileStruct file = + new TrackedFileStruct( + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + file.set(6, 0); + + assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot convert tracked file to DeleteFile: content type is %s, not POSITION_DELETES", + FileContent.DATA); + } + + @Test + void testPositionDeleteFileAdapterDelegatesAllFields() { + TrackingStruct tracking = createTracking(5L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.POSITION_DELETES, + "s3://bucket/pos-delete.parquet", + FileFormat.PARQUET, + 50L, + 512L); + file.set(6, 1); + file.set(8, 5); + file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); + file.set(12, ImmutableList.of(200L)); + + DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); + + assertThat(deleteFile.pos()).isEqualTo(5L); + assertThat(deleteFile.specId()).isEqualTo(1); + assertThat(deleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(deleteFile.location()).isEqualTo("s3://bucket/pos-delete.parquet"); + assertThat(deleteFile.format()).isEqualTo(FileFormat.PARQUET); + assertThat(deleteFile.recordCount()).isEqualTo(50L); + assertThat(deleteFile.fileSizeInBytes()).isEqualTo(512L); + assertThat(deleteFile.sortOrderId()).isEqualTo(5); + assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(deleteFile.firstRowId()).isNull(); + assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); + assertThat(deleteFile.splitOffsets()).containsExactly(200L); + assertThat(deleteFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + assertThat(deleteFile.equalityFieldIds()).isNull(); + assertThat(deleteFile.columnSizes()).isNull(); + } + @Test void testAsDVDeleteFileValidatesContentType() { TrackedFileStruct file = @@ -133,7 +203,7 @@ void testAsDVDeleteFileRejectsNonData() { file.set(9, createDeletionVector()); assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalStateException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot extract DV from tracked file: content type is %s, not DATA", FileContent.EQUALITY_DELETES); @@ -147,28 +217,13 @@ void testAsDVDeleteFileRejectsNullDV() { file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Cannot extract DV from tracked file: no deletion vector"); + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot create DV delete file: no deletion vector"); } @Test void testDVDeleteFileAdapterDelegatesAllFields() { - Types.StructType trackingWithPos = - Types.StructType.of( - ImmutableList.builder() - .addAll(Tracking.schema().fields()) - .add(MetadataColumns.ROW_POSITION) - .build()); - TrackingStruct tracking = new TrackingStruct(trackingWithPos); - - tracking.set(0, EntryStatus.ADDED.id()); - tracking.set(1, 42L); - tracking.set(2, 10L); - tracking.set(3, 11L); - tracking.set(5, 1000L); - tracking.setManifestLocation("s3://bucket/manifest.avro"); - tracking.set(8, 7L); - + TrackingStruct tracking = createTracking(7L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(2).build(); TrackedFileStruct file = @@ -252,21 +307,7 @@ void testDVDeleteFilePartitionExtracted() { @Test void testDataFileAdapterDelegatesAllFields() { - Types.StructType trackingWithPos = - Types.StructType.of( - ImmutableList.builder() - .addAll(Tracking.schema().fields()) - .add(MetadataColumns.ROW_POSITION) - .build()); - TrackingStruct tracking = new TrackingStruct(trackingWithPos); - - tracking.set(0, EntryStatus.ADDED.id()); - tracking.set(1, 42L); - tracking.set(2, 10L); - tracking.set(3, 11L); - tracking.set(5, 1000L); - tracking.setManifestLocation("s3://bucket/manifest.avro"); - tracking.set(8, 3L); + TrackingStruct tracking = createTracking(3L); TrackedFileStruct file = new TrackedFileStruct( @@ -303,22 +344,7 @@ void testDataFileAdapterDelegatesAllFields() { @Test void testEqualityDeleteFileAdapterDelegatesAllFields() { - Types.StructType trackingWithPos = - Types.StructType.of( - ImmutableList.builder() - .addAll(Tracking.schema().fields()) - .add(MetadataColumns.ROW_POSITION) - .build()); - TrackingStruct tracking = new TrackingStruct(trackingWithPos); - - tracking.set(0, EntryStatus.ADDED.id()); - tracking.set(1, 42L); - tracking.set(2, 10L); - tracking.set(3, 11L); - tracking.set(5, 1000L); - tracking.setManifestLocation("s3://bucket/manifest.avro"); - tracking.set(8, 5L); - + TrackingStruct tracking = createTracking(5L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); TrackedFileStruct file = @@ -625,13 +651,34 @@ void testSpecIdDefaultsToZeroWhenNull() { assertThat(dataFile.specId()).isEqualTo(0); } + private static final Types.StructType TRACKING_WITH_POS = + Types.StructType.of( + ImmutableList.builder() + .addAll(Tracking.schema().fields()) + .add(MetadataColumns.ROW_POSITION) + .build()); + + private static TrackingStruct createTracking(long manifestPos) { + // Uses the struct-type constructor because the builder creates with BASE_TYPE which does not + // include ROW_POSITION, so manifest position cannot be set on builder-created structs. + TrackingStruct tracking = new TrackingStruct(TRACKING_WITH_POS); + tracking.set(0, EntryStatus.ADDED.id()); + tracking.set(1, 42L); + tracking.set(2, 10L); + tracking.set(3, 11L); + tracking.set(5, 1000L); + tracking.setManifestLocation("s3://bucket/manifest.avro"); + tracking.set(8, manifestPos); + return tracking; + } + private static DeletionVectorStruct createDeletionVector() { - DeletionVectorStruct dv = new DeletionVectorStruct(DeletionVector.schema()); - dv.set(0, "s3://bucket/puffin/dv-file.bin"); - dv.set(1, 128L); - dv.set(2, 256L); - dv.set(3, 10L); - return dv; + return DeletionVectorStruct.builder() + .location("s3://bucket/puffin/dv-file.bin") + .offset(128L) + .sizeInBytes(256L) + .cardinality(10L) + .build(); } private static java.util.Map.Entry entry(int key, long value) { From b628520033bb16aecbf674faa49f605a1de7a528 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 11 May 2026 15:47:41 -0700 Subject: [PATCH 09/18] Incorporate PR feedback --- .../apache/iceberg/TrackedFileAdapters.java | 66 +- .../iceberg/TestTrackedFileAdapters.java | 744 +++++------------- 2 files changed, 188 insertions(+), 622 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 51f641c45077..538e284e8c1d 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -22,9 +22,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.types.Type; /** * Adapts {@link TrackedFile} entries to the {@link DataFile} and {@link DeleteFile} APIs. @@ -42,7 +40,7 @@ private TrackedFileAdapters() {} static DataFile asDataFile(TrackedFile file, Map specsById) { Preconditions.checkArgument( file.contentType() == FileContent.DATA, - "Cannot convert tracked file to DataFile: content type is %s, not DATA", + "Invalid content type for DataFile: %s", file.contentType()); return new TrackedDataFile(file, resolveSpec(file, specsById)); } @@ -50,7 +48,7 @@ static DataFile asDataFile(TrackedFile file, Map specsBy static DeleteFile asDVDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkArgument( file.contentType() == FileContent.DATA, - "Cannot extract DV from tracked file: content type is %s, not DATA", + "Invalid content type for DV delete file: %s", file.contentType()); return new TrackedDVDeleteFile(file, resolveSpec(file, specsById)); } @@ -58,7 +56,7 @@ static DeleteFile asDVDeleteFile(TrackedFile file, Map s static DeleteFile asPositionDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkArgument( file.contentType() == FileContent.POSITION_DELETES, - "Cannot convert tracked file to DeleteFile: content type is %s, not POSITION_DELETES", + "Invalid content type for position delete file: %s", file.contentType()); return new TrackedPositionDeleteFile(file, resolveSpec(file, specsById)); } @@ -66,7 +64,7 @@ static DeleteFile asPositionDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkArgument( file.contentType() == FileContent.EQUALITY_DELETES, - "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", + "Invalid content type for equality delete file: %s", file.contentType()); return new TrackedEqualityDeleteFile(file, resolveSpec(file, specsById)); } @@ -81,50 +79,14 @@ private static PartitionSpec resolveSpec( return spec; } - return PartitionSpec.unpartitioned(); - } - - // TODO: TrackedFile will likely get an explicit partition tuple field (using a union partition - // schema), replacing this transform-based derivation. Once that lands, this method should be - // removed and the adapter should read the tuple directly. - // - // This derives partition values by applying the partition transform to the lower bound of the - // source column stats. This is correct because each data file belongs to exactly one partition, - // so lower == upper for partition source columns. For non-identity transforms (bucket, truncate), - // the transform of the lower bound produces the correct partition value under this invariant. - @SuppressWarnings({"unchecked", "rawtypes"}) - static StructLike extractPartition(TrackedFile file, PartitionSpec spec) { - if (spec.isUnpartitioned()) { - return BaseFile.EMPTY_PARTITION_DATA; - } - - ContentStats stats = file.contentStats(); - if (stats == null) { - return new PartitionData(spec.partitionType()); - } - - PartitionData partition = new PartitionData(spec.partitionType()); - - for (int i = 0; i < spec.fields().size(); i += 1) { - PartitionField field = spec.fields().get(i); - - if (field.transform().isVoid()) { - partition.set(i, null); - continue; + for (PartitionSpec spec : specsById.values()) { + if (spec.isUnpartitioned()) { + return spec; } - - FieldStats fieldStats = stats.statsFor(field.sourceId()); - if (fieldStats == null || fieldStats.lowerBound() == null) { - partition.set(i, null); - continue; - } - - Type sourceType = spec.schema().findType(field.sourceId()); - Function boundTransform = field.transform().bind(sourceType); - partition.set(i, boundTransform.apply(fieldStats.lowerBound())); } - return partition; + throw new IllegalArgumentException( + "Cannot find unpartitioned spec in specs: " + specsById.keySet()); } /** @@ -170,9 +132,10 @@ public int specId() { return file.specId() != null ? file.specId() : spec.specId(); } + // TODO: return a real partition tuple (https://github.com/apache/iceberg/issues/16222) @Override public StructLike partition() { - return extractPartition(file, spec); + return null; } @Override @@ -186,12 +149,7 @@ public Long fileSequenceNumber() { } } - /** - * Shared base for adapters that delegate to a {@link TrackedFile} for content file fields. - * - *

Subclasses provide {@code content()}, {@code firstRowId()}, {@code equalityFieldIds()}, and - * the copy methods. - */ + /** Shared base for adapters that delegate to a {@link TrackedFile} for content file fields. */ private abstract static class TrackedContentFile> extends TrackedFileAdapter { private TrackedContentFile(TrackedFile file, PartitionSpec spec) { diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index ce1af8eed82f..b4f66573090b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -27,12 +27,13 @@ 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.Conversions; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; class TestTrackedFileAdapters { + private static final String MANIFEST_LOCATION = "s3://bucket/table/manifest.parquet"; + private static final Map UNPARTITIONED = ImmutableMap.of(0, PartitionSpec.unpartitioned()); @@ -41,20 +42,55 @@ private static Map specsById(PartitionSpec spec) { } @Test - void testAsDataFileValidatesContentType() { + void testDataFileAdapterDelegatesAllFields() { + TrackingStruct tracking = createTracking(3L); + ContentStats stats = createContentStats(); + TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + tracking, + FileContent.DATA, + "s3://bucket/data/file.parquet", + FileFormat.PARQUET, + 100L, + 1024L); file.set(6, 0); + file.set(7, stats); + file.set(8, 3); + file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); + file.set(12, ImmutableList.of(50L, 100L)); DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); - assertThat(dataFile).isNotNull(); + + assertThat(dataFile.pos()).isEqualTo(3L); + assertThat(dataFile.specId()).isEqualTo(0); assertThat(dataFile.content()).isEqualTo(FileContent.DATA); - assertThat(dataFile.location()).isEqualTo("s3://bucket/data.parquet"); + assertThat(dataFile.location()).isEqualTo("s3://bucket/data/file.parquet"); + assertThat(dataFile.format()).isEqualTo(FileFormat.PARQUET); + assertThat(dataFile.recordCount()).isEqualTo(100L); + assertThat(dataFile.fileSizeInBytes()).isEqualTo(1024L); + assertThat(dataFile.sortOrderId()).isEqualTo(3); + assertThat(dataFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(dataFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(dataFile.firstRowId()).isEqualTo(1000L); + assertThat(dataFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2, 3})); + assertThat(dataFile.splitOffsets()).containsExactly(50L, 100L); + assertThat(dataFile.manifestLocation()).isEqualTo(MANIFEST_LOCATION); + assertThat(dataFile.equalityFieldIds()).isNull(); + assertThat(dataFile.columnSizes()).isNull(); + assertThat(dataFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); + assertThat(dataFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); + assertThat(dataFile.nanValueCounts()).containsOnly(entry(2, 3L)); + assertThat(dataFile.lowerBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); + assertThat(dataFile.upperBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); } @Test - void testAsDataFileRejectsNonData() { + void testDataFileAdapterRejectsNonData() { TrackedFileStruct file = new TrackedFileStruct( null, @@ -67,80 +103,14 @@ void testAsDataFileRejectsNonData() { assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, UNPARTITIONED)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot convert tracked file to DataFile: content type is %s, not DATA", - FileContent.EQUALITY_DELETES); - } - - @Test - void testAsEqualityDeleteFileValidatesContentType() { - TrackedFileStruct file = - new TrackedFileStruct( - null, - FileContent.EQUALITY_DELETES, - "s3://bucket/eq-delete.avro", - FileFormat.AVRO, - 50L, - 512L); - file.set(6, 0); - file.set(13, ImmutableList.of(1, 2)); - - DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED); - assertThat(deleteFile).isNotNull(); - assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2); - } - - @Test - void testAsEqualityDeleteFileRejectsNonEqualityDeletes() { - TrackedFileStruct file = - new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, 0); - - assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot convert tracked file to DeleteFile: content type is %s, not EQUALITY_DELETES", - FileContent.DATA); - } - - @Test - void testAsPositionDeleteFileValidatesContentType() { - TrackedFileStruct file = - new TrackedFileStruct( - null, - FileContent.POSITION_DELETES, - "s3://bucket/pos-delete.parquet", - FileFormat.PARQUET, - 50L, - 512L); - file.set(6, 0); - - DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED); - assertThat(deleteFile).isNotNull(); - assertThat(deleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(deleteFile.location()).isEqualTo("s3://bucket/pos-delete.parquet"); - } - - @Test - void testAsPositionDeleteFileRejectsNonPositionDeletes() { - TrackedFileStruct file = - new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, 0); - - assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot convert tracked file to DeleteFile: content type is %s, not POSITION_DELETES", - FileContent.DATA); + .hasMessage("Invalid content type for DataFile: %s", FileContent.EQUALITY_DELETES); } @Test void testPositionDeleteFileAdapterDelegatesAllFields() { TrackingStruct tracking = createTracking(5L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + ContentStats stats = createContentStats(); TrackedFileStruct file = new TrackedFileStruct( @@ -151,6 +121,7 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { 50L, 512L); file.set(6, 1); + file.set(7, stats); file.set(8, 5); file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); file.set(12, ImmutableList.of(200L)); @@ -170,55 +141,92 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { assertThat(deleteFile.firstRowId()).isNull(); assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); assertThat(deleteFile.splitOffsets()).containsExactly(200L); - assertThat(deleteFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + assertThat(deleteFile.manifestLocation()).isEqualTo(MANIFEST_LOCATION); assertThat(deleteFile.equalityFieldIds()).isNull(); assertThat(deleteFile.columnSizes()).isNull(); + assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); + assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); + assertThat(deleteFile.nanValueCounts()).containsOnly(entry(2, 3L)); + assertThat(deleteFile.lowerBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); + assertThat(deleteFile.upperBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); } @Test - void testAsDVDeleteFileValidatesContentType() { + void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - file.set(9, createDeletionVector()); - DeleteFile dv = TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED); - assertThat(dv).isNotNull(); - assertThat(dv.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(dv.format()).isEqualTo(FileFormat.PUFFIN); + assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid content type for position delete file: %s", FileContent.DATA); } @Test - void testAsDVDeleteFileRejectsNonData() { + void testEqualityDeleteFileAdapterDelegatesAllFields() { + TrackingStruct tracking = createTracking(5L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + ContentStats stats = createContentStats(); + TrackedFileStruct file = new TrackedFileStruct( - null, + tracking, FileContent.EQUALITY_DELETES, "s3://bucket/eq-delete.avro", FileFormat.AVRO, 50L, 512L); - file.set(6, 0); - file.set(9, createDeletionVector()); + file.set(6, 1); + file.set(7, stats); + file.set(8, 5); + file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); + file.set(12, ImmutableList.of(200L)); + file.set(13, ImmutableList.of(1, 2, 3)); - assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot extract DV from tracked file: content type is %s, not DATA", - FileContent.EQUALITY_DELETES); + DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); + + assertThat(deleteFile.pos()).isEqualTo(5L); + assertThat(deleteFile.specId()).isEqualTo(1); + assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(deleteFile.location()).isEqualTo("s3://bucket/eq-delete.avro"); + assertThat(deleteFile.format()).isEqualTo(FileFormat.AVRO); + assertThat(deleteFile.recordCount()).isEqualTo(50L); + assertThat(deleteFile.fileSizeInBytes()).isEqualTo(512L); + assertThat(deleteFile.sortOrderId()).isEqualTo(5); + assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); + assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); + assertThat(deleteFile.firstRowId()).isNull(); + assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); + assertThat(deleteFile.splitOffsets()).containsExactly(200L); + assertThat(deleteFile.manifestLocation()).isEqualTo(MANIFEST_LOCATION); + assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2, 3); + assertThat(deleteFile.columnSizes()).isNull(); + assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); + assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); + assertThat(deleteFile.nanValueCounts()).containsOnly(entry(2, 3L)); + assertThat(deleteFile.lowerBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); + assertThat(deleteFile.upperBounds()) + .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) + .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); } @Test - void testAsDVDeleteFileRejectsNullDV() { + void testEqualityDeleteFileAdapterRejectsNonEqualityDeletes() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) + assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot create DV delete file: no deletion vector"); + .hasMessage("Invalid content type for equality delete file: %s", FileContent.DATA); } @Test @@ -226,16 +234,7 @@ void testDVDeleteFileAdapterDelegatesAllFields() { TrackingStruct tracking = createTracking(7L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(2).build(); - TrackedFileStruct file = - new TrackedFileStruct( - tracking, - FileContent.DATA, - "s3://bucket/data/file.parquet", - FileFormat.PARQUET, - 100L, - 1024L); - file.set(6, 2); - file.set(9, createDeletionVector()); + TrackedFileStruct file = createDataFileWithDV(tracking, 2); DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, specsById(spec)); @@ -254,7 +253,7 @@ void testDVDeleteFileAdapterDelegatesAllFields() { assertThat(dvFile.specId()).isEqualTo(2); assertThat(dvFile.dataSequenceNumber()).isEqualTo(10L); assertThat(dvFile.fileSequenceNumber()).isEqualTo(11L); - assertThat(dvFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); + assertThat(dvFile.manifestLocation()).isEqualTo(MANIFEST_LOCATION); // fields that should be null for DVs assertThat(dvFile.sortOrderId()).isNull(); @@ -271,171 +270,37 @@ void testDVDeleteFileAdapterDelegatesAllFields() { } @Test - void testDVDeleteFileAdapterDelegatesNullTracking() { - TrackedFileStruct file = - new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, 0); - file.set(9, createDeletionVector()); - - DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED); - - assertThat(dvFile.dataSequenceNumber()).isNull(); - assertThat(dvFile.fileSequenceNumber()).isNull(); - assertThat(dvFile.manifestLocation()).isNull(); - assertThat(dvFile.pos()).isNull(); - } - - @Test - void testDVDeleteFilePartitionExtracted() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "category", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); - - TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); - file.set(9, createDeletionVector()); - - DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, specsById(spec)); - - StructLike partition = dvFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); - } - - @Test - void testDataFileAdapterDelegatesAllFields() { - TrackingStruct tracking = createTracking(3L); - - TrackedFileStruct file = - new TrackedFileStruct( - tracking, - FileContent.DATA, - "s3://bucket/data/file.parquet", - FileFormat.PARQUET, - 100L, - 1024L); - file.set(6, 0); - file.set(8, 3); - file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); - file.set(12, ImmutableList.of(50L, 100L)); - - DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); - - assertThat(dataFile.pos()).isEqualTo(3L); - assertThat(dataFile.specId()).isEqualTo(0); - assertThat(dataFile.content()).isEqualTo(FileContent.DATA); - assertThat(dataFile.location()).isEqualTo("s3://bucket/data/file.parquet"); - assertThat(dataFile.format()).isEqualTo(FileFormat.PARQUET); - assertThat(dataFile.recordCount()).isEqualTo(100L); - assertThat(dataFile.fileSizeInBytes()).isEqualTo(1024L); - assertThat(dataFile.sortOrderId()).isEqualTo(3); - assertThat(dataFile.dataSequenceNumber()).isEqualTo(10L); - assertThat(dataFile.fileSequenceNumber()).isEqualTo(11L); - assertThat(dataFile.firstRowId()).isEqualTo(1000L); - assertThat(dataFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2, 3})); - assertThat(dataFile.splitOffsets()).containsExactly(50L, 100L); - assertThat(dataFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); - assertThat(dataFile.equalityFieldIds()).isNull(); - assertThat(dataFile.columnSizes()).isNull(); - } - - @Test - void testEqualityDeleteFileAdapterDelegatesAllFields() { - TrackingStruct tracking = createTracking(5L); - PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); - + void testDVDeleteFileAdapterRejectsNonData() { TrackedFileStruct file = new TrackedFileStruct( - tracking, + null, FileContent.EQUALITY_DELETES, "s3://bucket/eq-delete.avro", FileFormat.AVRO, 50L, 512L); - file.set(6, 1); - file.set(8, 5); - file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); - file.set(12, ImmutableList.of(200L)); - file.set(13, ImmutableList.of(1, 2, 3)); - - DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); + file.set(6, 0); + file.set(9, createDeletionVector()); - assertThat(deleteFile.pos()).isEqualTo(5L); - assertThat(deleteFile.specId()).isEqualTo(1); - assertThat(deleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(deleteFile.location()).isEqualTo("s3://bucket/eq-delete.avro"); - assertThat(deleteFile.format()).isEqualTo(FileFormat.AVRO); - assertThat(deleteFile.recordCount()).isEqualTo(50L); - assertThat(deleteFile.fileSizeInBytes()).isEqualTo(512L); - assertThat(deleteFile.sortOrderId()).isEqualTo(5); - assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); - assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); - assertThat(deleteFile.firstRowId()).isNull(); - assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); - assertThat(deleteFile.splitOffsets()).containsExactly(200L); - assertThat(deleteFile.manifestLocation()).isEqualTo("s3://bucket/manifest.avro"); - assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2, 3); - assertThat(deleteFile.columnSizes()).isNull(); + assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid content type for DV delete file: %s", FileContent.EQUALITY_DELETES); } @Test - void testAdapterDelegatesNullTracking() { + void testDVDeleteFileAdapterRejectsNullDV() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); file.set(6, 0); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); - - assertThat(dataFile.dataSequenceNumber()).isNull(); - assertThat(dataFile.fileSequenceNumber()).isNull(); - assertThat(dataFile.firstRowId()).isNull(); - assertThat(dataFile.manifestLocation()).isNull(); - assertThat(dataFile.pos()).isNull(); - } - - @Test - void testDataFileAdapterStatsFromContentStats() { - TrackedFileStruct file = createTrackedFileWithStats(); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); - - assertThat(dataFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); - assertThat(dataFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); - assertThat(dataFile.nanValueCounts()).containsOnly(entry(2, 3L)); - assertThat(dataFile.lowerBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); - assertThat(dataFile.upperBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); - assertThat(dataFile.columnSizes()).isNull(); - } - - @Test - void testEqualityDeleteFileAdapterStatsFromContentStats() { - TrackedFileStruct file = createTrackedFileWithStats(); - file.set(1, FileContent.EQUALITY_DELETES.id()); - file.set(13, ImmutableList.of(1)); - - DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED); - - assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); - assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); - assertThat(deleteFile.nanValueCounts()).containsOnly(entry(2, 3L)); - assertThat(deleteFile.lowerBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); - assertThat(deleteFile.upperBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); - assertThat(deleteFile.columnSizes()).isNull(); + assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot create DV delete file: no deletion vector"); } @Test - void testDataFileAdapterStatsNullWhenNoContentStats() { + void testNullContentStatsReturnsNullStats() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); @@ -451,326 +316,81 @@ void testDataFileAdapterStatsNullWhenNoContentStats() { } @Test - void testPartitionExtractedFromContentStatsWithIdentityTransform() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "category", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); - - TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - - StructLike partition = dataFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); - } - - @Test - void testPartitionExtractedWithYearTransform() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "ts", Types.DateType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).year("ts").build(); - - // date value 20546 = 2026-04-03 (days since epoch) - TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.DateType.get(), 20546); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - - StructLike partition = dataFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, Integer.class)).isEqualTo(56); - } - - @Test - void testPartitionExtractedWithBucketTransform() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "value", Types.IntegerType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 16).build(); - - TrackedFileStruct file = createTrackedFileWithFieldStats(2, Types.IntegerType.get(), 42); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - - StructLike partition = dataFile.partition(); - assertThat(partition).isNotNull(); - - // verify the bucket value is a valid bucket (0-15) - int bucket = partition.get(0, Integer.class); - assertThat(bucket).isBetween(0, 15); - } - - @Test - void testPartitionEmptyWhenNoContentStats() { - Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); - + void testNullSpecIdResolvesToUnpartitionedSpec() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, spec.specId()); - - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - assertThat(dataFile.partition()).isNotNull(); - assertThat(dataFile.partition().size()).isEqualTo(1); - assertThat(dataFile.partition().get(0, Integer.class)).isNull(); - } - - @Test - void testAsDataFileRejectsUnknownSpecId() { - TrackedFileStruct file = createTrackedFileWithStats(); - assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, ImmutableMap.of())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot find partition spec for spec ID"); - } - @Test - void testPartitionEmptyForUnpartitioned() { - PartitionSpec spec = PartitionSpec.unpartitioned(); - - TrackedFileStruct file = createTrackedFileWithStats(); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(5).build(); DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - assertThat(dataFile.partition()).isNotNull(); - assertThat(dataFile.partition().size()).isEqualTo(0); + assertThat(dataFile.specId()).isEqualTo(5); } @Test - void testPartitionWithMultipleFields() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "category", Types.StringType.get())); - - PartitionSpec spec = - PartitionSpec.builderFor(schema).identity("id").identity("category").build(); - - Types.StructType statsStruct = - Types.StructType.of( - Types.NestedField.optional( - 10000, - "1", - Types.StructType.of( - Types.NestedField.optional(10006, "lower_bound", Types.IntegerType.get()), - Types.NestedField.optional(10007, "upper_bound", Types.IntegerType.get()))), - Types.NestedField.optional( - 20000, - "2", - Types.StructType.of( - Types.NestedField.optional(20006, "lower_bound", Types.StringType.get()), - Types.NestedField.optional(20007, "upper_bound", Types.StringType.get())))); - - @SuppressWarnings("unchecked") - List> fieldStatsList = - ImmutableList.of( - (FieldStats) - BaseFieldStats.builder() - .fieldId(1) - .type(Types.IntegerType.get()) - .lowerBound(42) - .upperBound(42) - .build(), - (FieldStats) - BaseFieldStats.builder() - .fieldId(2) - .type(Types.StringType.get()) - .lowerBound("electronics") - .upperBound("electronics") - .build()); - - BaseContentStats stats = - BaseContentStats.builder() - .withStatsStruct(statsStruct) - .withFieldStats(fieldStatsList) - .build(); + void testNullSpecIdThrowsWhenNoUnpartitionedSpec() { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + PartitionSpec partitioned = PartitionSpec.builderFor(schema).identity("id").build(); TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, spec.specId()); - file.set(7, stats); - - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - - StructLike partition = dataFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, Integer.class)).isEqualTo(42); - assertThat(partition.get(1, CharSequence.class).toString()).isEqualTo("electronics"); - } - - @Test - void testPartitionWithVoidTransform() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").alwaysNull("data").build(); - - TrackedFileStruct file = createTrackedFileWithFieldStats(1, Types.IntegerType.get(), 42); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); - - StructLike partition = dataFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, Integer.class)).isEqualTo(42); - assertThat(partition.get(1, CharSequence.class)).isNull(); - } - - @Test - void testEqualityDeleteFilePartitionExtracted() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "category", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("category").build(); - - TrackedFileStruct file = createTrackedFileWithPartitionStats(spec); - file.set(1, FileContent.EQUALITY_DELETES.id()); - file.set(13, ImmutableList.of(1)); - DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); - - StructLike partition = deleteFile.partition(); - assertThat(partition).isNotNull(); - assertThat(partition.get(0, CharSequence.class).toString()).isEqualTo("electronics"); + assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, specsById(partitioned))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot find unpartitioned spec in specs"); } @Test - void testSpecIdDefaultsToZeroWhenNull() { + void testUnknownSpecIdThrows() { TrackedFileStruct file = new TrackedFileStruct( null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + file.set(6, 99); - DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); - assertThat(dataFile.specId()).isEqualTo(0); + assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot find partition spec for spec ID"); } - private static final Types.StructType TRACKING_WITH_POS = - Types.StructType.of( - ImmutableList.builder() - .addAll(Tracking.schema().fields()) - .add(MetadataColumns.ROW_POSITION) - .build()); - private static TrackingStruct createTracking(long manifestPos) { - // Uses the struct-type constructor because the builder creates with BASE_TYPE which does not - // include ROW_POSITION, so manifest position cannot be set on builder-created structs. - TrackingStruct tracking = new TrackingStruct(TRACKING_WITH_POS); - tracking.set(0, EntryStatus.ADDED.id()); - tracking.set(1, 42L); - tracking.set(2, 10L); - tracking.set(3, 11L); - tracking.set(5, 1000L); - tracking.setManifestLocation("s3://bucket/manifest.avro"); + TrackingStruct tracking = + TrackingStruct.builder() + .status(EntryStatus.ADDED) + .snapshotId(42L) + .dataSequenceNumber(10L) + .fileSequenceNumber(11L) + .firstRowId(1000L) + .build(); + // the builder doesn't support reader-side fields (manifestLocation, manifestPos) + tracking.setManifestLocation(MANIFEST_LOCATION); tracking.set(8, manifestPos); return tracking; } - private static DeletionVectorStruct createDeletionVector() { - return DeletionVectorStruct.builder() - .location("s3://bucket/puffin/dv-file.bin") - .offset(128L) - .sizeInBytes(256L) - .cardinality(10L) - .build(); - } - - private static java.util.Map.Entry entry(int key, long value) { - return java.util.Map.entry(key, value); - } - - @SuppressWarnings("unchecked") - private static TrackedFileStruct createTrackedFileWithPartitionStats(PartitionSpec spec) { - Types.StructType statsStruct = - Types.StructType.of( - Types.NestedField.optional( - 20000, - "2", - Types.StructType.of( - Types.NestedField.optional(20006, "lower_bound", Types.StringType.get()), - Types.NestedField.optional(20007, "upper_bound", Types.StringType.get())))); - - List> fieldStatsList = - ImmutableList.of( - (FieldStats) - BaseFieldStats.builder() - .fieldId(2) - .type(Types.StringType.get()) - .lowerBound("electronics") - .upperBound("electronics") - .build()); - - BaseContentStats stats = - BaseContentStats.builder() - .withStatsStruct(statsStruct) - .withFieldStats(fieldStatsList) - .build(); - + private static TrackedFileStruct createDataFileWithDV(Tracking tracking, int specId) { TrackedFileStruct file = new TrackedFileStruct( - null, + tracking, FileContent.DATA, "s3://bucket/data/file.parquet", FileFormat.PARQUET, 100L, 1024L); - file.set(6, spec.specId()); - file.set(7, stats); - + file.set(6, specId); + file.set(9, createDeletionVector()); return file; } - @SuppressWarnings("unchecked") - private static TrackedFileStruct createTrackedFileWithFieldStats( - int fieldId, Type type, T value) { - int statsFieldId = fieldId * 10000; - Types.StructType statsStruct = - Types.StructType.of( - Types.NestedField.optional( - statsFieldId, - Integer.toString(fieldId), - Types.StructType.of( - Types.NestedField.optional(statsFieldId + 6, "lower_bound", type), - Types.NestedField.optional(statsFieldId + 7, "upper_bound", type)))); - - List> fieldStatsList = - ImmutableList.of( - (FieldStats) - BaseFieldStats.builder() - .fieldId(fieldId) - .type(type) - .lowerBound(value) - .upperBound(value) - .build()); - - BaseContentStats stats = - BaseContentStats.builder() - .withStatsStruct(statsStruct) - .withFieldStats(fieldStatsList) - .build(); - - TrackedFileStruct file = - new TrackedFileStruct( - null, - FileContent.DATA, - "s3://bucket/data/file.parquet", - FileFormat.PARQUET, - 100L, - 1024L); - file.set(6, 0); - file.set(7, stats); - - return file; + private static DeletionVectorStruct createDeletionVector() { + return DeletionVectorStruct.builder() + .location("s3://bucket/puffin/dv-file.bin") + .offset(128L) + .sizeInBytes(256L) + .cardinality(10L) + .build(); } - @SuppressWarnings("unchecked") - private static TrackedFileStruct createTrackedFileWithStats() { + private static ContentStats createContentStats() { Types.StructType statsStruct = Types.StructType.of( Types.NestedField.optional( @@ -794,43 +414,31 @@ private static TrackedFileStruct createTrackedFileWithStats() { List> fieldStatsList = ImmutableList.of( - (FieldStats) - BaseFieldStats.builder() - .fieldId(1) - .type(Types.IntegerType.get()) - .valueCount(100L) - .nullValueCount(5L) - .lowerBound(1) - .upperBound(1000) - .build(), - (FieldStats) - BaseFieldStats.builder() - .fieldId(2) - .type(Types.FloatType.get()) - .valueCount(200L) - .nullValueCount(10L) - .nanValueCount(3L) - .lowerBound(1.0f) - .upperBound(100.0f) - .build()); - - BaseContentStats stats = - BaseContentStats.builder() - .withStatsStruct(statsStruct) - .withFieldStats(fieldStatsList) - .build(); - - TrackedFileStruct file = - new TrackedFileStruct( - null, - FileContent.DATA, - "s3://bucket/data/file.parquet", - FileFormat.PARQUET, - 100L, - 1024L); - file.set(6, 0); - file.set(7, stats); + BaseFieldStats.builder() + .fieldId(1) + .type(Types.IntegerType.get()) + .valueCount(100L) + .nullValueCount(5L) + .lowerBound(1) + .upperBound(1000) + .build(), + BaseFieldStats.builder() + .fieldId(2) + .type(Types.FloatType.get()) + .valueCount(200L) + .nullValueCount(10L) + .nanValueCount(3L) + .lowerBound(1.0f) + .upperBound(100.0f) + .build()); + + return BaseContentStats.builder() + .withStatsStruct(statsStruct) + .withFieldStats(fieldStatsList) + .build(); + } - return file; + private static Map.Entry entry(int key, long value) { + return Map.entry(key, value); } } From 6a19faf6e533805830b78c198a2d381bc84e88db Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 11 May 2026 16:52:35 -0700 Subject: [PATCH 10/18] support v3 DVs through the adapter --- .../apache/iceberg/TrackedFileAdapters.java | 18 +++++++ .../iceberg/TestTrackedFileAdapters.java | 47 +++++++++++++++++++ 2 files changed, 65 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 538e284e8c1d..dff32e3daec0 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -335,6 +335,24 @@ public List equalityFieldIds() { return null; } + @Override + public String referencedDataFile() { + DeletionVector dv = file().deletionVector(); + return dv != null ? file().location() : null; + } + + @Override + public Long contentOffset() { + DeletionVector dv = file().deletionVector(); + return dv != null ? dv.offset() : null; + } + + @Override + public Long contentSizeInBytes() { + DeletionVector dv = file().deletionVector(); + return dv != null ? dv.sizeInBytes() : null; + } + @Override public DeleteFile copy() { return new TrackedPositionDeleteFile(file().copy(), spec()); diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index b4f66573090b..ace122889345 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -167,6 +167,53 @@ void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { .hasMessage("Invalid content type for position delete file: %s", FileContent.DATA); } + @Test + void testPositionDeleteFileAdapterDelegatesDVFields() { + TrackingStruct tracking = createTracking(5L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.POSITION_DELETES, + "s3://bucket/dv.puffin", + FileFormat.PUFFIN, + 10L, + 256L); + file.set(6, 1); + file.set(9, createDeletionVector()); + + DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); + + assertThat(deleteFile.format()).isEqualTo(FileFormat.PUFFIN); + assertThat(deleteFile.referencedDataFile()).isEqualTo("s3://bucket/dv.puffin"); + assertThat(deleteFile.contentOffset()).isEqualTo(128L); + assertThat(deleteFile.contentSizeInBytes()).isEqualTo(256L); + } + + @Test + void testPositionDeleteFileAdapterWithoutDVReturnsNullDVFields() { + TrackingStruct tracking = createTracking(5L); + PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); + + TrackedFileStruct file = + new TrackedFileStruct( + tracking, + FileContent.POSITION_DELETES, + "s3://bucket/pos-delete.parquet", + FileFormat.PARQUET, + 50L, + 512L); + file.set(6, 1); + + DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); + + assertThat(deleteFile.format()).isEqualTo(FileFormat.PARQUET); + assertThat(deleteFile.referencedDataFile()).isNull(); + assertThat(deleteFile.contentOffset()).isNull(); + assertThat(deleteFile.contentSizeInBytes()).isNull(); + } + @Test void testEqualityDeleteFileAdapterDelegatesAllFields() { TrackingStruct tracking = createTracking(5L); From 7c61cbfcc3d2b83ee9602355e6f3d8c08891f7af Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 11 May 2026 17:10:36 -0700 Subject: [PATCH 11/18] Revert the whole v3 DV adapter --- .../apache/iceberg/TrackedFileAdapters.java | 18 ------- .../iceberg/TestTrackedFileAdapters.java | 47 ------------------- 2 files changed, 65 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index dff32e3daec0..538e284e8c1d 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -335,24 +335,6 @@ public List equalityFieldIds() { return null; } - @Override - public String referencedDataFile() { - DeletionVector dv = file().deletionVector(); - return dv != null ? file().location() : null; - } - - @Override - public Long contentOffset() { - DeletionVector dv = file().deletionVector(); - return dv != null ? dv.offset() : null; - } - - @Override - public Long contentSizeInBytes() { - DeletionVector dv = file().deletionVector(); - return dv != null ? dv.sizeInBytes() : null; - } - @Override public DeleteFile copy() { return new TrackedPositionDeleteFile(file().copy(), spec()); diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index ace122889345..b4f66573090b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -167,53 +167,6 @@ void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { .hasMessage("Invalid content type for position delete file: %s", FileContent.DATA); } - @Test - void testPositionDeleteFileAdapterDelegatesDVFields() { - TrackingStruct tracking = createTracking(5L); - PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); - - TrackedFileStruct file = - new TrackedFileStruct( - tracking, - FileContent.POSITION_DELETES, - "s3://bucket/dv.puffin", - FileFormat.PUFFIN, - 10L, - 256L); - file.set(6, 1); - file.set(9, createDeletionVector()); - - DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); - - assertThat(deleteFile.format()).isEqualTo(FileFormat.PUFFIN); - assertThat(deleteFile.referencedDataFile()).isEqualTo("s3://bucket/dv.puffin"); - assertThat(deleteFile.contentOffset()).isEqualTo(128L); - assertThat(deleteFile.contentSizeInBytes()).isEqualTo(256L); - } - - @Test - void testPositionDeleteFileAdapterWithoutDVReturnsNullDVFields() { - TrackingStruct tracking = createTracking(5L); - PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); - - TrackedFileStruct file = - new TrackedFileStruct( - tracking, - FileContent.POSITION_DELETES, - "s3://bucket/pos-delete.parquet", - FileFormat.PARQUET, - 50L, - 512L); - file.set(6, 1); - - DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); - - assertThat(deleteFile.format()).isEqualTo(FileFormat.PARQUET); - assertThat(deleteFile.referencedDataFile()).isNull(); - assertThat(deleteFile.contentOffset()).isNull(); - assertThat(deleteFile.contentSizeInBytes()).isNull(); - } - @Test void testEqualityDeleteFileAdapterDelegatesAllFields() { TrackingStruct tracking = createTracking(5L); From cd194e3212e0cedf81ada35b43f588cede6c8ea4 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 12 May 2026 15:26:30 -0700 Subject: [PATCH 12/18] Feedback --- .../apache/iceberg/TrackedFileAdapters.java | 41 ++++--------------- .../iceberg/TestTrackedFileAdapters.java | 2 +- 2 files changed, 10 insertions(+), 33 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 538e284e8c1d..4452c5ddc3d3 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -91,17 +91,15 @@ private static PartitionSpec resolveSpec( /** * Shared base for all tracked file adapters. Holds the common fields and implements the methods - * that delegate to {@link Tracking} and {@link PartitionSpec}. + * that delegate to {@link TrackedFile} and {@link PartitionSpec}. */ private abstract static class TrackedFileAdapter> implements ContentFile { private final TrackedFile file; - private final Tracking tracking; private final PartitionSpec spec; private TrackedFileAdapter(TrackedFile file, PartitionSpec spec) { this.file = file; - this.tracking = file.tracking(); this.spec = spec; } @@ -109,21 +107,23 @@ protected TrackedFile file() { return file; } - protected Tracking tracking() { - return tracking; - } - protected PartitionSpec spec() { return spec; } + private Tracking tracking() { + return file.tracking(); + } + @Override public Long pos() { + Tracking tracking = tracking(); return tracking != null ? tracking.manifestPos() : null; } @Override public String manifestLocation() { + Tracking tracking = tracking(); return tracking != null ? tracking.manifestLocation() : null; } @@ -140,11 +140,13 @@ public StructLike partition() { @Override public Long dataSequenceNumber() { + Tracking tracking = tracking(); return tracking != null ? tracking.dataSequenceNumber() : null; } @Override public Long fileSequenceNumber() { + Tracking tracking = tracking(); return tracking != null ? tracking.fileSequenceNumber() : null; } } @@ -244,11 +246,6 @@ public Long firstRowId() { return tracking() != null ? tracking().firstRowId() : null; } - @Override - public List equalityFieldIds() { - return null; - } - @Override public DataFile copy() { return new TrackedDataFile(file().copy(), spec()); @@ -282,11 +279,6 @@ public FileContent content() { return FileContent.EQUALITY_DELETES; } - @Override - public Long firstRowId() { - return null; - } - @Override public List equalityFieldIds() { return file().equalityIds(); @@ -325,11 +317,6 @@ public FileContent content() { return FileContent.POSITION_DELETES; } - @Override - public Long firstRowId() { - return null; - } - @Override public List equalityFieldIds() { return null; @@ -417,21 +404,11 @@ public Integer sortOrderId() { return null; } - @Override - public Long firstRowId() { - return null; - } - @Override public ByteBuffer keyMetadata() { return null; } - @Override - public List splitOffsets() { - return null; - } - @Override public List equalityFieldIds() { return null; diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index b4f66573090b..e64a3f0b080a 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -361,7 +361,7 @@ private static TrackingStruct createTracking(long manifestPos) { .fileSequenceNumber(11L) .firstRowId(1000L) .build(); - // the builder doesn't support reader-side fields (manifestLocation, manifestPos) + // manifestLocation and manifestPos are set by manifest readers, not written to manifests tracking.setManifestLocation(MANIFEST_LOCATION); tracking.set(8, manifestPos); return tracking; From 72bcd3478d9dff5abe9907862646c52a515d9e15 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 12 May 2026 15:35:52 -0700 Subject: [PATCH 13/18] Fix comment --- core/src/main/java/org/apache/iceberg/TrackedFile.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFile.java b/core/src/main/java/org/apache/iceberg/TrackedFile.java index 8a6335972888..edfa52d8e941 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFile.java @@ -34,7 +34,7 @@ interface TrackedFile { 134, "content_type", Types.IntegerType.get(), - "Type of content: 0=DATA, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST"); + "Type of content: 0=DATA, 1=POSITION_DELETES, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST"); Types.NestedField LOCATION = Types.NestedField.required(100, "location", Types.StringType.get(), "Location of the file"); Types.NestedField FILE_FORMAT = From 3ce09003abaf56a8087ad863dd18ffb931048006 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 12 May 2026 15:45:35 -0700 Subject: [PATCH 14/18] Fix failure --- core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 4452c5ddc3d3..852e1f5f3dc7 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -111,7 +111,7 @@ protected PartitionSpec spec() { return spec; } - private Tracking tracking() { + protected Tracking tracking() { return file.tracking(); } From 35fbdd879097d16da70c0ca676218a3453e76c1d Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 12 May 2026 15:54:35 -0700 Subject: [PATCH 15/18] Rebase --- .../iceberg/TestTrackedFileAdapters.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index e64a3f0b080a..32476c8be38b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -52,6 +52,7 @@ void testDataFileAdapterDelegatesAllFields() { FileContent.DATA, "s3://bucket/data/file.parquet", FileFormat.PARQUET, + null, 100L, 1024L); file.set(6, 0); @@ -97,6 +98,7 @@ void testDataFileAdapterRejectsNonData() { FileContent.EQUALITY_DELETES, "s3://bucket/delete.avro", FileFormat.AVRO, + null, 50L, 512L); file.set(6, 0); @@ -118,6 +120,7 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { FileContent.POSITION_DELETES, "s3://bucket/pos-delete.parquet", FileFormat.PARQUET, + null, 50L, 512L); file.set(6, 1); @@ -159,7 +162,7 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) @@ -179,6 +182,7 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { FileContent.EQUALITY_DELETES, "s3://bucket/eq-delete.avro", FileFormat.AVRO, + null, 50L, 512L); file.set(6, 1); @@ -221,7 +225,7 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { void testEqualityDeleteFileAdapterRejectsNonEqualityDeletes() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) @@ -277,6 +281,7 @@ void testDVDeleteFileAdapterRejectsNonData() { FileContent.EQUALITY_DELETES, "s3://bucket/eq-delete.avro", FileFormat.AVRO, + null, 50L, 512L); file.set(6, 0); @@ -291,7 +296,7 @@ void testDVDeleteFileAdapterRejectsNonData() { void testDVDeleteFileAdapterRejectsNullDV() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) @@ -303,7 +308,7 @@ void testDVDeleteFileAdapterRejectsNullDV() { void testNullContentStatsReturnsNullStats() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); file.set(6, 0); DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); @@ -319,7 +324,7 @@ void testNullContentStatsReturnsNullStats() { void testNullSpecIdResolvesToUnpartitionedSpec() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(5).build(); DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); @@ -333,7 +338,7 @@ void testNullSpecIdThrowsWhenNoUnpartitionedSpec() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, specsById(partitioned))) .isInstanceOf(IllegalArgumentException.class) @@ -344,7 +349,7 @@ void testNullSpecIdThrowsWhenNoUnpartitionedSpec() { void testUnknownSpecIdThrows() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, 100L, 1024L); + null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); file.set(6, 99); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, ImmutableMap.of())) @@ -374,6 +379,7 @@ private static TrackedFileStruct createDataFileWithDV(Tracking tracking, int spe FileContent.DATA, "s3://bucket/data/file.parquet", FileFormat.PARQUET, + null, 100L, 1024L); file.set(6, specId); From 5f6df84c8742df02c719b4c4bb397840a9d49947 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Tue, 12 May 2026 15:57:52 -0700 Subject: [PATCH 16/18] Fix spotless --- .../iceberg/TestTrackedFileAdapters.java | 56 ++++++++++++++++--- 1 file changed, 49 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index 32476c8be38b..b8fc7760c60b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -162,7 +162,13 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) @@ -225,7 +231,13 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { void testEqualityDeleteFileAdapterRejectsNonEqualityDeletes() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asEqualityDeleteFile(file, UNPARTITIONED)) @@ -296,7 +308,13 @@ void testDVDeleteFileAdapterRejectsNonData() { void testDVDeleteFileAdapterRejectsNullDV() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); file.set(6, 0); assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) @@ -308,7 +326,13 @@ void testDVDeleteFileAdapterRejectsNullDV() { void testNullContentStatsReturnsNullStats() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); file.set(6, 0); DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); @@ -324,7 +348,13 @@ void testNullContentStatsReturnsNullStats() { void testNullSpecIdResolvesToUnpartitionedSpec() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(5).build(); DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec)); @@ -338,7 +368,13 @@ void testNullSpecIdThrowsWhenNoUnpartitionedSpec() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, specsById(partitioned))) .isInstanceOf(IllegalArgumentException.class) @@ -349,7 +385,13 @@ void testNullSpecIdThrowsWhenNoUnpartitionedSpec() { void testUnknownSpecIdThrows() { TrackedFileStruct file = new TrackedFileStruct( - null, FileContent.DATA, "s3://bucket/data.parquet", FileFormat.PARQUET, null, 100L, 1024L); + null, + FileContent.DATA, + "s3://bucket/data.parquet", + FileFormat.PARQUET, + null, + 100L, + 1024L); file.set(6, 99); assertThatThrownBy(() -> TrackedFileAdapters.asDataFile(file, ImmutableMap.of())) From 9553f2e60d7afa1d24fb7ec39dd3374158e1df0d Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 13 May 2026 08:31:35 -0700 Subject: [PATCH 17/18] Fix ordinals after rebasing --- .../iceberg/TestTrackedFileAdapters.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index b8fc7760c60b..b6ab15032de5 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -56,10 +56,10 @@ void testDataFileAdapterDelegatesAllFields() { 100L, 1024L); file.set(6, 0); - file.set(7, stats); - file.set(8, 3); - file.set(11, ByteBuffer.wrap(new byte[] {1, 2, 3})); - file.set(12, ImmutableList.of(50L, 100L)); + file.set(8, stats); + file.set(9, 3); + file.set(12, ByteBuffer.wrap(new byte[] {1, 2, 3})); + file.set(13, ImmutableList.of(50L, 100L)); DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED); @@ -124,10 +124,10 @@ void testPositionDeleteFileAdapterDelegatesAllFields() { 50L, 512L); file.set(6, 1); - file.set(7, stats); - file.set(8, 5); - file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); - file.set(12, ImmutableList.of(200L)); + file.set(8, stats); + file.set(9, 5); + file.set(12, ByteBuffer.wrap(new byte[] {4, 5})); + file.set(13, ImmutableList.of(200L)); DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); @@ -192,11 +192,11 @@ void testEqualityDeleteFileAdapterDelegatesAllFields() { 50L, 512L); file.set(6, 1); - file.set(7, stats); - file.set(8, 5); - file.set(11, ByteBuffer.wrap(new byte[] {4, 5})); - file.set(12, ImmutableList.of(200L)); - file.set(13, ImmutableList.of(1, 2, 3)); + file.set(8, stats); + file.set(9, 5); + file.set(12, ByteBuffer.wrap(new byte[] {4, 5})); + file.set(13, ImmutableList.of(200L)); + file.set(14, ImmutableList.of(1, 2, 3)); DeleteFile deleteFile = TrackedFileAdapters.asEqualityDeleteFile(file, specsById(spec)); @@ -297,7 +297,7 @@ void testDVDeleteFileAdapterRejectsNonData() { 50L, 512L); file.set(6, 0); - file.set(9, createDeletionVector()); + file.set(10, createDeletionVector()); assertThatThrownBy(() -> TrackedFileAdapters.asDVDeleteFile(file, UNPARTITIONED)) .isInstanceOf(IllegalArgumentException.class) @@ -425,7 +425,7 @@ private static TrackedFileStruct createDataFileWithDV(Tracking tracking, int spe 100L, 1024L); file.set(6, specId); - file.set(9, createDeletionVector()); + file.set(10, createDeletionVector()); return file; } From 5ed9aeb539ba7c789d2d87c9c57ec2cf1410667d Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Thu, 14 May 2026 14:23:07 -0700 Subject: [PATCH 18/18] Remove position delete from adapter --- .../java/org/apache/iceberg/TrackedFile.java | 2 +- .../apache/iceberg/TrackedFileAdapters.java | 46 ------------- .../iceberg/TestTrackedFileAdapters.java | 68 ------------------- 3 files changed, 1 insertion(+), 115 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFile.java b/core/src/main/java/org/apache/iceberg/TrackedFile.java index edfa52d8e941..8a6335972888 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFile.java @@ -34,7 +34,7 @@ interface TrackedFile { 134, "content_type", Types.IntegerType.get(), - "Type of content: 0=DATA, 1=POSITION_DELETES, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST"); + "Type of content: 0=DATA, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST"); Types.NestedField LOCATION = Types.NestedField.required(100, "location", Types.StringType.get(), "Location of the file"); Types.NestedField FILE_FORMAT = diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java index 852e1f5f3dc7..525357439a4d 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java @@ -53,14 +53,6 @@ static DeleteFile asDVDeleteFile(TrackedFile file, Map s return new TrackedDVDeleteFile(file, resolveSpec(file, specsById)); } - static DeleteFile asPositionDeleteFile(TrackedFile file, Map specsById) { - Preconditions.checkArgument( - file.contentType() == FileContent.POSITION_DELETES, - "Invalid content type for position delete file: %s", - file.contentType()); - return new TrackedPositionDeleteFile(file, resolveSpec(file, specsById)); - } - static DeleteFile asEqualityDeleteFile(TrackedFile file, Map specsById) { Preconditions.checkArgument( file.contentType() == FileContent.EQUALITY_DELETES, @@ -305,44 +297,6 @@ public DeleteFile copyWithStats(Set requestedColumnIds) { } } - /** Adapts a TrackedFile POSITION_DELETES entry to the {@link DeleteFile} interface. */ - private static class TrackedPositionDeleteFile extends TrackedContentFile - implements DeleteFile { - private TrackedPositionDeleteFile(TrackedFile file, PartitionSpec spec) { - super(file, spec); - } - - @Override - public FileContent content() { - return FileContent.POSITION_DELETES; - } - - @Override - public List equalityFieldIds() { - return null; - } - - @Override - public DeleteFile copy() { - return new TrackedPositionDeleteFile(file().copy(), spec()); - } - - @Override - public DeleteFile copy(boolean withStats) { - return withStats ? copy() : copyWithoutStats(); - } - - @Override - public DeleteFile copyWithoutStats() { - return new TrackedPositionDeleteFile(file().copyWithoutStats(), spec()); - } - - @Override - public DeleteFile copyWithStats(Set requestedColumnIds) { - return new TrackedPositionDeleteFile(file().copyWithStats(requestedColumnIds), spec()); - } - } - /** * Adapts the deletion vector from a TrackedFile DATA entry to the {@link DeleteFile} interface. * diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java index b6ab15032de5..b04b5c04dd23 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java @@ -108,74 +108,6 @@ void testDataFileAdapterRejectsNonData() { .hasMessage("Invalid content type for DataFile: %s", FileContent.EQUALITY_DELETES); } - @Test - void testPositionDeleteFileAdapterDelegatesAllFields() { - TrackingStruct tracking = createTracking(5L); - PartitionSpec spec = PartitionSpec.builderFor(new Schema()).withSpecId(1).build(); - ContentStats stats = createContentStats(); - - TrackedFileStruct file = - new TrackedFileStruct( - tracking, - FileContent.POSITION_DELETES, - "s3://bucket/pos-delete.parquet", - FileFormat.PARQUET, - null, - 50L, - 512L); - file.set(6, 1); - file.set(8, stats); - file.set(9, 5); - file.set(12, ByteBuffer.wrap(new byte[] {4, 5})); - file.set(13, ImmutableList.of(200L)); - - DeleteFile deleteFile = TrackedFileAdapters.asPositionDeleteFile(file, specsById(spec)); - - assertThat(deleteFile.pos()).isEqualTo(5L); - assertThat(deleteFile.specId()).isEqualTo(1); - assertThat(deleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(deleteFile.location()).isEqualTo("s3://bucket/pos-delete.parquet"); - assertThat(deleteFile.format()).isEqualTo(FileFormat.PARQUET); - assertThat(deleteFile.recordCount()).isEqualTo(50L); - assertThat(deleteFile.fileSizeInBytes()).isEqualTo(512L); - assertThat(deleteFile.sortOrderId()).isEqualTo(5); - assertThat(deleteFile.dataSequenceNumber()).isEqualTo(10L); - assertThat(deleteFile.fileSequenceNumber()).isEqualTo(11L); - assertThat(deleteFile.firstRowId()).isNull(); - assertThat(deleteFile.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {4, 5})); - assertThat(deleteFile.splitOffsets()).containsExactly(200L); - assertThat(deleteFile.manifestLocation()).isEqualTo(MANIFEST_LOCATION); - assertThat(deleteFile.equalityFieldIds()).isNull(); - assertThat(deleteFile.columnSizes()).isNull(); - assertThat(deleteFile.valueCounts()).containsOnly(entry(1, 100L), entry(2, 200L)); - assertThat(deleteFile.nullValueCounts()).containsOnly(entry(1, 5L), entry(2, 10L)); - assertThat(deleteFile.nanValueCounts()).containsOnly(entry(2, 3L)); - assertThat(deleteFile.lowerBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 1.0f)); - assertThat(deleteFile.upperBounds()) - .containsEntry(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1000)) - .containsEntry(2, Conversions.toByteBuffer(Types.FloatType.get(), 100.0f)); - } - - @Test - void testPositionDeleteFileAdapterRejectsNonPositionDeletes() { - TrackedFileStruct file = - new TrackedFileStruct( - null, - FileContent.DATA, - "s3://bucket/data.parquet", - FileFormat.PARQUET, - null, - 100L, - 1024L); - file.set(6, 0); - - assertThatThrownBy(() -> TrackedFileAdapters.asPositionDeleteFile(file, UNPARTITIONED)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid content type for position delete file: %s", FileContent.DATA); - } - @Test void testEqualityDeleteFileAdapterDelegatesAllFields() { TrackingStruct tracking = createTracking(5L);