stevenzwu commented on code in PR #16100:
URL: https://github.com/apache/iceberg/pull/16100#discussion_r3211649793


##########
core/src/test/java/org/apache/iceberg/TestTrackedFileAdapters.java:
##########
@@ -0,0 +1,836 @@
+/*
+ * 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 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;
+import org.junit.jupiter.api.Test;
+
+class TestTrackedFileAdapters {
+
+  private static final Map<Integer, PartitionSpec> UNPARTITIONED =
+      ImmutableMap.of(0, PartitionSpec.unpartitioned());
+
+  private static Map<Integer, PartitionSpec> specsById(PartitionSpec spec) {
+    return ImmutableMap.of(spec.specId(), spec);
+  }
+
+  @Test
+  void testAsDataFileValidatesContentType() {
+    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).isNotNull();
+    assertThat(dataFile.content()).isEqualTo(FileContent.DATA);
+    assertThat(dataFile.location()).isEqualTo("s3://bucket/data.parquet");
+  }
+
+  @Test
+  void testAsDataFileRejectsNonData() {
+    TrackedFileStruct file =
+        new TrackedFileStruct(
+            null,
+            FileContent.EQUALITY_DELETES,
+            "s3://bucket/delete.avro",
+            FileFormat.AVRO,
+            50L,
+            512L);
+    file.set(6, 0);
+
+    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);
+  }
+
+  @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 =
+        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);
+  }
+
+  @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, 
UNPARTITIONED))
+        .isInstanceOf(IllegalArgumentException.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, 
UNPARTITIONED))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot create DV delete file: no deletion vector");
+  }
+
+  @Test
+  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());
+
+    DeleteFile dvFile = TrackedFileAdapters.asDVDeleteFile(file, 
specsById(spec));
+
+    // 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, 
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();
+
+    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}));
+    file.set(12, ImmutableList.of(200L));
+    file.set(13, ImmutableList.of(1, 2, 3));
+
+    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("s3://bucket/manifest.avro");
+    assertThat(deleteFile.equalityFieldIds()).containsExactly(1, 2, 3);
+    assertThat(deleteFile.columnSizes()).isNull();
+  }
+
+  @Test
+  void testAdapterDelegatesNullTracking() {
+    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();
+  }
+
+  @Test
+  void testDataFileAdapterStatsNullWhenNoContentStats() {
+    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.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, 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();
+
+    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();
+    DataFile dataFile = TrackedFileAdapters.asDataFile(file, specsById(spec));
+    assertThat(dataFile.partition()).isNotNull();
+    assertThat(dataFile.partition().size()).isEqualTo(0);
+  }
+
+  @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<FieldStats<?>> fieldStatsList =
+        ImmutableList.of(
+            (FieldStats<?>)
+                BaseFieldStats.<Integer>builder()
+                    .fieldId(1)
+                    .type(Types.IntegerType.get())
+                    .lowerBound(42)
+                    .upperBound(42)
+                    .build(),
+            (FieldStats<?>)
+                BaseFieldStats.<CharSequence>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(
+            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");
+  }
+
+  @Test
+  void testSpecIdDefaultsToZeroWhenNull() {
+    TrackedFileStruct file =
+        new TrackedFileStruct(
+            null, FileContent.DATA, "s3://bucket/data.parquet", 
FileFormat.PARQUET, 100L, 1024L);
+
+    DataFile dataFile = TrackedFileAdapters.asDataFile(file, UNPARTITIONED);
+    assertThat(dataFile.specId()).isEqualTo(0);
+  }
+
+  private static final Types.StructType TRACKING_WITH_POS =
+      Types.StructType.of(
+          ImmutableList.<Types.NestedField>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.

Review Comment:
   This justification doesn't hold. In `TrackingStruct.java`, `BASE_TYPE` 
(L33-43) already includes `MetadataColumns.ROW_POSITION` as its 9th field, and 
the builder's `build()` calls a constructor that does `super(BASE_TYPE, 
BASE_TYPE)` (L91), so builder-created structs do support row position 
projection.
   
   `manifestLocation` and `manifestPos` aren't part of the struct projection at 
all - they're plain instance fields (L55-56), "set by manifest readers, not 
written to manifests". They're assigned via `setManifestLocation(...)` and 
`set(8, ...)` regardless of which constructor was used.
   
   So the helper can just be:
   
   ```java
   TrackingStruct tracking = TrackingStruct.builder()
       .status(EntryStatus.ADDED)
       .snapshotId(42L)
       .dataSequenceNumber(10L)
       .fileSequenceNumber(11L)
       .firstRowId(1000L)
       .build();
   tracking.setManifestLocation("s3://bucket/manifest.avro");
   tracking.set(8, manifestPos);
   return tracking;
   ```
   
   



##########
core/src/main/java/org/apache/iceberg/TrackedFileAdapters.java:
##########
@@ -0,0 +1,547 @@
+/*
+ * 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.types.Type;
+
+/**
+ * Adapts {@link TrackedFile} entries to the {@link DataFile} and {@link 
DeleteFile} APIs.
+ *
+ * <p>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, 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, Map<Integer, PartitionSpec> 
specsById) {
+    Preconditions.checkArgument(
+        file.contentType() == FileContent.DATA,
+        "Cannot convert tracked file to DataFile: content type is %s, not 
DATA",
+        file.contentType());
+    return new TrackedDataFile(file, resolveSpec(file, specsById));
+  }
+
+  static DeleteFile asDVDeleteFile(TrackedFile file, Map<Integer, 
PartitionSpec> specsById) {
+    Preconditions.checkArgument(
+        file.contentType() == FileContent.DATA,
+        "Cannot extract DV from tracked file: content type is %s, not DATA",
+        file.contentType());
+    return new TrackedDVDeleteFile(file, resolveSpec(file, specsById));
+  }
+
+  static DeleteFile asPositionDeleteFile(TrackedFile file, Map<Integer, 
PartitionSpec> 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<Integer, 
PartitionSpec> specsById) {
+    Preconditions.checkArgument(
+        file.contentType() == FileContent.EQUALITY_DELETES,
+        "Cannot convert tracked file to DeleteFile: content type is %s, not 
EQUALITY_DELETES",
+        file.contentType());
+    return new TrackedEqualityDeleteFile(file, resolveSpec(file, specsById));
+  }
+
+  private static PartitionSpec resolveSpec(
+      TrackedFile file, Map<Integer, PartitionSpec> 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
+  //  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;
+      }
+
+      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;
+  }
+
+  /**
+   * 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 TrackedFileAdapter<F extends ContentFile<F>>
+      implements ContentFile<F> {
+    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;
+    }
+
+    protected TrackedFile file() {
+      return file;
+    }
+
+    protected Tracking tracking() {
+      return tracking;
+    }
+
+    protected PartitionSpec spec() {
+      return spec;
+    }
+
+    @Override
+    public Long pos() {
+      return tracking != null ? tracking.manifestPos() : null;
+    }
+
+    @Override
+    public String manifestLocation() {
+      return tracking != null ? tracking.manifestLocation() : null;
+    }
+
+    @Override
+    public int specId() {
+      return file.specId() != null ? file.specId() : spec.specId();
+    }
+
+    @Override
+    public StructLike partition() {
+      return extractPartition(file, spec);
+    }
+
+    @Override
+    public Long dataSequenceNumber() {
+      return tracking != null ? tracking.dataSequenceNumber() : null;
+    }
+
+    @Override
+    public Long fileSequenceNumber() {
+      return tracking != null ? tracking.fileSequenceNumber() : null;
+    }
+  }
+
+  /**
+   * Shared base for adapters that delegate to a {@link TrackedFile} for 
content file fields.
+   *
+   * <p>Subclasses provide {@code content()}, {@code firstRowId()}, {@code 
equalityFieldIds()}, and
+   * the copy methods.
+   */
+  private abstract static class TrackedContentFile<F extends ContentFile<F>>
+      extends TrackedFileAdapter<F> {
+    private TrackedContentFile(TrackedFile file, PartitionSpec spec) {
+      super(file, spec);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public CharSequence path() {
+      return file().location();
+    }
+
+    @Override
+    public String location() {
+      return file().location();
+    }
+
+    @Override
+    public FileFormat format() {
+      return file().fileFormat();
+    }
+
+    @Override
+    public long recordCount() {
+      return file().recordCount();
+    }
+
+    @Override
+    public long fileSizeInBytes() {
+      return file().fileSizeInBytes();
+    }
+
+    @Override
+    public Integer sortOrderId() {
+      return file().sortOrderId();
+    }
+
+    @Override
+    public ByteBuffer keyMetadata() {
+      return file().keyMetadata();
+    }
+
+    @Override
+    public List<Long> splitOffsets() {
+      return file().splitOffsets();
+    }
+
+    @Override
+    public Map<Integer, Long> columnSizes() {
+      return null;
+    }
+
+    @Override
+    public Map<Integer, Long> valueCounts() {
+      return MetricsUtil.valueCounts(file().contentStats());
+    }
+
+    @Override
+    public Map<Integer, Long> nullValueCounts() {
+      return MetricsUtil.nullValueCounts(file().contentStats());
+    }
+
+    @Override
+    public Map<Integer, Long> nanValueCounts() {
+      return MetricsUtil.nanValueCounts(file().contentStats());
+    }
+
+    @Override
+    public Map<Integer, ByteBuffer> lowerBounds() {
+      return MetricsUtil.lowerBounds(file().contentStats());
+    }
+
+    @Override
+    public Map<Integer, ByteBuffer> upperBounds() {
+      return MetricsUtil.upperBounds(file().contentStats());
+    }
+  }
+
+  /** Adapts a TrackedFile DATA entry to the {@link DataFile} interface. */
+  private static class TrackedDataFile extends TrackedContentFile<DataFile> 
implements DataFile {
+    private TrackedDataFile(TrackedFile file, PartitionSpec spec) {
+      super(file, spec);
+    }
+
+    @Override
+    public FileContent content() {
+      return FileContent.DATA;
+    }
+
+    @Override
+    public Long firstRowId() {
+      return tracking() != null ? tracking().firstRowId() : null;
+    }
+
+    @Override
+    public List<Integer> equalityFieldIds() {
+      return null;
+    }
+
+    @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<Integer> requestedColumnIds) {
+      return new TrackedDataFile(file().copyWithStats(requestedColumnIds), 
spec());
+    }
+  }
+
+  /** Adapts a TrackedFile EQUALITY_DELETES entry to the {@link DeleteFile} 
interface. */
+  private static class TrackedEqualityDeleteFile extends 
TrackedContentFile<DeleteFile>
+      implements DeleteFile {
+    private TrackedEqualityDeleteFile(TrackedFile file, PartitionSpec spec) {
+      super(file, spec);
+    }
+
+    @Override
+    public FileContent content() {
+      return FileContent.EQUALITY_DELETES;
+    }
+
+    @Override
+    public Long firstRowId() {
+      return null;
+    }
+
+    @Override
+    public List<Integer> equalityFieldIds() {
+      return file().equalityIds();
+    }
+
+    @Override
+    public DeleteFile copy() {
+      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<Integer> requestedColumnIds) {
+      return new 
TrackedEqualityDeleteFile(file().copyWithStats(requestedColumnIds), spec());
+    }
+  }
+
+  /** Adapts a TrackedFile POSITION_DELETES entry to the {@link DeleteFile} 
interface. */

