From 82a4e43ed17d28dc135434b5518c938b18742771 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Thu, 30 Oct 2025 17:04:47 -0700 Subject: [PATCH 01/14] API: Add V4 manifest TrackedFile interfaces and implementation Introduces the foundational types for V4 manifest format support: - TrackedFile interface as unified representation for all V4 entry types - DeletionVector and ManifestStats interfaces - GenericTrackedFile implementation and test --- .../org/apache/iceberg/DeletionVector.java | 65 ++ .../java/org/apache/iceberg/FileContent.java | 20 +- .../org/apache/iceberg/ManifestStats.java | 75 ++ .../java/org/apache/iceberg/TrackedFile.java | 211 +++++ .../java/org/apache/iceberg/TrackingInfo.java | 118 +++ .../apache/iceberg/GenericTrackedFile.java | 728 ++++++++++++++++++ .../iceberg/TestGenericTrackedFile.java | 418 ++++++++++ 7 files changed, 1633 insertions(+), 2 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/DeletionVector.java create mode 100644 api/src/main/java/org/apache/iceberg/ManifestStats.java create mode 100644 api/src/main/java/org/apache/iceberg/TrackedFile.java create mode 100644 api/src/main/java/org/apache/iceberg/TrackingInfo.java create mode 100644 core/src/main/java/org/apache/iceberg/GenericTrackedFile.java create mode 100644 core/src/test/java/org/apache/iceberg/TestGenericTrackedFile.java diff --git a/api/src/main/java/org/apache/iceberg/DeletionVector.java b/api/src/main/java/org/apache/iceberg/DeletionVector.java new file mode 100644 index 000000000000..7289a236fb49 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/DeletionVector.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.nio.ByteBuffer; +import org.apache.iceberg.types.Types; + +/** + * Deletion vector information for a tracked file entry in a V4 manifest. + * + *

This encapsulates both inline and out-of-line deletion vector content. This struct must be + * defined for types 1 (POSITION_DELETES) and 5 (MANIFEST_DV). + */ +public interface DeletionVector { + Types.NestedField OFFSET = + Types.NestedField.optional( + 144, "offset", Types.LongType.get(), "Offset in the file where the content starts"); + Types.NestedField SIZE_IN_BYTES = + Types.NestedField.optional( + 145, + "size_in_bytes", + Types.LongType.get(), + "Length of a referenced content stored in the file; required if offset is present"); + Types.NestedField INLINE_CONTENT = + Types.NestedField.optional( + 146, "inline_content", Types.BinaryType.get(), "Serialized bitmap for inline DVs"); + + /** + * Returns the offset in the file where the deletion vector content starts. + * + *

This is used for out-of-line deletion vectors stored in Puffin files. + */ + Long offset(); + + /** + * Returns the size in bytes of the deletion vector content. + * + *

Required if offset is present. + */ + Long sizeInBytes(); + + /** + * Returns the serialized bitmap for inline deletion vectors. + * + *

When present, the deletion vector is stored inline in the manifest rather than in a separate + * Puffin file. + */ + ByteBuffer inlineContent(); +} diff --git a/api/src/main/java/org/apache/iceberg/FileContent.java b/api/src/main/java/org/apache/iceberg/FileContent.java index 2c9a2fa51bd2..87993bb6e479 100644 --- a/api/src/main/java/org/apache/iceberg/FileContent.java +++ b/api/src/main/java/org/apache/iceberg/FileContent.java @@ -18,11 +18,27 @@ */ package org.apache.iceberg; -/** Content type stored in a file, one of DATA, POSITION_DELETES, or EQUALITY_DELETES. */ +/** + * Content type stored in a file. + * + *

For V1-V3 tables: DATA, POSITION_DELETES, or EQUALITY_DELETES. + * + *

For V4 tables: DATA, POSITION_DELETES, EQUALITY_DELETES, DATA_MANIFEST, DELETE_MANIFEST, or + * MANIFEST_DV. + */ public enum FileContent { DATA(0), POSITION_DELETES(1), - EQUALITY_DELETES(2); + EQUALITY_DELETES(2), + /** Data manifest entry (V4+ only) - references data files in a root manifest. */ + DATA_MANIFEST(3), + /** Delete manifest entry (V4+ only) - references delete files in a root manifest. */ + DELETE_MANIFEST(4), + /** + * Manifest deletion vector entry (V4+ only) - marks entries in a manifest as deleted without + * rewriting the manifest. + */ + MANIFEST_DV(5); private final int id; diff --git a/api/src/main/java/org/apache/iceberg/ManifestStats.java b/api/src/main/java/org/apache/iceberg/ManifestStats.java new file mode 100644 index 000000000000..5cc82e5b9a9d --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/ManifestStats.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.types.Types; + +/** + * Statistics for manifest entries in a V4 tracked file. + * + *

This encapsulates added/removed/existing files/row counts and min_sequence_number for a + * manifest. This must be defined when the content_type is a manifest (3 or 4), and null otherwise. + */ +public interface ManifestStats { + Types.NestedField ADDED_FILES_COUNT = + Types.NestedField.required( + 504, "added_files_count", Types.IntegerType.get(), "Number of files added"); + Types.NestedField EXISTING_FILES_COUNT = + Types.NestedField.required( + 505, "existing_files_count", Types.IntegerType.get(), "Number of existing files"); + Types.NestedField DELETED_FILES_COUNT = + Types.NestedField.required( + 506, "deleted_files_count", Types.IntegerType.get(), "Number of deleted files"); + Types.NestedField ADDED_ROWS_COUNT = + Types.NestedField.required( + 512, "added_rows_count", Types.LongType.get(), "Number of rows in added files"); + Types.NestedField EXISTING_ROWS_COUNT = + Types.NestedField.required( + 513, "existing_rows_count", Types.LongType.get(), "Number of rows in existing files"); + Types.NestedField DELETED_ROWS_COUNT = + Types.NestedField.required( + 514, "deleted_rows_count", Types.LongType.get(), "Number of rows in deleted files"); + Types.NestedField MIN_SEQUENCE_NUMBER = + Types.NestedField.required( + 516, + "min_sequence_number", + Types.LongType.get(), + "Minimum sequence number of files in this manifest"); + + /** Returns the number of files added by this manifest. */ + int addedFilesCount(); + + /** Returns the number of existing files referenced by this manifest. */ + int existingFilesCount(); + + /** Returns the number of deleted files in this manifest. */ + int deletedFilesCount(); + + /** Returns the number of rows in added files. */ + long addedRowsCount(); + + /** Returns the number of rows in existing files. */ + long existingRowsCount(); + + /** Returns the number of rows in deleted files. */ + long deletedRowsCount(); + + /** Returns the minimum sequence number of files in this manifest. */ + long minSequenceNumber(); +} diff --git a/api/src/main/java/org/apache/iceberg/TrackedFile.java b/api/src/main/java/org/apache/iceberg/TrackedFile.java new file mode 100644 index 000000000000..1829ab399f1e --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/TrackedFile.java @@ -0,0 +1,211 @@ +/* + * 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 org.apache.iceberg.types.Types; + +/** + * Represents a V4 content entry in a manifest file. + * + *

TrackedFile is the V4 equivalent of ContentFile. It provides a unified representation for all + * entry types in a V4 manifest: data files, delete files, manifests, and deletion vectors. + * + * @param the concrete class of a TrackedFile instance + */ +public interface TrackedFile { + // Field IDs from V4 specification + Types.NestedField CONTENT_TYPE = + Types.NestedField.required( + 134, + "content_type", + Types.IntegerType.get(), + "Type of content: 0=DATA, 1=POSITION_DELETES, 2=EQUALITY_DELETES, 3=DATA_MANIFEST, 4=DELETE_MANIFEST, 5=MANIFEST_DV"); + Types.NestedField LOCATION = + Types.NestedField.optional( + 100, + "location", + Types.StringType.get(), + "Location of the file. Optional if content_type is 5 and deletion_vector.inline_content is not null"); + Types.NestedField FILE_FORMAT = + Types.NestedField.required( + 101, + "file_format", + Types.StringType.get(), + "String file format name: avro, orc, parquet, or puffin"); + Types.NestedField PARTITION_SPEC_ID = + Types.NestedField.required( + 148, + "partition_spec_id", + Types.IntegerType.get(), + "ID of partition spec used to write manifest or data/delete files"); + Types.NestedField SORT_ORDER_ID = + Types.NestedField.optional( + 140, + "sort_order_id", + Types.IntegerType.get(), + "ID representing sort order for this file. Can only be set if content_type is 0"); + Types.NestedField RECORD_COUNT = + Types.NestedField.required( + 103, + "record_count", + Types.LongType.get(), + "Number of records in this file, or the cardinality of a deletion vector"); + Types.NestedField FILE_SIZE_IN_BYTES = + Types.NestedField.optional( + 104, + "file_size_in_bytes", + Types.LongType.get(), + "Total file size in bytes. Must be defined if location is defined"); + Types.NestedField KEY_METADATA = + Types.NestedField.optional( + 131, + "key_metadata", + Types.BinaryType.get(), + "Implementation-specific key metadata for encryption"); + Types.NestedField SPLIT_OFFSETS = + Types.NestedField.optional( + 132, + "split_offsets", + Types.ListType.ofRequired(133, Types.LongType.get()), + "Split offsets for the data file. Must be sorted ascending"); + Types.NestedField EQUALITY_IDS = + Types.NestedField.optional( + 135, + "equality_ids", + Types.ListType.ofRequired(136, Types.IntegerType.get()), + "Field ids used to determine row equality in equality delete files. Required when content=2"); + Types.NestedField REFERENCED_FILE = + Types.NestedField.optional( + 143, + "referenced_file", + Types.StringType.get(), + "Location of data file that a DV references if content_type is 1 or 5." + + " Location of affiliated data manifest if content_type is 4 or null if delete manifest is unaffiliated"); + + /** + * Returns the tracking information for this entry. + * + *

Contains status, snapshot ID, sequence numbers, and first-row-id. Optional - may be null if + * tracking info is inherited. + */ + TrackingInfo trackingInfo(); + + /** + * Returns the type of content stored by this entry. + * + *

One of: DATA, POSITION_DELETES, EQUALITY_DELETES, DATA_MANIFEST, DELETE_MANIFEST, or + * MANIFEST_DV. + */ + FileContent contentType(); + + /** + * Returns the location of the file. + * + *

Optional if content_type is MANIFEST_DV and deletion_vector has inline content. + */ + String location(); + + /** Returns the format of the file (avro, orc, parquet, or puffin). */ + FileFormat fileFormat(); + + /** + * Returns the deletion vector for this entry. + * + *

Must be defined if content_type is POSITION_DELETES or MANIFEST_DV. Must be null for all + * other types. + */ + DeletionVector deletionVector(); + + /** Returns the ID of the partition spec used to write this file or manifest. */ + int partitionSpecId(); + + /** + * Returns the ID representing sort order for this file. + * + *

Can only be set if content_type is DATA. + */ + Integer sortOrderId(); + + /** Returns the number of records in this file, or the cardinality of a deletion vector. */ + long recordCount(); + + /** + * Returns the total file size in bytes. + * + *

Must be defined if location is defined. + */ + Long fileSizeInBytes(); + + /** + * Returns the content stats for this entry. + * + *

TODO: Define ContentStats structure per V4 proposal. + */ + Object contentStats(); + + /** + * Returns the manifest stats for this entry. + * + *

Must be set if content_type is DATA_MANIFEST or DELETE_MANIFEST, otherwise must be null. + */ + ManifestStats manifestStats(); + + /** Returns metadata about how this file is encrypted, or null if stored in plain text. */ + ByteBuffer keyMetadata(); + + /** + * Returns list of recommended split locations, if applicable, null otherwise. + * + *

Must be sorted in ascending order. + */ + List splitOffsets(); + + /** + * Returns the set of field IDs used for equality comparison, in equality delete files. + * + *

Required when content_type is EQUALITY_DELETES, must be null otherwise. + */ + List equalityIds(); + + /** + * Returns the location of the referenced file. + * + *

For POSITION_DELETES or MANIFEST_DV: location of data file that the DV references. + * + *

For DELETE_MANIFEST: location of affiliated data manifest, or null if unaffiliated. + */ + String referencedFile(); + + /** + * Copies this tracked file. + * + *

Manifest readers can reuse file instances; use this method to copy data when collecting + * files from tasks. + */ + F copy(); + + /** + * Copies this tracked file without stats. + * + *

Use this method to copy data without stats when collecting files. + */ + F copyWithoutStats(); +} diff --git a/api/src/main/java/org/apache/iceberg/TrackingInfo.java b/api/src/main/java/org/apache/iceberg/TrackingInfo.java new file mode 100644 index 000000000000..b467e2463d6a --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/TrackingInfo.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.types.Types; + +/** + * Tracking information for a tracked file entry in a V4 manifest. + * + *

This groups the status, snapshot, and sequence number information for the entry. This enables + * accessing the fields for the entry and provides an isolated structure that can be modified. + */ +public interface TrackingInfo { + /** Status of an entry in a tracked file */ + enum Status { + EXISTING(0), + ADDED(1), + DELETED(2); + + private final int id; + + Status(int id) { + this.id = id; + } + + public int id() { + return id; + } + } + + Types.NestedField STATUS = + Types.NestedField.required(0, "status", Types.IntegerType.get(), "Entry status"); + Types.NestedField SNAPSHOT_ID = + Types.NestedField.optional( + 1, + "snapshot_id", + Types.LongType.get(), + "Snapshot ID where the file was added, or deleted if status is 2. Inherited when null."); + Types.NestedField SEQUENCE_NUMBER = + Types.NestedField.optional( + 3, + "sequence_number", + Types.LongType.get(), + "Data sequence number of the file. Inherited when null and status is 1 (added). Must be equal to file_sequence_number if content_type is 3 or 4."); + Types.NestedField FILE_SEQUENCE_NUMBER = + Types.NestedField.optional( + 4, + "file_sequence_number", + Types.LongType.get(), + "File sequence number indicating when the file was added. Inherited when null and status is added. Must be equal to sequence_number if content_type is 3 or 4."); + Types.NestedField FIRST_ROW_ID = + Types.NestedField.optional( + 142, + "first_row_id", + Types.LongType.get(), + "The _row_id for the first row in the data file if content_type is 0. If content_type is 3, this is the starting _row_id to assign to rows added by ADDED data files."); + + /** + * Returns the status of the entry. + * + *

Status values: + * + *

+ */ + Status status(); + + /** + * Returns the snapshot ID where the file was added or deleted. + * + *

Inherited when null. + */ + Long snapshotId(); + + /** + * Returns the data sequence number of the file. + * + *

Inherited when null and status is 1 (added). Must be equal to file_sequence_number if + * content_type is 3 or 4. + */ + Long sequenceNumber(); + + /** + * Returns the file sequence number indicating when the file was added. + * + *

Inherited when null and status is added. Must be equal to sequence_number if content_type is + * 3 or 4. + */ + Long fileSequenceNumber(); + + /** + * Returns the starting row ID for the file. + * + *

If content_type is 0 (DATA), this is the _row_id for the first row in the data file. If + * content_type is 3 (DATA_MANIFEST), this is the starting _row_id to assign to rows added by + * ADDED data files. + */ + Long firstRowId(); +} diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java new file mode 100644 index 000000000000..ecc5fe922c85 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -0,0 +1,728 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.SupportsIndexProjection; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.ByteBuffers; + +/** Generic implementation of {@link TrackedFile} for V4 manifests. */ +public class GenericTrackedFile extends SupportsIndexProjection + implements TrackedFile, + IndexedRecord, + StructLike, + SpecificData.SchemaConstructable, + Serializable { + + private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); + + // Tracking metadata + private TrackingInfo.Status status = null; + private Long snapshotId = null; + private Long sequenceNumber = null; + private Long fileSequenceNumber = null; + private Long firstRowId = null; + + // Common file metadata + private FileContent contentType = FileContent.DATA; + private String location = null; + private FileFormat fileFormat = null; + private long recordCount = -1L; + private Long fileSizeInBytes = null; + + // Deletion vector fields + private Long deletionVectorOffset = null; + private Long deletionVectorSizeInBytes = null; + private byte[] deletionVectorInlineContent = null; + + // Partition and ordering + private int partitionSpecId = -1; + private Integer sortOrderId = null; + + // Optional metadata + private byte[] keyMetadata = null; + private long[] splitOffsets = null; + private int[] equalityIds = null; + private String referencedFile = null; + + // Manifest stats (for manifest entries) + private Integer addedFilesCount = null; + private Integer existingFilesCount = null; + private Integer deletedFilesCount = null; + private Long addedRowsCount = null; + private Long existingRowsCount = null; + private Long deletedRowsCount = null; + private Long minSequenceNumber = null; + + // Content stats placeholder (TODO: implement ContentStats) + private Object contentStats = null; + + // Cached schema for Avro + private transient Schema avroSchema = null; + + // Base type that corresponds to positions for get/set + // All fields are made optional to support different content types + static final Types.StructType BASE_TYPE = + Types.StructType.of( + TrackedFile.CONTENT_TYPE, + TrackedFile.LOCATION, + TrackedFile.FILE_FORMAT, + TrackedFile.PARTITION_SPEC_ID, + TrackedFile.SORT_ORDER_ID, + TrackedFile.RECORD_COUNT, + TrackedFile.FILE_SIZE_IN_BYTES, + TrackedFile.KEY_METADATA, + TrackedFile.SPLIT_OFFSETS, + TrackedFile.EQUALITY_IDS, + TrackedFile.REFERENCED_FILE, + // Tracking info fields + TrackingInfo.STATUS.asOptional(), + TrackingInfo.SNAPSHOT_ID, + TrackingInfo.SEQUENCE_NUMBER, + TrackingInfo.FILE_SEQUENCE_NUMBER, + TrackingInfo.FIRST_ROW_ID, + // Deletion vector fields + DeletionVector.OFFSET, + DeletionVector.SIZE_IN_BYTES, + DeletionVector.INLINE_CONTENT, + // Manifest stats fields + ManifestStats.ADDED_FILES_COUNT.asOptional(), + ManifestStats.EXISTING_FILES_COUNT.asOptional(), + ManifestStats.DELETED_FILES_COUNT.asOptional(), + ManifestStats.ADDED_ROWS_COUNT.asOptional(), + ManifestStats.EXISTING_ROWS_COUNT.asOptional(), + ManifestStats.DELETED_ROWS_COUNT.asOptional(), + ManifestStats.MIN_SEQUENCE_NUMBER.asOptional()); + + /** Used by Avro reflection to instantiate this class when reading manifest files. */ + public GenericTrackedFile(Schema avroSchema) { + this(AvroSchemaUtil.convert(avroSchema).asStructType()); + this.avroSchema = avroSchema; + } + + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericTrackedFile(Types.StructType projection) { + super(BASE_TYPE, projection); + this.avroSchema = AvroSchemaUtil.convert(projection, "tracked_file"); + } + + public GenericTrackedFile() { + super(BASE_TYPE.fields().size()); + } + + /** + * Full constructor for creating a tracked file. + * + * @param contentType the type of content (DATA, POSITION_DELETES, etc.) + * @param location the file location + * @param fileFormat the file format + * @param partitionSpecId the partition spec ID + * @param recordCount the number of records + */ + public GenericTrackedFile( + FileContent contentType, + String location, + FileFormat fileFormat, + int partitionSpecId, + long recordCount, + Long fileSizeInBytes) { + super(BASE_TYPE.fields().size()); + this.contentType = contentType; + this.location = location; + this.fileFormat = fileFormat; + this.partitionSpecId = partitionSpecId; + this.recordCount = recordCount; + this.fileSizeInBytes = fileSizeInBytes; + } + + /** + * Copy constructor. + * + * @param toCopy a tracked file to copy + * @param copyStats whether to copy stats + */ + private GenericTrackedFile(GenericTrackedFile toCopy, boolean copyStats) { + super(toCopy); + + // Tracking metadata + this.status = toCopy.status; + this.snapshotId = toCopy.snapshotId; + this.sequenceNumber = toCopy.sequenceNumber; + this.fileSequenceNumber = toCopy.fileSequenceNumber; + this.firstRowId = toCopy.firstRowId; + + // Common file metadata + this.contentType = toCopy.contentType; + this.location = toCopy.location; + this.fileFormat = toCopy.fileFormat; + this.recordCount = toCopy.recordCount; + this.fileSizeInBytes = toCopy.fileSizeInBytes; + + // Deletion vector + this.deletionVectorOffset = toCopy.deletionVectorOffset; + this.deletionVectorSizeInBytes = toCopy.deletionVectorSizeInBytes; + this.deletionVectorInlineContent = + toCopy.deletionVectorInlineContent != null + ? Arrays.copyOf( + toCopy.deletionVectorInlineContent, toCopy.deletionVectorInlineContent.length) + : null; + + // Partition and ordering + this.partitionSpecId = toCopy.partitionSpecId; + this.sortOrderId = toCopy.sortOrderId; + + // Optional metadata + this.keyMetadata = + toCopy.keyMetadata != null + ? Arrays.copyOf(toCopy.keyMetadata, toCopy.keyMetadata.length) + : null; + this.splitOffsets = + toCopy.splitOffsets != null + ? Arrays.copyOf(toCopy.splitOffsets, toCopy.splitOffsets.length) + : null; + this.equalityIds = + toCopy.equalityIds != null + ? Arrays.copyOf(toCopy.equalityIds, toCopy.equalityIds.length) + : null; + this.referencedFile = toCopy.referencedFile; + + if (copyStats) { + // Manifest stats + this.addedFilesCount = toCopy.addedFilesCount; + this.existingFilesCount = toCopy.existingFilesCount; + this.deletedFilesCount = toCopy.deletedFilesCount; + this.addedRowsCount = toCopy.addedRowsCount; + this.existingRowsCount = toCopy.existingRowsCount; + this.deletedRowsCount = toCopy.deletedRowsCount; + this.minSequenceNumber = toCopy.minSequenceNumber; + this.contentStats = toCopy.contentStats; + } + } + + @Override + public TrackingInfo trackingInfo() { + if (status == null + && snapshotId == null + && sequenceNumber == null + && fileSequenceNumber == null + && firstRowId == null) { + return null; + } + + return new TrackingInfo() { + @Override + public TrackingInfo.Status status() { + return status; + } + + @Override + public Long snapshotId() { + return snapshotId; + } + + @Override + public Long sequenceNumber() { + return sequenceNumber; + } + + @Override + public Long fileSequenceNumber() { + return fileSequenceNumber; + } + + @Override + public Long firstRowId() { + return firstRowId; + } + }; + } + + public void setStatus(TrackingInfo.Status status) { + this.status = status; + } + + public void setSnapshotId(Long snapshotId) { + this.snapshotId = snapshotId; + } + + public void setSequenceNumber(Long sequenceNumber) { + this.sequenceNumber = sequenceNumber; + } + + public void setFileSequenceNumber(Long fileSequenceNumber) { + this.fileSequenceNumber = fileSequenceNumber; + } + + public void setFirstRowId(Long firstRowId) { + this.firstRowId = firstRowId; + } + + @Override + public FileContent contentType() { + return contentType; + } + + public void setContentType(FileContent contentType) { + this.contentType = contentType; + } + + @Override + public String location() { + return location; + } + + public void setLocation(String location) { + this.location = location; + } + + @Override + public FileFormat fileFormat() { + return fileFormat; + } + + public void setFileFormat(FileFormat fileFormat) { + this.fileFormat = fileFormat; + } + + @Override + public DeletionVector deletionVector() { + if (deletionVectorOffset == null + && deletionVectorSizeInBytes == null + && deletionVectorInlineContent == null) { + return null; + } + + return new DeletionVector() { + @Override + public Long offset() { + return deletionVectorOffset; + } + + @Override + public Long sizeInBytes() { + return deletionVectorSizeInBytes; + } + + @Override + public ByteBuffer inlineContent() { + return deletionVectorInlineContent != null + ? ByteBuffer.wrap(deletionVectorInlineContent) + : null; + } + }; + } + + public void setDeletionVectorOffset(Long offset) { + this.deletionVectorOffset = offset; + } + + public void setDeletionVectorSizeInBytes(Long sizeInBytes) { + this.deletionVectorSizeInBytes = sizeInBytes; + } + + public void setDeletionVectorInlineContent(ByteBuffer inlineContent) { + this.deletionVectorInlineContent = ByteBuffers.toByteArray(inlineContent); + } + + @Override + public int partitionSpecId() { + return partitionSpecId; + } + + public void setPartitionSpecId(int partitionSpecId) { + this.partitionSpecId = partitionSpecId; + } + + @Override + public Integer sortOrderId() { + return sortOrderId; + } + + public void setSortOrderId(Integer sortOrderId) { + this.sortOrderId = sortOrderId; + } + + @Override + public long recordCount() { + return recordCount; + } + + public void setRecordCount(long recordCount) { + this.recordCount = recordCount; + } + + @Override + public Long fileSizeInBytes() { + return fileSizeInBytes; + } + + public void setFileSizeInBytes(Long fileSizeInBytes) { + this.fileSizeInBytes = fileSizeInBytes; + } + + @Override + public Object contentStats() { + return contentStats; + } + + @Override + public ManifestStats manifestStats() { + if (addedFilesCount == null + && existingFilesCount == null + && deletedFilesCount == null + && addedRowsCount == null + && existingRowsCount == null + && deletedRowsCount == null + && minSequenceNumber == null) { + return null; + } + + return new ManifestStats() { + @Override + public int addedFilesCount() { + return addedFilesCount != null ? addedFilesCount : 0; + } + + @Override + public int existingFilesCount() { + return existingFilesCount != null ? existingFilesCount : 0; + } + + @Override + public int deletedFilesCount() { + return deletedFilesCount != null ? deletedFilesCount : 0; + } + + @Override + public long addedRowsCount() { + return addedRowsCount != null ? addedRowsCount : 0L; + } + + @Override + public long existingRowsCount() { + return existingRowsCount != null ? existingRowsCount : 0L; + } + + @Override + public long deletedRowsCount() { + return deletedRowsCount != null ? deletedRowsCount : 0L; + } + + @Override + public long minSequenceNumber() { + return minSequenceNumber != null ? minSequenceNumber : 0L; + } + }; + } + + public void setAddedFilesCount(Integer count) { + this.addedFilesCount = count; + } + + public void setExistingFilesCount(Integer count) { + this.existingFilesCount = count; + } + + public void setDeletedFilesCount(Integer count) { + this.deletedFilesCount = count; + } + + public void setAddedRowsCount(Long count) { + this.addedRowsCount = count; + } + + public void setExistingRowsCount(Long count) { + this.existingRowsCount = count; + } + + public void setDeletedRowsCount(Long count) { + this.deletedRowsCount = count; + } + + public void setMinSequenceNumber(Long sequenceNumber) { + this.minSequenceNumber = sequenceNumber; + } + + @Override + public ByteBuffer keyMetadata() { + return keyMetadata != null ? ByteBuffer.wrap(keyMetadata) : null; + } + + public void setKeyMetadata(ByteBuffer keyMetadata) { + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); + } + + @Override + public List splitOffsets() { + return ArrayUtil.toUnmodifiableLongList(splitOffsets); + } + + public void setSplitOffsets(List offsets) { + this.splitOffsets = ArrayUtil.toLongArray(offsets); + } + + @Override + public List equalityIds() { + return ArrayUtil.toUnmodifiableIntList(equalityIds); + } + + public void setEqualityIds(List ids) { + this.equalityIds = ArrayUtil.toIntArray(ids); + } + + @Override + public String referencedFile() { + return referencedFile; + } + + public void setReferencedFile(String referencedFile) { + this.referencedFile = referencedFile; + } + + @Override + public GenericTrackedFile copy() { + return new GenericTrackedFile(this, true); + } + + @Override + public GenericTrackedFile copyWithoutStats() { + return new GenericTrackedFile(this, false); + } + + @Override + public Schema getSchema() { + if (avroSchema == null) { + this.avroSchema = getAvroSchema(); + } + return avroSchema; + } + + @Override + public void put(int i, Object v) { + set(i, v); + } + + @Override + protected void internalSet(int pos, T value) { + switch (pos) { + case 0: + this.contentType = value != null ? FILE_CONTENT_VALUES[(Integer) value] : null; + return; + case 1: + this.location = (String) value; + return; + case 2: + this.fileFormat = value != null ? FileFormat.fromString(value.toString()) : null; + return; + case 3: + this.partitionSpecId = value != null ? (Integer) value : -1; + return; + case 4: + this.sortOrderId = (Integer) value; + return; + case 5: + this.recordCount = value != null ? (Long) value : -1L; + return; + case 6: + this.fileSizeInBytes = (Long) value; + return; + case 7: + this.keyMetadata = ByteBuffers.toByteArray((ByteBuffer) value); + return; + case 8: + this.splitOffsets = ArrayUtil.toLongArray((List) value); + return; + case 9: + this.equalityIds = ArrayUtil.toIntArray((List) value); + return; + case 10: + this.referencedFile = (String) value; + return; + case 11: + this.status = value != null ? TrackingInfo.Status.values()[(Integer) value] : null; + return; + case 12: + this.snapshotId = (Long) value; + return; + case 13: + this.sequenceNumber = (Long) value; + return; + case 14: + this.fileSequenceNumber = (Long) value; + return; + case 15: + this.firstRowId = (Long) value; + return; + case 16: + this.deletionVectorOffset = (Long) value; + return; + case 17: + this.deletionVectorSizeInBytes = (Long) value; + return; + case 18: + this.deletionVectorInlineContent = ByteBuffers.toByteArray((ByteBuffer) value); + return; + case 19: + this.addedFilesCount = (Integer) value; + return; + case 20: + this.existingFilesCount = (Integer) value; + return; + case 21: + this.deletedFilesCount = (Integer) value; + return; + case 22: + this.addedRowsCount = (Long) value; + return; + case 23: + this.existingRowsCount = (Long) value; + return; + case 24: + this.deletedRowsCount = (Long) value; + return; + case 25: + this.minSequenceNumber = (Long) value; + return; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + } + } + + @Override + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); + } + + @Override + public Object get(int pos) { + return getByPos(pos); + } + + private Object getByPos(int pos) { + switch (pos) { + case 0: + return contentType != null ? contentType.id() : null; + case 1: + return location; + case 2: + return fileFormat != null ? fileFormat.name().toLowerCase(Locale.ROOT) : null; + case 3: + return partitionSpecId; + case 4: + return sortOrderId; + case 5: + return recordCount; + case 6: + return fileSizeInBytes; + case 7: + return keyMetadata(); + case 8: + return splitOffsets(); + case 9: + return equalityIds(); + case 10: + return referencedFile; + case 11: + return status != null ? status.id() : null; + case 12: + return snapshotId; + case 13: + return sequenceNumber; + case 14: + return fileSequenceNumber; + case 15: + return firstRowId; + case 16: + return deletionVectorOffset; + case 17: + return deletionVectorSizeInBytes; + case 18: + return deletionVectorInlineContent != null + ? ByteBuffer.wrap(deletionVectorInlineContent) + : null; + case 19: + return addedFilesCount; + case 20: + return existingFilesCount; + case 21: + return deletedFilesCount; + case 22: + return addedRowsCount; + case 23: + return existingRowsCount; + case 24: + return deletedRowsCount; + case 25: + return minSequenceNumber; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + } + } + + private Schema getAvroSchema() { + return AvroSchemaUtil.convert( + BASE_TYPE, ImmutableMap.of(BASE_TYPE, GenericTrackedFile.class.getName())); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("content_type", contentType) + .add("location", location) + .add("file_format", fileFormat) + .add("partition_spec_id", partitionSpecId) + .add("record_count", recordCount) + .add("file_size_in_bytes", fileSizeInBytes) + .add("sort_order_id", sortOrderId) + .add("key_metadata", keyMetadata == null ? "null" : "(redacted)") + .add("split_offsets", splitOffsets) + .add("equality_ids", equalityIds) + .add("referenced_file", referencedFile) + .add("status", status) + .add("snapshot_id", snapshotId) + .add("sequence_number", sequenceNumber) + .add("file_sequence_number", fileSequenceNumber) + .add("first_row_id", firstRowId) + .add("deletion_vector_offset", deletionVectorOffset) + .add("deletion_vector_size_in_bytes", deletionVectorSizeInBytes) + .add( + "deletion_vector_inline_content", + deletionVectorInlineContent == null ? "null" : "(inline)") + .add("added_files_count", addedFilesCount) + .add("existing_files_count", existingFilesCount) + .add("deleted_files_count", deletedFilesCount) + .add("added_rows_count", addedRowsCount) + .add("existing_rows_count", existingRowsCount) + .add("deleted_rows_count", deletedRowsCount) + .add("min_sequence_number", minSequenceNumber) + .toString(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestGenericTrackedFile.java b/core/src/test/java/org/apache/iceberg/TestGenericTrackedFile.java new file mode 100644 index 000000000000..15b307726ce3 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestGenericTrackedFile.java @@ -0,0 +1,418 @@ +/* + * 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 java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.util.List; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestGenericTrackedFile { + + @TempDir private Path temp; + + private static Schema getTrackedFileSchema() { + return new Schema(GenericTrackedFile.BASE_TYPE.fields()); + } + + @Test + public void testAvroRoundTripDataFile() throws IOException { + // Create a tracked file representing a data file + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.DATA); + original.setLocation("s3://bucket/table/data/file1.parquet"); + original.setFileFormat(FileFormat.PARQUET); + original.setPartitionSpecId(0); + original.setRecordCount(1000L); + original.setFileSizeInBytes(50000L); + original.setSortOrderId(1); + original.setSplitOffsets(ImmutableList.of(0L, 10000L, 20000L)); + + // Set tracking info + original.setStatus(TrackingInfo.Status.ADDED); + original.setSnapshotId(12345L); + original.setSequenceNumber(100L); + original.setFileSequenceNumber(100L); + original.setFirstRowId(0L); + + // Set key metadata + original.setKeyMetadata(ByteBuffer.wrap(new byte[] {1, 2, 3, 4})); + + // Write to file + OutputFile outputFile = Files.localOutput(temp.resolve("tracked-file.parquet").toFile()); + List written; + try (FileAppender appender = + InternalData.write(FileFormat.PARQUET, outputFile) + .schema(getTrackedFileSchema()) + .named("tracked_file") + .build()) { + appender.add(original); + written = ImmutableList.of(original); + } + + // Read back + InputFile inputFile = outputFile.toInputFile(); + List read; + try (CloseableIterable files = + InternalData.read(FileFormat.PARQUET, inputFile) + .setRootType(GenericTrackedFile.class) + .project(getTrackedFileSchema()) + .build()) { + read = Lists.newArrayList(files); + } + + // Verify + assertThat(read).hasSize(1); + GenericTrackedFile roundTripped = read.get(0); + + assertThat(roundTripped.contentType()).isEqualTo(FileContent.DATA); + assertThat(roundTripped.location()).isEqualTo("s3://bucket/table/data/file1.parquet"); + assertThat(roundTripped.fileFormat()).isEqualTo(FileFormat.PARQUET); + assertThat(roundTripped.partitionSpecId()).isEqualTo(0); + assertThat(roundTripped.recordCount()).isEqualTo(1000L); + assertThat(roundTripped.fileSizeInBytes()).isEqualTo(50000L); + assertThat(roundTripped.sortOrderId()).isEqualTo(1); + assertThat(roundTripped.splitOffsets()).containsExactly(0L, 10000L, 20000L); + + // Verify tracking info + TrackingInfo trackingInfo = roundTripped.trackingInfo(); + assertThat(trackingInfo).isNotNull(); + assertThat(trackingInfo.status()).isEqualTo(TrackingInfo.Status.ADDED); + assertThat(trackingInfo.snapshotId()).isEqualTo(12345L); + assertThat(trackingInfo.sequenceNumber()).isEqualTo(100L); + assertThat(trackingInfo.fileSequenceNumber()).isEqualTo(100L); + assertThat(trackingInfo.firstRowId()).isEqualTo(0L); + + // Verify key metadata + assertThat(roundTripped.keyMetadata()).isEqualTo(ByteBuffer.wrap(new byte[] {1, 2, 3, 4})); + } + + @Test + public void testAvroRoundTripPositionDeletesWithDeletionVector() throws IOException { + // Create a tracked file representing position deletes with external deletion vector + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.POSITION_DELETES); + original.setLocation("s3://bucket/table/deletes/dv1.puffin"); + original.setFileFormat(FileFormat.PUFFIN); + original.setPartitionSpecId(0); + original.setRecordCount(50L); // 50 deleted positions + original.setFileSizeInBytes(1000L); + original.setReferencedFile("s3://bucket/table/data/file1.parquet"); + + // Set deletion vector (external) + original.setDeletionVectorOffset(100L); + original.setDeletionVectorSizeInBytes(500L); + + // Set tracking info + original.setStatus(TrackingInfo.Status.ADDED); + original.setSnapshotId(12346L); + original.setSequenceNumber(101L); + original.setFileSequenceNumber(101L); + + // Write to file + OutputFile outputFile = Files.localOutput(temp.resolve("dv-tracked-file.avro").toFile()); + try (FileAppender appender = + InternalData.write(FileFormat.AVRO, outputFile) + .schema(getTrackedFileSchema()) + .named("tracked_file") + .build()) { + appender.add(original); + } + + // Read back + InputFile inputFile = outputFile.toInputFile(); + List read; + try (CloseableIterable files = + InternalData.read(FileFormat.AVRO, inputFile) + .setRootType(GenericTrackedFile.class) + .project(getTrackedFileSchema()) + .build()) { + read = Lists.newArrayList(files); + } + + // Verify + assertThat(read).hasSize(1); + GenericTrackedFile roundTripped = read.get(0); + + assertThat(roundTripped.contentType()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(roundTripped.location()).isEqualTo("s3://bucket/table/deletes/dv1.puffin"); + assertThat(roundTripped.fileFormat()).isEqualTo(FileFormat.PUFFIN); + assertThat(roundTripped.recordCount()).isEqualTo(50L); + assertThat(roundTripped.referencedFile()).isEqualTo("s3://bucket/table/data/file1.parquet"); + + // Verify deletion vector + DeletionVector dv = roundTripped.deletionVector(); + assertThat(dv).isNotNull(); + assertThat(dv.offset()).isEqualTo(100L); + assertThat(dv.sizeInBytes()).isEqualTo(500L); + assertThat(dv.inlineContent()).isNull(); + } + + @Test + public void testAvroRoundTripManifestDVWithInlineContent() throws IOException { + // Create a tracked file representing a manifest deletion vector with inline content + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.MANIFEST_DV); + original.setFileFormat(FileFormat.PUFFIN); // Even inline DVs need a format + original.setPartitionSpecId(0); + original.setRecordCount(3L); // 3 deleted manifest entries + original.setReferencedFile("s3://bucket/table/metadata/manifest1.avro"); + + // Set inline deletion vector + byte[] inlineBitmap = new byte[] {(byte) 0b00000111}; // positions 0, 1, 2 deleted + original.setDeletionVectorInlineContent(ByteBuffer.wrap(inlineBitmap)); + + // Set tracking info + original.setStatus(TrackingInfo.Status.ADDED); + original.setSnapshotId(12347L); + + // Write to file + OutputFile outputFile = + Files.localOutput(temp.resolve("manifest-dv-tracked-file.avro").toFile()); + try (FileAppender appender = + InternalData.write(FileFormat.AVRO, outputFile) + .schema(getTrackedFileSchema()) + .named("tracked_file") + .build()) { + appender.add(original); + } + + // Read back + InputFile inputFile = outputFile.toInputFile(); + List read; + try (CloseableIterable files = + InternalData.read(FileFormat.AVRO, inputFile) + .setRootType(GenericTrackedFile.class) + .project(getTrackedFileSchema()) + .build()) { + read = Lists.newArrayList(files); + } + + // Verify + assertThat(read).hasSize(1); + GenericTrackedFile roundTripped = read.get(0); + + assertThat(roundTripped.contentType()).isEqualTo(FileContent.MANIFEST_DV); + assertThat(roundTripped.location()).isNull(); // Can be null for inline DVs + assertThat(roundTripped.recordCount()).isEqualTo(3L); + assertThat(roundTripped.referencedFile()) + .isEqualTo("s3://bucket/table/metadata/manifest1.avro"); + + // Verify inline deletion vector + DeletionVector dv = roundTripped.deletionVector(); + assertThat(dv).isNotNull(); + assertThat(dv.offset()).isNull(); + assertThat(dv.sizeInBytes()).isNull(); + assertThat(dv.inlineContent()).isEqualTo(ByteBuffer.wrap(inlineBitmap)); + } + + @Test + public void testAvroRoundTripDataManifestWithStats() throws IOException { + // Create a tracked file representing a data manifest entry + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.DATA_MANIFEST); + original.setLocation("s3://bucket/table/metadata/manifest-data-1.avro"); + original.setFileFormat(FileFormat.AVRO); + original.setPartitionSpecId(0); + original.setRecordCount(100L); // 100 entries in the manifest + original.setFileSizeInBytes(25000L); + + // Set manifest stats + original.setAddedFilesCount(10); + original.setExistingFilesCount(85); + original.setDeletedFilesCount(5); + original.setAddedRowsCount(10000L); + original.setExistingRowsCount(850000L); + original.setDeletedRowsCount(5000L); + original.setMinSequenceNumber(50L); + + // Set tracking info + original.setStatus(TrackingInfo.Status.EXISTING); + original.setSnapshotId(12348L); + original.setSequenceNumber(102L); + original.setFileSequenceNumber(100L); + original.setFirstRowId(100000L); // Starting row ID for new data files + + // Write to file + OutputFile outputFile = + Files.localOutput(temp.resolve("data-manifest-tracked-file.avro").toFile()); + try (FileAppender appender = + InternalData.write(FileFormat.AVRO, outputFile) + .schema(getTrackedFileSchema()) + .named("tracked_file") + .build()) { + appender.add(original); + } + + // Read back + InputFile inputFile = outputFile.toInputFile(); + List read; + try (CloseableIterable files = + InternalData.read(FileFormat.AVRO, inputFile) + .setRootType(GenericTrackedFile.class) + .project(getTrackedFileSchema()) + .build()) { + read = Lists.newArrayList(files); + } + + // Verify + assertThat(read).hasSize(1); + GenericTrackedFile roundTripped = read.get(0); + + assertThat(roundTripped.contentType()).isEqualTo(FileContent.DATA_MANIFEST); + assertThat(roundTripped.location()) + .isEqualTo("s3://bucket/table/metadata/manifest-data-1.avro"); + assertThat(roundTripped.fileFormat()).isEqualTo(FileFormat.AVRO); + assertThat(roundTripped.recordCount()).isEqualTo(100L); + assertThat(roundTripped.fileSizeInBytes()).isEqualTo(25000L); + + // Verify manifest stats + ManifestStats stats = roundTripped.manifestStats(); + assertThat(stats).isNotNull(); + assertThat(stats.addedFilesCount()).isEqualTo(10); + assertThat(stats.existingFilesCount()).isEqualTo(85); + assertThat(stats.deletedFilesCount()).isEqualTo(5); + assertThat(stats.addedRowsCount()).isEqualTo(10000L); + assertThat(stats.existingRowsCount()).isEqualTo(850000L); + assertThat(stats.deletedRowsCount()).isEqualTo(5000L); + assertThat(stats.minSequenceNumber()).isEqualTo(50L); + + // Verify tracking info + TrackingInfo trackingInfo = roundTripped.trackingInfo(); + assertThat(trackingInfo).isNotNull(); + assertThat(trackingInfo.status()).isEqualTo(TrackingInfo.Status.EXISTING); + assertThat(trackingInfo.firstRowId()).isEqualTo(100000L); + } + + @Test + public void testAvroRoundTripEqualityDeletes() throws IOException { + // Create a tracked file representing equality deletes + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.EQUALITY_DELETES); + original.setLocation("s3://bucket/table/deletes/eq-delete-1.parquet"); + original.setFileFormat(FileFormat.PARQUET); + original.setPartitionSpecId(0); + original.setRecordCount(25L); + original.setFileSizeInBytes(5000L); + original.setEqualityIds(ImmutableList.of(1, 2, 5)); // Field IDs for equality comparison + original.setSortOrderId(1); + + // Set tracking info + original.setStatus(TrackingInfo.Status.ADDED); + original.setSnapshotId(12349L); + original.setSequenceNumber(103L); + original.setFileSequenceNumber(103L); + + // Write to file + OutputFile outputFile = + Files.localOutput(temp.resolve("equality-delete-tracked-file.avro").toFile()); + try (FileAppender appender = + InternalData.write(FileFormat.AVRO, outputFile) + .schema(getTrackedFileSchema()) + .named("tracked_file") + .build()) { + appender.add(original); + } + + // Read back + InputFile inputFile = outputFile.toInputFile(); + List read; + try (CloseableIterable files = + InternalData.read(FileFormat.AVRO, inputFile) + .setRootType(GenericTrackedFile.class) + .project(getTrackedFileSchema()) + .build()) { + read = Lists.newArrayList(files); + } + + // Verify + assertThat(read).hasSize(1); + GenericTrackedFile roundTripped = read.get(0); + + assertThat(roundTripped.contentType()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(roundTripped.location()).isEqualTo("s3://bucket/table/deletes/eq-delete-1.parquet"); + assertThat(roundTripped.fileFormat()).isEqualTo(FileFormat.PARQUET); + assertThat(roundTripped.recordCount()).isEqualTo(25L); + assertThat(roundTripped.equalityIds()).containsExactly(1, 2, 5); + assertThat(roundTripped.sortOrderId()).isEqualTo(1); + } + + @Test + public void testCopy() { + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.DATA); + original.setLocation("s3://bucket/table/data/file1.parquet"); + original.setFileFormat(FileFormat.PARQUET); + original.setPartitionSpecId(0); + original.setRecordCount(1000L); + original.setFileSizeInBytes(50000L); + + // Set manifest stats (should be copied) + original.setAddedFilesCount(10); + original.setMinSequenceNumber(50L); + + GenericTrackedFile copy = original.copy(); + + // Verify copy is equal but separate instance + assertThat(copy).isNotSameAs(original); + assertThat(copy.contentType()).isEqualTo(original.contentType()); + assertThat(copy.location()).isEqualTo(original.location()); + assertThat(copy.recordCount()).isEqualTo(original.recordCount()); + + // Verify stats were copied + assertThat(copy.manifestStats()).isNotNull(); + assertThat(copy.manifestStats().addedFilesCount()).isEqualTo(10); + } + + @Test + public void testCopyWithoutStats() { + GenericTrackedFile original = new GenericTrackedFile(); + original.setContentType(FileContent.DATA); + original.setLocation("s3://bucket/table/data/file1.parquet"); + original.setFileFormat(FileFormat.PARQUET); + original.setPartitionSpecId(0); + original.setRecordCount(1000L); + original.setFileSizeInBytes(50000L); + + // Set manifest stats (should NOT be copied) + original.setAddedFilesCount(10); + original.setMinSequenceNumber(50L); + + GenericTrackedFile copy = original.copyWithoutStats(); + + // Verify copy is equal but stats are dropped + assertThat(copy).isNotSameAs(original); + assertThat(copy.contentType()).isEqualTo(original.contentType()); + assertThat(copy.location()).isEqualTo(original.location()); + assertThat(copy.recordCount()).isEqualTo(original.recordCount()); + + // Verify stats were NOT copied + assertThat(copy.manifestStats()).isNull(); + } +} From c278fcc678b40e9353d00df4b05688a153007008 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 31 Oct 2025 12:49:21 -0700 Subject: [PATCH 02/14] Fix checkstyle error --- core/src/main/java/org/apache/iceberg/GenericTrackedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java index ecc5fe922c85..5f4da037ced9 100644 --- a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -467,8 +467,8 @@ public void setDeletedRowsCount(Long count) { this.deletedRowsCount = count; } - public void setMinSequenceNumber(Long sequenceNumber) { - this.minSequenceNumber = sequenceNumber; + public void setMinSequenceNumber(Long minSeqNum) { + this.minSequenceNumber = minSeqNum; } @Override From 5925ff16b430c632e176830a6db7269d99e9ec02 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 31 Oct 2025 14:11:53 -0700 Subject: [PATCH 03/14] Add position, setters --- .../java/org/apache/iceberg/TrackedFile.java | 25 +++++++++++++++++++ .../apache/iceberg/GenericTrackedFile.java | 16 ++++++++++++ 2 files changed, 41 insertions(+) diff --git a/api/src/main/java/org/apache/iceberg/TrackedFile.java b/api/src/main/java/org/apache/iceberg/TrackedFile.java index 1829ab399f1e..32b1c3152dd1 100644 --- a/api/src/main/java/org/apache/iceberg/TrackedFile.java +++ b/api/src/main/java/org/apache/iceberg/TrackedFile.java @@ -208,4 +208,29 @@ public interface TrackedFile { *

Use this method to copy data without stats when collecting files. */ F copyWithoutStats(); + + /** + * Returns the ordinal position in the manifest. + * + *

Used for applying manifest deletion vectors. + */ + Long pos(); + + /** Set the status for this tracked file entry. */ + void setStatus(TrackingInfo.Status status); + + /** Set the snapshot ID for this tracked file entry. */ + void setSnapshotId(Long snapshotId); + + /** Set the data sequence number for this tracked file entry. */ + void setSequenceNumber(Long sequenceNumber); + + /** Set the file sequence number for this tracked file entry. */ + void setFileSequenceNumber(Long fileSequenceNumber); + + /** Set the first row ID for this tracked file entry. */ + void setFirstRowId(Long firstRowId); + + /** Set the ordinal position in the manifest. */ + void setPos(Long position); } diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java index 5f4da037ced9..a73a4a537eb5 100644 --- a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -50,6 +50,7 @@ public class GenericTrackedFile extends SupportsIndexProjection private Long sequenceNumber = null; private Long fileSequenceNumber = null; private Long firstRowId = null; + private Long position = null; // Common file metadata private FileContent contentType = FileContent.DATA; @@ -265,26 +266,41 @@ public Long firstRowId() { }; } + @Override + public Long pos() { + return position; + } + + @Override public void setStatus(TrackingInfo.Status status) { this.status = status; } + @Override public void setSnapshotId(Long snapshotId) { this.snapshotId = snapshotId; } + @Override public void setSequenceNumber(Long sequenceNumber) { this.sequenceNumber = sequenceNumber; } + @Override public void setFileSequenceNumber(Long fileSequenceNumber) { this.fileSequenceNumber = fileSequenceNumber; } + @Override public void setFirstRowId(Long firstRowId) { this.firstRowId = firstRowId; } + @Override + public void setPos(Long position) { + this.position = position; + } + @Override public FileContent contentType() { return contentType; From d6302a0586b8aaf196eb8364d281783509fe6b24 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 31 Oct 2025 14:33:50 -0700 Subject: [PATCH 04/14] Fix checkstyle error --- .../src/main/java/org/apache/iceberg/GenericTrackedFile.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java index a73a4a537eb5..702ed744fe2b 100644 --- a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -179,6 +179,7 @@ private GenericTrackedFile(GenericTrackedFile toCopy, boolean copyStats) { this.sequenceNumber = toCopy.sequenceNumber; this.fileSequenceNumber = toCopy.fileSequenceNumber; this.firstRowId = toCopy.firstRowId; + this.position = toCopy.position; // Common file metadata this.contentType = toCopy.contentType; @@ -297,8 +298,8 @@ public void setFirstRowId(Long firstRowId) { } @Override - public void setPos(Long position) { - this.position = position; + public void setPos(Long pos) { + this.position = pos; } @Override From bbb44cb563ce3af52367593d4bedc310e3f6780b Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 07:12:29 -0800 Subject: [PATCH 05/14] implement reader --- .../iceberg/InheritableTrackedMetadata.java | 36 +++ .../InheritableTrackedMetadataFactory.java | 93 ++++++ .../org/apache/iceberg/TrackedFileReader.java | 169 +++++++++++ .../apache/iceberg/TrackedFileReaders.java | 76 +++++ .../apache/iceberg/TestTrackedFileReader.java | 268 ++++++++++++++++++ 5 files changed, 642 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/InheritableTrackedMetadata.java create mode 100644 core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java create mode 100644 core/src/main/java/org/apache/iceberg/TrackedFileReader.java create mode 100644 core/src/main/java/org/apache/iceberg/TrackedFileReaders.java create mode 100644 core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java diff --git a/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadata.java b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadata.java new file mode 100644 index 000000000000..6039e27fa98b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadata.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.Serializable; + +/** + * Interface for applying inheritable metadata to tracked file entries in V4 manifests. + * + *