Review Comment:
   `TrackedPositionDeleteFile` doesn't override `referencedDataFile()`, 
`contentOffset()`, or `contentSizeInBytes()`, so all three fall through to the 
`DeleteFile` interface defaults of `null`. But `V4Metadata.java` (cases 
17/18/19, L495-509) writes those slots out for `POSITION_DELETES` entries, so 
V4 manifests can carry V2-style position delete metadata — a V2 delete file 
with `referenced_data_file` and content offset/size.
   
   The read-side gap: `TrackedFile` (interface) doesn't expose 
`referencedDataFile`/`contentOffset`/`contentSizeInBytes`, so this adapter has 
no way to surface them even if they were in the manifest. So either:
   
   1. V4 manifests aren't expected to round-trip V2 position delete metadata 
(worth a code comment here saying these are intentionally always null), or
   2. `TrackedFile` is missing fields for position-delete-specific metadata, 
and this adapter silently drops them when reading V2 position deletes from a V4 
manifest.
   
   The test `testPositionDeleteFileAdapterDelegatesAllFields` doesn't exercise 
these three fields, so the gap isn't visible there.
   
   Side note while we're here: `TrackedFile.java:37` documents `content_type` 
as `"0=DATA, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST"` — 
`1=POSITION_DELETES` is omitted, but this PR's `asPositionDeleteFile` implies 
it's valid. Worth updating that doc to match (separate file, but the 
inconsistency is exposed by this PR).
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to