Similar to {@link InheritableMetadata} but for V4 TrackedFile entries. + */ +interface InheritableTrackedMetadata extends Serializable { + /** + * Apply inheritable metadata to a tracked file entry. + * + * @param entry the tracked file entry + * @return the entry with metadata applied + */ + TrackedFile apply(TrackedFile entry); +} diff --git a/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java new file mode 100644 index 000000000000..973fbde377ac --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Factory for creating {@link InheritableTrackedMetadata} instances. */ +class InheritableTrackedMetadataFactory { + + private InheritableTrackedMetadataFactory() {} + + /** + * Creates inheritable metadata from explicit values. + * + * @param snapshotId the snapshot ID + * @param sequenceNumber the sequence number + * @return inheritable metadata instance + */ + static InheritableTrackedMetadata create(long snapshotId, long sequenceNumber) { + return new BaseInheritableTrackedMetadata(snapshotId, sequenceNumber); + } + + /** + * Creates inheritable metadata from a tracked file representing a manifest (for reading leaf + * manifests). + * + * @param manifestEntry the DATA_MANIFEST or DELETE_MANIFEST tracked file from root + * @return inheritable metadata instance + */ + static InheritableTrackedMetadata fromTrackedFile(TrackedFile manifestEntry) { + Preconditions.checkArgument( + manifestEntry.contentType() == FileContent.DATA_MANIFEST + || manifestEntry.contentType() == FileContent.DELETE_MANIFEST, + "Can only create metadata from manifest entries, got: %s", + manifestEntry.contentType()); + + TrackingInfo tracking = manifestEntry.trackingInfo(); + Long snapshotId = tracking != null ? tracking.snapshotId() : null; + Long sequenceNumber = tracking != null ? tracking.sequenceNumber() : null; + + Preconditions.checkArgument( + snapshotId != null, "Manifest entry must have snapshot ID: %s", manifestEntry.location()); + + return new BaseInheritableTrackedMetadata( + snapshotId, sequenceNumber != null ? sequenceNumber : 0L); + } + + static class BaseInheritableTrackedMetadata implements InheritableTrackedMetadata { + private final long snapshotId; + private final long sequenceNumber; + + private BaseInheritableTrackedMetadata(long snapshotId, long sequenceNumber) { + this.snapshotId = snapshotId; + this.sequenceNumber = sequenceNumber; + } + + @Override + public TrackedFile apply(TrackedFile entry) { + TrackingInfo tracking = entry.trackingInfo(); + + if (tracking == null || tracking.snapshotId() == null) { + entry.setSnapshotId(snapshotId); + } + + // in v1 tables, the sequence number is not persisted and can be safely defaulted to 0 + // in v2+ tables, the sequence number should be inherited iff the entry status is ADDED + if (tracking == null || tracking.sequenceNumber() == null) { + if (sequenceNumber == 0 || (tracking != null && tracking.status() == TrackingInfo.Status.ADDED)) { + entry.setSequenceNumber(sequenceNumber); + entry.setFileSequenceNumber(sequenceNumber); + } + } + + return entry; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileReader.java b/core/src/main/java/org/apache/iceberg/TrackedFileReader.java new file mode 100644 index 000000000000..ad170b6bbffb --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackedFileReader.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.Collection; +import java.util.List; +import java.util.function.Function; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; + +/** + * Reader for V4 manifest files containing TrackedFile entries. + * + *

Supports reading both root manifests and leaf manifests. Returns TrackedFile entries which can + * represent data files, delete files, or manifest references. + */ +class TrackedFileReader extends CloseableGroup implements CloseableIterable> { + static final ImmutableList ALL_COLUMNS = ImmutableList.of("*"); + + private final InputFile file; + private final InheritableTrackedMetadata inheritableMetadata; + private final Long manifestFirstRowId; + + private Collection columns = null; + + protected TrackedFileReader( + InputFile file, + InheritableTrackedMetadata inheritableMetadata, + Long manifestFirstRowId) { + this.file = file; + this.inheritableMetadata = inheritableMetadata; + this.manifestFirstRowId = manifestFirstRowId; + } + + public TrackedFileReader select(Collection newColumns) { + this.columns = newColumns; + return this; + } + + public CloseableIterable> entries() { + return entries(false); + } + + public CloseableIterable> liveEntries() { + return entries(true); + } + + private CloseableIterable> entries(boolean onlyLive) { + CloseableIterable> entries = open(columns); + return onlyLive ? filterLiveEntries(entries) : entries; + } + + private CloseableIterable> open(Collection cols) { + Schema projection = buildProjection(cols); + + FileFormat format = FileFormat.fromFileName(file.location()); + + CloseableIterable entries = + InternalData.read(format, file) + .project(projection) + .setRootType(GenericTrackedFile.class) + .build(); + + addCloseable(entries); + + CloseableIterable> transformed = + CloseableIterable.transform(entries, inheritableMetadata::apply); + + transformed = CloseableIterable.transform(transformed, rowIdAssigner(manifestFirstRowId)); + + transformed = assignPositions(transformed); + + return transformed; + } + + private Schema buildProjection(Collection cols) { + if (cols == null || ALL_COLUMNS.equals(cols)) { + return new Schema(GenericTrackedFile.BASE_TYPE.fields()); + } + + List fields = Lists.newArrayList(); + for (String column : cols) { + Types.NestedField field = GenericTrackedFile.BASE_TYPE.field(column); + if (field != null) { + fields.add(field); + } + } + + return new Schema(fields); + } + + private CloseableIterable> filterLiveEntries( + CloseableIterable> entries) { + return CloseableIterable.filter( + entries, + entry -> { + TrackingInfo tracking = entry.trackingInfo(); + return tracking == null || tracking.status() != TrackingInfo.Status.DELETED; + }); + } + + private static Function, TrackedFile> rowIdAssigner(Long firstRowId) { + if (firstRowId == null) { + return entry -> entry; + } + + return new Function<>() { + private long nextRowId = firstRowId; + + @Override + public TrackedFile apply(TrackedFile entry) { + if (entry.contentType() == FileContent.DATA) { + TrackingInfo tracking = entry.trackingInfo(); + if (tracking != null + && tracking.status() != TrackingInfo.Status.DELETED + && tracking.firstRowId() == null) { + entry.setFirstRowId(nextRowId); + nextRowId += entry.recordCount(); + } + } + return entry; + } + }; + } + + private CloseableIterable> assignPositions( + CloseableIterable> entries) { + return CloseableIterable.transform( + entries, + new Function<>() { + private long position = 0; + + @Override + public TrackedFile apply(TrackedFile entry) { + entry.setPos(position++); + return entry; + } + }); + } + + @Override + @SuppressWarnings("unchecked") + public CloseableIterator> iterator() { + return (CloseableIterator>) + (CloseableIterator) + CloseableIterable.transform(liveEntries(), TrackedFile::copy).iterator(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileReaders.java b/core/src/main/java/org/apache/iceberg/TrackedFileReaders.java new file mode 100644 index 000000000000..ddcd23bda9ee --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackedFileReaders.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.Map; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Factory methods for creating {@link TrackedFileReader} instances. */ +public class TrackedFileReaders { + + private TrackedFileReaders() {} + + /** + * Create a reader for a root manifest. + * + * @param rootManifestPath path to the root manifest file + * @param io file IO for reading + * @param snapshotId snapshot ID for metadata inheritance + * @param sequenceNumber sequence number for metadata inheritance + * @param firstRowId starting first row ID for data files (can be null) + * @return a TrackedFileReader for the root manifest + */ + public static TrackedFileReader readRoot( + String rootManifestPath, FileIO io, long snapshotId, long sequenceNumber, Long firstRowId) { + InputFile inputFile = io.newInputFile(rootManifestPath); + InheritableTrackedMetadata metadata = + InheritableTrackedMetadataFactory.create(snapshotId, sequenceNumber); + + return new TrackedFileReader(inputFile, metadata, firstRowId); + } + + /** + * Create a reader for a leaf manifest referenced from a root manifest. + * + * @param manifestEntry the DATA_MANIFEST or DELETE_MANIFEST entry from root + * @param io file IO for reading + * @param specsById map of partition specs by ID + * @return a TrackedFileReader for the leaf manifest + */ + public static TrackedFileReader readLeaf( + TrackedFile manifestEntry, FileIO io, Map specsById) { + Preconditions.checkArgument( + manifestEntry.contentType() == FileContent.DATA_MANIFEST + || manifestEntry.contentType() == FileContent.DELETE_MANIFEST, + "Can only read manifest entries, got: %s", + manifestEntry.contentType()); + + InputFile inputFile = io.newInputFile(manifestEntry.location()); + + InheritableTrackedMetadata metadata = + InheritableTrackedMetadataFactory.fromTrackedFile(manifestEntry); + + TrackingInfo tracking = manifestEntry.trackingInfo(); + Long firstRowId = tracking != null ? tracking.firstRowId() : null; + + return new TrackedFileReader(inputFile, metadata, firstRowId); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java b/core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java new file mode 100644 index 000000000000..01b833e1bcbd --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java @@ -0,0 +1,268 @@ +/* + * 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 java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.iceberg.inmemory.InMemoryFileIO; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestTrackedFileReader { + + private final FileIO io = new InMemoryFileIO(); + private static final long SNAPSHOT_ID = 12345L; + private static final long SEQUENCE_NUMBER = 100L; + + @TempDir private Path temp; + + @Test + public void testReadFlatManifest() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L, 50000L); + file1.setStatus(TrackingInfo.Status.ADDED); + file1.setSnapshotId(SNAPSHOT_ID); + file1.setSequenceNumber(SEQUENCE_NUMBER); + file1.setFileSequenceNumber(SEQUENCE_NUMBER); + + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L, 100000L); + file2.setStatus(TrackingInfo.Status.EXISTING); + file2.setSnapshotId(SNAPSHOT_ID - 1); + file2.setSequenceNumber(SEQUENCE_NUMBER - 1); + file2.setFileSequenceNumber(SEQUENCE_NUMBER - 1); + + String manifestPath = writeManifest(file1, file2); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> files = Lists.newArrayList(reader); + + assertThat(files).hasSize(2); + + TrackedFile read1 = files.get(0); + assertThat(read1.location()).isNotNull(); + assertThat(read1.location()).endsWith("file1.parquet"); + assertThat(read1.recordCount()).isEqualTo(1000L); + assertThat(read1.fileSizeInBytes()).isEqualTo(50000L); + assertThat(read1.pos()).isNotNull(); + assertThat(read1.pos()).isEqualTo(0L); + + TrackedFile read2 = files.get(1); + assertThat(read2.location()).endsWith("file2.parquet"); + assertThat(read2.recordCount()).isEqualTo(2000L); + assertThat(read2.pos()).isEqualTo(1L); + } + + @Test + public void testInheritSnapshotId() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L, 50000L); + file1.setStatus(TrackingInfo.Status.ADDED); + + String manifestPath = writeManifest(file1); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> files = Lists.newArrayList(reader); + + assertThat(files).hasSize(1); + TrackedFile read = files.get(0); + + TrackingInfo tracking = read.trackingInfo(); + assertThat(tracking).isNotNull(); + assertThat(tracking.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(tracking.sequenceNumber()).isEqualTo(SEQUENCE_NUMBER); + assertThat(tracking.fileSequenceNumber()).isEqualTo(SEQUENCE_NUMBER); + } + + @Test + public void testInheritSequenceNumberForAddedOnly() throws IOException { + GenericTrackedFile added = createDataFile("added.parquet", 1000L, 50000L); + added.setStatus(TrackingInfo.Status.ADDED); + added.setSnapshotId(SNAPSHOT_ID); + + GenericTrackedFile existing = createDataFile("existing.parquet", 2000L, 100000L); + existing.setStatus(TrackingInfo.Status.EXISTING); + existing.setSnapshotId(SNAPSHOT_ID - 1); + existing.setSequenceNumber(SEQUENCE_NUMBER - 10); + existing.setFileSequenceNumber(SEQUENCE_NUMBER - 10); + + String manifestPath = writeManifest(added, existing); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> files = Lists.newArrayList(reader); + + assertThat(files).hasSize(2); + + TrackedFile readAdded = files.get(0); + assertThat(readAdded.trackingInfo().status()).isEqualTo(TrackingInfo.Status.ADDED); + assertThat(readAdded.trackingInfo().sequenceNumber()).isEqualTo(SEQUENCE_NUMBER); + + TrackedFile readExisting = files.get(1); + assertThat(readExisting.trackingInfo().status()).isEqualTo(TrackingInfo.Status.EXISTING); + assertThat(readExisting.trackingInfo().sequenceNumber()).isEqualTo(SEQUENCE_NUMBER - 10); + } + + @Test + public void testRowIdAssignment() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L, 50000L); + file1.setStatus(TrackingInfo.Status.ADDED); + file1.setSnapshotId(SNAPSHOT_ID); + + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L, 100000L); + file2.setStatus(TrackingInfo.Status.ADDED); + file2.setSnapshotId(SNAPSHOT_ID); + + String manifestPath = writeManifest(file1, file2); + + long startingRowId = 1000L; + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, startingRowId); + + List> files = Lists.newArrayList(reader); + + assertThat(files).hasSize(2); + + TrackedFile read1 = files.get(0); + assertThat(read1.trackingInfo().firstRowId()).isEqualTo(1000L); + + TrackedFile read2 = files.get(1); + assertThat(read2.trackingInfo().firstRowId()).isEqualTo(2000L); + } + + @Test + public void testLiveEntriesFilterDeleted() throws IOException { + GenericTrackedFile added = createDataFile("added.parquet", 1000L, 50000L); + added.setStatus(TrackingInfo.Status.ADDED); + added.setSnapshotId(SNAPSHOT_ID); + + GenericTrackedFile deleted = createDataFile("deleted.parquet", 2000L, 100000L); + deleted.setStatus(TrackingInfo.Status.DELETED); + deleted.setSnapshotId(SNAPSHOT_ID); + + GenericTrackedFile existing = createDataFile("existing.parquet", 3000L, 150000L); + existing.setStatus(TrackingInfo.Status.EXISTING); + existing.setSnapshotId(SNAPSHOT_ID - 1); + + String manifestPath = writeManifest(added, deleted, existing); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> liveFiles = Lists.newArrayList(reader.liveEntries()); + + assertThat(liveFiles).hasSize(2); + + List locations = Lists.newArrayList(); + List statuses = Lists.newArrayList(); + for (TrackedFile file : liveFiles) { + locations.add(file.location()); + statuses.add(file.trackingInfo().status()); + } + + assertThat(locations) + .anyMatch(loc -> loc.endsWith("added.parquet")) + .anyMatch(loc -> loc.endsWith("existing.parquet")); + assertThat(statuses) + .containsExactlyInAnyOrder(TrackingInfo.Status.ADDED, TrackingInfo.Status.EXISTING); + assertThat(locations).noneMatch(loc -> loc.endsWith("deleted.parquet")); + } + + @Test + public void testColumnProjection() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L, 50000L); + file1.setStatus(TrackingInfo.Status.ADDED); + file1.setSnapshotId(SNAPSHOT_ID); + file1.setSortOrderId(5); + + String manifestPath = writeManifest(file1); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null) + .select(ImmutableList.of("location", "record_count")); + + List> files = Lists.newArrayList(reader); + + assertThat(files).hasSize(1); + TrackedFile read = files.get(0); + assertThat(read.location()).endsWith("file1.parquet"); + assertThat(read.recordCount()).isEqualTo(1000L); + } + + @Test + public void testPositionTracking() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L, 50000L); + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L, 100000L); + GenericTrackedFile file3 = createDataFile("file3.parquet", 3000L, 150000L); + + String manifestPath = writeManifest(file1, file2, file3); + + TrackedFileReader reader = + TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> files = Lists.newArrayList(reader.entries()); + + assertThat(files).hasSize(3); + + List positions = Lists.newArrayList(); + for (TrackedFile file : files) { + assertThat(file.pos()).isNotNull(); + positions.add(file.pos()); + } + + assertThat(positions).containsExactlyInAnyOrder(0L, 1L, 2L); + } + + private GenericTrackedFile createDataFile(String filename, long recordCount, long fileSize) { + GenericTrackedFile file = new GenericTrackedFile(); + file.setContentType(FileContent.DATA); + file.setLocation("s3://bucket/table/data/" + filename); + file.setFileFormat(FileFormat.PARQUET); + file.setPartitionSpecId(0); + file.setRecordCount(recordCount); + file.setFileSizeInBytes(fileSize); + return file; + } + + private String writeManifest(GenericTrackedFile... files) throws IOException { + OutputFile outputFile = io.newOutputFile("manifest-" + System.nanoTime() + ".parquet"); + + try (FileAppender appender = + InternalData.write(FileFormat.PARQUET, outputFile) + .schema(new Schema(GenericTrackedFile.BASE_TYPE.fields())) + .named("tracked_file") + .build()) { + for (GenericTrackedFile file : files) { + appender.add(file); + } + } + + return outputFile.location(); + } +} From 3ae5e168fe3b88297655642e2f1cc75fa3b0a78d Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 07:34:57 -0800 Subject: [PATCH 06/14] Checkstyle and formatter --- .../apache/iceberg/InheritableTrackedMetadataFactory.java | 3 ++- .../src/main/java/org/apache/iceberg/TrackedFileReader.java | 6 ++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java index 973fbde377ac..9e97514476ef 100644 --- a/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java +++ b/core/src/main/java/org/apache/iceberg/InheritableTrackedMetadataFactory.java @@ -81,7 +81,8 @@ public TrackedFile apply(TrackedFile entry) { // in v1 tables, the sequence number is not persisted and can be safely defaulted to 0 // in v2+ tables, the sequence number should be inherited iff the entry status is ADDED if (tracking == null || tracking.sequenceNumber() == null) { - if (sequenceNumber == 0 || (tracking != null && tracking.status() == TrackingInfo.Status.ADDED)) { + if (sequenceNumber == 0 + || (tracking != null && tracking.status() == TrackingInfo.Status.ADDED)) { entry.setSequenceNumber(sequenceNumber); entry.setFileSequenceNumber(sequenceNumber); } diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileReader.java b/core/src/main/java/org/apache/iceberg/TrackedFileReader.java index ad170b6bbffb..14c61c65729d 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileReader.java +++ b/core/src/main/java/org/apache/iceberg/TrackedFileReader.java @@ -45,9 +45,7 @@ class TrackedFileReader extends CloseableGroup implements CloseableIterable columns = null; protected TrackedFileReader( - InputFile file, - InheritableTrackedMetadata inheritableMetadata, - Long manifestFirstRowId) { + InputFile file, InheritableTrackedMetadata inheritableMetadata, Long manifestFirstRowId) { this.file = file; this.inheritableMetadata = inheritableMetadata; this.manifestFirstRowId = manifestFirstRowId; @@ -95,7 +93,7 @@ private CloseableIterable> open(Collection cols) { } private Schema buildProjection(Collection cols) { - if (cols == null || ALL_COLUMNS.equals(cols)) { + if (cols == null || cols.containsAll(ALL_COLUMNS)) { return new Schema(GenericTrackedFile.BASE_TYPE.fields()); } From 2fbd88629552c7d035d55a50bee3f21b6f217c63 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 07:53:29 -0800 Subject: [PATCH 07/14] Rename file readers --- ...dFileReader.java => V4ManifestReader.java} | 6 ++-- ...ileReaders.java => V4ManifestReaders.java} | 18 +++++------ ...eReader.java => TestV4ManifestReader.java} | 30 +++++++++---------- 3 files changed, 27 insertions(+), 27 deletions(-) rename core/src/main/java/org/apache/iceberg/{TrackedFileReader.java => V4ManifestReader.java} (96%) rename core/src/main/java/org/apache/iceberg/{TrackedFileReaders.java => V4ManifestReaders.java} (84%) rename core/src/test/java/org/apache/iceberg/{TestTrackedFileReader.java => TestV4ManifestReader.java} (91%) diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileReader.java b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java similarity index 96% rename from core/src/main/java/org/apache/iceberg/TrackedFileReader.java rename to core/src/main/java/org/apache/iceberg/V4ManifestReader.java index 14c61c65729d..1673bd9ea3c8 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileReader.java +++ b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java @@ -35,7 +35,7 @@ *

Supports reading both root manifests and leaf manifests. Returns TrackedFile entries which can * represent data files, delete files, or manifest references. */ -class TrackedFileReader extends CloseableGroup implements CloseableIterable> { +class V4ManifestReader extends CloseableGroup implements CloseableIterable> { static final ImmutableList ALL_COLUMNS = ImmutableList.of("*"); private final InputFile file; @@ -44,14 +44,14 @@ class TrackedFileReader extends CloseableGroup implements CloseableIterable columns = null; - protected TrackedFileReader( + protected V4ManifestReader( InputFile file, InheritableTrackedMetadata inheritableMetadata, Long manifestFirstRowId) { this.file = file; this.inheritableMetadata = inheritableMetadata; this.manifestFirstRowId = manifestFirstRowId; } - public TrackedFileReader select(Collection newColumns) { + public V4ManifestReader select(Collection newColumns) { this.columns = newColumns; return this; } diff --git a/core/src/main/java/org/apache/iceberg/TrackedFileReaders.java b/core/src/main/java/org/apache/iceberg/V4ManifestReaders.java similarity index 84% rename from core/src/main/java/org/apache/iceberg/TrackedFileReaders.java rename to core/src/main/java/org/apache/iceberg/V4ManifestReaders.java index ddcd23bda9ee..1891462bc5b5 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedFileReaders.java +++ b/core/src/main/java/org/apache/iceberg/V4ManifestReaders.java @@ -23,10 +23,10 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Factory methods for creating {@link TrackedFileReader} instances. */ -public class TrackedFileReaders { +/** Factory methods for creating {@link V4ManifestReader} instances. */ +public class V4ManifestReaders { - private TrackedFileReaders() {} + private V4ManifestReaders() {} /** * Create a reader for a root manifest. @@ -36,15 +36,15 @@ private TrackedFileReaders() {} * @param snapshotId snapshot ID for metadata inheritance * @param sequenceNumber sequence number for metadata inheritance * @param firstRowId starting first row ID for data files (can be null) - * @return a TrackedFileReader for the root manifest + * @return a V4ManifestReader for the root manifest */ - public static TrackedFileReader readRoot( + public static V4ManifestReader readRoot( String rootManifestPath, FileIO io, long snapshotId, long sequenceNumber, Long firstRowId) { InputFile inputFile = io.newInputFile(rootManifestPath); InheritableTrackedMetadata metadata = InheritableTrackedMetadataFactory.create(snapshotId, sequenceNumber); - return new TrackedFileReader(inputFile, metadata, firstRowId); + return new V4ManifestReader(inputFile, metadata, firstRowId); } /** @@ -53,9 +53,9 @@ public static TrackedFileReader readRoot( * @param manifestEntry the DATA_MANIFEST or DELETE_MANIFEST entry from root * @param io file IO for reading * @param specsById map of partition specs by ID - * @return a TrackedFileReader for the leaf manifest + * @return a V4ManifestReader for the leaf manifest */ - public static TrackedFileReader readLeaf( + public static V4ManifestReader readLeaf( TrackedFile manifestEntry, FileIO io, Map specsById) { Preconditions.checkArgument( manifestEntry.contentType() == FileContent.DATA_MANIFEST @@ -71,6 +71,6 @@ public static TrackedFileReader readLeaf( TrackingInfo tracking = manifestEntry.trackingInfo(); Long firstRowId = tracking != null ? tracking.firstRowId() : null; - return new TrackedFileReader(inputFile, metadata, firstRowId); + return new V4ManifestReader(inputFile, metadata, firstRowId); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java b/core/src/test/java/org/apache/iceberg/TestV4ManifestReader.java similarity index 91% rename from core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java rename to core/src/test/java/org/apache/iceberg/TestV4ManifestReader.java index 01b833e1bcbd..c860698014bc 100644 --- a/core/src/test/java/org/apache/iceberg/TestTrackedFileReader.java +++ b/core/src/test/java/org/apache/iceberg/TestV4ManifestReader.java @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -public class TestTrackedFileReader { +public class TestV4ManifestReader { private final FileIO io = new InMemoryFileIO(); private static final long SNAPSHOT_ID = 12345L; @@ -56,8 +56,8 @@ public void testReadFlatManifest() throws IOException { String manifestPath = writeManifest(file1, file2); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); List> files = Lists.newArrayList(reader); @@ -84,8 +84,8 @@ public void testInheritSnapshotId() throws IOException { String manifestPath = writeManifest(file1); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); List> files = Lists.newArrayList(reader); @@ -113,8 +113,8 @@ public void testInheritSequenceNumberForAddedOnly() throws IOException { String manifestPath = writeManifest(added, existing); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); List> files = Lists.newArrayList(reader); @@ -142,8 +142,8 @@ public void testRowIdAssignment() throws IOException { String manifestPath = writeManifest(file1, file2); long startingRowId = 1000L; - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, startingRowId); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, startingRowId); List> files = Lists.newArrayList(reader); @@ -172,8 +172,8 @@ public void testLiveEntriesFilterDeleted() throws IOException { String manifestPath = writeManifest(added, deleted, existing); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); List> liveFiles = Lists.newArrayList(reader.liveEntries()); @@ -203,8 +203,8 @@ public void testColumnProjection() throws IOException { String manifestPath = writeManifest(file1); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null) + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null) .select(ImmutableList.of("location", "record_count")); List> files = Lists.newArrayList(reader); @@ -223,8 +223,8 @@ public void testPositionTracking() throws IOException { String manifestPath = writeManifest(file1, file2, file3); - TrackedFileReader reader = - TrackedFileReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader reader = + V4ManifestReaders.readRoot(manifestPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); List> files = Lists.newArrayList(reader.entries()); From 8e4849b8900e80804ef54fbbe6d7b1bdb02ee85e Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 09:09:09 -0800 Subject: [PATCH 08/14] implement root manifest reader --- .../apache/iceberg/RootManifestReader.java | 119 +++++++ .../iceberg/TestRootManifestReader.java | 298 ++++++++++++++++++ 2 files changed, 417 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/RootManifestReader.java create mode 100644 core/src/test/java/org/apache/iceberg/TestRootManifestReader.java diff --git a/core/src/main/java/org/apache/iceberg/RootManifestReader.java b/core/src/main/java/org/apache/iceberg/RootManifestReader.java new file mode 100644 index 000000000000..52410e408575 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/RootManifestReader.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * Reader for V4 root manifests that recursively expands manifest references. + * + *

Root manifests can contain direct file entries (DATA, POSITION_DELETES, EQUALITY_DELETES) as + * well as references to leaf manifests (DATA_MANIFEST, DELETE_MANIFEST). + * + *

RootManifestReader is the orchestrator reader that handles the recursive expansion of manifest + * references. + */ +public class RootManifestReader extends CloseableGroup { + private final V4ManifestReader rootReader; + private final FileIO io; + private final Map specsById; + + public RootManifestReader( + String rootManifestPath, + FileIO io, + Map specsById, + long snapshotId, + long sequenceNumber, + Long firstRowId) { + this.rootReader = + V4ManifestReaders.readRoot(rootManifestPath, io, snapshotId, sequenceNumber, firstRowId); + this.io = io; + this.specsById = specsById; + addCloseable(rootReader); + } + + /** + * Returns all TrackedFiles from the root manifest and all referenced leaf manifests. + * + *

This includes: + * + *

+ * + *

Returns TrackedFile entries (not converted to DataFile/DeleteFile yet). + * + *

TODO: Add manifest DV support. + * + * @return iterable of all tracked files + */ + public CloseableIterable> allTrackedFiles() { + return CloseableIterable.concat(Lists.newArrayList(directFiles(), expandManifests())); + } + + /** + * Returns direct file entries from the root manifest. + * + *

These are DATA, POSITION_DELETES, or EQUALITY_DELETES entries stored directly in the root + * manifest + */ + private CloseableIterable> directFiles() { + return CloseableIterable.filter( + rootReader.liveEntries(), + tf -> + tf.contentType() == FileContent.DATA + || tf.contentType() == FileContent.POSITION_DELETES + || tf.contentType() == FileContent.EQUALITY_DELETES); + } + + /** + * Expands manifest references (DATA_MANIFEST and DELETE_MANIFEST) to their contained files. + * + *

Loads all root entries to identify manifests, then reads each leaf manifest serially. + * + *

TODO: Add parallel manifest reading support via ExecutorService (like + * ManifestGroup.planWith). This would use ParallelIterable to read leaf manifests concurrently. + * + *

TODO: Add manifest DV support - group MANIFEST_DV entries by referencedFile and apply to + * leaf readers. + */ + private CloseableIterable> expandManifests() { + List> rootEntries = Lists.newArrayList(rootReader.liveEntries()); + + List>> allLeafFiles = Lists.newArrayList(); + + for (TrackedFile manifestEntry : rootEntries) { + if (manifestEntry.contentType() == FileContent.DATA_MANIFEST + || manifestEntry.contentType() == FileContent.DELETE_MANIFEST) { + V4ManifestReader leafReader = V4ManifestReaders.readLeaf(manifestEntry, io, specsById); + addCloseable(leafReader); + allLeafFiles.add(leafReader.liveEntries()); + } + } + + return CloseableIterable.concat(allLeafFiles); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestRootManifestReader.java b/core/src/test/java/org/apache/iceberg/TestRootManifestReader.java new file mode 100644 index 000000000000..b9b254af6512 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestRootManifestReader.java @@ -0,0 +1,298 @@ +/* + * 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 java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.inmemory.InMemoryFileIO; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestRootManifestReader { + + private final FileIO io = new InMemoryFileIO(); + private final Map specsById; + + private static final long SNAPSHOT_ID = 12345L; + private static final long SEQUENCE_NUMBER = 100L; + + @TempDir private Path temp; + + public TestRootManifestReader() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.unpartitioned(); + this.specsById = ImmutableMap.of(spec.specId(), spec); + } + + @Test + public void testRootWithOnlyDirectFiles() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + file1.setStatus(TrackingInfo.Status.ADDED); + + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L); + file2.setStatus(TrackingInfo.Status.EXISTING); + + String rootPath = writeRootManifest(file1, file2); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(2); + assertThat(allFiles.get(0).contentType()).isEqualTo(FileContent.DATA); + assertThat(allFiles.get(1).contentType()).isEqualTo(FileContent.DATA); + } + + @Test + public void testRootWithOnlyManifests() throws IOException { + GenericTrackedFile dataFile1 = createDataFile("data1.parquet", 1000L); + GenericTrackedFile dataFile2 = createDataFile("data2.parquet", 2000L); + String dataManifestPath = writeLeafManifest(dataFile1, dataFile2); + + GenericTrackedFile manifestEntry = createManifestEntry(dataManifestPath, 2, 2000L); + + String rootPath = writeRootManifest(manifestEntry); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(2); + assertThat(allFiles.get(0).location()).endsWith("data1.parquet"); + assertThat(allFiles.get(1).location()).endsWith("data2.parquet"); + } + + @Test + public void testRootWithMixedDirectAndManifests() throws IOException { + GenericTrackedFile directFile = createDataFile("direct.parquet", 500L); + directFile.setStatus(TrackingInfo.Status.ADDED); + + GenericTrackedFile leafFile1 = createDataFile("leaf1.parquet", 1000L); + GenericTrackedFile leafFile2 = createDataFile("leaf2.parquet", 2000L); + String leafManifestPath = writeLeafManifest(leafFile1, leafFile2); + + GenericTrackedFile manifestEntry = createManifestEntry(leafManifestPath, 2, 3000L); + + String rootPath = writeRootManifest(directFile, manifestEntry); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(3); + + List locations = Lists.newArrayList(); + for (TrackedFile file : allFiles) { + locations.add(file.location()); + } + + assertThat(locations) + .anyMatch(loc -> loc.endsWith("direct.parquet")) + .anyMatch(loc -> loc.endsWith("leaf1.parquet")) + .anyMatch(loc -> loc.endsWith("leaf2.parquet")); + } + + @Test + public void testMultipleDataManifests() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L); + String manifest1Path = writeLeafManifest(file1, file2); + + GenericTrackedFile file3 = createDataFile("file3.parquet", 3000L); + String manifest2Path = writeLeafManifest(file3); + + GenericTrackedFile manifestEntry1 = createManifestEntry(manifest1Path, 2, 3000L); + GenericTrackedFile manifestEntry2 = createManifestEntry(manifest2Path, 1, 3000L); + + String rootPath = writeRootManifest(manifestEntry1, manifestEntry2); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(3); + assertThat(allFiles).allMatch(tf -> tf.contentType() == FileContent.DATA); + } + + @Test + public void testDeleteManifests() throws IOException { + GenericTrackedFile deleteFile1 = createDeleteFile("delete1.parquet", 100L); + GenericTrackedFile deleteFile2 = createDeleteFile("delete2.parquet", 200L); + String deleteManifestPath = writeLeafManifest(deleteFile1, deleteFile2); + + GenericTrackedFile manifestEntry = createDeleteManifestEntry(deleteManifestPath, 2, 300L); + + String rootPath = writeRootManifest(manifestEntry); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(2); + assertThat(allFiles).allMatch(tf -> tf.contentType() == FileContent.POSITION_DELETES); + } + + @Test + public void testMixedDataAndDeleteManifests() throws IOException { + GenericTrackedFile dataFile = createDataFile("data.parquet", 1000L); + String dataManifestPath = writeLeafManifest(dataFile); + + GenericTrackedFile deleteFile = createDeleteFile("delete.parquet", 100L); + String deleteManifestPath = writeLeafManifest(deleteFile); + + GenericTrackedFile dataManifestEntry = createManifestEntry(dataManifestPath, 1, 1000L); + GenericTrackedFile deleteManifestEntry = createDeleteManifestEntry(deleteManifestPath, 1, 100L); + + String rootPath = writeRootManifest(dataManifestEntry, deleteManifestEntry); + + RootManifestReader reader = + new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + + List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); + + assertThat(allFiles).hasSize(2); + + long dataFiles = allFiles.stream().filter(tf -> tf.contentType() == FileContent.DATA).count(); + long deleteFiles = + allFiles.stream().filter(tf -> tf.contentType() == FileContent.POSITION_DELETES).count(); + + assertThat(dataFiles).isEqualTo(1); + assertThat(deleteFiles).isEqualTo(1); + } + + private GenericTrackedFile createDataFile(String filename, long recordCount) { + GenericTrackedFile file = new GenericTrackedFile(); + file.setContentType(FileContent.DATA); + file.setLocation("s3://bucket/table/data/" + filename); + file.setFileFormat(FileFormat.PARQUET); + file.setPartitionSpecId(0); + file.setRecordCount(recordCount); + file.setFileSizeInBytes(recordCount * 100); + file.setStatus(TrackingInfo.Status.ADDED); + file.setSnapshotId(SNAPSHOT_ID); + return file; + } + + private GenericTrackedFile createDeleteFile(String filename, long recordCount) { + GenericTrackedFile file = new GenericTrackedFile(); + file.setContentType(FileContent.POSITION_DELETES); + file.setLocation("s3://bucket/table/deletes/" + filename); + file.setFileFormat(FileFormat.PARQUET); + file.setPartitionSpecId(0); + file.setRecordCount(recordCount); + file.setFileSizeInBytes(recordCount * 50); + file.setStatus(TrackingInfo.Status.ADDED); + file.setSnapshotId(SNAPSHOT_ID); + return file; + } + + private GenericTrackedFile createManifestEntry( + String manifestLocation, int fileCount, long totalRows) { + GenericTrackedFile entry = new GenericTrackedFile(); + entry.setContentType(FileContent.DATA_MANIFEST); + entry.setLocation(manifestLocation); + entry.setFileFormat(FileFormat.PARQUET); + entry.setPartitionSpecId(0); + entry.setRecordCount(fileCount); + entry.setFileSizeInBytes(10000L); + + entry.setAddedFilesCount(fileCount); + entry.setExistingFilesCount(0); + entry.setDeletedFilesCount(0); + entry.setAddedRowsCount(totalRows); + entry.setExistingRowsCount(0L); + entry.setDeletedRowsCount(0L); + entry.setMinSequenceNumber(SEQUENCE_NUMBER); + + entry.setStatus(TrackingInfo.Status.ADDED); + entry.setSnapshotId(SNAPSHOT_ID); + entry.setSequenceNumber(SEQUENCE_NUMBER); + entry.setFileSequenceNumber(SEQUENCE_NUMBER); + + return entry; + } + + private GenericTrackedFile createDeleteManifestEntry( + String manifestLocation, int fileCount, long totalRows) { + GenericTrackedFile entry = new GenericTrackedFile(); + entry.setContentType(FileContent.DELETE_MANIFEST); + entry.setLocation(manifestLocation); + entry.setFileFormat(FileFormat.PARQUET); + entry.setPartitionSpecId(0); + entry.setRecordCount(fileCount); + entry.setFileSizeInBytes(5000L); + + entry.setAddedFilesCount(fileCount); + entry.setExistingFilesCount(0); + entry.setDeletedFilesCount(0); + entry.setAddedRowsCount(totalRows); + entry.setExistingRowsCount(0L); + entry.setDeletedRowsCount(0L); + entry.setMinSequenceNumber(SEQUENCE_NUMBER); + + entry.setStatus(TrackingInfo.Status.ADDED); + entry.setSnapshotId(SNAPSHOT_ID); + entry.setSequenceNumber(SEQUENCE_NUMBER); + entry.setFileSequenceNumber(SEQUENCE_NUMBER); + + return entry; + } + + private String writeRootManifest(GenericTrackedFile... entries) throws IOException { + return writeManifest("root-manifest", entries); + } + + private String writeLeafManifest(GenericTrackedFile... entries) throws IOException { + return writeManifest("leaf-manifest", entries); + } + + private String writeManifest(String prefix, GenericTrackedFile... entries) throws IOException { + OutputFile outputFile = io.newOutputFile(prefix + "-" + System.nanoTime() + ".parquet"); + + try (FileAppender appender = + InternalData.write(FileFormat.PARQUET, outputFile) + .schema(new Schema(GenericTrackedFile.BASE_TYPE.fields())) + .named("tracked_file") + .build()) { + for (GenericTrackedFile entry : entries) { + appender.add(entry); + } + } + + return outputFile.location(); + } +} From cff48fe9451ca49ddd81fd6a1f6b574ccd6aae18 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 10:21:33 -0800 Subject: [PATCH 09/14] implement adapter for asData and asDelete --- .../java/org/apache/iceberg/TrackedFile.java | 31 +++ .../apache/iceberg/GenericTrackedFile.java | 21 ++ .../iceberg/TrackedDataFileAdapter.java | 179 +++++++++++++++++ .../iceberg/TrackedDeleteFileAdapter.java | 190 ++++++++++++++++++ 4 files changed, 421 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java create mode 100644 core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java diff --git a/api/src/main/java/org/apache/iceberg/TrackedFile.java b/api/src/main/java/org/apache/iceberg/TrackedFile.java index 32b1c3152dd1..757c940c8295 100644 --- a/api/src/main/java/org/apache/iceberg/TrackedFile.java +++ b/api/src/main/java/org/apache/iceberg/TrackedFile.java @@ -100,6 +100,12 @@ public interface TrackedFile { "Location of data file that a DV references if content_type is 1 or 5." + " Location of affiliated data manifest if content_type is 4 or null if delete manifest is unaffiliated"); + /** + * Returns the path of the manifest which this file is referenced in or null if it was not read + * from a manifest. + */ + String manifestLocation(); + /** * Returns the tracking information for this entry. * @@ -233,4 +239,29 @@ public interface TrackedFile { /** Set the ordinal position in the manifest. */ void setPos(Long position); + + /** + * Returns this TrackedFile as a DataFile. + * + *

This method creates an adapter that implements the DataFile interface, allowing TrackedFile + * instances to be used where DataFile is expected. The adapter returns null for partition data + * and column-level statistics. + * + * @return a DataFile view of this TrackedFile + * @throws IllegalStateException if the content type is not DATA + */ + DataFile asDataFile(); + + /** + * Returns this TrackedFile as a DeleteFile. + * + *

This method creates an adapter that implements the DeleteFile interface, allowing + * TrackedFile instances to be used where DeleteFile is expected. The adapter returns null for + * partition data and column-level statistics. + * + * @return a DeleteFile view of this TrackedFile + * @throws IllegalStateException if the content type is not a delete type (POSITION_DELETES or + * EQUALITY_DELETES) + */ + DeleteFile asDeleteFile(); } diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java index 702ed744fe2b..6b2046771853 100644 --- a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -45,6 +45,7 @@ public class GenericTrackedFile extends SupportsIndexProjection private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); // Tracking metadata + private String manifestLocation = null; private TrackingInfo.Status status = null; private Long snapshotId = null; private Long sequenceNumber = null; @@ -174,6 +175,7 @@ private GenericTrackedFile(GenericTrackedFile toCopy, boolean copyStats) { super(toCopy); // Tracking metadata + this.manifestLocation = toCopy.manifestLocation; this.status = toCopy.status; this.snapshotId = toCopy.snapshotId; this.sequenceNumber = toCopy.sequenceNumber; @@ -229,6 +231,15 @@ private GenericTrackedFile(GenericTrackedFile toCopy, boolean copyStats) { } } + @Override + public String manifestLocation() { + return manifestLocation; + } + + public void setManifestLocation(String manifestLocation) { + this.manifestLocation = manifestLocation; + } + @Override public TrackingInfo trackingInfo() { if (status == null @@ -709,6 +720,16 @@ private Schema getAvroSchema() { BASE_TYPE, ImmutableMap.of(BASE_TYPE, GenericTrackedFile.class.getName())); } + @Override + public DataFile asDataFile() { + return new TrackedDataFileAdapter(this); + } + + @Override + public DeleteFile asDeleteFile() { + return new TrackedDeleteFileAdapter(this); + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java b/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java new file mode 100644 index 000000000000..eaabff798fd3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Adapter that wraps a TrackedFile and presents it as a DataFile. + * + *

This adapter allows TrackedFile instances to be used in contexts that expect DataFile. The + * adapter returns null for partition data and column-level statistics, as these are not stored in + * TrackedFile. + */ +class TrackedDataFileAdapter implements DataFile { + private final TrackedFile trackedFile; + + TrackedDataFileAdapter(TrackedFile trackedFile) { + if (trackedFile.contentType() != FileContent.DATA) { + throw new IllegalStateException( + "Cannot convert TrackedFile with content type " + + trackedFile.contentType() + + " to DataFile"); + } + this.trackedFile = trackedFile; + } + + @Override + public String manifestLocation() { + return trackedFile.manifestLocation(); + } + + @Override + public Long pos() { + return trackedFile.pos(); + } + + @Override + public int specId() { + return trackedFile.partitionSpecId(); + } + + @Override + public FileContent content() { + return trackedFile.contentType(); + } + + @Override + public CharSequence path() { + return trackedFile.location(); + } + + @Override + public String location() { + return trackedFile.location(); + } + + @Override + public FileFormat format() { + return trackedFile.fileFormat(); + } + + @Override + public StructLike partition() { + return null; + } + + @Override + public long recordCount() { + return trackedFile.recordCount(); + } + + @Override + public long fileSizeInBytes() { + Long size = trackedFile.fileSizeInBytes(); + return size != null ? size : 0L; + } + + @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 ByteBuffer keyMetadata() { + return trackedFile.keyMetadata(); + } + + @Override + public List splitOffsets() { + return trackedFile.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return null; + } + + @Override + public Integer sortOrderId() { + return trackedFile.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + TrackingInfo trackingInfo = trackedFile.trackingInfo(); + return trackingInfo != null ? trackingInfo.sequenceNumber() : null; + } + + @Override + public Long fileSequenceNumber() { + TrackingInfo trackingInfo = trackedFile.trackingInfo(); + return trackingInfo != null ? trackingInfo.fileSequenceNumber() : null; + } + + @Override + public Long firstRowId() { + TrackingInfo trackingInfo = trackedFile.trackingInfo(); + return trackingInfo != null ? trackingInfo.firstRowId() : null; + } + + @Override + public DataFile copy() { + return new TrackedDataFileAdapter((TrackedFile) trackedFile.copy()); + } + + @Override + public DataFile copyWithoutStats() { + return new TrackedDataFileAdapter((TrackedFile) trackedFile.copyWithoutStats()); + } + + @Override + public DataFile copyWithStats(Set requestedColumnIds) { + return copyWithoutStats(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java b/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java new file mode 100644 index 000000000000..e32cfd699571 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Adapter that wraps a TrackedFile and presents it as a DeleteFile. + * + *

This adapter allows TrackedFile instances to be used in contexts that expect DeleteFile. The + * adapter returns null for partition data and column-level statistics, as these are not stored in + * TrackedFile. + */ +class TrackedDeleteFileAdapter implements DeleteFile { + private final TrackedFile trackedFile; + + TrackedDeleteFileAdapter(TrackedFile trackedFile) { + FileContent contentType = trackedFile.contentType(); + if (contentType != FileContent.POSITION_DELETES + && contentType != FileContent.EQUALITY_DELETES) { + throw new IllegalStateException( + "Cannot convert TrackedFile with content type " + contentType + " to DeleteFile"); + } + this.trackedFile = trackedFile; + } + + @Override + public String manifestLocation() { + return trackedFile.manifestLocation(); + } + + @Override + public Long pos() { + return trackedFile.pos(); + } + + @Override + public int specId() { + return trackedFile.partitionSpecId(); + } + + @Override + public FileContent content() { + return trackedFile.contentType(); + } + + @Override + public CharSequence path() { + return trackedFile.location(); + } + + @Override + public String location() { + return trackedFile.location(); + } + + @Override + public FileFormat format() { + return trackedFile.fileFormat(); + } + + @Override + public StructLike partition() { + return null; + } + + @Override + public long recordCount() { + return trackedFile.recordCount(); + } + + @Override + public long fileSizeInBytes() { + Long size = trackedFile.fileSizeInBytes(); + return size != null ? size : 0L; + } + + @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 ByteBuffer keyMetadata() { + return trackedFile.keyMetadata(); + } + + @Override + public List splitOffsets() { + return trackedFile.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return trackedFile.equalityIds(); + } + + @Override + public Integer sortOrderId() { + return trackedFile.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + TrackingInfo trackingInfo = trackedFile.trackingInfo(); + return trackingInfo != null ? trackingInfo.sequenceNumber() : null; + } + + @Override + public Long fileSequenceNumber() { + TrackingInfo trackingInfo = trackedFile.trackingInfo(); + return trackingInfo != null ? trackingInfo.fileSequenceNumber() : null; + } + + @Override + public String referencedDataFile() { + return trackedFile.referencedFile(); + } + + @Override + public Long contentOffset() { + DeletionVector dv = trackedFile.deletionVector(); + return dv != null ? dv.offset() : null; + } + + @Override + public Long contentSizeInBytes() { + DeletionVector dv = trackedFile.deletionVector(); + return dv != null ? dv.sizeInBytes() : null; + } + + @Override + public DeleteFile copy() { + return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copy()); + } + + @Override + public DeleteFile copyWithoutStats() { + return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copyWithoutStats()); + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return copyWithoutStats(); + } +} From e53cbb73052c8c21776eaeb25dcb2579e176c632 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 3 Nov 2025 10:31:43 -0800 Subject: [PATCH 10/14] Trigger PR refresh From 0380105d66146d4efc61b06068bdb047dfd92c25 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Fri, 7 Nov 2025 15:35:28 -0800 Subject: [PATCH 11/14] Implement manifest expander --- .../java/org/apache/iceberg/TrackedFile.java | 51 +-- .../apache/iceberg/GenericTrackedFile.java | 8 +- .../org/apache/iceberg/ManifestExpander.java | 420 ++++++++++++++++++ .../apache/iceberg/RootManifestReader.java | 119 ----- .../iceberg/TrackedDataFileAdapter.java | 16 +- .../iceberg/TrackedDeleteFileAdapter.java | 16 +- .../org/apache/iceberg/V4ManifestReader.java | 2 +- ...tReader.java => TestManifestExpander.java} | 235 +++++++++- 8 files changed, 694 insertions(+), 173 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/ManifestExpander.java delete mode 100644 core/src/main/java/org/apache/iceberg/RootManifestReader.java rename core/src/test/java/org/apache/iceberg/{TestRootManifestReader.java => TestManifestExpander.java} (53%) diff --git a/api/src/main/java/org/apache/iceberg/TrackedFile.java b/api/src/main/java/org/apache/iceberg/TrackedFile.java index 757c940c8295..34783fea1d0f 100644 --- a/api/src/main/java/org/apache/iceberg/TrackedFile.java +++ b/api/src/main/java/org/apache/iceberg/TrackedFile.java @@ -222,6 +222,32 @@ public interface TrackedFile { */ Long pos(); + /** + * Converts this tracked file to a DataFile. + * + *

Only valid when content_type is DATA. Extracts partition from contentStats using the + * provided spec. + * + * @param spec the partition spec (needed to know which transforms to apply) + * @return a DataFile representation + * @throws IllegalStateException if content_type is not DATA + * @throws UnsupportedOperationException if ContentStats not yet implemented + */ + DataFile asDataFile(PartitionSpec spec); + + /** + * Converts this tracked file to a DeleteFile. + * + *

Only valid when content_type is POSITION_DELETES or EQUALITY_DELETES. Extracts partition + * from contentStats using the provided spec. + * + * @param spec the partition spec (needed to know which transforms to apply) + * @return a DeleteFile representation + * @throws IllegalStateException if content_type is not a delete type + * @throws UnsupportedOperationException if ContentStats not yet implemented + */ + DeleteFile asDeleteFile(PartitionSpec spec); + /** Set the status for this tracked file entry. */ void setStatus(TrackingInfo.Status status); @@ -239,29 +265,4 @@ public interface TrackedFile { /** Set the ordinal position in the manifest. */ void setPos(Long position); - - /** - * Returns this TrackedFile as a DataFile. - * - *

This method creates an adapter that implements the DataFile interface, allowing TrackedFile - * instances to be used where DataFile is expected. The adapter returns null for partition data - * and column-level statistics. - * - * @return a DataFile view of this TrackedFile - * @throws IllegalStateException if the content type is not DATA - */ - DataFile asDataFile(); - - /** - * Returns this TrackedFile as a DeleteFile. - * - *

This method creates an adapter that implements the DeleteFile interface, allowing - * TrackedFile instances to be used where DeleteFile is expected. The adapter returns null for - * partition data and column-level statistics. - * - * @return a DeleteFile view of this TrackedFile - * @throws IllegalStateException if the content type is not a delete type (POSITION_DELETES or - * EQUALITY_DELETES) - */ - DeleteFile asDeleteFile(); } diff --git a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java index 6b2046771853..a7095f0deb21 100644 --- a/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericTrackedFile.java @@ -721,13 +721,13 @@ private Schema getAvroSchema() { } @Override - public DataFile asDataFile() { - return new TrackedDataFileAdapter(this); + public DataFile asDataFile(PartitionSpec spec) { + return new TrackedDataFileAdapter(this, spec); } @Override - public DeleteFile asDeleteFile() { - return new TrackedDeleteFileAdapter(this); + public DeleteFile asDeleteFile(PartitionSpec spec) { + return new TrackedDeleteFileAdapter(this, spec); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestExpander.java b/core/src/main/java/org/apache/iceberg/ManifestExpander.java new file mode 100644 index 000000000000..4e70423d80f9 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestExpander.java @@ -0,0 +1,420 @@ +/* + * 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.apache.iceberg.expressions.Expressions.alwaysTrue; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.metrics.ScanMetrics; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +/** + * Expands V4 root manifests and plans file scan tasks. + * + *

Root manifests can contain direct file entries (DATA, POSITION_DELETES, EQUALITY_DELETES) as + * well as references to leaf manifests (DATA_MANIFEST, DELETE_MANIFEST). Handles recursive + * expansion of manifest references and creates scan tasks, similar to ManifestGroup in V3. + * + *

Currently returns DataFileScanInfo (TrackedFile data + deletes). Full FileScanTask creation + * blocked on ContentStats implementation. + * + *

TODO: (after ContentStats is ready): + * + *

    + *
  1. Implement TrackedFile.asDataFile(spec) + *
  2. Implement TrackedFile.asDeleteFile(spec) + *
  3. Add planFiles() method that returns CloseableIterable<FileScanTask> + *
  4. Handle equality deletes + *
  5. Add manifest-level filtering using ManifestStats + *
+ */ +public class ManifestExpander extends CloseableGroup { + private final V4ManifestReader rootReader; + private final FileIO io; + private final Map specsById; + + private static final Types.StructType FILE_METADATA_TYPE = + Types.StructType.of( + TrackedFile.CONTENT_TYPE, + TrackedFile.RECORD_COUNT, + TrackedFile.FILE_SIZE_IN_BYTES, + TrackedFile.PARTITION_SPEC_ID, + TrackedFile.SORT_ORDER_ID); + + private Expression dataFilter; + private Expression fileFilter; + private Evaluator fileFilterEvaluator; + private Map>> deletesByPath; + + private boolean ignoreDeleted; + + private boolean ignoreExisting; + + private List columns; + + private boolean caseSensitive; + + @SuppressWarnings("UnusedVariable") + private ScanMetrics scanMetrics; + + public ManifestExpander( + V4ManifestReader rootReader, FileIO io, Map specsById) { + this.rootReader = rootReader; + this.io = io; + this.specsById = specsById; + this.dataFilter = alwaysTrue(); + this.fileFilter = alwaysTrue(); + this.ignoreDeleted = false; + this.ignoreExisting = false; + this.columns = V4ManifestReader.ALL_COLUMNS; + this.caseSensitive = true; + this.scanMetrics = ScanMetrics.noop(); + addCloseable(rootReader); + } + + private Map>> buildDeleteIndex() { + if (deletesByPath != null) { + return deletesByPath; + } + + Map>> index = Maps.newHashMap(); + + try (CloseableIterable> allFiles = allTrackedFiles()) { + for (TrackedFile entry : allFiles) { + if (entry.contentType() == FileContent.POSITION_DELETES && entry.referencedFile() != null) { + index.computeIfAbsent(entry.referencedFile(), k -> Lists.newArrayList()).add(entry); + } + } + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to build delete index"); + } + + this.deletesByPath = index; + return index; + } + + private List> deleteFilesForDataFile(TrackedFile dataFile) { + Map>> index = buildDeleteIndex(); + List> deletes = index.get(dataFile.location()); + + if (deletes == null || deletes.isEmpty()) { + return Collections.emptyList(); + } + + TrackingInfo dataTracking = dataFile.trackingInfo(); + Long dataSeq = dataTracking != null ? dataTracking.sequenceNumber() : null; + + if (dataSeq == null) { + return deletes; + } + + List> filtered = Lists.newArrayList(); + for (TrackedFile delete : deletes) { + TrackingInfo deleteTracking = delete.trackingInfo(); + Long deleteSeq = deleteTracking != null ? deleteTracking.sequenceNumber() : null; + + if (deleteSeq == null || deleteSeq <= dataSeq) { + filtered.add(delete); + } + } + + return filtered; + } + + public ManifestExpander filterData(Expression expression) { + this.dataFilter = Expressions.and(dataFilter, expression); + return this; + } + + public ManifestExpander filterFiles(Expression expression) { + this.fileFilter = Expressions.and(fileFilter, expression); + this.fileFilterEvaluator = null; + return this; + } + + public ManifestExpander ignoreDeleted() { + this.ignoreDeleted = true; + return this; + } + + public ManifestExpander ignoreExisting() { + this.ignoreExisting = true; + return this; + } + + public ManifestExpander select(List selectedColumns) { + this.columns = Lists.newArrayList(selectedColumns); + return this; + } + + public ManifestExpander caseSensitive(boolean isCaseSensitive) { + this.caseSensitive = isCaseSensitive; + return this; + } + + public ManifestExpander scanMetrics(ScanMetrics metrics) { + this.scanMetrics = metrics; + return this; + } + + /** + * Plans file scan tasks for table scans with delete matching. + * + *

This is the main entry point for query planning, similar to ManifestGroup.planFiles() in V3. + * + *

Filters and matching: + * + *

    + *
  • Entry-level: Filters by status (ignoreDeleted/ignoreExisting) + *
  • Content-type: Returns only DATA files + *
  • File-level: Filters by file metadata (record_count, file_size, etc.) + *
  • Delete matching: Path-based matching for POSITION_DELETES with referencedFile + *
+ * + *

Returns DataFileScanInfo which pairs each data file with matched delete files. + * + *

TODO: When ContentStats is implemented, add planFiles() method that returns + * CloseableIterable<FileScanTask>: + * + *

{@code
+   * public CloseableIterable planFiles() {
+   *     return CloseableIterable.transform(planDataFiles(), this::createFileScanTask);
+   * }
+   *
+   * private FileScanTask createFileScanTask(DataFileScanInfo scanInfo) {
+   *     TrackedFile tf = scanInfo.dataFile();
+   *     PartitionSpec spec = specsById.get(tf.partitionSpecId());
+   *
+   *     // TrackedFile.asDataFile(spec) extracts partition from contentStats internally
+   *     DataFile dataFile = tf.asDataFile(spec);
+   *
+   *     // Convert TrackedFile deletes → DeleteFile array
+   *     DeleteFile[] deleteFiles = convertDeleteFiles(scanInfo.deleteFiles(), spec);
+   *
+   *     // Build residual evaluator
+   *     ResidualEvaluator residuals = ResidualEvaluator.of(spec, dataFilter, caseSensitive);
+   *
+   *     // Create task
+   *     return new BaseFileScanTask(
+   *         dataFile,
+   *         deleteFiles,
+   *         SchemaParser.toJson(spec.schema()),
+   *         PartitionSpecParser.toJson(spec),
+   *         residuals);
+   * }
+   *
+   * private DeleteFile[] convertDeleteFiles(
+   *     List> deleteTrackedFiles,
+   *     PartitionSpec spec) {
+   *     DeleteFile[] deleteFiles = new DeleteFile[deleteTrackedFiles.size()];
+   *     for (int i = 0; i < deleteTrackedFiles.size(); i++) {
+   *         // TrackedFile.asDeleteFile(spec) extracts partition from contentStats internally
+   *         deleteFiles[i] = deleteTrackedFiles.get(i).asDeleteFile(spec);
+   *     }
+   *     return deleteFiles;
+   * }
+   * }
+ * + *

TODO: When ContentStats is implemented: + * + *

    + *
  • Equality deletes (match by partition + equality fields) + *
+ * + * @return iterable of DataFileScanInfo (data file + matched deletes) + */ + public CloseableIterable planDataFiles() { + CloseableIterable> allFiles = + CloseableIterable.concat(Lists.newArrayList(directFiles(), expandManifests())); + + allFiles = CloseableIterable.filter(allFiles, tf -> tf.contentType() == FileContent.DATA); + + allFiles = applyStatusFilters(allFiles); + + allFiles = applyFileFilter(allFiles); + + buildDeleteIndex(); + + return CloseableIterable.transform( + allFiles, + dataFile -> { + List> deletes = deleteFilesForDataFile(dataFile); + return new DataFileScanInfo(dataFile, deletes); + }); + } + + /** + * Information needed to scan a data file with associated delete files. + * + *

This is the V4 equivalent of the information used to create FileScanTask in V3. When + * ContentStats is implemented, this will be converted to BaseFileScanTask. + * + *

Contains: + * + *

    + *
  • Data file (as TrackedFile) + *
  • Matched delete files (as TrackedFile list) + *
+ * + *

TODO: When ContentStats available, convert this to FileScanTask by: + * + *

    + *
  1. Call TrackedFile.asDataFile(spec) - extracts partition from contentStats internally + *
  2. Call TrackedFile.asDeleteFile(spec) for deletes - extracts partition internally + *
  3. Create BaseFileScanTask with DataFile, DeleteFile array, and residuals + *
+ */ + public static class DataFileScanInfo { + private final TrackedFile dataFile; + private final List> deleteFiles; + + DataFileScanInfo(TrackedFile dataFile, List> deleteFiles) { + this.dataFile = dataFile; + this.deleteFiles = deleteFiles; + } + + public TrackedFile dataFile() { + return dataFile; + } + + public List> deleteFiles() { + return deleteFiles; + } + } + + /** + * Returns all TrackedFiles from the root manifest and all referenced leaf manifests. + * + *

This includes: + * + *

    + *
  • Direct file entries in root (DATA, POSITION_DELETES, EQUALITY_DELETES) + *
  • Files from expanded DATA_MANIFEST entries + *
  • Files from expanded DELETE_MANIFEST entries + *
+ * + *

Returns TrackedFile entries (not converted to DataFile/DeleteFile yet). + * + *

TODO: Add manifest DV support. + * + * @return iterable of all tracked files + */ + public CloseableIterable> allTrackedFiles() { + return CloseableIterable.concat(Lists.newArrayList(directFiles(), expandManifests())); + } + + private CloseableIterable> applyStatusFilters( + CloseableIterable> entries) { + CloseableIterable> filtered = entries; + + if (ignoreDeleted) { + filtered = + CloseableIterable.filter( + filtered, + tf -> { + TrackingInfo tracking = tf.trackingInfo(); + return tracking == null || tracking.status() != TrackingInfo.Status.DELETED; + }); + } + + if (ignoreExisting) { + filtered = + CloseableIterable.filter( + filtered, + tf -> { + TrackingInfo tracking = tf.trackingInfo(); + return tracking != null && tracking.status() != TrackingInfo.Status.EXISTING; + }); + } + + return filtered; + } + + private CloseableIterable> applyFileFilter( + CloseableIterable> entries) { + if (fileFilter == null || fileFilter == alwaysTrue()) { + return entries; + } + + return CloseableIterable.filter(entries, this::evaluateFileFilter); + } + + private boolean evaluateFileFilter(TrackedFile file) { + if (fileFilterEvaluator == null) { + fileFilterEvaluator = new Evaluator(FILE_METADATA_TYPE, fileFilter, caseSensitive); + } + return fileFilterEvaluator.eval((StructLike) file); + } + + /** + * Returns direct file entries from the root manifest. + * + *

These are DATA, POSITION_DELETES, or EQUALITY_DELETES entries stored directly in the root + * manifest + */ + private CloseableIterable> directFiles() { + return CloseableIterable.filter( + rootReader.liveEntries(), + tf -> + tf.contentType() == FileContent.DATA + || tf.contentType() == FileContent.POSITION_DELETES + || tf.contentType() == FileContent.EQUALITY_DELETES); + } + + /** + * Expands manifest references (DATA_MANIFEST and DELETE_MANIFEST) to their contained files. + * + *

Loads all root entries to identify manifests, then reads each leaf manifest serially. + * + *

TODO: Add parallel manifest reading support via ExecutorService (like + * ManifestGroup.planWith). This would use ParallelIterable to read leaf manifests concurrently. + * + *

TODO: Add manifest DV support - group MANIFEST_DV entries by referencedFile and apply to + * leaf readers. + */ + private CloseableIterable> expandManifests() { + List> rootEntries = Lists.newArrayList(rootReader.liveEntries()); + + List>> allLeafFiles = Lists.newArrayList(); + + for (TrackedFile manifestEntry : rootEntries) { + if (manifestEntry.contentType() == FileContent.DATA_MANIFEST + || manifestEntry.contentType() == FileContent.DELETE_MANIFEST) { + V4ManifestReader leafReader = + V4ManifestReaders.readLeaf(manifestEntry, io, specsById).select(columns); + addCloseable(leafReader); + allLeafFiles.add(leafReader.liveEntries()); + } + } + + return CloseableIterable.concat(allLeafFiles); + } +} diff --git a/core/src/main/java/org/apache/iceberg/RootManifestReader.java b/core/src/main/java/org/apache/iceberg/RootManifestReader.java deleted file mode 100644 index 52410e408575..000000000000 --- a/core/src/main/java/org/apache/iceberg/RootManifestReader.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg; - -import java.util.List; -import java.util.Map; -import org.apache.iceberg.io.CloseableGroup; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** - * Reader for V4 root manifests that recursively expands manifest references. - * - *

Root manifests can contain direct file entries (DATA, POSITION_DELETES, EQUALITY_DELETES) as - * well as references to leaf manifests (DATA_MANIFEST, DELETE_MANIFEST). - * - *

RootManifestReader is the orchestrator reader that handles the recursive expansion of manifest - * references. - */ -public class RootManifestReader extends CloseableGroup { - private final V4ManifestReader rootReader; - private final FileIO io; - private final Map specsById; - - public RootManifestReader( - String rootManifestPath, - FileIO io, - Map specsById, - long snapshotId, - long sequenceNumber, - Long firstRowId) { - this.rootReader = - V4ManifestReaders.readRoot(rootManifestPath, io, snapshotId, sequenceNumber, firstRowId); - this.io = io; - this.specsById = specsById; - addCloseable(rootReader); - } - - /** - * Returns all TrackedFiles from the root manifest and all referenced leaf manifests. - * - *

This includes: - * - *

    - *
  • Direct file entries in root (DATA, POSITION_DELETES, EQUALITY_DELETES) - *
  • Files from expanded DATA_MANIFEST entries - *
  • Files from expanded DELETE_MANIFEST entries - *
- * - *

Returns TrackedFile entries (not converted to DataFile/DeleteFile yet). - * - *

TODO: Add manifest DV support. - * - * @return iterable of all tracked files - */ - public CloseableIterable> allTrackedFiles() { - return CloseableIterable.concat(Lists.newArrayList(directFiles(), expandManifests())); - } - - /** - * Returns direct file entries from the root manifest. - * - *

These are DATA, POSITION_DELETES, or EQUALITY_DELETES entries stored directly in the root - * manifest - */ - private CloseableIterable> directFiles() { - return CloseableIterable.filter( - rootReader.liveEntries(), - tf -> - tf.contentType() == FileContent.DATA - || tf.contentType() == FileContent.POSITION_DELETES - || tf.contentType() == FileContent.EQUALITY_DELETES); - } - - /** - * Expands manifest references (DATA_MANIFEST and DELETE_MANIFEST) to their contained files. - * - *

Loads all root entries to identify manifests, then reads each leaf manifest serially. - * - *

TODO: Add parallel manifest reading support via ExecutorService (like - * ManifestGroup.planWith). This would use ParallelIterable to read leaf manifests concurrently. - * - *

TODO: Add manifest DV support - group MANIFEST_DV entries by referencedFile and apply to - * leaf readers. - */ - private CloseableIterable> expandManifests() { - List> rootEntries = Lists.newArrayList(rootReader.liveEntries()); - - List>> allLeafFiles = Lists.newArrayList(); - - for (TrackedFile manifestEntry : rootEntries) { - if (manifestEntry.contentType() == FileContent.DATA_MANIFEST - || manifestEntry.contentType() == FileContent.DELETE_MANIFEST) { - V4ManifestReader leafReader = V4ManifestReaders.readLeaf(manifestEntry, io, specsById); - addCloseable(leafReader); - allLeafFiles.add(leafReader.liveEntries()); - } - } - - return CloseableIterable.concat(allLeafFiles); - } -} diff --git a/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java b/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java index eaabff798fd3..5c3bb9a829a3 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java +++ b/core/src/main/java/org/apache/iceberg/TrackedDataFileAdapter.java @@ -27,13 +27,18 @@ * Adapter that wraps a TrackedFile and presents it as a DataFile. * *

This adapter allows TrackedFile instances to be used in contexts that expect DataFile. The - * adapter returns null for partition data and column-level statistics, as these are not stored in - * TrackedFile. + * adapter returns null for partition data and column-level statistics until ContentStats is + * implemented. + * + *

TODO: When ContentStats is implemented, use spec to extract partition from contentStats. */ class TrackedDataFileAdapter implements DataFile { private final TrackedFile trackedFile; - TrackedDataFileAdapter(TrackedFile trackedFile) { + @SuppressWarnings("UnusedVariable") + private final PartitionSpec spec; + + TrackedDataFileAdapter(TrackedFile trackedFile, PartitionSpec spec) { if (trackedFile.contentType() != FileContent.DATA) { throw new IllegalStateException( "Cannot convert TrackedFile with content type " @@ -41,6 +46,7 @@ class TrackedDataFileAdapter implements DataFile { + " to DataFile"); } this.trackedFile = trackedFile; + this.spec = spec; } @Override @@ -164,12 +170,12 @@ public Long firstRowId() { @Override public DataFile copy() { - return new TrackedDataFileAdapter((TrackedFile) trackedFile.copy()); + return new TrackedDataFileAdapter((TrackedFile) trackedFile.copy(), spec); } @Override public DataFile copyWithoutStats() { - return new TrackedDataFileAdapter((TrackedFile) trackedFile.copyWithoutStats()); + return new TrackedDataFileAdapter((TrackedFile) trackedFile.copyWithoutStats(), spec); } @Override diff --git a/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java b/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java index e32cfd699571..8dc12bb70a3c 100644 --- a/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java +++ b/core/src/main/java/org/apache/iceberg/TrackedDeleteFileAdapter.java @@ -27,13 +27,18 @@ * Adapter that wraps a TrackedFile and presents it as a DeleteFile. * *

This adapter allows TrackedFile instances to be used in contexts that expect DeleteFile. The - * adapter returns null for partition data and column-level statistics, as these are not stored in - * TrackedFile. + * adapter returns null for partition data and column-level statistics until ContentStats is + * implemented. + * + *

TODO: When ContentStats is implemented, use spec to extract partition from contentStats. */ class TrackedDeleteFileAdapter implements DeleteFile { private final TrackedFile trackedFile; - TrackedDeleteFileAdapter(TrackedFile trackedFile) { + @SuppressWarnings("UnusedVariable") + private final PartitionSpec spec; + + TrackedDeleteFileAdapter(TrackedFile trackedFile, PartitionSpec spec) { FileContent contentType = trackedFile.contentType(); if (contentType != FileContent.POSITION_DELETES && contentType != FileContent.EQUALITY_DELETES) { @@ -41,6 +46,7 @@ class TrackedDeleteFileAdapter implements DeleteFile { "Cannot convert TrackedFile with content type " + contentType + " to DeleteFile"); } this.trackedFile = trackedFile; + this.spec = spec; } @Override @@ -175,12 +181,12 @@ public Long contentSizeInBytes() { @Override public DeleteFile copy() { - return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copy()); + return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copy(), spec); } @Override public DeleteFile copyWithoutStats() { - return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copyWithoutStats()); + return new TrackedDeleteFileAdapter((TrackedFile) trackedFile.copyWithoutStats(), spec); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V4ManifestReader.java b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java index 1673bd9ea3c8..7db3cf9af9fc 100644 --- a/core/src/main/java/org/apache/iceberg/V4ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java @@ -33,7 +33,7 @@ * Reader for V4 manifest files containing TrackedFile entries. * *

Supports reading both root manifests and leaf manifests. Returns TrackedFile entries which can - * represent data files, delete files, or manifest references. + * represent data files, delete files, or manifest references. TODO: implement caching. */ class V4ManifestReader extends CloseableGroup implements CloseableIterable> { static final ImmutableList ALL_COLUMNS = ImmutableList.of("*"); diff --git a/core/src/test/java/org/apache/iceberg/TestRootManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java similarity index 53% rename from core/src/test/java/org/apache/iceberg/TestRootManifestReader.java rename to core/src/test/java/org/apache/iceberg/TestManifestExpander.java index b9b254af6512..c6f1d10113de 100644 --- a/core/src/test/java/org/apache/iceberg/TestRootManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java @@ -34,7 +34,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -public class TestRootManifestReader { +public class TestManifestExpander { private final FileIO io = new InMemoryFileIO(); private final Map specsById; @@ -44,7 +44,7 @@ public class TestRootManifestReader { @TempDir private Path temp; - public TestRootManifestReader() { + public TestManifestExpander() { Schema schema = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), @@ -63,8 +63,9 @@ public void testRootWithOnlyDirectFiles() throws IOException { String rootPath = writeRootManifest(file1, file2); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -83,8 +84,9 @@ public void testRootWithOnlyManifests() throws IOException { String rootPath = writeRootManifest(manifestEntry); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -106,8 +108,9 @@ public void testRootWithMixedDirectAndManifests() throws IOException { String rootPath = writeRootManifest(directFile, manifestEntry); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -138,8 +141,9 @@ public void testMultipleDataManifests() throws IOException { String rootPath = writeRootManifest(manifestEntry1, manifestEntry2); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -157,8 +161,9 @@ public void testDeleteManifests() throws IOException { String rootPath = writeRootManifest(manifestEntry); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -179,8 +184,9 @@ public void testMixedDataAndDeleteManifests() throws IOException { String rootPath = writeRootManifest(dataManifestEntry, deleteManifestEntry); - RootManifestReader reader = - new RootManifestReader(rootPath, io, specsById, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander reader = new ManifestExpander(rootReader, io, specsById); List> allFiles = Lists.newArrayList(reader.allTrackedFiles()); @@ -194,6 +200,207 @@ public void testMixedDataAndDeleteManifests() throws IOException { assertThat(deleteFiles).isEqualTo(1); } + @Test + public void testPlanDataFilesOnlyReturnsData() throws IOException { + GenericTrackedFile dataFile = createDataFile("data.parquet", 1000L); + GenericTrackedFile deleteFile = createDeleteFile("delete.parquet", 100L); + + String rootPath = writeRootManifest(dataFile, deleteFile); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).dataFile().contentType()).isEqualTo(FileContent.DATA); + assertThat(scanInfos.get(0).dataFile().location()).endsWith("data.parquet"); + } + + @Test + public void testIgnoreDeleted() throws IOException { + GenericTrackedFile added = createDataFile("added.parquet", 1000L); + added.setStatus(TrackingInfo.Status.ADDED); + + GenericTrackedFile deleted = createDataFile("deleted.parquet", 2000L); + deleted.setStatus(TrackingInfo.Status.DELETED); + + GenericTrackedFile existing = createDataFile("existing.parquet", 3000L); + existing.setStatus(TrackingInfo.Status.EXISTING); + + String rootPath = writeRootManifest(added, deleted, existing); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById).ignoreDeleted(); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(2); + assertThat(scanInfos).noneMatch(info -> info.dataFile().location().endsWith("deleted.parquet")); + } + + @Test + public void testIgnoreExisting() throws IOException { + GenericTrackedFile added = createDataFile("added.parquet", 1000L); + added.setStatus(TrackingInfo.Status.ADDED); + + GenericTrackedFile existing = createDataFile("existing.parquet", 2000L); + existing.setStatus(TrackingInfo.Status.EXISTING); + + String rootPath = writeRootManifest(added, existing); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById).ignoreExisting(); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).dataFile().location()).endsWith("added.parquet"); + } + + @Test + public void testPlanDataFilesFromManifests() throws IOException { + GenericTrackedFile dataFile1 = createDataFile("data1.parquet", 1000L); + GenericTrackedFile dataFile2 = createDataFile("data2.parquet", 2000L); + String dataManifestPath = writeLeafManifest(dataFile1, dataFile2); + + GenericTrackedFile manifestEntry = createManifestEntry(dataManifestPath, 2, 3000L); + + String rootPath = writeRootManifest(manifestEntry); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(2); + assertThat(scanInfos).allMatch(info -> info.dataFile().contentType() == FileContent.DATA); + } + + @Test + public void testDeleteMatching() throws IOException { + String dataFilePath = "s3://bucket/table/data/file1.parquet"; + GenericTrackedFile dataFile = createDataFile("file1.parquet", 1000L); + dataFile.setLocation(dataFilePath); + + GenericTrackedFile deleteFile = createDeleteFile("delete1.parquet", 50L); + deleteFile.setReferencedFile(dataFilePath); + + String rootPath = writeRootManifest(dataFile, deleteFile); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + ManifestExpander.DataFileScanInfo scanInfo = scanInfos.get(0); + + assertThat(scanInfo.dataFile().location()).isEqualTo(dataFilePath); + assertThat(scanInfo.deleteFiles()).hasSize(1); + assertThat(scanInfo.deleteFiles().get(0).location()).endsWith("delete1.parquet"); + } + + @Test + public void testMultipleDeletesMatchToSameFile() throws IOException { + String dataFilePath = "s3://bucket/table/data/file1.parquet"; + GenericTrackedFile dataFile = createDataFile("file1.parquet", 1000L); + dataFile.setLocation(dataFilePath); + + GenericTrackedFile deleteFile1 = createDeleteFile("delete1.parquet", 50L); + deleteFile1.setReferencedFile(dataFilePath); + + GenericTrackedFile deleteFile2 = createDeleteFile("delete2.parquet", 30L); + deleteFile2.setReferencedFile(dataFilePath); + + String rootPath = writeRootManifest(dataFile, deleteFile1, deleteFile2); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).deleteFiles()).hasSize(2); + } + + @Test + public void testDeleteWithoutReferencedFileNotMatched() throws IOException { + GenericTrackedFile dataFile = createDataFile("data.parquet", 1000L); + + GenericTrackedFile deleteFile = createDeleteFile("delete.parquet", 50L); + + String rootPath = writeRootManifest(dataFile, deleteFile); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).deleteFiles()).isEmpty(); + } + + @Test + public void testDataFileWithNoDeletes() throws IOException { + GenericTrackedFile dataFile = createDataFile("data.parquet", 1000L); + + String rootPath = writeRootManifest(dataFile); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).deleteFiles()).isEmpty(); + } + + @Test + public void testSequenceNumberFiltering() throws IOException { + String dataFilePath = "s3://bucket/table/data/file1.parquet"; + GenericTrackedFile dataFile = createDataFile("file1.parquet", 1000L); + dataFile.setLocation(dataFilePath); + dataFile.setSequenceNumber(100L); + + GenericTrackedFile delete1 = createDeleteFile("delete1.parquet", 50L); + delete1.setReferencedFile(dataFilePath); + delete1.setSequenceNumber(95L); + + GenericTrackedFile delete2 = createDeleteFile("delete2.parquet", 30L); + delete2.setReferencedFile(dataFilePath); + delete2.setSequenceNumber(105L); + + String rootPath = writeRootManifest(dataFile, delete1, delete2); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).deleteFiles()).hasSize(1); + assertThat(scanInfos.get(0).deleteFiles().get(0).location()).endsWith("delete1.parquet"); + } + private GenericTrackedFile createDataFile(String filename, long recordCount) { GenericTrackedFile file = new GenericTrackedFile(); file.setContentType(FileContent.DATA); From f278ae4fded991dc477c727973393ce391e8903c Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Mon, 10 Nov 2025 11:35:23 -0800 Subject: [PATCH 12/14] Support for inline metadata DVs: 32 bit roaring bitmaps for now --- .../org/apache/iceberg/ManifestExpander.java | 30 ++++- .../org/apache/iceberg/V4ManifestReader.java | 52 ++++++++ .../apache/iceberg/TestManifestExpander.java | 123 ++++++++++++++++++ 3 files changed, 201 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestExpander.java b/core/src/main/java/org/apache/iceberg/ManifestExpander.java index 4e70423d80f9..3ed66eaa0d59 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestExpander.java +++ b/core/src/main/java/org/apache/iceberg/ManifestExpander.java @@ -32,6 +32,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.metrics.ScanMetrics; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; @@ -394,22 +395,27 @@ private CloseableIterable> directFiles() { * *

Loads all root entries to identify manifests, then reads each leaf manifest serially. * + *

Applies manifest DVs (MANIFEST_DV entries) to skip deleted positions in leaf manifests. + * *

TODO: Add parallel manifest reading support via ExecutorService (like * ManifestGroup.planWith). This would use ParallelIterable to read leaf manifests concurrently. - * - *

TODO: Add manifest DV support - group MANIFEST_DV entries by referencedFile and apply to - * leaf readers. */ private CloseableIterable> expandManifests() { List> rootEntries = Lists.newArrayList(rootReader.liveEntries()); + Map> dvByTarget = indexManifestDVs(rootEntries); + List>> allLeafFiles = Lists.newArrayList(); for (TrackedFile manifestEntry : rootEntries) { if (manifestEntry.contentType() == FileContent.DATA_MANIFEST || manifestEntry.contentType() == FileContent.DELETE_MANIFEST) { + TrackedFile dv = dvByTarget.get(manifestEntry.location()); + V4ManifestReader leafReader = - V4ManifestReaders.readLeaf(manifestEntry, io, specsById).select(columns); + V4ManifestReaders.readLeaf(manifestEntry, io, specsById) + .select(columns) + .withDeletionVector(dv); addCloseable(leafReader); allLeafFiles.add(leafReader.liveEntries()); } @@ -417,4 +423,20 @@ private CloseableIterable> expandManifests() { return CloseableIterable.concat(allLeafFiles); } + + private Map> indexManifestDVs(List> rootEntries) { + Map> index = Maps.newHashMap(); + + for (TrackedFile entry : rootEntries) { + if (entry.contentType() == FileContent.MANIFEST_DV) { + String target = entry.referencedFile(); + TrackedFile existing = index.put(target, entry); + + Preconditions.checkState( + existing == null, "Multiple MANIFEST_DVs found for manifest: %s", target); + } + } + + return index; + } } diff --git a/core/src/main/java/org/apache/iceberg/V4ManifestReader.java b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java index 7db3cf9af9fc..818ab12ac0fb 100644 --- a/core/src/main/java/org/apache/iceberg/V4ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/V4ManifestReader.java @@ -18,16 +18,23 @@ */ package org.apache.iceberg; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; import java.util.function.Function; +import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.roaringbitmap.RoaringBitmap; /** * Reader for V4 manifest files containing TrackedFile entries. @@ -43,6 +50,7 @@ class V4ManifestReader extends CloseableGroup implements CloseableIterable columns = null; + private TrackedFile manifestDV = null; protected V4ManifestReader( InputFile file, InheritableTrackedMetadata inheritableMetadata, Long manifestFirstRowId) { @@ -56,6 +64,11 @@ public V4ManifestReader select(Collection newColumns) { return this; } + public V4ManifestReader withDeletionVector(TrackedFile dv) { + this.manifestDV = dv; + return this; + } + public CloseableIterable> entries() { return entries(false); } @@ -89,9 +102,48 @@ private CloseableIterable> open(Collection cols) { transformed = assignPositions(transformed); + if (manifestDV != null) { + RoaringBitmap deletedPositions = deserializeManifestDV(manifestDV); + transformed = + CloseableIterable.filter( + transformed, + entry -> { + Long pos = entry.pos(); + // positions are 0-based and should not exceed Integer.MAX_VALUE + return pos == null || !deletedPositions.contains(pos.intValue()); + }); + } + return transformed; } + private static RoaringBitmap deserializeManifestDV(TrackedFile manifestDV) { + Preconditions.checkArgument( + manifestDV.contentType() == FileContent.MANIFEST_DV, + "Expected MANIFEST_DV, got: %s", + manifestDV.contentType()); + + DeletionVector dvInfo = manifestDV.deletionVector(); + Preconditions.checkNotNull(dvInfo, "MANIFEST_DV must have deletion_vector"); + + Preconditions.checkNotNull( + dvInfo.inlineContent(), + "Manifest DV must have inline content (External not supported): %s", + manifestDV.referencedFile()); + + ByteBuffer buffer = dvInfo.inlineContent(); + byte[] bytes = new byte[buffer.remaining()]; + buffer.asReadOnlyBuffer().get(bytes); + + RoaringBitmap bitmap = new RoaringBitmap(); + try { + bitmap.deserialize(new DataInputStream(new ByteArrayInputStream(bytes))); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to deserialize Roaring bitmap from manifest DV"); + } + return bitmap; + } + private Schema buildProjection(Collection cols) { if (cols == null || cols.containsAll(ALL_COLUMNS)) { return new Schema(GenericTrackedFile.BASE_TYPE.fields()); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestExpander.java b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java index c6f1d10113de..4d2f75619084 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestExpander.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java @@ -19,8 +19,12 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.file.Path; import java.util.List; import java.util.Map; @@ -33,6 +37,7 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.roaringbitmap.RoaringBitmap; public class TestManifestExpander { @@ -502,4 +507,122 @@ private String writeManifest(String prefix, GenericTrackedFile... entries) throw return outputFile.location(); } + + @Test + public void testManifestDVFiltersPositions() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L); + GenericTrackedFile file3 = createDataFile("file3.parquet", 3000L); + String leafManifestPath = writeLeafManifest(file1, file2, file3); + + GenericTrackedFile manifestEntry = createManifestEntry(leafManifestPath, 3, 6000L); + + GenericTrackedFile manifestDV = createManifestDV(leafManifestPath, new long[] {1}); + + String rootPath = writeRootManifest(manifestEntry, manifestDV); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(2); + assertThat(scanInfos) + .anyMatch(info -> info.dataFile().location().endsWith("file1.parquet")) + .anyMatch(info -> info.dataFile().location().endsWith("file3.parquet")); + assertThat(scanInfos).noneMatch(info -> info.dataFile().location().endsWith("file2.parquet")); + } + + @Test + public void testManifestWithoutDV() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + String leafManifestPath = writeLeafManifest(file1); + + GenericTrackedFile manifestEntry = createManifestEntry(leafManifestPath, 1, 1000L); + + String rootPath = writeRootManifest(manifestEntry); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(1); + assertThat(scanInfos.get(0).dataFile().location()).endsWith("file1.parquet"); + } + + @Test + public void testMultipleManifestDVsForSameManifest() throws IOException { + String leafManifestPath = writeLeafManifest(createDataFile("file1.parquet", 1000L)); + + GenericTrackedFile manifestEntry = createManifestEntry(leafManifestPath, 1, 1000L); + GenericTrackedFile manifestDV1 = createManifestDV(leafManifestPath, new long[] {0}); + GenericTrackedFile manifestDV2 = createManifestDV(leafManifestPath, new long[] {1}); + + String rootPath = writeRootManifest(manifestEntry, manifestDV1, manifestDV2); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + assertThatThrownBy(() -> Lists.newArrayList(expander.allTrackedFiles())) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Multiple MANIFEST_DVs found for manifest"); + } + + @Test + public void testManifestDVDeletesMultiplePositions() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L); + GenericTrackedFile file3 = createDataFile("file3.parquet", 3000L); + GenericTrackedFile file4 = createDataFile("file4.parquet", 4000L); + String leafManifestPath = writeLeafManifest(file1, file2, file3, file4); + + GenericTrackedFile manifestEntry = createManifestEntry(leafManifestPath, 4, 10000L); + + GenericTrackedFile manifestDV = createManifestDV(leafManifestPath, new long[] {0, 2}); + + String rootPath = writeRootManifest(manifestEntry, manifestDV); + + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = new ManifestExpander(rootReader, io, specsById); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(2); + assertThat(scanInfos) + .anyMatch(info -> info.dataFile().location().endsWith("file2.parquet")) + .anyMatch(info -> info.dataFile().location().endsWith("file4.parquet")); + } + + private GenericTrackedFile createManifestDV(String targetManifest, long[] deletedPositions) + throws IOException { + RoaringBitmap bitmap = new RoaringBitmap(); + for (long pos : deletedPositions) { + // positions are 0-based and should not exceed Integer.MAX_VALUE + bitmap.add((int) pos); + } + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + bitmap.serialize(new DataOutputStream(baos)); + byte[] serialized = baos.toByteArray(); + + GenericTrackedFile dv = new GenericTrackedFile(); + dv.setContentType(FileContent.MANIFEST_DV); + dv.setFileFormat(FileFormat.PUFFIN); + dv.setPartitionSpecId(0); + dv.setRecordCount(deletedPositions.length); + dv.setReferencedFile(targetManifest); + dv.setDeletionVectorInlineContent(ByteBuffer.wrap(serialized)); + dv.setStatus(TrackingInfo.Status.ADDED); + dv.setSnapshotId(SNAPSHOT_ID); + + return dv; + } } From 170d4bff2b4403028a3ef3262d7f8474d2f36d72 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 12 Nov 2025 07:35:33 -0800 Subject: [PATCH 13/14] Stub for snapshot integration --- .../org/apache/iceberg/DataTableScan.java | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/DataTableScan.java b/core/src/main/java/org/apache/iceberg/DataTableScan.java index 4d23dd525e07..0b48b6c37656 100644 --- a/core/src/main/java/org/apache/iceberg/DataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/DataTableScan.java @@ -63,8 +63,17 @@ protected TableScan newRefinedScan(Table table, Schema schema, TableScanContext @Override public CloseableIterable doPlanFiles() { Snapshot snapshot = snapshot(); - FileIO io = table().io(); + + int formatVersion = + table() instanceof HasTableOperations + ? ((HasTableOperations) table()).operations().current().formatVersion() + : 2; + + if (formatVersion >= 4) { + return planV4Files(snapshot, io); + } + List dataManifests = snapshot.dataManifests(io); List deleteManifests = snapshot.deleteManifests(io); scanMetrics().totalDataManifests().increment((long) dataManifests.size()); @@ -89,4 +98,28 @@ public CloseableIterable doPlanFiles() { return manifestGroup.planFiles(); } + + private CloseableIterable planV4Files(Snapshot snapshot, FileIO io) { + V4ManifestReader rootReader = + V4ManifestReaders.readRoot( + snapshot.manifestListLocation(), + io, + snapshot.snapshotId(), + snapshot.sequenceNumber(), + snapshot.firstRowId()); + + @SuppressWarnings("UnusedVariable") + ManifestExpander expander = + new ManifestExpander(rootReader, io, specs()) + .filterData(filter()) + .ignoreDeleted() + .select(scanColumns()) + .caseSensitive(isCaseSensitive()) + .scanMetrics(scanMetrics()); + + // TODO(anoop): implement file scan task creation after content stats + // implementation supports TrackedFile.asDataFile(spec) + throw new UnsupportedOperationException( + "V4 scans require ContentStats implementation for FileScanTask creation."); + } } From 889db0d03baa04d5a6d90a5a2c31215a72390ed2 Mon Sep 17 00:00:00 2001 From: Anoop Johnson Date: Wed, 12 Nov 2025 07:58:01 -0800 Subject: [PATCH 14/14] Implement parallel expansion --- .../org/apache/iceberg/ManifestExpander.java | 22 +++++++--- .../apache/iceberg/TestManifestExpander.java | 41 +++++++++++++++++++ 2 files changed, 58 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestExpander.java b/core/src/main/java/org/apache/iceberg/ManifestExpander.java index 3ed66eaa0d59..bd4c3e79ab6f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestExpander.java +++ b/core/src/main/java/org/apache/iceberg/ManifestExpander.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -36,6 +37,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ParallelIterable; /** * Expands V4 root manifests and plans file scan tasks. @@ -86,6 +88,8 @@ public class ManifestExpander extends CloseableGroup { @SuppressWarnings("UnusedVariable") private ScanMetrics scanMetrics; + private ExecutorService executorService; + public ManifestExpander( V4ManifestReader rootReader, FileIO io, Map specsById) { this.rootReader = rootReader; @@ -186,6 +190,11 @@ public ManifestExpander scanMetrics(ScanMetrics metrics) { return this; } + public ManifestExpander planWith(ExecutorService newExecutorService) { + this.executorService = newExecutorService; + return this; + } + /** * Plans file scan tasks for table scans with delete matching. * @@ -393,12 +402,11 @@ private CloseableIterable> directFiles() { /** * Expands manifest references (DATA_MANIFEST and DELETE_MANIFEST) to their contained files. * - *

Loads all root entries to identify manifests, then reads each leaf manifest serially. + *

Loads all root entries to identify manifests, then reads each leaf manifest. If an + * ExecutorService is provided via planWith(), manifests are read in parallel using + * ParallelIterable. Otherwise, manifests are read serially. * *

Applies manifest DVs (MANIFEST_DV entries) to skip deleted positions in leaf manifests. - * - *

TODO: Add parallel manifest reading support via ExecutorService (like - * ManifestGroup.planWith). This would use ParallelIterable to read leaf manifests concurrently. */ private CloseableIterable> expandManifests() { List> rootEntries = Lists.newArrayList(rootReader.liveEntries()); @@ -421,7 +429,11 @@ private CloseableIterable> expandManifests() { } } - return CloseableIterable.concat(allLeafFiles); + if (executorService != null) { + return new ParallelIterable<>(allLeafFiles, executorService); + } else { + return CloseableIterable.concat(allLeafFiles); + } } private Map> indexManifestDVs(List> rootEntries) { diff --git a/core/src/test/java/org/apache/iceberg/TestManifestExpander.java b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java index 4d2f75619084..f29ba0f7fcfc 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestExpander.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestExpander.java @@ -28,6 +28,8 @@ import java.nio.file.Path; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.iceberg.inmemory.InMemoryFileIO; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; @@ -625,4 +627,43 @@ private GenericTrackedFile createManifestDV(String targetManifest, long[] delete return dv; } + + @Test + public void testParallelManifestReading() throws IOException { + GenericTrackedFile file1 = createDataFile("file1.parquet", 1000L); + String manifest1Path = writeLeafManifest(file1); + + GenericTrackedFile file2 = createDataFile("file2.parquet", 2000L); + GenericTrackedFile file3 = createDataFile("file3.parquet", 3000L); + String manifest2Path = writeLeafManifest(file2, file3); + + GenericTrackedFile manifestEntry1 = createManifestEntry(manifest1Path, 1, 1000L); + GenericTrackedFile manifestEntry2 = createManifestEntry(manifest2Path, 2, 5000L); + + String rootPath = writeRootManifest(manifestEntry1, manifestEntry2); + + ExecutorService executor = Executors.newFixedThreadPool(2); + try { + V4ManifestReader rootReader = + V4ManifestReaders.readRoot(rootPath, io, SNAPSHOT_ID, SEQUENCE_NUMBER, null); + ManifestExpander expander = + new ManifestExpander(rootReader, io, specsById).planWith(executor); + + List scanInfos = + Lists.newArrayList(expander.planDataFiles()); + + assertThat(scanInfos).hasSize(3); + List locations = Lists.newArrayList(); + for (ManifestExpander.DataFileScanInfo info : scanInfos) { + locations.add(info.dataFile().location()); + } + + assertThat(locations) + .anyMatch(loc -> loc.endsWith("file1.parquet")) + .anyMatch(loc -> loc.endsWith("file2.parquet")) + .anyMatch(loc -> loc.endsWith("file3.parquet")); + } finally { + executor.shutdown(); + } + } }