diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index a992ca3f0449..ab44214e7717 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -27,6 +27,21 @@ acceptedBreaks: - code: "java.method.addedToInterface" new: "method ThisT org.apache.iceberg.SnapshotUpdate::scanManifestsWith(java.util.concurrent.ExecutorService)" justification: "Accept all changes prior to introducing API compatibility checks" + - code: "java.method.addedToInterface" + new: "method java.lang.Iterable org.apache.iceberg.Snapshot::addedFiles(org.apache.iceberg.io.FileIO)" + justification: "Allow adding a new method to the interface - old method is deprecated" + - code: "java.method.addedToInterface" + new: "method java.lang.Iterable org.apache.iceberg.Snapshot::deletedFiles(org.apache.iceberg.io.FileIO)" + justification: "Allow adding a new method to the interface - old method is deprecated" + - code: "java.method.addedToInterface" + new: "method java.util.List org.apache.iceberg.Snapshot::allManifests(org.apache.iceberg.io.FileIO)" + justification: "Allow adding a new method to the interface - old method is deprecated" + - code: "java.method.addedToInterface" + new: "method java.util.List org.apache.iceberg.Snapshot::dataManifests(org.apache.iceberg.io.FileIO)" + justification: "Allow adding a new method to the interface - old method is deprecated" + - code: "java.method.addedToInterface" + new: "method java.util.List org.apache.iceberg.Snapshot::deleteManifests(org.apache.iceberg.io.FileIO)" + justification: "Allow adding a new method to the interface - old method is deprecated" - code: "java.method.addedToInterface" new: "method long org.apache.iceberg.actions.ExpireSnapshots.Result::deletedEqualityDeleteFilesCount()" justification: "Interface is backward compatible, very unlikely anyone implements this Result bean interface" diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index d10a6c3a49d1..bdb3da4160d9 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import org.apache.iceberg.io.FileIO; /** * A snapshot of the data in a table at a point in time. @@ -68,23 +69,54 @@ public interface Snapshot extends Serializable { * Return all {@link ManifestFile} instances for either data or delete manifests in this snapshot. * * @return a list of ManifestFile + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#allManifests(FileIO)} instead. */ + @Deprecated List allManifests(); + /** + * Return all {@link ManifestFile} instances for either data or delete manifests in this snapshot. + * + * @param io a {@link FileIO} instance used for reading files from storage + * @return a list of ManifestFile + */ + List allManifests(FileIO io); + /** * Return a {@link ManifestFile} for each data manifest in this snapshot. * * @return a list of ManifestFile + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#dataManifests(FileIO)} instead. */ + @Deprecated List dataManifests(); + /** + * Return a {@link ManifestFile} for each data manifest in this snapshot. + * + * @param io a {@link FileIO} instance used for reading files from storage + * @return a list of ManifestFile + */ + List dataManifests(FileIO io); + /** * Return a {@link ManifestFile} for each delete manifest in this snapshot. * * @return a list of ManifestFile + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#deleteManifests(FileIO)} instead. */ + @Deprecated List deleteManifests(); + + /** + * Return a {@link ManifestFile} for each delete manifest in this snapshot. + * + * @param io a {@link FileIO} instance used for reading files from storage + * @return a list of ManifestFile + */ + List deleteManifests(FileIO io); + /** * Return the name of the {@link DataOperations data operation} that produced this snapshot. * @@ -107,9 +139,22 @@ public interface Snapshot extends Serializable { * record_count, and file_size_in_bytes. Other columns will be null. * * @return all files added to the table in this snapshot. + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#addedFiles(FileIO)} instead. */ + @Deprecated Iterable addedFiles(); + /** + * Return all files added to the table in this snapshot. + *

+ * The files returned include the following columns: file_path, file_format, partition, + * record_count, and file_size_in_bytes. Other columns will be null. + * + * @param io a {@link FileIO} instance used for reading files from storage + * @return all files added to the table in this snapshot. + */ + Iterable addedFiles(FileIO io); + /** * Return all files deleted from the table in this snapshot. *

@@ -117,9 +162,22 @@ public interface Snapshot extends Serializable { * record_count, and file_size_in_bytes. Other columns will be null. * * @return all files deleted from the table in this snapshot. + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#deletedFiles(FileIO)} instead. */ + @Deprecated Iterable deletedFiles(); + /** + * Return all files deleted from the table in this snapshot. + *

+ * The files returned include the following columns: file_path, file_format, partition, + * record_count, and file_size_in_bytes. Other columns will be null. + * + * @param io a {@link FileIO} instance used for reading files from storage + * @return all files deleted from the table in this snapshot. + */ + Iterable deletedFiles(FileIO io); + /** * Return the location of this snapshot's manifest list, or null if it is not separate. * diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java index f2d6c62140fc..92c1dcc12618 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java @@ -97,7 +97,8 @@ public void testParallelCommitMultiThreadSingleCommit() { table.refresh(); Assert.assertEquals("Commits should all succeed sequentially", nThreads, table.history().size()); - Assert.assertEquals("Should have all manifests", nThreads, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should have all manifests", nThreads, + table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -138,7 +139,7 @@ public void testParallelCommitMultiThreadMultiCommit() { table.refresh(); Assert.assertEquals("Commits should all succeed sequentially", 20, table.history().size()); - Assert.assertEquals("should have 20 manifests", 20, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("should have 20 manifests", 20, table.currentSnapshot().allManifests(table.io()).size()); } } diff --git a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java index 65b7e6192818..e51692f18168 100644 --- a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java @@ -66,7 +66,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return reachableManifests(Snapshot::dataManifests); + return reachableManifests(snapshot -> snapshot.dataManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/AllDeleteFilesTable.java b/core/src/main/java/org/apache/iceberg/AllDeleteFilesTable.java index 7b00a68d5154..19f299aa84dc 100644 --- a/core/src/main/java/org/apache/iceberg/AllDeleteFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllDeleteFilesTable.java @@ -66,7 +66,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return reachableManifests(Snapshot::deleteManifests); + return reachableManifests(snapshot -> snapshot.deleteManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java index f9d0bcbbe616..a2de86183fc9 100644 --- a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java @@ -84,7 +84,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable doPlanFiles() { - CloseableIterable manifests = reachableManifests(Snapshot::allManifests); + CloseableIterable manifests = + reachableManifests(snapshot -> snapshot.allManifests(tableOps().io())); String schemaString = SchemaParser.toJson(schema()); String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); diff --git a/core/src/main/java/org/apache/iceberg/AllFilesTable.java b/core/src/main/java/org/apache/iceberg/AllFilesTable.java index e4eccc8f52dd..470a9e42ad92 100644 --- a/core/src/main/java/org/apache/iceberg/AllFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllFilesTable.java @@ -66,7 +66,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return reachableManifests(Snapshot::allManifests); + return reachableManifests(snapshot -> snapshot.allManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java index 999175e063ad..c8dd98f99161 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java @@ -125,7 +125,7 @@ protected CloseableIterable doPlanFiles() { } else { return StaticDataTask.of( io.newInputFile(tableOps().current().metadataFileLocation()), - MANIFEST_FILE_SCHEMA, schema(), snap.allManifests(), + MANIFEST_FILE_SCHEMA, schema(), snap.allManifests(io), manifest -> ManifestsTable.manifestFileToRow(specs.get(manifest.partitionSpecId()), manifest) ); } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 4d6c7b34cbc5..f0e8e7904fb4 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -162,7 +162,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { @Override public List apply(TableMetadata base) { - List currentManifests = base.currentSnapshot().dataManifests(); + List currentManifests = base.currentSnapshot().dataManifests(ops.io()); Set currentManifestSet = ImmutableSet.copyOf(currentManifests); validateDeletedManifests(currentManifestSet); @@ -183,7 +183,7 @@ public List apply(TableMetadata base) { List apply = Lists.newArrayList(); Iterables.addAll(apply, newManifestsWithMetadata); apply.addAll(keptManifests); - apply.addAll(base.currentSnapshot().deleteManifests()); + apply.addAll(base.currentSnapshot().deleteManifests(ops.io())); return apply; } diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 7626d4d2b6c7..6422a437db06 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -35,7 +35,12 @@ class BaseSnapshot implements Snapshot { private static final long INITIAL_SEQUENCE_NUMBER = 0; + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; {@link FileIO} should be passed to methods which require it + */ + @Deprecated private final FileIO io; + private final long snapshotId; private final Long parentId; private final long sequenceNumber; @@ -84,6 +89,25 @@ class BaseSnapshot implements Snapshot { this.manifestListLocation = manifestList; } + BaseSnapshot(long sequenceNumber, + long snapshotId, + Long parentId, + long timestampMillis, + String operation, + Map summary, + Integer schemaId, + String manifestList) { + this.io = null; + this.sequenceNumber = sequenceNumber; + this.snapshotId = snapshotId; + this.parentId = parentId; + this.timestampMillis = timestampMillis; + this.operation = operation; + this.summary = summary; + this.schemaId = schemaId; + this.manifestListLocation = manifestList; + } + BaseSnapshot(FileIO io, long snapshotId, Long parentId, @@ -131,14 +155,14 @@ public Integer schemaId() { return schemaId; } - private void cacheManifests() { - if (io == null) { - throw new IllegalStateException("Cannot cache changes: FileIO is null"); + private void cacheManifests(FileIO fileIO) { + if (fileIO == null) { + throw new IllegalArgumentException("Cannot cache changes: FileIO is null"); } if (allManifests == null) { // if manifests isn't set, then the snapshotFile is set and should be read to get the list - this.allManifests = ManifestLists.read(io.newInputFile(manifestListLocation)); + this.allManifests = ManifestLists.read(fileIO.newInputFile(manifestListLocation)); } if (dataManifests == null || deleteManifests == null) { @@ -150,41 +174,102 @@ private void cacheManifests() { } @Override + public List allManifests(FileIO fileIO) { + if (allManifests == null) { + cacheManifests(fileIO); + } + return allManifests; + } + + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#allManifests(FileIO)} instead. + */ + @Override + @Deprecated public List allManifests() { if (allManifests == null) { - cacheManifests(); + cacheManifests(io); } return allManifests; } @Override + public List dataManifests(FileIO fileIO) { + if (dataManifests == null) { + cacheManifests(fileIO); + } + return dataManifests; + } + + + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#dataManifests(FileIO)} instead. + */ + @Override + @Deprecated public List dataManifests() { if (dataManifests == null) { - cacheManifests(); + cacheManifests(io); } return dataManifests; } @Override + public List deleteManifests(FileIO fileIO) { + if (deleteManifests == null) { + cacheManifests(fileIO); + } + return deleteManifests; + } + + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#deleteManifests(FileIO)} instead. + */ + @Override + @Deprecated public List deleteManifests() { if (deleteManifests == null) { - cacheManifests(); + cacheManifests(io); } return deleteManifests; } @Override + public List addedFiles(FileIO fileIO) { + if (cachedAdds == null) { + cacheChanges(fileIO); + } + return cachedAdds; + } + + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#addedFiles(FileIO)} instead. + */ + @Override + @Deprecated public List addedFiles() { if (cachedAdds == null) { - cacheChanges(); + cacheChanges(io); } return cachedAdds; } @Override + public List deletedFiles(FileIO fileIO) { + if (cachedDeletes == null) { + cacheChanges(fileIO); + } + return cachedDeletes; + } + + /** + * @deprecated since 0.14.0, will be removed in 1.0.0; Use {@link Snapshot#deletedFiles(FileIO)} instead. + */ + @Override + @Deprecated public List deletedFiles() { if (cachedDeletes == null) { - cacheChanges(); + cacheChanges(io); } return cachedDeletes; } @@ -194,18 +279,18 @@ public String manifestListLocation() { return manifestListLocation; } - private void cacheChanges() { - if (io == null) { - throw new IllegalStateException("Cannot cache changes: FileIO is null"); + private void cacheChanges(FileIO fileIO) { + if (fileIO == null) { + throw new IllegalArgumentException("Cannot cache changes: FileIO is null"); } ImmutableList.Builder adds = ImmutableList.builder(); ImmutableList.Builder deletes = ImmutableList.builder(); // read only manifests that were created by this snapshot - Iterable changedManifests = Iterables.filter(dataManifests(), + Iterable changedManifests = Iterables.filter(dataManifests(fileIO), manifest -> Objects.equal(manifest.snapshotId(), snapshotId)); - try (CloseableIterable> entries = new ManifestGroup(io, changedManifests) + try (CloseableIterable> entries = new ManifestGroup(fileIO, changedManifests) .ignoreExisting() .entries()) { for (ManifestEntry entry : entries) { diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 70410b682434..b64f328c32cd 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -472,7 +472,7 @@ private static Set committedFiles(TableOperations ops, Set snapsho Snapshot snap = ops.current().snapshot(snapshotId); if (snap != null) { committedFiles.add(snap.manifestListLocation()); - snap.allManifests() + snap.allManifests(ops.io()) .forEach(manifest -> committedFiles.add(manifest.path())); } else { return null; diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 1a948de994a6..203da2992a37 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -83,7 +83,7 @@ public static void dropTableData(FileIO io, TableMetadata metadata) { Set manifestsToDelete = Sets.newHashSet(); for (Snapshot snapshot : metadata.snapshots()) { // add all manifests to the delete set because both data and delete files should be removed - Iterables.addAll(manifestsToDelete, snapshot.allManifests()); + Iterables.addAll(manifestsToDelete, snapshot.allManifests(io)); // add the manifest list to the delete set, if present if (snapshot.manifestListLocation() != null) { manifestListsToDelete.add(snapshot.manifestListLocation()); diff --git a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java index b506af331649..1b96743ed34c 100644 --- a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java +++ b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.iceberg.exceptions.CherrypickAncestorCommitException; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.PropertyUtil; @@ -37,6 +38,7 @@ */ class CherryPickOperation extends MergingSnapshotProducer { + private final FileIO io; private final Map specsById; private Snapshot cherrypickSnapshot = null; private boolean requireFastForward = false; @@ -44,6 +46,7 @@ class CherryPickOperation extends MergingSnapshotProducer { CherryPickOperation(String tableName, TableOperations ops) { super(tableName, ops); + this.io = ops.io(); this.specsById = ops.current().specsById(); } @@ -76,7 +79,7 @@ public CherryPickOperation cherrypick(long snapshotId) { set(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, String.valueOf(snapshotId)); // Pick modifications from the snapshot - for (DataFile addedFile : cherrypickSnapshot.addedFiles()) { + for (DataFile addedFile : cherrypickSnapshot.addedFiles(io)) { add(addedFile); } @@ -103,13 +106,13 @@ public CherryPickOperation cherrypick(long snapshotId) { // copy adds from the picked snapshot this.replacedPartitions = PartitionSet.create(specsById); - for (DataFile addedFile : cherrypickSnapshot.addedFiles()) { + for (DataFile addedFile : cherrypickSnapshot.addedFiles(io)) { add(addedFile); replacedPartitions.add(addedFile.specId(), addedFile.partition()); } // copy deletes from the picked snapshot - for (DataFile deletedFile : cherrypickSnapshot.deletedFiles()) { + for (DataFile deletedFile : cherrypickSnapshot.deletedFiles(io)) { delete(deletedFile); } @@ -147,7 +150,7 @@ protected void validate(TableMetadata base) { // this is only called after apply() passes off to super, but check fast-forward status just in case if (!isFastForward(base)) { validateNonAncestor(base, cherrypickSnapshot.snapshotId()); - validateReplacedPartitions(base, cherrypickSnapshot.parentId(), replacedPartitions); + validateReplacedPartitions(base, cherrypickSnapshot.parentId(), replacedPartitions, io); WapUtil.validateWapPublish(base, cherrypickSnapshot.snapshotId()); } } @@ -196,11 +199,12 @@ private static void validateNonAncestor(TableMetadata meta, long snapshotId) { } private static void validateReplacedPartitions(TableMetadata meta, Long parentId, - PartitionSet replacedPartitions) { + PartitionSet replacedPartitions, FileIO io) { if (replacedPartitions != null && meta.currentSnapshot() != null) { ValidationException.check(parentId == null || isCurrentAncestor(meta, parentId), "Cannot cherry-pick overwrite, based on non-ancestor of the current state: %s", parentId); - List newFiles = SnapshotUtil.newFiles(parentId, meta.currentSnapshot().snapshotId(), meta::snapshot); + List newFiles = SnapshotUtil.newFiles( + parentId, meta.currentSnapshot().snapshotId(), meta::snapshot, io); for (DataFile newFile : newFiles) { ValidationException.check(!replacedPartitions.contains(newFile.specId(), newFile.partition()), "Cannot cherry-pick replace partitions with changed partition: %s", diff --git a/core/src/main/java/org/apache/iceberg/DataFilesTable.java b/core/src/main/java/org/apache/iceberg/DataFilesTable.java index 03a4176c0604..f9fcdd525dc5 100644 --- a/core/src/main/java/org/apache/iceberg/DataFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/DataFilesTable.java @@ -61,7 +61,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return CloseableIterable.withNoopClose(snapshot().dataManifests()); + return CloseableIterable.withNoopClose(snapshot().dataManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/DataTableScan.java b/core/src/main/java/org/apache/iceberg/DataTableScan.java index 0bd3e0fedd10..98d7f11c2783 100644 --- a/core/src/main/java/org/apache/iceberg/DataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/DataTableScan.java @@ -20,6 +20,7 @@ package org.apache.iceberg; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.PropertyUtil; @@ -79,7 +80,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche public CloseableIterable doPlanFiles() { Snapshot snapshot = snapshot(); - ManifestGroup manifestGroup = new ManifestGroup(table().io(), snapshot.dataManifests(), snapshot.deleteManifests()) + FileIO io = table().io(); + ManifestGroup manifestGroup = new ManifestGroup(io, snapshot.dataManifests(io), snapshot.deleteManifests(io)) .caseSensitive(isCaseSensitive()) .select(colStats() ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS) .filterData(filter()) @@ -90,7 +92,7 @@ public CloseableIterable doPlanFiles() { manifestGroup = manifestGroup.ignoreResiduals(); } - if (snapshot.dataManifests().size() > 1 && + if (snapshot.dataManifests(io).size() > 1 && (PLAN_SCANS_WITH_WORKER_POOL || context().planWithCustomizedExecutor())) { manifestGroup = manifestGroup.planWith(planExecutor()); } diff --git a/core/src/main/java/org/apache/iceberg/DeleteFilesTable.java b/core/src/main/java/org/apache/iceberg/DeleteFilesTable.java index 591b0dec49e3..f69c7d6508e4 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFilesTable.java @@ -61,7 +61,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return CloseableIterable.withNoopClose(snapshot().deleteManifests()); + return CloseableIterable.withNoopClose(snapshot().deleteManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 15f9dfe308a4..a13089ff9ada 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -141,7 +141,7 @@ public List apply(TableMetadata base) { Iterables.addAll(newManifests, appendManifestsWithMetadata); if (base.currentSnapshot() != null) { - newManifests.addAll(base.currentSnapshot().allManifests()); + newManifests.addAll(base.currentSnapshot().allManifests(ops.io())); } return newManifests; diff --git a/core/src/main/java/org/apache/iceberg/FilesTable.java b/core/src/main/java/org/apache/iceberg/FilesTable.java index fafad83d93f6..f2531915077d 100644 --- a/core/src/main/java/org/apache/iceberg/FilesTable.java +++ b/core/src/main/java/org/apache/iceberg/FilesTable.java @@ -61,7 +61,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable manifests() { - return CloseableIterable.withNoopClose(snapshot().allManifests()); + return CloseableIterable.withNoopClose(snapshot().allManifests(tableOps().io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/FindFiles.java b/core/src/main/java/org/apache/iceberg/FindFiles.java index 17d0753c36c0..473b37611344 100644 --- a/core/src/main/java/org/apache/iceberg/FindFiles.java +++ b/core/src/main/java/org/apache/iceberg/FindFiles.java @@ -203,7 +203,7 @@ public CloseableIterable collect() { } // when snapshot is not null - CloseableIterable> entries = new ManifestGroup(ops.io(), snapshot.dataManifests()) + CloseableIterable> entries = new ManifestGroup(ops.io(), snapshot.dataManifests(ops.io())) .specsById(ops.current().specsById()) .filterData(rowFilter) .filterFiles(fileFilter) diff --git a/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java b/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java index fda938f81609..d293a07cd3ab 100644 --- a/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java @@ -76,7 +76,7 @@ public CloseableIterable planFiles() { Set snapshotIds = Sets.newHashSet(Iterables.transform(snapshots, Snapshot::snapshotId)); Set manifests = FluentIterable .from(snapshots) - .transformAndConcat(Snapshot::dataManifests) + .transformAndConcat(snapshot -> snapshot.dataManifests(tableOps().io())) .filter(manifestFile -> snapshotIds.contains(manifestFile.snapshotId())) .toSet(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java b/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java index 17357cfcd8ee..5085dece05dc 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java @@ -90,7 +90,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche @Override protected CloseableIterable doPlanFiles() { // return entries from both data and delete manifests - CloseableIterable manifests = CloseableIterable.withNoopClose(snapshot().allManifests()); + CloseableIterable manifests = + CloseableIterable.withNoopClose(snapshot().allManifests(tableOps().io())); String schemaString = SchemaParser.toJson(schema()); String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); Expression filter = shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestsTable.java b/core/src/main/java/org/apache/iceberg/ManifestsTable.java index 85063c3cd173..b7b081d3814d 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/ManifestsTable.java @@ -80,7 +80,7 @@ protected DataTask task(TableScan scan) { return StaticDataTask.of( ops.io().newInputFile(location != null ? location : ops.current().metadataFileLocation()), - schema(), scan.schema(), scan.snapshot().allManifests(), + schema(), scan.schema(), scan.snapshot().allManifests(ops.io()), manifest -> { PartitionSpec spec = specs.get(manifest.partitionSpecId()); return ManifestsTable.manifestFileToRow(spec, manifest); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 507b21ba86e4..e184b50e8284 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -652,13 +652,13 @@ private Pair, Set> validationHistory(TableMetadata base if (matchingOperations.contains(currentSnapshot.operation())) { newSnapshots.add(currentSnapshot.snapshotId()); if (content == ManifestContent.DATA) { - for (ManifestFile manifest : currentSnapshot.dataManifests()) { + for (ManifestFile manifest : currentSnapshot.dataManifests(ops.io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } } } else { - for (ManifestFile manifest : currentSnapshot.deleteManifests()) { + for (ManifestFile manifest : currentSnapshot.deleteManifests(ops.io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } @@ -687,14 +687,14 @@ public List apply(TableMetadata base) { // filter any existing manifests List filtered = filterManager.filterManifests( - base.schema(), current != null ? current.dataManifests() : null); + base.schema(), current != null ? current.dataManifests(ops.io()) : null); long minDataSequenceNumber = filtered.stream() .map(ManifestFile::minSequenceNumber) .filter(seq -> seq != ManifestWriter.UNASSIGNED_SEQ) // filter out unassigned in rewritten manifests .reduce(base.lastSequenceNumber(), Math::min); deleteFilterManager.dropDeleteFilesOlderThan(minDataSequenceNumber); List filteredDeletes = deleteFilterManager.filterManifests( - base.schema(), current != null ? current.deleteManifests() : null); + base.schema(), current != null ? current.deleteManifests(ops.io()) : null); // only keep manifests that have live data files or that were written by this commit Predicate shouldKeep = manifest -> diff --git a/core/src/main/java/org/apache/iceberg/MicroBatches.java b/core/src/main/java/org/apache/iceberg/MicroBatches.java index 798c3379b1fd..56c7f4a371b1 100644 --- a/core/src/main/java/org/apache/iceberg/MicroBatches.java +++ b/core/src/main/java/org/apache/iceberg/MicroBatches.java @@ -113,8 +113,8 @@ public MicroBatch generate(long startFileIndex, long targetSizeInBytes, boolean Preconditions.checkArgument(startFileIndex >= 0, "startFileIndex is unexpectedly smaller than 0"); Preconditions.checkArgument(targetSizeInBytes > 0, "targetSizeInBytes should be larger than 0"); - List manifests = scanAllFiles ? snapshot.dataManifests() : - snapshot.dataManifests().stream().filter(m -> m.snapshotId().equals(snapshot.snapshotId())) + List manifests = scanAllFiles ? snapshot.dataManifests(io) : + snapshot.dataManifests(io).stream().filter(m -> m.snapshotId().equals(snapshot.snapshotId())) .collect(Collectors.toList()); List> manifestIndexes = indexManifests(manifests); diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index 38ba3496d4b4..92f8864de05e 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -24,6 +24,7 @@ import java.util.Map; import org.apache.iceberg.expressions.ManifestEvaluator; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; @@ -156,7 +157,8 @@ static CloseableIterable planFiles(StaticTableScan scan) { return ManifestEvaluator.forRowFilter(scan.filter(), transformedSpec, caseSensitive); }); - ManifestGroup manifestGroup = new ManifestGroup(table.io(), snapshot.dataManifests(), snapshot.deleteManifests()) + FileIO io = table.io(); + ManifestGroup manifestGroup = new ManifestGroup(io, snapshot.dataManifests(io), snapshot.deleteManifests(io)) .caseSensitive(caseSensitive) .filterManifests(m -> evalCache.get(m.partitionSpecId()).eval(m)) .select(scan.colStats() ? DataTableScan.SCAN_WITH_STATS_COLUMNS : DataTableScan.SCAN_COLUMNS) @@ -167,7 +169,7 @@ static CloseableIterable planFiles(StaticTableScan scan) { manifestGroup = manifestGroup.ignoreResiduals(); } - if (scan.snapshot().dataManifests().size() > 1 && + if (scan.snapshot().dataManifests(io).size() > 1 && (PLAN_SCANS_WITH_WORKER_POOL || scan.context().planWithCustomizedExecutor())) { manifestGroup = manifestGroup.planWith(scan.context().planExecutor()); } diff --git a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java index f73c451e2366..d920c5348f94 100644 --- a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java +++ b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java @@ -456,7 +456,7 @@ private CloseableIterable readManifestFiles(Snapshot snapshot) { .build(); } else { - return CloseableIterable.withNoopClose(snapshot.allManifests()); + return CloseableIterable.withNoopClose(snapshot.allManifests(ops.io())); } } } diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index 2bca3b74d775..6ae9d1eceb45 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -160,7 +160,7 @@ public Map build() { removeTimeFilters(filters, Expressions.rewriteNot(scan.filter())); Expression rowFilter = joinFilters(filters); - Iterable manifests = table.currentSnapshot().dataManifests(); + Iterable manifests = table.currentSnapshot().dataManifests(ops.io()); boolean filterByTimestamp = !timeFilters.isEmpty(); Set snapshotsInTimeRange = Sets.newHashSet(); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 0f7eed5e491e..b6e699a0e24a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -330,7 +330,7 @@ public void commit() { // id in case another commit was added between this commit and the refresh. Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); if (saved != null) { - cleanUncommitted(Sets.newHashSet(saved.allManifests())); + cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); // also clean up unused manifest lists created by multiple attempts for (String manifestList : manifestLists) { if (!saved.manifestListLocation().equals(manifestList)) { diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index 70e25e9314a5..8115920e770c 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -30,6 +30,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileIO; 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.Iterables; @@ -227,7 +228,8 @@ private static Iterable toIds(Iterable snapshots) { return Iterables.transform(snapshots, Snapshot::snapshotId); } - public static List newFiles(Long baseSnapshotId, long latestSnapshotId, Function lookup) { + public static List newFiles( + Long baseSnapshotId, long latestSnapshotId, Function lookup, FileIO io) { List newFiles = Lists.newArrayList(); Snapshot lastSnapshot = null; for (Snapshot currentSnapshot : ancestorsOf(latestSnapshotId, lookup)) { @@ -236,7 +238,7 @@ public static List newFiles(Long baseSnapshotId, long latestSnapshotId return newFiles; } - Iterables.addAll(newFiles, currentSnapshot.addedFiles()); + Iterables.addAll(newFiles, currentSnapshot.addedFiles(io)); } ValidationException.check(Objects.equals(lastSnapshot.parentId(), baseSnapshotId), diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 5abfb2c0330a..3a3ab92b5186 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -318,12 +318,12 @@ void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile... newFiles) { Assert.assertEquals("Should not change delete manifests", - old != null ? Sets.newHashSet(old.deleteManifests()) : ImmutableSet.of(), - Sets.newHashSet(snap.deleteManifests())); - List oldManifests = old != null ? old.dataManifests() : ImmutableList.of(); + old != null ? Sets.newHashSet(old.deleteManifests(FILE_IO)) : ImmutableSet.of(), + Sets.newHashSet(snap.deleteManifests(FILE_IO))); + List oldManifests = old != null ? old.dataManifests(FILE_IO) : ImmutableList.of(); // copy the manifests to a modifiable list and remove the existing manifests - List newManifests = Lists.newArrayList(snap.dataManifests()); + List newManifests = Lists.newArrayList(snap.dataManifests(FILE_IO)); for (ManifestFile oldManifest : oldManifests) { Assert.assertTrue("New snapshot should contain old manifests", newManifests.remove(oldManifest)); diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java b/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java index 322eb921d267..bff43e98a23a 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java @@ -387,7 +387,7 @@ public void testUnpartitionedTableSequenceNumbers() throws IOException { .commit(); Assert.assertEquals("Table should contain 2 delete files", - 2, (long) unpartitioned.currentSnapshot().deleteManifests().get(0).addedFilesCount()); + 2, (long) unpartitioned.currentSnapshot().deleteManifests(unpartitioned.io()).get(0).addedFilesCount()); List tasks = Lists.newArrayList(unpartitioned.newScan().planFiles().iterator()); Assert.assertEquals("Should have one task", 1, tasks.size()); @@ -425,7 +425,7 @@ public void testPartitionedTableWithExistingDeleteFile() { .commit(); Assert.assertEquals("Should have two delete manifests", - 2, table.currentSnapshot().deleteManifests().size()); + 2, table.currentSnapshot().deleteManifests(table.io()).size()); // merge delete manifests table.newAppend() @@ -433,13 +433,13 @@ public void testPartitionedTableWithExistingDeleteFile() { .commit(); Assert.assertEquals("Should have one delete manifest", - 1, table.currentSnapshot().deleteManifests().size()); + 1, table.currentSnapshot().deleteManifests(table.io()).size()); Assert.assertEquals("Should have zero added delete file", - 0, table.currentSnapshot().deleteManifests().get(0).addedFilesCount().intValue()); + 0, table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount().intValue()); Assert.assertEquals("Should have zero deleted delete file", - 0, table.currentSnapshot().deleteManifests().get(0).deletedFilesCount().intValue()); + 0, table.currentSnapshot().deleteManifests(table.io()).get(0).deletedFilesCount().intValue()); Assert.assertEquals("Should have two existing delete files", - 2, table.currentSnapshot().deleteManifests().get(0).existingFilesCount().intValue()); + 2, table.currentSnapshot().deleteManifests(table.io()).get(0).existingFilesCount().intValue()); List tasks = Lists.newArrayList(table.newScan().filter(equal(bucket("data", BUCKETS_NUMBER), 0)) diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index bce0d90612b6..cc0e84cb72dd 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -90,8 +90,8 @@ public void testMultipleDeletes() { Assert.assertEquals("Metadata should be at version 2", 2L, (long) version()); Snapshot delete = readMetadata().currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, delete.allManifests().size()); - validateManifestEntries(delete.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, delete.allManifests(FILE_IO).size()); + validateManifestEntries(delete.allManifests(table.io()).get(0), ids(delete.snapshotId(), append.snapshotId(), append.snapshotId()), files(FILE_A, FILE_B, FILE_C), statuses(Status.DELETED, Status.EXISTING, Status.EXISTING)); @@ -102,8 +102,8 @@ public void testMultipleDeletes() { Assert.assertEquals("Metadata should be at version 3", 3L, (long) version()); Snapshot delete2 = readMetadata().currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, delete2.allManifests().size()); - validateManifestEntries(delete2.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, delete2.allManifests(FILE_IO).size()); + validateManifestEntries(delete2.allManifests(FILE_IO).get(0), ids(delete2.snapshotId(), append.snapshotId()), files(FILE_B, FILE_C), statuses(Status.DELETED, Status.EXISTING)); @@ -148,8 +148,8 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { .commit(); Snapshot initialSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests().size()); - validateManifestEntries(initialSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), files(firstDataFile, secondDataFile), statuses(Status.ADDED, Status.ADDED)); @@ -160,8 +160,8 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { .commit(); Snapshot deleteSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests().size()); - validateManifestEntries(deleteSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(deleteSnapshot.allManifests(FILE_IO).get(0), ids(deleteSnapshot.snapshotId(), initialSnapshot.snapshotId()), files(firstDataFile, secondDataFile), statuses(Status.DELETED, Status.EXISTING)); @@ -173,8 +173,8 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { .commit(); Snapshot finalSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, finalSnapshot.allManifests().size()); - validateManifestEntries(finalSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, finalSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(finalSnapshot.allManifests(FILE_IO).get(0), ids(finalSnapshot.snapshotId()), files(secondDataFile), statuses(Status.DELETED)); @@ -189,8 +189,8 @@ public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { .commit(); Snapshot initialSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests().size()); - validateManifestEntries(initialSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), files(DATA_FILE_BUCKET_0_IDS_0_2, DATA_FILE_BUCKET_0_IDS_8_10), statuses(Status.ADDED, Status.ADDED)); @@ -201,8 +201,8 @@ public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { .commit(); Snapshot deleteSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests().size()); - validateManifestEntries(deleteSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(deleteSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), deleteSnapshot.snapshotId()), files(DATA_FILE_BUCKET_0_IDS_0_2, DATA_FILE_BUCKET_0_IDS_8_10), statuses(Status.EXISTING, Status.DELETED)); @@ -217,8 +217,8 @@ public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { .commit(); Snapshot initialSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests().size()); - validateManifestEntries(initialSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), files(DATA_FILE_BUCKET_0_IDS_0_2, DATA_FILE_BUCKET_0_IDS_8_10), statuses(Status.ADDED, Status.ADDED)); @@ -232,8 +232,8 @@ public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { .commit(); Snapshot deleteSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests().size()); - validateManifestEntries(deleteSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(deleteSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), deleteSnapshot.snapshotId()), files(DATA_FILE_BUCKET_0_IDS_0_2, DATA_FILE_BUCKET_0_IDS_8_10), statuses(Status.EXISTING, Status.DELETED)); @@ -295,8 +295,8 @@ public void testDeleteCaseSensitivity() { .commit(); Snapshot deleteSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests().size()); - validateManifestEntries(deleteSnapshot.allManifests().get(0), + Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + validateManifestEntries(deleteSnapshot.allManifests(FILE_IO).get(0), ids(deleteSnapshot.snapshotId()), files(DATA_FILE_BUCKET_0_IDS_0_2), statuses(Status.DELETED)); diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index baa90c378e4d..0db8e6c6bc78 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -75,7 +75,7 @@ public void testEntriesTableScan() { FileScanTask file = Iterables.getOnlyElement(scan.planFiles()); Assert.assertEquals("Data file should be the table's manifest", - Iterables.getOnlyElement(table.currentSnapshot().allManifests()).path(), file.file().path()); + Iterables.getOnlyElement(table.currentSnapshot().allManifests(table.io())).path(), file.file().path()); Assert.assertEquals("Should contain 2 data file records", 2, file.file().recordCount()); } @@ -124,10 +124,10 @@ public void testSplitPlanningWithDefaultMetadataSplitSize() { int splitSize = (int) TableProperties.METADATA_SPLIT_SIZE_DEFAULT; // default split size is 32 MB Table entriesTable = new ManifestEntriesTable(table.ops(), table); - Assert.assertEquals(1, entriesTable.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, entriesTable.currentSnapshot().allManifests(table.io()).size()); int expectedSplits = - ((int) entriesTable.currentSnapshot().allManifests().get(0).length() + splitSize - 1) / splitSize; + ((int) entriesTable.currentSnapshot().allManifests(table.io()).get(0).length() + splitSize - 1) / splitSize; TableScan scan = entriesTable.newScan(); @@ -157,10 +157,11 @@ public void testEntriesTableWithDeleteManifests() throws Exception { List files = ImmutableList.copyOf(scan.planFiles()); Assert.assertEquals("Data file should be the table's manifest", - Iterables.getOnlyElement(table.currentSnapshot().dataManifests()).path(), files.get(0).file().path()); + Iterables.getOnlyElement(table.currentSnapshot().dataManifests(table.io())).path(), files.get(0).file().path()); Assert.assertEquals("Should contain 2 data file records", 2, files.get(0).file().recordCount()); Assert.assertEquals("Delete file should be in the table manifest", - Iterables.getOnlyElement(table.currentSnapshot().deleteManifests()).path(), files.get(1).file().path()); + Iterables.getOnlyElement(table.currentSnapshot().deleteManifests(table.io())).path(), + files.get(1).file().path()); Assert.assertEquals("Should contain 1 delete file record", 1, files.get(1).file().recordCount()); } diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 0740e35cbb95..e0ffacd73778 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -113,11 +113,11 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { long commitId = snap.snapshotId(); - validateManifest(snap.allManifests().get(0), + validateManifest(snap.allManifests(FILE_IO).get(0), seqs(1, 1), ids(commitId, commitId), files(FILE_C, FILE_D)); - validateManifest(snap.allManifests().get(1), + validateManifest(snap.allManifests(FILE_IO).get(1), seqs(1, 1), ids(commitId, commitId), files(FILE_A, FILE_B)); @@ -137,7 +137,7 @@ public void testNonEmptyTableAppend() { TableMetadata base = readMetadata(); Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); - List v2manifests = base.currentSnapshot().allManifests(); + List v2manifests = base.currentSnapshot().allManifests(FILE_IO); Assert.assertEquals("Should have one existing manifest", 1, v2manifests.size()); // prepare a new append @@ -163,7 +163,7 @@ public void testNoMerge() { TableMetadata base = readMetadata(); Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); - List v3manifests = base.currentSnapshot().allManifests(); + List v3manifests = base.currentSnapshot().allManifests(FILE_IO); Assert.assertEquals("Should have 2 existing manifests", 2, v3manifests.size()); // prepare a new append @@ -193,7 +193,7 @@ public void testRefreshBeforeApply() { TableMetadata base = readMetadata(); Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); - List v2manifests = base.currentSnapshot().allManifests(); + List v2manifests = base.currentSnapshot().allManifests(FILE_IO); Assert.assertEquals("Should have 1 existing manifest", 1, v2manifests.size()); // commit from the stale table @@ -220,7 +220,7 @@ public void testRefreshBeforeCommit() { TableMetadata base = readMetadata(); Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); - List v2manifests = base.currentSnapshot().allManifests(); + List v2manifests = base.currentSnapshot().allManifests(FILE_IO); Assert.assertEquals("Should have 1 existing manifest", 1, v2manifests.size()); append.commit(); @@ -230,10 +230,10 @@ public void testRefreshBeforeCommit() { // apply was called before the conflicting commit, but the commit was still consistent validateSnapshot(base.currentSnapshot(), committed.currentSnapshot(), FILE_D); - List committedManifests = Lists.newArrayList(committed.currentSnapshot().allManifests()); - committedManifests.removeAll(base.currentSnapshot().allManifests()); + List committedManifests = Lists.newArrayList(committed.currentSnapshot().allManifests(FILE_IO)); + committedManifests.removeAll(base.currentSnapshot().allManifests(FILE_IO)); Assert.assertEquals("Should reused manifest created by apply", - pending.allManifests().get(0), committedManifests.get(0)); + pending.allManifests(FILE_IO).get(0), committedManifests.get(0)); } @Test @@ -244,7 +244,7 @@ public void testFailure() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); AssertHelpers.assertThrows("Should retry 4 times and throw last failure", @@ -262,7 +262,7 @@ public void testAppendManifestCleanup() throws IOException { ManifestFile manifest = writeManifest(FILE_A, FILE_B); AppendFiles append = table.newFastAppend().appendManifest(manifest); Snapshot pending = append.apply(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); AssertHelpers.assertThrows("Should retry 4 times and throw last failure", @@ -281,7 +281,7 @@ public void testRecoveryWithManifestList() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); append.commit(); @@ -291,7 +291,7 @@ public void testRecoveryWithManifestList() { validateSnapshot(null, metadata.currentSnapshot(), FILE_B); Assert.assertTrue("Should commit same new manifest", new File(newManifest.path()).exists()); Assert.assertTrue("Should commit the same new manifest", - metadata.currentSnapshot().allManifests().contains(newManifest)); + metadata.currentSnapshot().allManifests(FILE_IO).contains(newManifest)); } @Test @@ -304,7 +304,7 @@ public void testRecoveryWithoutManifestList() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); append.commit(); @@ -314,7 +314,7 @@ public void testRecoveryWithoutManifestList() { validateSnapshot(null, metadata.currentSnapshot(), FILE_B); Assert.assertTrue("Should commit same new manifest", new File(newManifest.path()).exists()); Assert.assertTrue("Should commit the same new manifest", - metadata.currentSnapshot().allManifests().contains(newManifest)); + metadata.currentSnapshot().allManifests(FILE_IO).contains(newManifest)); } @Test @@ -334,7 +334,7 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { .commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(FILE_IO); Assert.assertEquals("Should have 1 committed manifest", 1, manifests.size()); validateManifestEntries(manifests.get(0), diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java index eccbe4d65199..a04ddc30061f 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java @@ -47,19 +47,19 @@ public void testDelete() { .commit(); Assert.assertEquals("Table should have one append manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); table.newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); Assert.assertEquals("Table should have one delete manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); table.newAppend().commit(); Assert.assertEquals("Table should have no manifests", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -74,7 +74,7 @@ public void testPartialDelete() { Snapshot s1 = table.currentSnapshot(); Assert.assertEquals("Table should have one append manifest", - 1, s1.allManifests().size()); + 1, s1.allManifests(table.io()).size()); table.newDelete() .deleteFile(FILE_B) @@ -82,13 +82,13 @@ public void testPartialDelete() { Snapshot s2 = table.currentSnapshot(); Assert.assertEquals("Table should have one mixed manifest", - 1, s2.allManifests().size()); + 1, s2.allManifests(table.io()).size()); table.newAppend().commit(); Snapshot s3 = table.currentSnapshot(); Assert.assertEquals("Table should have the same manifests", - s2.allManifests(), s3.allManifests()); + s2.allManifests(table.io()), s3.allManifests(table.io())); } @Test @@ -102,7 +102,7 @@ public void testOverwrite() { .commit(); Assert.assertEquals("Table should have one append manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); table.newOverwrite() .overwriteByRowFilter(Expressions.alwaysTrue()) @@ -111,7 +111,7 @@ public void testOverwrite() { .commit(); Assert.assertEquals("Table should have one delete manifest and one append manifest", - 2, table.currentSnapshot().allManifests().size()); + 2, table.currentSnapshot().allManifests(table.io()).size()); table.newOverwrite() .overwriteByRowFilter(Expressions.alwaysTrue()) @@ -120,6 +120,6 @@ public void testOverwrite() { .commit(); Assert.assertEquals("Table should have one delete manifest and one append manifest", - 2, table.currentSnapshot().allManifests().size()); + 2, table.currentSnapshot().allManifests(table.io()).size()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index fd42197e8749..2060bf572cdd 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -68,11 +68,11 @@ public void testEmptyTableAppend() { V2Assert.assertEquals("Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); Assert.assertEquals("Should create 1 manifest for initial write", - 1, committedSnapshot.allManifests().size()); + 1, committedSnapshot.allManifests(table.io()).size()); long snapshotId = committedSnapshot.snapshotId(); - validateManifest(committedSnapshot.allManifests().get(0), + validateManifest(committedSnapshot.allManifests(table.io()).get(0), seqs(1, 1), ids(snapshotId, snapshotId), files(FILE_A, FILE_B), @@ -96,10 +96,11 @@ public void testEmptyTableAppendManifest() throws IOException { Assert.assertNotNull("Should create a snapshot", table.currentSnapshot()); V1Assert.assertEquals("Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals("Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should create 1 manifest for initial write", 1, committedSnapshot.allManifests().size()); + Assert.assertEquals("Should create 1 manifest for initial write", + 1, committedSnapshot.allManifests(table.io()).size()); long snapshotId = committedSnapshot.snapshotId(); - validateManifest(committedSnapshot.allManifests().get(0), + validateManifest(committedSnapshot.allManifests(table.io()).get(0), seqs(1, 1), ids(snapshotId, snapshotId), files(FILE_A, FILE_B), @@ -130,17 +131,17 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { V1Assert.assertEquals("Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals("Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); Assert.assertEquals("Should create 2 manifests for initial write", - 2, committedSnapshot.allManifests().size()); + 2, committedSnapshot.allManifests(table.io()).size()); long snapshotId = committedSnapshot.snapshotId(); - validateManifest(committedSnapshot.allManifests().get(0), + validateManifest(committedSnapshot.allManifests(table.io()).get(0), seqs(1, 1), ids(snapshotId, snapshotId), files(FILE_C, FILE_D), statuses(Status.ADDED, Status.ADDED)); - validateManifest(committedSnapshot.allManifests().get(1), + validateManifest(committedSnapshot.allManifests(table.io()).get(1), seqs(1, 1), ids(snapshotId, snapshotId), files(FILE_A, FILE_B), @@ -194,9 +195,9 @@ public void testMergeWithAppendFilesAndManifest() throws IOException { long snapshotId = committedSnapshot.snapshotId(); - Assert.assertEquals("Should create 1 merged manifest", 1, committedSnapshot.allManifests().size()); + Assert.assertEquals("Should create 1 merged manifest", 1, committedSnapshot.allManifests(table.io()).size()); - validateManifest(committedSnapshot.allManifests().get(0), + validateManifest(committedSnapshot.allManifests(table.io()).get(0), seqs(1, 1, 1, 1), ids(snapshotId, snapshotId, snapshotId, snapshotId), files(FILE_C, FILE_D, FILE_A, FILE_B), @@ -226,8 +227,8 @@ public void testMergeWithExistingManifest() { validateSnapshot(null, commitBefore, 1, FILE_A, FILE_B); Assert.assertEquals("Should create 1 manifest for initial write", - 1, commitBefore.allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, commitBefore.allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1, 1), ids(baseId, baseId), @@ -244,8 +245,8 @@ public void testMergeWithExistingManifest() { Snapshot committedAfter = table.currentSnapshot(); Assert.assertEquals("Should contain 1 merged manifest for second write", - 1, committedAfter.allManifests().size()); - ManifestFile newManifest = committedAfter.allManifests().get(0); + 1, committedAfter.allManifests(table.io()).size()); + ManifestFile newManifest = committedAfter.allManifests(table.io()).get(0); Assert.assertNotEquals("Should not contain manifest from initial write", initialManifest, newManifest); @@ -290,13 +291,13 @@ public void testManifestMergeMinCount() throws IOException { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); Assert.assertEquals("Should contain 2 merged manifest for first write", - 2, readMetadata().currentSnapshot().allManifests().size()); - validateManifest(snap1.allManifests().get(0), + 2, readMetadata().currentSnapshot().allManifests(table.io()).size()); + validateManifest(snap1.allManifests(table.io()).get(0), seqs(1), ids(commitId1), files(FILE_A), statuses(Status.ADDED)); - validateManifest(snap1.allManifests().get(1), + validateManifest(snap1.allManifests(table.io()).get(1), seqs(1, 1), ids(commitId1, commitId1), files(FILE_C, FILE_D), @@ -315,18 +316,18 @@ public void testManifestMergeMinCount() throws IOException { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); Assert.assertEquals("Should contain 3 merged manifest for second write", - 3, readMetadata().currentSnapshot().allManifests().size()); - validateManifest(snap2.allManifests().get(0), + 3, readMetadata().currentSnapshot().allManifests(table.io()).size()); + validateManifest(snap2.allManifests(table.io()).get(0), seqs(2), ids(commitId2), files(FILE_A), statuses(Status.ADDED)); - validateManifest(snap2.allManifests().get(1), + validateManifest(snap2.allManifests(table.io()).get(1), seqs(2, 2), ids(commitId2, commitId2), files(FILE_C, FILE_D), statuses(Status.ADDED, Status.ADDED)); - validateManifest(snap2.allManifests().get(2), + validateManifest(snap2.allManifests(table.io()).get(2), seqs(1, 1, 1), ids(commitId1, commitId1, commitId1), files(FILE_A, FILE_C, FILE_D), @@ -348,8 +349,12 @@ public void testManifestsMergeIntoOne() throws IOException { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); long commitId1 = snap1.snapshotId(); - Assert.assertEquals("Should contain 1 manifest", 1, snap1.allManifests().size()); - validateManifest(snap1.allManifests().get(0), seqs(1), ids(commitId1), files(FILE_A), statuses(Status.ADDED)); + Assert.assertEquals("Should contain 1 manifest", 1, snap1.allManifests(table.io()).size()); + validateManifest(snap1.allManifests(table.io()).get(0), + seqs(1), + ids(commitId1), + files(FILE_A), + statuses(Status.ADDED)); table.newAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); @@ -359,13 +364,13 @@ public void testManifestsMergeIntoOne() throws IOException { V2Assert.assertEquals("Last sequence number should be 2", 2, base.lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals("Should contain 2 manifests", 2, snap2.allManifests().size()); - validateManifest(snap2.allManifests().get(0), + Assert.assertEquals("Should contain 2 manifests", 2, snap2.allManifests(table.io()).size()); + validateManifest(snap2.allManifests(table.io()).get(0), seqs(2), ids(commitId2), files(FILE_B), statuses(Status.ADDED)); - validateManifest(snap2.allManifests().get(1), + validateManifest(snap2.allManifests(table.io()).get(1), seqs(1), ids(commitId1), files(FILE_A), @@ -382,19 +387,19 @@ public void testManifestsMergeIntoOne() throws IOException { V2Assert.assertEquals("Last sequence number should be 3", 3, base.lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals("Should contain 3 manifests", 3, snap3.allManifests().size()); + Assert.assertEquals("Should contain 3 manifests", 3, snap3.allManifests(table.io()).size()); long commitId3 = snap3.snapshotId(); - validateManifest(snap3.allManifests().get(0), + validateManifest(snap3.allManifests(table.io()).get(0), seqs(3), ids(commitId3), files(FILE_C), statuses(Status.ADDED)); - validateManifest(snap3.allManifests().get(1), + validateManifest(snap3.allManifests(table.io()).get(1), seqs(2), ids(commitId2), files(FILE_B), statuses(Status.ADDED)); - validateManifest(snap3.allManifests().get(2), + validateManifest(snap3.allManifests(table.io()).get(2), seqs(1), ids(commitId1), files(FILE_A), @@ -416,8 +421,8 @@ public void testManifestsMergeIntoOne() throws IOException { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); long commitId4 = snap4.snapshotId(); - Assert.assertEquals("Should only contains 1 merged manifest", 1, snap4.allManifests().size()); - validateManifest(snap4.allManifests().get(0), + Assert.assertEquals("Should only contains 1 merged manifest", 1, snap4.allManifests(table.io()).size()); + validateManifest(snap4.allManifests(table.io()).get(0), seqs(4, 3, 2, 1), ids(commitId4, commitId3, commitId2, commitId1), files(FILE_D, FILE_C, FILE_B, FILE_A), @@ -449,23 +454,23 @@ public void testManifestDoNotMergeMinCount() throws IOException { Snapshot committed = table.currentSnapshot(); Assert.assertEquals("Should contain 3 merged manifest after 1st write write", - 3, committed.allManifests().size()); + 3, committed.allManifests(table.io()).size()); long snapshotId = table.currentSnapshot().snapshotId(); - validateManifest(committed.allManifests().get(0), + validateManifest(committed.allManifests(table.io()).get(0), seqs(1, 1), ids(snapshotId, snapshotId), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED) ); - validateManifest(committed.allManifests().get(1), + validateManifest(committed.allManifests(table.io()).get(1), seqs(1), ids(snapshotId), files(FILE_C), statuses(Status.ADDED) ); - validateManifest(committed.allManifests().get(2), + validateManifest(committed.allManifests(table.io()).get(2), seqs(1), ids(snapshotId), files(FILE_D), @@ -496,8 +501,8 @@ public void testMergeWithExistingManifestAfterDelete() { TableMetadata base = readMetadata(); long baseId = base.currentSnapshot().snapshotId(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1, 1), ids(baseId, baseId), @@ -516,8 +521,8 @@ public void testMergeWithExistingManifestAfterDelete() { TableMetadata delete = readMetadata(); long deleteId = delete.currentSnapshot().snapshotId(); Assert.assertEquals("Should create 1 filtered manifest for delete", - 1, delete.currentSnapshot().allManifests().size()); - ManifestFile deleteManifest = delete.currentSnapshot().allManifests().get(0); + 1, delete.currentSnapshot().allManifests(table.io()).size()); + ManifestFile deleteManifest = delete.currentSnapshot().allManifests(table.io()).get(0); validateManifest(deleteManifest, seqs(2, 1), @@ -536,8 +541,8 @@ public void testMergeWithExistingManifestAfterDelete() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should contain 1 merged manifest for second write", - 1, committedSnapshot.allManifests().size()); - ManifestFile newManifest = committedSnapshot.allManifests().get(0); + 1, committedSnapshot.allManifests(table.io()).size()); + ManifestFile newManifest = committedSnapshot.allManifests(table.io()).get(0); Assert.assertNotEquals("Should not contain manifest from initial write", initialManifest, newManifest); @@ -573,7 +578,7 @@ public void testMinMergeCount() { validateSnapshot(snap1, snap2, 2, FILE_B); Assert.assertEquals("Should have 2 manifests from setup writes", - 2, readMetadata().currentSnapshot().allManifests().size()); + 2, readMetadata().currentSnapshot().allManifests(table.io()).size()); table.newAppend() .appendFile(FILE_C) @@ -584,8 +589,8 @@ public void testMinMergeCount() { TableMetadata base = readMetadata(); Assert.assertEquals("Should have 3 unmerged manifests", - 3, base.currentSnapshot().allManifests().size()); - Set unmerged = Sets.newHashSet(base.currentSnapshot().allManifests()); + 3, base.currentSnapshot().allManifests(table.io()).size()); + Set unmerged = Sets.newHashSet(base.currentSnapshot().allManifests(table.io())); table.newAppend() .appendFile(FILE_D) @@ -596,8 +601,8 @@ public void testMinMergeCount() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should contain 1 merged manifest after the 4th write", - 1, committed.allManifests().size()); - ManifestFile newManifest = committed.allManifests().get(0); + 1, committed.allManifests(table.io()).size()); + ManifestFile newManifest = committed.allManifests(table.io()).get(0); Assert.assertFalse("Should not contain previous manifests", unmerged.contains(newManifest)); long lastSnapshotId = committed.snapshotId(); @@ -631,8 +636,8 @@ public void testMergeSizeTargetWithExistingManifest() { TableMetadata base = readMetadata(); long baseId = base.currentSnapshot().snapshotId(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1, 1), ids(baseId, baseId), @@ -650,8 +655,8 @@ public void testMergeSizeTargetWithExistingManifest() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should contain 2 unmerged manifests after second write", - 2, committed.allManifests().size()); - ManifestFile newManifest = committed.allManifests().get(0); + 2, committed.allManifests(table.io()).size()); + ManifestFile newManifest = committed.allManifests(table.io()).get(0); Assert.assertNotEquals("Should not contain manifest from initial write", initialManifest, newManifest); @@ -663,7 +668,7 @@ public void testMergeSizeTargetWithExistingManifest() { statuses(Status.ADDED, Status.ADDED) ); - validateManifest(committed.allManifests().get(1), + validateManifest(committed.allManifests(table.io()).get(1), seqs(1, 1), ids(baseId, baseId), files(initialManifest), @@ -684,8 +689,8 @@ public void testChangedPartitionSpec() { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1, 1), ids(commitId, commitId), @@ -722,10 +727,10 @@ public void testChangedPartitionSpec() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should use 2 manifest files", - 2, lastSnapshot.allManifests().size()); + 2, lastSnapshot.allManifests(table.io()).size()); // new manifest comes first - validateManifest(lastSnapshot.allManifests().get(0), + validateManifest(lastSnapshot.allManifests(table.io()).get(0), seqs(2), ids(lastSnapshot.snapshotId()), files(newFileY), @@ -733,7 +738,7 @@ public void testChangedPartitionSpec() { ); Assert.assertEquals("Second manifest should be the initial manifest with the old spec", - initialManifest, lastSnapshot.allManifests().get(1)); + initialManifest, lastSnapshot.allManifests(table.io()).get(1)); } @Test @@ -757,8 +762,8 @@ public void testChangedPartitionSpecMergeExisting() { TableMetadata base = readMetadata(); Assert.assertEquals("Should contain 2 manifests", - 2, base.currentSnapshot().allManifests().size()); - ManifestFile manifest = base.currentSnapshot().allManifests().get(0); + 2, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile manifest = base.currentSnapshot().allManifests(table.io()).get(0); // build the new spec using the table's schema, which uses fresh IDs PartitionSpec newSpec = PartitionSpec.builderFor(base.schema()) @@ -789,17 +794,17 @@ public void testChangedPartitionSpecMergeExisting() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should use 2 manifest files", - 2, lastSnapshot.allManifests().size()); + 2, lastSnapshot.allManifests(table.io()).size()); Assert.assertFalse("First manifest should not be in the new snapshot", - lastSnapshot.allManifests().contains(manifest)); + lastSnapshot.allManifests(table.io()).contains(manifest)); - validateManifest(lastSnapshot.allManifests().get(0), + validateManifest(lastSnapshot.allManifests(table.io()).get(0), seqs(3), ids(lastSnapshot.snapshotId()), files(newFileY), statuses(Status.ADDED) ); - validateManifest(lastSnapshot.allManifests().get(1), + validateManifest(lastSnapshot.allManifests(table.io()).get(1), seqs(2, 1), ids(id2, id1), files(FILE_B, FILE_A), @@ -822,7 +827,7 @@ public void testFailure() { V2Assert.assertEquals("Last sequence number should be 1", 1, base.lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1), ids(baseId), files(FILE_A), statuses(Status.ADDED)); table.ops().failCommits(5); @@ -830,8 +835,8 @@ public void testFailure() { AppendFiles append = table.newAppend().appendFile(FILE_B); Snapshot pending = append.apply(); - Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests().size()); - ManifestFile newManifest = pending.allManifests().get(0); + Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests(table.io()).size()); + ManifestFile newManifest = pending.allManifests(table.io()).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); validateManifest(newManifest, @@ -844,9 +849,9 @@ public void testFailure() { V2Assert.assertEquals("Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should only contain 1 manifest file", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); - validateManifest(table.currentSnapshot().allManifests().get(0), + validateManifest(table.currentSnapshot().allManifests(table.io()).get(0), seqs(1), ids(baseId), files(initialManifest), @@ -865,7 +870,7 @@ public void testAppendManifestCleanup() throws IOException { ManifestFile manifest = writeManifest(FILE_A, FILE_B); AppendFiles append = table.newAppend().appendManifest(manifest); Snapshot pending = append.apply(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(table.io()).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); AssertHelpers.assertThrows("Should retry 4 times and throw last failure", @@ -891,7 +896,7 @@ public void testRecovery() { long baseId = base.currentSnapshot().snapshotId(); V2Assert.assertEquals("Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1), ids(baseId), files(FILE_A), statuses(Status.ADDED)); table.ops().failCommits(3); @@ -899,8 +904,8 @@ public void testRecovery() { AppendFiles append = table.newAppend().appendFile(FILE_B); Snapshot pending = append.apply(); - Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests().size()); - ManifestFile newManifest = pending.allManifests().get(0); + Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests(table.io()).size()); + ManifestFile newManifest = pending.allManifests(table.io()).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); validateManifest(newManifest, @@ -921,11 +926,11 @@ public void testRecovery() { TableMetadata metadata = readMetadata(); Assert.assertTrue("Should reuse the new manifest", new File(newManifest.path()).exists()); Assert.assertEquals("Should commit the same new manifest during retry", - Lists.newArrayList(newManifest), metadata.currentSnapshot().allManifests()); + Lists.newArrayList(newManifest), metadata.currentSnapshot().allManifests(table.io())); Assert.assertEquals("Should only contain 1 merged manifest file", - 1, table.currentSnapshot().allManifests().size()); - ManifestFile manifestFile = snapshot.allManifests().get(0); + 1, table.currentSnapshot().allManifests(table.io()).size()); + ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(2, 1), ids(snapshotId, baseId), @@ -954,9 +959,9 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { long snapshotId = snapshot.snapshotId(); validateSnapshot(null, snapshot, 1, FILE_A, FILE_B); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 committed manifest", 1, manifests.size()); - ManifestFile manifestFile = snapshot.allManifests().get(0); + ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(1, 1), ids(snapshotId, snapshotId), @@ -999,8 +1004,8 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO long commitId1 = snap1.snapshotId(); validateSnapshot(null, snap1, 1, FILE_A, FILE_B); - Assert.assertEquals("Should have only 1 manifest", 1, snap1.allManifests().size()); - validateManifest(table.currentSnapshot().allManifests().get(0), + Assert.assertEquals("Should have only 1 manifest", 1, snap1.allManifests(table.io()).size()); + validateManifest(table.currentSnapshot().allManifests(table.io()).get(0), seqs(1, 1), ids(commitId1, commitId1), files(FILE_A, FILE_B), @@ -1018,8 +1023,8 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO V2Assert.assertEquals("Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Manifests should be merged into 1", 1, snap2.allManifests().size()); - validateManifest(table.currentSnapshot().allManifests().get(0), + Assert.assertEquals("Manifests should be merged into 1", 1, snap2.allManifests(table.io()).size()); + validateManifest(table.currentSnapshot().allManifests(table.io()).get(0), seqs(2, 2, 1, 1), ids(commitId2, commitId2, commitId1, commitId1), files(FILE_C, FILE_D, FILE_A, FILE_B), @@ -1142,8 +1147,8 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); validateManifest(initialManifest, seqs(1), ids(commitId), files(FILE_A), statuses(Status.ADDED)); // build the new spec using the table's schema, which uses fresh IDs @@ -1175,20 +1180,20 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); Assert.assertEquals("Should use 2 manifest files", - 2, committedSnapshot.allManifests().size()); + 2, committedSnapshot.allManifests(table.io()).size()); // new manifest comes first - validateManifest(committedSnapshot.allManifests().get(0), + validateManifest(committedSnapshot.allManifests(table.io()).get(0), seqs(2), ids(committedSnapshot.snapshotId()), files(newFile), statuses(Status.ADDED) ); Assert.assertEquals("Second manifest should be the initial manifest with the old spec", - initialManifest, committedSnapshot.allManifests().get(1)); + initialManifest, committedSnapshot.allManifests(table.io()).get(1)); // field ids of manifest entries in two manifests with different specs of the same source field should be different - ManifestEntry entry = ManifestFiles.read(committedSnapshot.allManifests().get(0), FILE_IO) + ManifestEntry entry = ManifestFiles.read(committedSnapshot.allManifests(table.io()).get(0), FILE_IO) .entries().iterator().next(); Types.NestedField field = ((PartitionData) entry.file().partition()).getPartitionType().fields().get(0); Assert.assertEquals(1000, field.fieldId()); @@ -1197,7 +1202,7 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { Assert.assertEquals(1001, field.fieldId()); Assert.assertEquals("data_bucket", field.name()); - entry = ManifestFiles.read(committedSnapshot.allManifests().get(1), FILE_IO).entries().iterator().next(); + entry = ManifestFiles.read(committedSnapshot.allManifests(table.io()).get(1), FILE_IO).entries().iterator().next(); field = ((PartitionData) entry.file().partition()).getPartitionType().fields().get(0); Assert.assertEquals(1000, field.fieldId()); Assert.assertEquals("data_bucket", field.name()); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 0f1ead9a59bb..12ffd34bc3ed 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -109,7 +109,7 @@ public void setupTable() throws Exception { table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Moves file entries to DELETED state table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Removes all entries Assert.assertEquals("Current snapshot should be made empty", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } } @@ -317,7 +317,7 @@ public void testPartitionSpecEvolutionRemovalV1() { table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Moves file entries to DELETED state table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Removes all entries Assert.assertEquals("Current snapshot should be made empty", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } Table metadataTable = createMetadataTable(); @@ -392,7 +392,7 @@ public void testPartitionSpecEvolutionRemovalV2() { table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Moves file entries to DELETED state table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Removes all entries Assert.assertEquals("Current snapshot should be made empty", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } Table metadataTable = createMetadataTable(); @@ -453,7 +453,7 @@ public void testPartitionSpecEvolutionAdditiveV1() { table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Moves file entries to DELETED state table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Removes all entries Assert.assertEquals("Current snapshot should be made empty", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } Table metadataTable = createMetadataTable(); @@ -528,7 +528,7 @@ public void testPartitionSpecEvolutionAdditiveV2() { table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Moves file entries to DELETED state table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); // Removes all entries Assert.assertEquals("Current snapshot should be made empty", - 0, table.currentSnapshot().allManifests().size()); + 0, table.currentSnapshot().allManifests(table.io()).size()); } Table metadataTable = createMetadataTable(); diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 3292b6028199..b21e78c47b8d 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -136,9 +136,9 @@ public void testOverwriteWithoutAppend() { Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals("Table should have one manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(overwriteId, baseId), files(FILE_0_TO_4, FILE_5_TO_9), statuses(Status.DELETED, Status.EXISTING)); @@ -174,15 +174,15 @@ public void testOverwriteWithAppendOutsideOfDelete() { Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals("Table should have 2 manifests", - 2, table.currentSnapshot().allManifests().size()); + 2, table.currentSnapshot().allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(overwriteId), files(FILE_10_TO_14), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(overwriteId, baseId), files(FILE_0_TO_4, FILE_5_TO_9), statuses(Status.DELETED, Status.EXISTING)); @@ -205,9 +205,9 @@ public void testOverwriteWithMergedAppendOutsideOfDelete() { Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals("Table should have one merged manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(overwriteId, overwriteId, baseId), files(FILE_10_TO_14, FILE_0_TO_4, FILE_5_TO_9), statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 44871aade9e6..7ed319acb5cf 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -96,7 +96,7 @@ public void testExpireOlderThanWithDelete() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -106,7 +106,7 @@ public void testExpireOlderThanWithDelete() { Snapshot secondSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create replace manifest with a rewritten manifest", - 1, secondSnapshot.allManifests().size()); + 1, secondSnapshot.allManifests(table.io()).size()); table.newAppend() .appendFile(FILE_B) @@ -132,9 +132,9 @@ public void testExpireOlderThanWithDelete() { Assert.assertEquals("Should remove expired manifest lists and deleted data file", Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot.allManifests().get(0).path(), // manifest was rewritten for delete + firstSnapshot.allManifests(table.io()).get(0).path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - secondSnapshot.allManifests().get(0).path(), // manifest contained only deletes, was dropped + secondSnapshot.allManifests(table.io()).get(0).path(), // manifest contained only deletes, was dropped FILE_A.path()), // deleted deletedFiles); } @@ -153,7 +153,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -163,7 +163,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Snapshot secondSnapshot = table.currentSnapshot(); Assert.assertEquals("Should replace manifest with a rewritten manifest", - 1, secondSnapshot.allManifests().size()); + 1, secondSnapshot.allManifests(table.io()).size()); table.newFastAppend() // do not merge to keep the last snapshot's manifest valid .appendFile(FILE_C) @@ -189,7 +189,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Assert.assertEquals("Should remove expired manifest lists and deleted data file", Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot.allManifests().get(0).path(), // manifest was rewritten for delete + firstSnapshot.allManifests(table.io()).get(0).path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired FILE_A.path()), // deleted deletedFiles); @@ -209,7 +209,7 @@ public void testExpireOlderThanWithRollback() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -218,8 +218,8 @@ public void testExpireOlderThanWithRollback() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests()); - secondSnapshotManifests.removeAll(firstSnapshot.allManifests()); + Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); Assert.assertEquals("Should add one new manifest for append", 1, secondSnapshotManifests.size()); table.manageSnapshots() @@ -256,7 +256,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -265,8 +265,8 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests()); - secondSnapshotManifests.removeAll(firstSnapshot.allManifests()); + Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); Assert.assertEquals("Should add one new manifest for append", 1, secondSnapshotManifests.size()); table.manageSnapshots() @@ -740,7 +740,7 @@ public void dataFilesCleanup() throws IOException { t4 = System.currentTimeMillis(); } - List manifests = table.currentSnapshot().dataManifests(); + List manifests = table.currentSnapshot().dataManifests(table.io()); ManifestFile newManifest = writeManifest( "manifest-file-1.avro", @@ -788,7 +788,7 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { t4 = System.currentTimeMillis(); } - List manifests = table.currentSnapshot().dataManifests(); + List manifests = table.currentSnapshot().dataManifests(table.io()); ManifestFile newManifest = writeManifest( "manifest-file-1.avro", @@ -907,13 +907,13 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB.addedFiles().forEach(i -> { + snapshotB.addedFiles(table.io()).forEach(i -> { expectedDeletes.add(i.path().toString()); }); // ManifestList should be deleted too expectedDeletes.add(snapshotB.manifestListLocation()); - snapshotB.dataManifests().forEach(file -> { + snapshotB.dataManifests(table.io()).forEach(file -> { // Only the manifest of B should be deleted. if (file.snapshotId() == snapshotB.snapshotId()) { expectedDeletes.add(file.path()); @@ -982,7 +982,7 @@ public void testWithCherryPickTableSnapshot() { // Make sure no dataFiles are deleted for the B, C, D snapshot Lists.newArrayList(snapshotB, snapshotC, snapshotD).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -1035,7 +1035,7 @@ public void testWithExpiringStagedThenCherrypick() { // Make sure no dataFiles are deleted for the staged snapshot Lists.newArrayList(snapshotB).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -1048,7 +1048,7 @@ public void testWithExpiringStagedThenCherrypick() { // Make sure no dataFiles are deleted for the staged and cherry-pick Lists.newArrayList(snapshotB, snapshotD).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -1162,8 +1162,8 @@ public void testExpireWithDeleteFiles() { .addDeletes(FILE_A_DELETES) .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals("Should have 1 data manifest", 1, secondSnapshot.dataManifests().size()); - Assert.assertEquals("Should have 1 delete manifest", 1, secondSnapshot.deleteManifests().size()); + Assert.assertEquals("Should have 1 data manifest", 1, secondSnapshot.dataManifests(table.io()).size()); + Assert.assertEquals("Should have 1 delete manifest", 1, secondSnapshot.deleteManifests(table.io()).size()); // FILE_A and FILE_A_DELETES move into "DELETED" state table.newRewrite() @@ -1173,7 +1173,7 @@ public void testExpireWithDeleteFiles() { .validateFromSnapshot(secondSnapshot.snapshotId()) .commit(); Snapshot thirdSnapshot = table.currentSnapshot(); - Set manifestOfDeletedFiles = thirdSnapshot.allManifests().stream().filter( + Set manifestOfDeletedFiles = thirdSnapshot.allManifests(table.io()).stream().filter( ManifestFile::hasDeletedFiles).collect(Collectors.toSet()); Assert.assertEquals("Should have two manifests of deleted files", 2, manifestOfDeletedFiles.size()); @@ -1199,7 +1199,7 @@ public void testExpireWithDeleteFiles() { .add(firstSnapshot.manifestListLocation()) .add(secondSnapshot.manifestListLocation()) .add(thirdSnapshot.manifestListLocation()) - .addAll(secondSnapshot.allManifests().stream().map(ManifestFile::path).collect(Collectors.toList())) + .addAll(secondSnapshot.allManifests(FILE_IO).stream().map(ManifestFile::path).collect(Collectors.toList())) .addAll(manifestOfDeletedFiles.stream().map(ManifestFile::path).collect(Collectors.toList())) .build(), deletedFiles); diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index e5a97e9a9a9c..776c465ca676 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -92,15 +92,15 @@ public void testReplaceOnePartition() { long replaceId = readMetadata().currentSnapshot().snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals("Table should have 2 manifests", - 2, table.currentSnapshot().allManifests().size()); + 2, table.currentSnapshot().allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(replaceId), files(FILE_E), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(replaceId, baseId), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); @@ -126,9 +126,9 @@ public void testReplaceAndMergeOnePartition() { long replaceId = readMetadata().currentSnapshot().snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals("Table should have 1 manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(replaceId, replaceId, baseId), files(FILE_E, FILE_A, FILE_B), statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); @@ -164,12 +164,12 @@ public void testReplaceWithUnpartitionedTable() throws IOException { long replaceId = replaceMetadata.currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 2 manifests", - 2, replaceMetadata.currentSnapshot().allManifests().size()); + 2, replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).size()); - validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(0), + validateManifestEntries(replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(0), ids(replaceId), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(1), + validateManifestEntries(replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(1), ids(replaceId), files(FILE_A), statuses(Status.DELETED)); } @@ -206,9 +206,9 @@ public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { long replaceId = replaceMetadata.currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 1 manifest", - 1, replaceMetadata.currentSnapshot().allManifests().size()); + 1, replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).size()); - validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(0), + validateManifestEntries(replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(0), ids(replaceId, replaceId), files(FILE_B, FILE_A), statuses(Status.ADDED, Status.DELETED)); } @@ -253,15 +253,15 @@ public void testValidationSuccess() { long replaceId = readMetadata().currentSnapshot().snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals("Table should have 2 manifests", - 2, table.currentSnapshot().allManifests().size()); + 2, table.currentSnapshot().allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(replaceId), files(FILE_G), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(baseId, baseId), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); @@ -288,12 +288,12 @@ public void testValidationNotInvoked() { long replaceId = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 2 manifest", - 2, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 2, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(replaceId, replaceId), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(replaceId), files(FILE_E), statuses(Status.DELETED)); @@ -373,12 +373,12 @@ public void testConcurrentReplaceNoConflict() { long id3 = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 2 manifests", - 2, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 2, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(id3), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(id2), files(FILE_A), statuses(Status.ADDED)); @@ -466,16 +466,16 @@ public void testAppendReplaceNoConflict() { long id3 = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 3 manifests", - 3, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 3, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(id3), files(FILE_A), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(id2), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(2), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(2), ids(id3), files(FILE_A), statuses(Status.DELETED)); @@ -597,16 +597,16 @@ public void testDeleteReplaceNoConflict() { long id3 = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 3 manifest", - 3, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 3, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(id3), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(id1), files(FILE_A), statuses(Status.ADDED)); - validateDeleteManifest(table.currentSnapshot().allManifests().get(2), + validateDeleteManifest(table.currentSnapshot().allManifests(table.io()).get(2), seqs(2), ids(id2), files(FILE_A_DELETES), @@ -667,12 +667,12 @@ public void testOverwriteReplaceNoConflict() { long finalId = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 2 manifest", - 2, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 2, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(finalId), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(finalId), files(FILE_B), statuses(Status.DELETED)); @@ -730,16 +730,16 @@ public void testValidateOnlyDeletes() { long finalId = readMetadata().currentSnapshot().snapshotId(); Assert.assertEquals("Table should have 3 manifest", - 3, table.currentSnapshot().allManifests().size()); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + 3, table.currentSnapshot().allManifests(table.io()).size()); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(finalId), files(FILE_B), statuses(Status.ADDED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(1), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(1), ids(finalId), files(FILE_B), statuses(Status.DELETED)); - validateManifestEntries(table.currentSnapshot().allManifests().get(2), + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(2), ids(baseId), files(FILE_A), statuses(Status.ADDED)); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index a0b8c60d770d..4bd52da70e1b 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -139,26 +139,26 @@ public void testDeleteWithDuplicateEntriesInManifest() { TableMetadata base = readMetadata(); long baseSnapshotId = base.currentSnapshot().snapshotId(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); Snapshot pending = table.newRewrite() .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)) .apply(); Assert.assertEquals("Should contain 2 manifest", - 2, pending.allManifests().size()); + 2, pending.allManifests(table.io()).size()); Assert.assertFalse("Should not contain manifest from initial write", - pending.allManifests().contains(initialManifest)); + pending.allManifests(table.io()).contains(initialManifest)); long pendingId = pending.snapshotId(); - validateManifestEntries(pending.allManifests().get(0), + validateManifestEntries(pending.allManifests(table.io()).get(0), ids(pendingId), files(FILE_C), statuses(ADDED)); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pendingId, pendingId, baseSnapshotId), files(FILE_A, FILE_A, FILE_B), statuses(DELETED, DELETED, EXISTING)); @@ -179,26 +179,26 @@ public void testAddAndDelete() { TableMetadata base = readMetadata(); long baseSnapshotId = base.currentSnapshot().snapshotId(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); Snapshot pending = table.newRewrite() .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)) .apply(); Assert.assertEquals("Should contain 2 manifest", - 2, pending.allManifests().size()); + 2, pending.allManifests(table.io()).size()); Assert.assertFalse("Should not contain manifest from initial write", - pending.allManifests().contains(initialManifest)); + pending.allManifests(table.io()).contains(initialManifest)); long pendingId = pending.snapshotId(); - validateManifestEntries(pending.allManifests().get(0), + validateManifestEntries(pending.allManifests(table.io()).get(0), ids(pendingId), files(FILE_C), statuses(ADDED)); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pendingId, baseSnapshotId), files(FILE_A, FILE_B), statuses(DELETED, EXISTING)); @@ -223,8 +223,8 @@ public void testRewriteDataAndDeleteFiles() { TableMetadata base = readMetadata(); Snapshot baseSnap = base.currentSnapshot(); long baseSnapshotId = baseSnap.snapshotId(); - Assert.assertEquals("Should create 2 manifests for initial write", 2, baseSnap.allManifests().size()); - List initialManifests = baseSnap.allManifests(); + Assert.assertEquals("Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); + List initialManifests = baseSnap.allManifests(table.io()); validateManifestEntries(initialManifests.get(0), ids(baseSnapshotId, baseSnapshotId, baseSnapshotId), @@ -243,22 +243,22 @@ public void testRewriteDataAndDeleteFiles() { ImmutableSet.of(FILE_D), ImmutableSet.of()) .apply(); - Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests().size()); + Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse("Should not contain manifest from initial write", - pending.allManifests().stream().anyMatch(initialManifests::contains)); + pending.allManifests(table.io()).stream().anyMatch(initialManifests::contains)); long pendingId = pending.snapshotId(); - validateManifestEntries(pending.allManifests().get(0), + validateManifestEntries(pending.allManifests(table.io()).get(0), ids(pendingId), files(FILE_D), statuses(ADDED)); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pendingId, baseSnapshotId, baseSnapshotId), files(FILE_A, FILE_B, FILE_C), statuses(DELETED, EXISTING, EXISTING)); - validateDeleteManifest(pending.allManifests().get(2), + validateDeleteManifest(pending.allManifests(table.io()).get(2), seqs(2, 1), ids(pendingId, baseSnapshotId), files(FILE_A_DELETES, FILE_B_DELETES), @@ -284,8 +284,8 @@ public void testRewriteDataAndAssignOldSequenceNumber() { TableMetadata base = readMetadata(); Snapshot baseSnap = base.currentSnapshot(); long baseSnapshotId = baseSnap.snapshotId(); - Assert.assertEquals("Should create 2 manifests for initial write", 2, baseSnap.allManifests().size()); - List initialManifests = baseSnap.allManifests(); + Assert.assertEquals("Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); + List initialManifests = baseSnap.allManifests(table.io()); validateManifestEntries(initialManifests.get(0), ids(baseSnapshotId, baseSnapshotId, baseSnapshotId), @@ -304,12 +304,12 @@ public void testRewriteDataAndAssignOldSequenceNumber() { .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber) .apply(); - Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests().size()); + Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse("Should not contain data manifest from initial write", - pending.dataManifests().stream().anyMatch(initialManifests::contains)); + pending.dataManifests(table.io()).stream().anyMatch(initialManifests::contains)); long pendingId = pending.snapshotId(); - ManifestFile newManifest = pending.allManifests().get(0); + ManifestFile newManifest = pending.allManifests(table.io()).get(0); validateManifestEntries(newManifest, ids(pendingId), files(FILE_D), statuses(ADDED)); for (ManifestEntry entry : ManifestFiles.read(newManifest, FILE_IO).entries()) { Assert.assertEquals("Should have old sequence number for manifest entries", @@ -318,12 +318,12 @@ public void testRewriteDataAndAssignOldSequenceNumber() { Assert.assertEquals("Should use new sequence number for the manifest file", oldSequenceNumber + 1, newManifest.sequenceNumber()); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pendingId, baseSnapshotId, baseSnapshotId), files(FILE_A, FILE_B, FILE_C), statuses(DELETED, EXISTING, EXISTING)); - validateDeleteManifest(pending.allManifests().get(2), + validateDeleteManifest(pending.allManifests(table.io()).get(2), seqs(1, 1), ids(baseSnapshotId, baseSnapshotId), files(FILE_A_DELETES, FILE_B_DELETES), @@ -345,9 +345,9 @@ public void testFailure() { .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); + Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_B), statuses(ADDED)); @@ -385,22 +385,22 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { ImmutableSet.of(FILE_D), ImmutableSet.of()); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); - ManifestFile manifest3 = pending.allManifests().get(2); + Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); + ManifestFile manifest3 = pending.allManifests(table.io()).get(2); - validateManifestEntries(pending.allManifests().get(0), + validateManifestEntries(pending.allManifests(table.io()).get(0), ids(pending.snapshotId()), files(FILE_D), statuses(ADDED)); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pending.snapshotId(), baseSnapshotId, baseSnapshotId), files(FILE_A, FILE_B, FILE_C), statuses(DELETED, EXISTING, EXISTING)); - validateDeleteManifest(pending.allManifests().get(2), + validateDeleteManifest(pending.allManifests(table.io()).get(2), seqs(2, 2), ids(pending.snapshotId(), pending.snapshotId()), files(FILE_A_DELETES, FILE_B_DELETES), @@ -428,9 +428,9 @@ public void testRecovery() { RewriteFiles rewrite = table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); + Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_B), statuses(ADDED)); @@ -444,7 +444,7 @@ public void testRecovery() { TableMetadata metadata = readMetadata(); Assert.assertTrue("Should commit the manifest for append", - metadata.currentSnapshot().allManifests().contains(manifest2)); + metadata.currentSnapshot().allManifests(table.io()).contains(manifest2)); // 2 manifests added by rewrite and 1 original manifest should be found. Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); @@ -471,10 +471,10 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { ImmutableSet.of(FILE_D), ImmutableSet.of()); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); - ManifestFile manifest3 = pending.allManifests().get(2); + Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); + ManifestFile manifest3 = pending.allManifests(table.io()).get(2); validateManifestEntries(manifest1, ids(pending.snapshotId()), @@ -501,7 +501,7 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { TableMetadata metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); Assert.assertEquals("Should committed the manifests", - metadata.currentSnapshot().allManifests(), committedManifests); + metadata.currentSnapshot().allManifests(table.io()), committedManifests); // As commit success all the manifests added with rewrite should be available. Assert.assertEquals("Only 5 manifest should exist", 5, listManifestFiles().size()); @@ -526,10 +526,10 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { ); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); - ManifestFile manifest3 = pending.allManifests().get(2); + Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); + ManifestFile manifest3 = pending.allManifests(table.io()).get(2); validateManifestEntries(manifest1, ids(baseSnapshotId), @@ -557,7 +557,7 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); Assert.assertEquals("Should committed the manifests", - metadata.currentSnapshot().allManifests(), committedManifests); + metadata.currentSnapshot().allManifests(table.io()), committedManifests); // As commit success all the manifests added with rewrite should be available. Assert.assertEquals("4 manifests should exist", 4, listManifestFiles().size()); @@ -581,9 +581,9 @@ public void testRemoveAllDeletes() { ); Snapshot pending = rewrite.apply(); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests().size()); - ManifestFile manifest1 = pending.allManifests().get(0); - ManifestFile manifest2 = pending.allManifests().get(1); + Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + ManifestFile manifest1 = pending.allManifests(table.io()).get(0); + ManifestFile manifest2 = pending.allManifests(table.io()).get(1); validateManifestEntries(manifest1, ids(pending.snapshotId()), @@ -604,7 +604,7 @@ public void testRemoveAllDeletes() { TableMetadata metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2); Assert.assertTrue("Should committed the manifests", - metadata.currentSnapshot().allManifests().containsAll(committedManifests)); + metadata.currentSnapshot().allManifests(table.io()).containsAll(committedManifests)); // As commit success all the manifests added with rewrite should be available. Assert.assertEquals("4 manifests should exist", 4, listManifestFiles().size()); @@ -621,7 +621,7 @@ public void testDeleteNonExistentFile() { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); + 1, base.currentSnapshot().allManifests(table.io()).size()); AssertHelpers.assertThrows("Expected an exception", ValidationException.class, @@ -643,7 +643,7 @@ public void testAlreadyDeletedFile() { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); + 1, base.currentSnapshot().allManifests(table.io()).size()); RewriteFiles rewrite = table.newRewrite(); Snapshot pending = rewrite @@ -651,16 +651,16 @@ public void testAlreadyDeletedFile() { .apply(); Assert.assertEquals("Should contain 2 manifest", - 2, pending.allManifests().size()); + 2, pending.allManifests(table.io()).size()); long pendingId = pending.snapshotId(); - validateManifestEntries(pending.allManifests().get(0), + validateManifestEntries(pending.allManifests(table.io()).get(0), ids(pendingId), files(FILE_B), statuses(ADDED)); - validateManifestEntries(pending.allManifests().get(1), + validateManifestEntries(pending.allManifests(table.io()).get(1), ids(pendingId, base.currentSnapshot().snapshotId()), files(FILE_A), statuses(DELETED)); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 1c5c3b583d19..ebcd45a24aaf 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -68,13 +68,13 @@ public void testRewriteManifestsAppendedDirectly() throws IOException { .commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); table.rewriteManifests() .clusterBy(file -> "") .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(1, manifests.size()); validateManifestEntries(manifests.get(0), @@ -97,7 +97,7 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { .appendManifest(newManifest) .commit(); - Assert.assertEquals(1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); AtomicInteger scanThreadsIndex = new AtomicInteger(0); table.rewriteManifests() .clusterBy(file -> "") @@ -109,7 +109,7 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { })) .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(1, manifests.size()); Assert.assertTrue("Thread should be created in provided pool", scanThreadsIndex.get() > 0); } @@ -134,13 +134,13 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio .commit(); long fileAppendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); table.rewriteManifests() .clusterBy(file -> "") .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Manifests must be merged into 1", 1, manifests.size()); // get the correct file order @@ -171,7 +171,7 @@ public void testReplaceManifestsSeparate() { .commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); // cluster by path will split the manifest into two @@ -179,7 +179,7 @@ public void testReplaceManifestsSeparate() { .clusterBy(file -> file.path()) .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(2, manifests.size()); manifests.sort(Comparator.comparing(ManifestFile::path)); @@ -206,7 +206,7 @@ public void testReplaceManifestsConsolidate() throws IOException { .commit(); long appendIdB = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); // cluster by constant will combine manifests into one @@ -214,7 +214,7 @@ public void testReplaceManifestsConsolidate() throws IOException { .clusterBy(file -> "file") .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(1, manifests.size()); // get the file order correct @@ -255,7 +255,7 @@ public void testReplaceManifestsWithFilter() throws IOException { .commit(); long appendIdC = table.currentSnapshot().snapshotId(); - Assert.assertEquals(3, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(3, table.currentSnapshot().allManifests(table.io()).size()); // keep the file A manifest, combine the other two @@ -270,7 +270,7 @@ public void testReplaceManifestsWithFilter() throws IOException { }) .commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(2, manifests.size()); // get the file order correct @@ -305,14 +305,14 @@ public void testReplaceManifestsMaxSize() { .commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); // cluster by constant will combine manifests into one but small target size will create one per entry BaseRewriteManifests rewriteManifests = spy((BaseRewriteManifests) table.rewriteManifests()); when(rewriteManifests.getManifestTargetSizeBytes()).thenReturn(1L); rewriteManifests.clusterBy(file -> "file").commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(2, manifests.size()); manifests.sort(Comparator.comparing(ManifestFile::path)); @@ -354,13 +354,13 @@ public void testConcurrentRewriteManifest() throws IOException { }) .commit(); - Assert.assertEquals(2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); // commit the rewrite manifests in progress - this should perform a full rewrite as the manifest // with file B is no longer part of the snapshot rewrite.commit(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(1, manifests.size()); // get the file order correct @@ -400,7 +400,7 @@ public void testAppendDuringRewriteManifest() { .commit(); long appendIdB = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); // commit the rewrite manifests in progress rewrite.commit(); @@ -408,7 +408,7 @@ public void testAppendDuringRewriteManifest() { // the rewrite should only affect the first manifest, so we will end up with 2 manifests even though we // have a single cluster key, rewritten one should be the first in the list - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(2, manifests.size()); validateManifestEntries(manifests.get(0), @@ -438,13 +438,13 @@ public void testRewriteManifestDuringAppend() { .clusterBy(file -> "file") .commit(); - Assert.assertEquals(1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); // commit the append in progress append.commit(); long appendIdB = table.currentSnapshot().snapshotId(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals(2, manifests.size()); // last append should be the first in the list @@ -469,7 +469,7 @@ public void testBasicManifestReplacement() throws IOException { .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -493,7 +493,7 @@ public void testBasicManifestReplacement() throws IOException { rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(3, manifests.size()); validateSummary(snapshot, 1, 1, 2, 0); @@ -531,7 +531,7 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -555,7 +555,7 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(3, manifests.size()); validateSummary(snapshot, 1, 1, 2, 0); @@ -595,8 +595,8 @@ public void testWithMultiplePartitionSpec() throws IOException { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); int initialPartitionSpecId = initialManifest.partitionSpecId(); @@ -632,13 +632,13 @@ public void testWithMultiplePartitionSpec() throws IOException { .commit(); Assert.assertEquals("Should use 3 manifest files", - 3, table.currentSnapshot().allManifests().size()); + 3, table.currentSnapshot().allManifests(table.io()).size()); RewriteManifests rewriteManifests = table.rewriteManifests(); // try to cluster in 1 manifest file, but because of 2 partition specs // we should still have 2 manifest files. rewriteManifests.clusterBy(dataFile -> "file").commit(); - List manifestFiles = table.currentSnapshot().allManifests(); + List manifestFiles = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Rewrite manifest should produce 2 manifest files", 2, manifestFiles.size()); @@ -669,8 +669,8 @@ public void testManifestSizeWithMultiplePartitionSpec() throws IOException { TableMetadata base = readMetadata(); Assert.assertEquals("Should create 1 manifest for initial write", - 1, base.currentSnapshot().allManifests().size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests().get(0); + 1, base.currentSnapshot().allManifests(table.io()).size()); + ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); int initialPartitionSpecId = initialManifest.partitionSpecId(); // build the new spec using the table's schema, which uses fresh IDs @@ -705,7 +705,7 @@ public void testManifestSizeWithMultiplePartitionSpec() throws IOException { .commit(); Assert.assertEquals("Rewrite manifests should produce 3 manifest files", - 3, table.currentSnapshot().allManifests().size()); + 3, table.currentSnapshot().allManifests(table.io()).size()); // cluster by constant will combine manifests into one but small target size will create one per entry BaseRewriteManifests rewriteManifests = spy((BaseRewriteManifests) table.rewriteManifests()); @@ -713,7 +713,7 @@ public void testManifestSizeWithMultiplePartitionSpec() throws IOException { // rewriteManifests should produce 4 manifestFiles, because of targetByteSize=1 rewriteManifests.clusterBy(dataFile -> "file").commit(); - List manifestFiles = table.currentSnapshot().allManifests(); + List manifestFiles = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should use 4 manifest files", 4, manifestFiles.size()); @@ -745,7 +745,7 @@ public void testManifestReplacementConcurrentAppend() throws IOException { .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -767,12 +767,12 @@ public void testManifestReplacementConcurrentAppend() throws IOException { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals(2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(3, manifests.size()); validateSummary(snapshot, 1, 1, 2, 0); @@ -810,7 +810,7 @@ public void testManifestReplacementConcurrentDelete() throws IOException { .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -840,7 +840,7 @@ public void testManifestReplacementConcurrentDelete() throws IOException { rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(3, manifests.size()); validateSummary(snapshot, 1, 1, 2, 0); @@ -874,7 +874,7 @@ public void testManifestReplacementConcurrentConflictingDelete() throws IOExcept .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -908,7 +908,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -946,7 +946,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { .commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(3, manifests.size()); validateSummary(snapshot, 3, 1, 2, 2); @@ -977,7 +977,7 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -1012,7 +1012,7 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(2, manifests.size()); validateSummary(snapshot, 3, 0, 2, 1); @@ -1039,7 +1039,7 @@ public void testInvalidUsage() throws IOException { .commit(); Snapshot snapshot = table.currentSnapshot(); - List manifests = snapshot.allManifests(); + List manifests = snapshot.allManifests(table.io()); Assert.assertEquals(1, manifests.size()); ManifestFile manifest = manifests.get(0); @@ -1085,7 +1085,7 @@ public void testManifestReplacementFailure() throws IOException { .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -1094,7 +1094,7 @@ public void testManifestReplacementFailure() throws IOException { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - List secondSnapshotManifests = secondSnapshot.allManifests(); + List secondSnapshotManifests = secondSnapshot.allManifests(table.io()); Assert.assertEquals(2, secondSnapshotManifests.size()); ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0); @@ -1132,7 +1132,7 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE .commit(); Snapshot firstSnapshot = table.currentSnapshot(); - List firstSnapshotManifests = firstSnapshot.allManifests(); + List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); Assert.assertEquals(1, firstSnapshotManifests.size()); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); @@ -1141,7 +1141,7 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - List secondSnapshotManifests = secondSnapshot.allManifests(); + List secondSnapshotManifests = secondSnapshot.allManifests(table.io()); Assert.assertEquals(2, secondSnapshotManifests.size()); ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0); diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 65fe1afb01b1..4e317b18891a 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -55,17 +55,17 @@ public void testAddDeleteFile() { Assert.assertEquals("Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); Assert.assertEquals("Delta commit should use operation 'overwrite'", DataOperations.OVERWRITE, snap.operation()); - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests().size()); + Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); validateManifest( - snap.dataManifests().get(0), + snap.dataManifests(table.io()).get(0), seqs(1), ids(snap.snapshotId()), files(FILE_A), statuses(Status.ADDED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(1, 1), ids(snap.snapshotId(), snap.snapshotId()), files(FILE_A_DELETES, FILE_B_DELETES), @@ -107,7 +107,7 @@ public void testValidateDataFilesExistDefaults() { deleteSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); table.newRowDelta() .addDeletes(FILE_B_DELETES) @@ -116,8 +116,8 @@ public void testValidateDataFilesExistDefaults() { .commit(); Assert.assertEquals("Table should have one new delete manifest", - 1, table.currentSnapshot().deleteManifests().size()); - ManifestFile deletes = table.currentSnapshot().deleteManifests().get(0); + 1, table.currentSnapshot().deleteManifests(table.io()).size()); + ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0); validateDeleteManifest(deletes, seqs(4), ids(table.currentSnapshot().snapshotId()), @@ -155,7 +155,7 @@ public void testValidateDataFilesExistOverwrite() { deleteSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); } @Test @@ -187,7 +187,7 @@ public void testValidateDataFilesExistReplacePartitions() { deleteSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); } @Test @@ -219,10 +219,10 @@ public void testValidateDataFilesExistFromSnapshot() { Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests().size()); + Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests(table.io()).size()); // manifest with FILE_A2 added validateManifest( - snap.dataManifests().get(0), + snap.dataManifests(table.io()).get(0), seqs(2), ids(replaceSnapshotId), files(FILE_A2), @@ -230,15 +230,15 @@ public void testValidateDataFilesExistFromSnapshot() { // manifest with FILE_A deleted validateManifest( - snap.dataManifests().get(1), + snap.dataManifests(table.io()).get(1), seqs(2, 1), ids(replaceSnapshotId, appendSnapshotId), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); - Assert.assertEquals("Should have 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(3), ids(snap.snapshotId()), files(FILE_A_DELETES), @@ -274,7 +274,7 @@ public void testValidateDataFilesExistRewrite() { deleteSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); } @Test @@ -307,7 +307,7 @@ public void testValidateDataFilesExistValidateDeletes() { deleteSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); } @Test @@ -338,7 +338,7 @@ public void testValidateNoConflicts() { appendSnapshotId, table.currentSnapshot().snapshotId()); Assert.assertEquals("Table should not have any delete manifests", - 0, table.currentSnapshot().deleteManifests().size()); + 0, table.currentSnapshot().deleteManifests(table.io()).size()); } @Test @@ -369,10 +369,10 @@ public void testValidateNoConflictsFromSnapshot() { Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests().size()); + Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests(table.io()).size()); // manifest with FILE_A2 added validateManifest( - snap.dataManifests().get(0), + snap.dataManifests(table.io()).get(0), seqs(2), ids(validateFromSnapshotId), files(FILE_A2), @@ -380,15 +380,15 @@ public void testValidateNoConflictsFromSnapshot() { // manifest with FILE_A added validateManifest( - snap.dataManifests().get(1), + snap.dataManifests(table.io()).get(1), seqs(1), ids(appendSnapshotId), files(FILE_A), statuses(Status.ADDED)); - Assert.assertEquals("Should have 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(3), ids(snap.snapshotId()), files(FILE_A_DELETES), @@ -416,17 +416,17 @@ public void testOverwriteWithDeleteFile() { Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests().size()); + Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); validateManifest( - snap.dataManifests().get(0), + snap.dataManifests(table.io()).get(0), seqs(2), ids(snap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(2, 1), ids(snap.snapshotId(), deltaSnapshotId), files(FILE_A_DELETES, FILE_B_DELETES), @@ -454,25 +454,25 @@ public void testReplacePartitionsWithDeleteFile() { Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should produce 2 data manifests", 2, snap.dataManifests().size()); - int deleteManifestPos = snap.dataManifests().get(0).deletedFilesCount() > 0 ? 0 : 1; + Assert.assertEquals("Should produce 2 data manifests", 2, snap.dataManifests(table.io()).size()); + int deleteManifestPos = snap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1; validateManifest( - snap.dataManifests().get(deleteManifestPos), + snap.dataManifests(table.io()).get(deleteManifestPos), seqs(2), ids(snap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); int appendManifestPos = deleteManifestPos == 0 ? 1 : 0; validateManifest( - snap.dataManifests().get(appendManifestPos), + snap.dataManifests(table.io()).get(appendManifestPos), seqs(2), ids(snap.snapshotId()), files(FILE_A2), statuses(Status.ADDED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(2, 1), ids(snap.snapshotId(), deltaSnapshotId), files(FILE_A_DELETES, FILE_B_DELETES), @@ -500,17 +500,17 @@ public void testDeleteByExpressionWithDeleteFile() { Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests().size()); + Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); validateManifest( - snap.dataManifests().get(0), + snap.dataManifests(table.io()).get(0), seqs(2), ids(snap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); validateDeleteManifest( - snap.deleteManifests().get(0), + snap.deleteManifests(table.io()).get(0), seqs(2, 2), ids(snap.snapshotId(), snap.snapshotId()), files(FILE_A_DELETES, FILE_B_DELETES), @@ -537,17 +537,17 @@ public void testDeleteDataFileWithDeleteFile() { Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should produce 1 data manifest", 1, deleteSnap.dataManifests().size()); + Assert.assertEquals("Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size()); validateManifest( - deleteSnap.dataManifests().get(0), + deleteSnap.dataManifests(table.io()).get(0), seqs(2), ids(deleteSnap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, deleteSnap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size()); validateDeleteManifest( - deleteSnap.deleteManifests().get(0), + deleteSnap.deleteManifests(table.io()).get(0), seqs(1), ids(deltaSnapshotId), files(FILE_A_DELETES), @@ -564,10 +564,10 @@ public void testDeleteDataFileWithDeleteFile() { Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should have 0 data manifests", 0, nextSnap.dataManifests().size()); - Assert.assertEquals("Should produce 1 delete manifest", 1, nextSnap.deleteManifests().size()); + Assert.assertEquals("Should have 0 data manifests", 0, nextSnap.dataManifests(table.io()).size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size()); validateDeleteManifest( - nextSnap.deleteManifests().get(0), + nextSnap.deleteManifests(table.io()).get(0), seqs(3), ids(nextSnap.snapshotId()), files(FILE_A_DELETES), @@ -594,17 +594,17 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should produce 1 data manifest", 1, deleteSnap.dataManifests().size()); + Assert.assertEquals("Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size()); validateManifest( - deleteSnap.dataManifests().get(0), + deleteSnap.dataManifests(table.io()).get(0), seqs(2), ids(deleteSnap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, deleteSnap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size()); validateDeleteManifest( - deleteSnap.deleteManifests().get(0), + deleteSnap.deleteManifests(table.io()).get(0), seqs(1), ids(deltaSnapshotId), files(FILE_A_DELETES), @@ -619,25 +619,25 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); Assert.assertEquals("Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); - Assert.assertEquals("Should have 2 data manifests", 2, nextSnap.dataManifests().size()); - int deleteManifestPos = nextSnap.dataManifests().get(0).deletedFilesCount() > 0 ? 0 : 1; + Assert.assertEquals("Should have 2 data manifests", 2, nextSnap.dataManifests(table.io()).size()); + int deleteManifestPos = nextSnap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1; validateManifest( - nextSnap.dataManifests().get(deleteManifestPos), + nextSnap.dataManifests(table.io()).get(deleteManifestPos), seqs(2), ids(deleteSnap.snapshotId()), files(FILE_A), statuses(Status.DELETED)); int appendManifestPos = deleteManifestPos == 0 ? 1 : 0; validateManifest( - nextSnap.dataManifests().get(appendManifestPos), + nextSnap.dataManifests(table.io()).get(appendManifestPos), seqs(3), ids(nextSnap.snapshotId()), files(FILE_B), statuses(Status.ADDED)); - Assert.assertEquals("Should produce 1 delete manifest", 1, nextSnap.deleteManifests().size()); + Assert.assertEquals("Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size()); validateDeleteManifest( - nextSnap.deleteManifests().get(0), + nextSnap.deleteManifests(table.io()).get(0), seqs(1), ids(deltaSnapshotId), files(FILE_A_DELETES), @@ -705,8 +705,8 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { rowDelta.commit(); Assert.assertEquals("Table should have one new delete manifest", - 1, table.currentSnapshot().deleteManifests().size()); - ManifestFile deletes = table.currentSnapshot().deleteManifests().get(0); + 1, table.currentSnapshot().deleteManifests(table.io()).size()); + ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0); validateDeleteManifest(deletes, seqs(4), ids(table.currentSnapshot().snapshotId()), @@ -841,18 +841,18 @@ public void testAddDeleteFilesMultipleSpecs() { summary.get(CHANGED_PARTITION_PREFIX + "data=xyz").contains(ADDED_FILES_PROP + "=1")); // 3 appends + 1 row delta - Assert.assertEquals("Should have 4 data manifest", 4, snapshot.dataManifests().size()); + Assert.assertEquals("Should have 4 data manifest", 4, snapshot.dataManifests(table.io()).size()); validateManifest( - snapshot.dataManifests().get(0), + snapshot.dataManifests(table.io()).get(0), seqs(4), ids(snapshot.snapshotId()), files(dataFile), statuses(Status.ADDED)); // each delete file goes into a separate manifest as the specs are different - Assert.assertEquals("Should produce 3 delete manifest", 3, snapshot.deleteManifests().size()); + Assert.assertEquals("Should produce 3 delete manifest", 3, snapshot.deleteManifests(table.io()).size()); - ManifestFile firstDeleteManifest = snapshot.deleteManifests().get(2); + ManifestFile firstDeleteManifest = snapshot.deleteManifests(table.io()).get(2); Assert.assertEquals("Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId()); validateDeleteManifest( firstDeleteManifest, @@ -861,7 +861,7 @@ public void testAddDeleteFilesMultipleSpecs() { files(firstDeleteFile), statuses(Status.ADDED)); - ManifestFile secondDeleteManifest = snapshot.deleteManifests().get(1); + ManifestFile secondDeleteManifest = snapshot.deleteManifests(table.io()).get(1); Assert.assertEquals("Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId()); validateDeleteManifest( secondDeleteManifest, @@ -870,7 +870,7 @@ public void testAddDeleteFilesMultipleSpecs() { files(secondDeleteFile), statuses(Status.ADDED)); - ManifestFile thirdDeleteManifest = snapshot.deleteManifests().get(0); + ManifestFile thirdDeleteManifest = snapshot.deleteManifests(table.io()).get(0); Assert.assertEquals("Spec must match", thirdSnapshotDataFile.specId(), thirdDeleteManifest.partitionSpecId()); validateDeleteManifest( thirdDeleteManifest, @@ -919,8 +919,8 @@ public void testManifestMergingMultipleSpecs() { Snapshot thirdSnapshot = table.currentSnapshot(); // 2 appends and 1 row delta where delete files belong to different specs - Assert.assertEquals("Should have 2 data manifest", 2, thirdSnapshot.dataManifests().size()); - Assert.assertEquals("Should have 2 delete manifest", 2, thirdSnapshot.deleteManifests().size()); + Assert.assertEquals("Should have 2 data manifest", 2, thirdSnapshot.dataManifests(table.io()).size()); + Assert.assertEquals("Should have 2 delete manifest", 2, thirdSnapshot.deleteManifests(table.io()).size()); // commit two more delete files to the same specs to trigger merging DeleteFile thirdDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); @@ -934,10 +934,10 @@ public void testManifestMergingMultipleSpecs() { Snapshot fourthSnapshot = table.currentSnapshot(); // make sure merging respects spec boundaries - Assert.assertEquals("Should have 2 data manifest", 2, fourthSnapshot.dataManifests().size()); - Assert.assertEquals("Should have 2 delete manifest", 2, fourthSnapshot.deleteManifests().size()); + Assert.assertEquals("Should have 2 data manifest", 2, fourthSnapshot.dataManifests(table.io()).size()); + Assert.assertEquals("Should have 2 delete manifest", 2, fourthSnapshot.deleteManifests(table.io()).size()); - ManifestFile firstDeleteManifest = fourthSnapshot.deleteManifests().get(1); + ManifestFile firstDeleteManifest = fourthSnapshot.deleteManifests(table.io()).get(1); Assert.assertEquals("Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId()); validateDeleteManifest( firstDeleteManifest, @@ -946,7 +946,7 @@ public void testManifestMergingMultipleSpecs() { files(thirdDeleteFile, firstDeleteFile), statuses(Status.ADDED, Status.EXISTING)); - ManifestFile secondDeleteManifest = fourthSnapshot.deleteManifests().get(0); + ManifestFile secondDeleteManifest = fourthSnapshot.deleteManifests(table.io()).get(0); Assert.assertEquals("Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId()); validateDeleteManifest( secondDeleteManifest, @@ -1219,7 +1219,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { rewriteFiles.commit(); table.refresh(); - List dataManifests = table.currentSnapshot().dataManifests(); + List dataManifests = table.currentSnapshot().dataManifests(table.io()); Assert.assertEquals("should have 1 data manifest", 1, dataManifests.size()); ManifestFile mergedDataManifest = dataManifests.get(0); diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index ab4406aab7f9..d596dce16a60 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -37,7 +37,7 @@ public void testRewrite() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -46,7 +46,7 @@ public void testRewrite() { table.newFastAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap1, snap2, 2, FILE_B); validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B)); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); @@ -54,7 +54,7 @@ public void testRewrite() { table.rewriteManifests().clusterBy(file -> "").commit(); Snapshot snap3 = table.currentSnapshot(); - ManifestFile newManifest = snap3.allManifests().stream() + ManifestFile newManifest = snap3.allManifests(table.io()).stream() .filter(manifest -> manifest.snapshotId() == snap3.snapshotId()) .collect(Collectors.toList()).get(0); @@ -96,7 +96,7 @@ public void testCommitConflict() { appendA.commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -107,7 +107,7 @@ public void testCommitConflict() { table.newFastAppend().appendFile(FILE_D).commit(); Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap1, snap2, 2, FILE_D); validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_D)); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); @@ -116,7 +116,7 @@ public void testCommitConflict() { appendFiles.commit(); Snapshot snap3 = table.currentSnapshot(); long commitId3 = snap3.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C)); validateSnapshot(snap2, snap3, 3, FILE_C); V2Assert.assertEquals("Snapshot sequence number should be 3", 3, snap3.sequenceNumber()); @@ -128,7 +128,7 @@ public void testRollBack() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -137,7 +137,7 @@ public void testRollBack() { table.newFastAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap1, snap2, 2, FILE_B); validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B)); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); @@ -151,7 +151,7 @@ public void testRollBack() { table.newFastAppend().appendFile(FILE_C).commit(); Snapshot snap4 = table.currentSnapshot(); long commitId4 = snap4.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap3, snap4, 3, FILE_C); validateManifest(manifestFile, seqs(3), ids(commitId4), files(FILE_C)); V2Assert.assertEquals("Snapshot sequence number should be 1", 3, snap4.sequenceNumber()); @@ -165,7 +165,7 @@ public void testSingleTransaction() { txn.commitTransaction(); Snapshot snap = table.currentSnapshot(); long commitId = snap.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber()); @@ -187,7 +187,7 @@ public void testConcurrentTransaction() { txn1.commitTransaction(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile1 = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile1 = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile1, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -196,7 +196,7 @@ public void testConcurrentTransaction() { txn2.commitTransaction(); Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap1, snap2, 2, FILE_B); validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B)); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); @@ -205,7 +205,7 @@ public void testConcurrentTransaction() { txn3.commitTransaction(); Snapshot snap3 = table.currentSnapshot(); long commitId3 = snap3.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().stream() + manifestFile = table.currentSnapshot().allManifests(table.io()).stream() .filter(manifest -> manifest.snapshotId() == commitId3) .collect(Collectors.toList()).get(0); validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C)); @@ -216,7 +216,7 @@ public void testConcurrentTransaction() { txn4.commitTransaction(); Snapshot snap4 = table.currentSnapshot(); long commitId4 = snap4.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().stream() + manifestFile = table.currentSnapshot().allManifests(table.io()).stream() .filter(manifest -> manifest.snapshotId() == commitId4) .collect(Collectors.toList()).get(0); validateManifest(manifestFile, seqs(4), ids(commitId4), files(FILE_A), statuses(Status.DELETED)); @@ -230,7 +230,7 @@ public void testMultipleOperationsTransaction() { txn.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = txn.table().currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = snap1.allManifests().get(0); + ManifestFile manifestFile = snap1.allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -245,7 +245,7 @@ public void testMultipleOperationsTransaction() { Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - manifestFile = snap2.allManifests().stream() + manifestFile = snap2.allManifests(table.io()).stream() .filter(manifest -> manifest.snapshotId() == commitId2) .collect(Collectors.toList()).get(0); @@ -259,7 +259,7 @@ public void testExpirationInTransaction() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -268,7 +268,7 @@ public void testExpirationInTransaction() { table.newAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); long commitId2 = snap2.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(snap1, snap2, 2, FILE_B); validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B)); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); @@ -288,7 +288,7 @@ public void testTransactionFailure() { .commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = table.currentSnapshot().allManifests().get(0); + ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A, FILE_B); validateManifest(manifestFile, seqs(1, 1), ids(commitId1, commitId1), files(FILE_A, FILE_B)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -316,7 +316,7 @@ public void testCherryPicking() { .commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = snap1.allManifests().get(0); + ManifestFile manifestFile = snap1.allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -342,7 +342,7 @@ public void testCherryPicking() { Snapshot snap3 = table.currentSnapshot(); long commitId3 = snap3.snapshotId(); - manifestFile = snap3.allManifests().get(0); + manifestFile = snap3.allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C)); validateSnapshot(snap2, snap3, 3, FILE_C); V2Assert.assertEquals("Snapshot sequence number should be 3", 3, snap3.sequenceNumber()); @@ -352,7 +352,7 @@ public void testCherryPicking() { table.manageSnapshots().cherrypick(stagedSnapshot.snapshotId()).commit(); Snapshot snap4 = table.currentSnapshot(); long commitId4 = snap4.snapshotId(); - manifestFile = table.currentSnapshot().allManifests().get(0); + manifestFile = table.currentSnapshot().allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(4), ids(commitId4), files(FILE_B)); validateSnapshot(snap3, snap4, 4, FILE_B); V2Assert.assertEquals("Snapshot sequence number should be 4", 4, snap4.sequenceNumber()); @@ -366,7 +366,7 @@ public void testCherryPickFastForward() { .commit(); Snapshot snap1 = table.currentSnapshot(); long commitId1 = snap1.snapshotId(); - ManifestFile manifestFile = snap1.allManifests().get(0); + ManifestFile manifestFile = snap1.allManifests(table.io()).get(0); validateSnapshot(null, snap1, 1, FILE_A); validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A)); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -388,7 +388,7 @@ public void testCherryPickFastForward() { table.manageSnapshots().cherrypick(stagedSnapshot.snapshotId()).commit(); Snapshot snap3 = table.currentSnapshot(); long commitId3 = snap3.snapshotId(); - manifestFile = snap3.allManifests().get(0); + manifestFile = snap3.allManifests(table.io()).get(0); validateManifest(manifestFile, seqs(2), ids(commitId3), files(FILE_B)); validateSnapshot(snap2, snap3, 2, FILE_B); V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap3.sequenceNumber()); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index d918accb7355..11d8f0ac2f14 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -42,7 +42,7 @@ public void testAppendFilesFromTable() { .commit(); // collect data files from deserialization - Iterable filesToAdd = table.currentSnapshot().addedFiles(); + Iterable filesToAdd = table.currentSnapshot().addedFiles(table.io()); table.newDelete().deleteFile(FILE_A).deleteFile(FILE_B).commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 22f0b0ccdbc6..130129ab3477 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -47,7 +47,7 @@ public void testJsonConversion() { Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); Assert.assertEquals("Files should match", - expected.allManifests(), snapshot.allManifests()); + expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); Assert.assertNull("Operation should be null", snapshot.operation()); Assert.assertNull("Summary should be null", snapshot.summary()); Assert.assertEquals("Schema ID should match", Integer.valueOf(1), snapshot.schemaId()); @@ -63,7 +63,7 @@ public void testJsonConversionWithoutSchemaId() { Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); Assert.assertEquals("Files should match", - expected.allManifests(), snapshot.allManifests()); + expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); Assert.assertNull("Operation should be null", snapshot.operation()); Assert.assertNull("Summary should be null", snapshot.summary()); Assert.assertNull("Schema ID should be null", snapshot.schemaId()); @@ -95,7 +95,7 @@ public void testJsonConversionWithOperation() { Assert.assertEquals("Manifest list should match", expected.manifestListLocation(), snapshot.manifestListLocation()); Assert.assertEquals("Files should match", - expected.allManifests(), snapshot.allManifests()); + expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); Assert.assertEquals("Operation should match", expected.operation(), snapshot.operation()); Assert.assertEquals("Summary should match", @@ -127,7 +127,7 @@ public void testJsonConversionWithManifestList() throws IOException { ops.io(), id, parentId, expected.timestampMillis(), null, null, 4, manifests); Assert.assertEquals("Files should match in memory list", - inMemory.allManifests(), expected.allManifests()); + inMemory.allManifests(ops.io()), expected.allManifests(ops.io())); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json); @@ -143,7 +143,7 @@ public void testJsonConversionWithManifestList() throws IOException { Assert.assertEquals("Manifest list should match", expected.manifestListLocation(), snapshot.manifestListLocation()); Assert.assertEquals("Files should match", - expected.allManifests(), snapshot.allManifests()); + expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); Assert.assertNull("Operation should be null", snapshot.operation()); Assert.assertNull("Summary should be null", snapshot.summary()); Assert.assertEquals("Schema ID should match", expected.schemaId(), snapshot.schemaId()); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java index 75ebd23cc0eb..d828baaea4b6 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java @@ -79,7 +79,7 @@ public void testSnapshotStatsForAddedFiles() { .commit(); Snapshot snapshot = table.currentSnapshot(); - Iterable addedFiles = snapshot.addedFiles(); + Iterable addedFiles = snapshot.addedFiles(table.io()); Assert.assertEquals(1, Iterables.size(addedFiles)); DataFile dataFile = Iterables.getOnlyElement(addedFiles); Assert.assertNotNull("Value counts should be not null", dataFile.valueCounts()); diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 51484bdddc98..7d9ec39a669b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -156,14 +156,14 @@ public void testJsonConversion() throws Exception { Assert.assertEquals("Parent snapshot ID should match", (Long) previousSnapshotId, metadata.currentSnapshot().parentId()); Assert.assertEquals("Current snapshot files should match", - currentSnapshot.allManifests(), metadata.currentSnapshot().allManifests()); + currentSnapshot.allManifests(ops.io()), metadata.currentSnapshot().allManifests(ops.io())); Assert.assertEquals("Schema ID for current snapshot should match", (Integer) 7, metadata.currentSnapshot().schemaId()); Assert.assertEquals("Previous snapshot ID should match", previousSnapshotId, metadata.snapshot(previousSnapshotId).snapshotId()); Assert.assertEquals("Previous snapshot files should match", - previousSnapshot.allManifests(), - metadata.snapshot(previousSnapshotId).allManifests()); + previousSnapshot.allManifests(ops.io()), + metadata.snapshot(previousSnapshotId).allManifests(ops.io())); Assert.assertNull("Previous snapshot's schema ID should be null", metadata.snapshot(previousSnapshotId).schemaId()); Assert.assertEquals("Refs map should match", @@ -231,14 +231,14 @@ public void testBackwardCompat() throws Exception { Assert.assertEquals("Parent snapshot ID should match", (Long) previousSnapshotId, metadata.currentSnapshot().parentId()); Assert.assertEquals("Current snapshot files should match", - currentSnapshot.allManifests(), metadata.currentSnapshot().allManifests()); + currentSnapshot.allManifests(ops.io()), metadata.currentSnapshot().allManifests(ops.io())); Assert.assertNull("Current snapshot's schema ID should be null", metadata.currentSnapshot().schemaId()); Assert.assertEquals("Previous snapshot ID should match", previousSnapshotId, metadata.snapshot(previousSnapshotId).snapshotId()); Assert.assertEquals("Previous snapshot files should match", - previousSnapshot.allManifests(), - metadata.snapshot(previousSnapshotId).allManifests()); + previousSnapshot.allManifests(ops.io()), + metadata.snapshot(previousSnapshotId).allManifests(ops.io())); Assert.assertEquals("Snapshot logs should match", expected.previousFiles(), metadata.previousFiles()); Assert.assertNull("Previous snapshot's schema ID should be null", diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 83cb41b0581b..c6817622506b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -69,8 +69,8 @@ public void testPartitionAppend() throws IOException { .appendFile(dataFile) .commit(); long id = table.currentSnapshot().snapshotId(); - Assert.assertEquals(table.currentSnapshot().allManifests().size(), 1); - validateManifestEntries(table.currentSnapshot().allManifests().get(0), + Assert.assertEquals(table.currentSnapshot().allManifests(table.io()).size(), 1); + validateManifestEntries(table.currentSnapshot().allManifests(table.io()).get(0), ids(id), files(dataFile), statuses(ManifestEntry.Status.ADDED)); diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 36eb7fbc692d..6c0fd69312bf 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -125,16 +125,16 @@ public void testMultipleOperationTransaction() { Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version()); Assert.assertEquals("Table should have one manifest after commit", - 1, readMetadata().currentSnapshot().allManifests().size()); + 1, readMetadata().currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Table snapshot should be the delete snapshot", deleteSnapshot, readMetadata().currentSnapshot()); - validateManifestEntries(readMetadata().currentSnapshot().allManifests().get(0), + validateManifestEntries(readMetadata().currentSnapshot().allManifests(table.io()).get(0), ids(deleteSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); Assert.assertEquals("Table should have a snapshot for each operation", 2, readMetadata().snapshots().size()); - validateManifestEntries(readMetadata().snapshots().get(0).allManifests().get(0), + validateManifestEntries(readMetadata().snapshots().get(0).allManifests(table.io()).get(0), ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); } @@ -176,16 +176,16 @@ public void testMultipleOperationTransactionFromTable() { Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version()); Assert.assertEquals("Table should have one manifest after commit", - 1, readMetadata().currentSnapshot().allManifests().size()); + 1, readMetadata().currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Table snapshot should be the delete snapshot", deleteSnapshot, readMetadata().currentSnapshot()); - validateManifestEntries(readMetadata().currentSnapshot().allManifests().get(0), + validateManifestEntries(readMetadata().currentSnapshot().allManifests(table.io()).get(0), ids(deleteSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); Assert.assertEquals("Table should have a snapshot for each operation", 2, readMetadata().snapshots().size()); - validateManifestEntries(readMetadata().snapshots().get(0).allManifests().get(0), + validateManifestEntries(readMetadata().snapshots().get(0).allManifests(table.io()).get(0), ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); } @@ -293,7 +293,7 @@ public void testTransactionRetry() { .appendFile(FILE_B) .commit(); - Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests()); + Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests(table.io())); Assert.assertSame("Base metadata should not change when commit is created", base, readMetadata()); @@ -307,7 +307,7 @@ public void testTransactionRetry() { Assert.assertEquals("Table should be on version 2 after commit", 2, (int) version()); Assert.assertEquals("Should reuse manifests from initial append commit", - appendManifests, Sets.newHashSet(table.currentSnapshot().allManifests())); + appendManifests, Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))); } @Test @@ -332,7 +332,7 @@ public void testTransactionRetryMergeAppend() { .appendFile(FILE_B) .commit(); - Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests()); + Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests(table.io())); Assert.assertSame("Base metadata should not change when commit is created", base, readMetadata()); @@ -346,7 +346,7 @@ public void testTransactionRetryMergeAppend() { Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); - Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests()); + Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); @@ -357,7 +357,7 @@ public void testTransactionRetryMergeAppend() { expectedManifests.addAll(conflictAppendManifests); Assert.assertEquals("Should reuse manifests from initial append commit and conflicting append", - expectedManifests, Sets.newHashSet(table.currentSnapshot().allManifests())); + expectedManifests, Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))); } @Test @@ -388,8 +388,8 @@ public void testMultipleUpdateTransactionRetryMergeCleanup() { .commit(); Assert.assertEquals("Append should create one manifest", - 1, txn.table().currentSnapshot().allManifests().size()); - ManifestFile appendManifest = txn.table().currentSnapshot().allManifests().get(0); + 1, txn.table().currentSnapshot().allManifests(table.io()).size()); + ManifestFile appendManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); Assert.assertSame("Base metadata should not change when commit is created", base, readMetadata()); @@ -403,7 +403,7 @@ public void testMultipleUpdateTransactionRetryMergeCleanup() { Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); - Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests()); + Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); @@ -414,9 +414,9 @@ public void testMultipleUpdateTransactionRetryMergeCleanup() { previousManifests.addAll(conflictAppendManifests); Assert.assertEquals("Should merge both commit manifests into a single manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); Assert.assertFalse("Should merge both commit manifests into a new manifest", - previousManifests.contains(table.currentSnapshot().allManifests().get(0))); + previousManifests.contains(table.currentSnapshot().allManifests(table.io()).get(0))); Assert.assertFalse("Append manifest should be deleted", new File(appendManifest.path()).exists()); } @@ -476,8 +476,8 @@ public void testTransactionRetryMergeCleanup() { .commit(); Assert.assertEquals("Append should create one manifest", - 1, txn.table().currentSnapshot().allManifests().size()); - ManifestFile appendManifest = txn.table().currentSnapshot().allManifests().get(0); + 1, txn.table().currentSnapshot().allManifests(table.io()).size()); + ManifestFile appendManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); Assert.assertSame("Base metadata should not change when commit is created", base, readMetadata()); @@ -491,7 +491,7 @@ public void testTransactionRetryMergeCleanup() { Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); - Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests()); + Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); @@ -502,9 +502,9 @@ public void testTransactionRetryMergeCleanup() { previousManifests.addAll(conflictAppendManifests); Assert.assertEquals("Should merge both commit manifests into a single manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); Assert.assertFalse("Should merge both commit manifests into a new manifest", - previousManifests.contains(table.currentSnapshot().allManifests().get(0))); + previousManifests.contains(table.currentSnapshot().allManifests(table.io()).get(0))); Assert.assertFalse("Append manifest should be deleted", new File(appendManifest.path()).exists()); } @@ -525,8 +525,9 @@ public void testTransactionRetryAndAppendManifests() throws Exception { .commit(); Assert.assertEquals("Table should be on version 2 after append", 2, (int) version()); - Assert.assertEquals("Append should create one manifest", 1, table.currentSnapshot().allManifests().size()); - ManifestFile v1manifest = table.currentSnapshot().allManifests().get(0); + Assert.assertEquals("Append should create one manifest", + 1, table.currentSnapshot().allManifests(table.io()).size()); + ManifestFile v1manifest = table.currentSnapshot().allManifests(table.io()).get(0); TableMetadata base = readMetadata(); @@ -549,8 +550,8 @@ public void testTransactionRetryAndAppendManifests() throws Exception { Assert.assertEquals("Table should be on version 2 after txn create", 2, (int) version()); Assert.assertEquals("Append should have one merged manifest", - 1, txn.table().currentSnapshot().allManifests().size()); - ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests().get(0); + 1, txn.table().currentSnapshot().allManifests(table.io()).size()); + ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); // find the initial copy of the appended manifest String copiedAppendManifest = Iterables.getOnlyElement(Iterables.filter( @@ -580,7 +581,7 @@ public void testTransactionRetryAndAppendManifests() throws Exception { Assert.assertFalse("Append manifest should be deleted", new File(mergedManifest.path()).exists()); Assert.assertEquals("Should merge all commit manifests into a single manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -600,7 +601,8 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th .commit(); Assert.assertEquals("Table should be on version 2 after append", 2, (int) version()); - Assert.assertEquals("Append should create one manifest", 1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Append should create one manifest", + 1, table.currentSnapshot().allManifests(table.io()).size()); TableMetadata base = readMetadata(); @@ -615,8 +617,8 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th Assert.assertEquals("Table should be on version 2 after txn create", 2, (int) version()); Assert.assertEquals("Append should have one merged manifest", - 1, txn.table().currentSnapshot().allManifests().size()); - ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests().get(0); + 1, txn.table().currentSnapshot().allManifests(table.io()).size()); + ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); // cause the transaction commit to fail and retry table.newAppend() @@ -638,7 +640,7 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th Assert.assertFalse("Merged append manifest should be deleted", new File(mergedManifest.path()).exists()); Assert.assertEquals("Should merge all commit manifests into a single manifest", - 1, table.currentSnapshot().allManifests().size()); + 1, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -670,7 +672,7 @@ public void testTransactionFastAppends() { txn.commitTransaction(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Expected 2 manifests", 2, manifests.size()); } @@ -693,7 +695,7 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException .commit(); long secondSnapshotId = table.currentSnapshot().snapshotId(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests after 2 appends", 2, manifests.size()); ManifestFile newManifest = writeManifest( @@ -717,7 +719,7 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException Assert.assertTrue("Append manifest should not be deleted", new File(newManifest.path()).exists()); - List finalManifests = table.currentSnapshot().allManifests(); + List finalManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 final manifest", 1, finalManifests.size()); validateManifestEntries(finalManifests.get(0), @@ -749,7 +751,7 @@ public void testSimpleTransactionNotDeletingMetadataOnUnknownSate() throws IOExc // Make sure metadata files still exist Snapshot current = table.currentSnapshot(); - List manifests = current.allManifests(); + List manifests = current.allManifests(table.io()); Assert.assertEquals("Should have 1 manifest file", 1, manifests.size()); Assert.assertTrue("Manifest file should exist", new File(manifests.get(0).path()).exists()); Assert.assertEquals("Should have 2 files in metadata", 2, countAllMetadataFiles(tableDir)); diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java index 9e27526bdd5a..8d9c52b7bd48 100644 --- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java +++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java @@ -67,8 +67,8 @@ public void testPartitionedTableWithPartitionEqDeletes() { .appendFile(FILE_C) .commit(); - List dataManifests = table.currentSnapshot().dataManifests(); - List deleteManifests = table.currentSnapshot().deleteManifests(); + List dataManifests = table.currentSnapshot().dataManifests(table.io()); + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); Assert.assertEquals("Should have zero delete manifest file", @@ -85,8 +85,8 @@ public void testPartitionedTableWithPartitionEqDeletes() { .addDeletes(FILE_A_EQ_1) .commit(); - dataManifests = table.currentSnapshot().dataManifests(); - deleteManifests = table.currentSnapshot().deleteManifests(); + dataManifests = table.currentSnapshot().dataManifests(ops.io()); + deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); Assert.assertEquals("Should have one delete manifest file", @@ -106,8 +106,8 @@ public void testPartitionedTableWithPartitionEqDeletes() { // first commit after row-delta changes table.newDelete().deleteFile(FILE_B).commit(); - dataManifests = table.currentSnapshot().dataManifests(); - deleteManifests = table.currentSnapshot().deleteManifests(); + dataManifests = table.currentSnapshot().dataManifests(ops.io()); + deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); Assert.assertEquals("Should have one delete manifest file", @@ -126,8 +126,8 @@ public void testPartitionedTableWithPartitionEqDeletes() { // second commit after row-delta changes table.newDelete().deleteFile(FILE_C).commit(); - dataManifests = table.currentSnapshot().dataManifests(); - deleteManifests = table.currentSnapshot().deleteManifests(); + dataManifests = table.currentSnapshot().dataManifests(ops.io()); + deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); Assert.assertEquals("Should have one delete manifest file", @@ -212,7 +212,7 @@ public void testPartitionedTableWithExistingDeleteFile() { .commit(); Assert.assertEquals("Should have two delete manifests", - 2, table.currentSnapshot().deleteManifests().size()); + 2, table.currentSnapshot().deleteManifests(table.io()).size()); // merge delete manifests table.newAppend() @@ -220,13 +220,13 @@ public void testPartitionedTableWithExistingDeleteFile() { .commit(); Assert.assertEquals("Should have one delete manifest", - 1, table.currentSnapshot().deleteManifests().size()); + 1, table.currentSnapshot().deleteManifests(table.io()).size()); Assert.assertEquals("Should have zero added delete file", - 0, table.currentSnapshot().deleteManifests().get(0).addedFilesCount().intValue()); + 0, table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount().intValue()); Assert.assertEquals("Should have zero deleted delete file", - 0, table.currentSnapshot().deleteManifests().get(0).deletedFilesCount().intValue()); + 0, table.currentSnapshot().deleteManifests(table.io()).get(0).deletedFilesCount().intValue()); Assert.assertEquals("Should have two existing delete files", - 2, table.currentSnapshot().deleteManifests().get(0).existingFilesCount().intValue()); + 2, table.currentSnapshot().deleteManifests(table.io()).get(0).existingFilesCount().intValue()); List tasks = Lists.newArrayList(table.newScan().filter(equal(bucket("data", BUCKETS_NUMBER), 0)) diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index 107a758b4857..c1efec46018f 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -140,9 +140,10 @@ public void testCurrentSnapshotOperation() { Assert.assertEquals("Current snapshot should be what we rolled back to", wapSnapshot.snapshotId(), base.currentSnapshot().snapshotId()); Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals("Should contain manifests for both files", 2, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 2, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); } @@ -168,9 +169,10 @@ public void testSetCurrentSnapshotNoWAP() { Assert.assertEquals("Current snapshot should be what we rolled back to", firstSnapshotId, base.currentSnapshot().snapshotId()); Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals("Should contain manifests for both files", 1, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 1, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 3, base.snapshotLog().size()); } @@ -214,9 +216,9 @@ public void testRollbackOnInvalidNonAncestor() { base.currentSnapshot().snapshotId()); Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); Assert.assertEquals("Should contain manifests for one snapshot", 1, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); } @@ -334,9 +336,10 @@ public void testWithCherryPicking() { Assert.assertEquals("Current snapshot should be fast-forwarded to wap snapshot", wapSnapshot.snapshotId(), base.currentSnapshot().snapshotId()); Assert.assertEquals("Should have two snapshots", 2, base.snapshots().size()); - Assert.assertEquals("Should contain manifests for both files", 2, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 2, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); } @@ -398,9 +401,9 @@ public void testWithTwoPhaseCherryPicking() { parentSnapshot.snapshotId() + 1, base.currentSnapshot().snapshotId()); Assert.assertEquals("Should contain manifests for both files", 2, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Parent snapshot id should change to latest snapshot before commit", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, @@ -417,9 +420,10 @@ public void testWithTwoPhaseCherryPicking() { Assert.assertEquals("Current snapshot should be set to one after wap snapshot", parentSnapshot.snapshotId() + 1 /* one fast-forwarded snapshot */ + 1, base.currentSnapshot().snapshotId()); - Assert.assertEquals("Should contain manifests for both files", 3, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 3, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Parent snapshot id should change to latest snapshot before commit", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 3, @@ -482,7 +486,7 @@ public void testWithCommitsBetweenCherryPicking() { Assert.assertEquals("Current snapshot should carry over the parent snapshot", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Should contain manifests for two files", 2, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); @@ -498,9 +502,9 @@ public void testWithCommitsBetweenCherryPicking() { Assert.assertEquals("Current snapshot should be set to one after wap snapshot", parentSnapshot.snapshotId() + 1, base.currentSnapshot().snapshotId()); Assert.assertEquals("Should contain manifests for three files", 3, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Parent snapshot id should point to same snapshot", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 3, @@ -518,9 +522,9 @@ public void testWithCommitsBetweenCherryPicking() { Assert.assertEquals("Current snapshot should be set to one after wap snapshot", parentSnapshot.snapshotId() + 1, base.currentSnapshot().snapshotId()); Assert.assertEquals("Should contain manifests for four files", 4, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Parent snapshot id should point to same snapshot", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 4, @@ -575,9 +579,9 @@ public void testWithCherryPickingWithCommitRetry() { parentSnapshot.snapshotId() + 1, base.currentSnapshot().snapshotId()); Assert.assertEquals("Should contain manifests for both files", 2, - base.currentSnapshot().allManifests().size()); + base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should not contain redundant append due to retry", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Parent snapshot id should change to latest snapshot before commit", parentSnapshot.snapshotId(), base.currentSnapshot().parentId().longValue()); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, @@ -622,9 +626,10 @@ public void testCherrypickingAncestor() { Assert.assertEquals("Current snapshot should be fast-forwarded to wap snapshot", wapSnapshot.snapshotId(), base.currentSnapshot().snapshotId()); Assert.assertEquals("Should have two snapshots", 2, base.snapshots().size()); - Assert.assertEquals("Should contain manifests for both files", 2, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 2, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); @@ -677,9 +682,10 @@ public void testDuplicateCherrypick() { base = readMetadata(); Assert.assertEquals("Should have three snapshots", 3, base.snapshots().size()); - Assert.assertEquals("Should contain manifests for both files", 2, base.currentSnapshot().allManifests().size()); + Assert.assertEquals("Should contain manifests for both files", + 2, base.currentSnapshot().allManifests(table.io()).size()); Assert.assertEquals("Should contain append from last commit", 1, - Iterables.size(base.currentSnapshot().addedFiles())); + Iterables.size(base.currentSnapshot().addedFiles(table.io()))); Assert.assertEquals("Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java index 6db1eef7c044..4dbe28378887 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java @@ -51,8 +51,8 @@ public void dropTableDataDeletesExpectedFiles() { Set snapshotSet = Sets.newHashSet(table.snapshots()); Set manifestListLocations = manifestListLocations(snapshotSet); - Set manifestLocations = manifestLocations(snapshotSet); - Set dataLocations = dataLocations(snapshotSet); + Set manifestLocations = manifestLocations(snapshotSet, table.io()); + Set dataLocations = dataLocations(snapshotSet, table.io()); Set metadataLocations = metadataLocations(tableMetadata); Assert.assertEquals("should have 2 manifest lists", 2, manifestListLocations.size()); Assert.assertEquals("should have 3 metadata locations", 3, metadataLocations.size()); @@ -94,8 +94,8 @@ public void dropTableDataDoNotThrowWhenDeletesFail() { CatalogUtil.dropTableData(fileIO, tableMetadata); Mockito.verify(fileIO, Mockito.times( - manifestListLocations(snapshotSet).size() + manifestLocations(snapshotSet).size() + - dataLocations(snapshotSet).size() + metadataLocations(tableMetadata).size())) + manifestListLocations(snapshotSet).size() + manifestLocations(snapshotSet, fileIO).size() + + dataLocations(snapshotSet, table.io()).size() + metadataLocations(tableMetadata).size())) .deleteFile(ArgumentMatchers.anyString()); } @@ -113,7 +113,7 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { Set snapshotSet = Sets.newHashSet(table.snapshots()); Set manifestListLocations = manifestListLocations(snapshotSet); - Set manifestLocations = manifestLocations(snapshotSet); + Set manifestLocations = manifestLocations(snapshotSet, table.io()); Set metadataLocations = metadataLocations(tableMetadata); Assert.assertEquals("should have 2 manifest lists", 2, manifestListLocations.size()); Assert.assertEquals("should have 4 metadata locations", 4, metadataLocations.size()); @@ -141,16 +141,16 @@ private Set manifestListLocations(Set snapshotSet) { .collect(Collectors.toSet()); } - private Set manifestLocations(Set snapshotSet) { + private Set manifestLocations(Set snapshotSet, FileIO io) { return snapshotSet.stream() - .flatMap(snapshot -> snapshot.allManifests().stream()) + .flatMap(snapshot -> snapshot.allManifests(io).stream()) .map(ManifestFile::path) .collect(Collectors.toSet()); } - private Set dataLocations(Set snapshotSet) { + private Set dataLocations(Set snapshotSet, FileIO io) { return snapshotSet.stream() - .flatMap(snapshot -> StreamSupport.stream(snapshot.addedFiles().spliterator(), false)) + .flatMap(snapshot -> StreamSupport.stream(snapshot.addedFiles(io).spliterator(), false)) .map(dataFile -> dataFile.path().toString()) .collect(Collectors.toSet()); } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java index edd76cf375b2..9a73b6f13872 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -319,7 +319,7 @@ public void testFastAppend() throws Exception { TableMetadata metadata = readMetadataVersion(3); Assert.assertEquals("Current snapshot should contain 2 manifests", - 2, metadata.currentSnapshot().allManifests().size()); + 2, metadata.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -342,7 +342,7 @@ public void testMergeAppend() throws Exception { TableMetadata metadata = readMetadataVersion(5); Assert.assertEquals("Current snapshot should contain 1 merged manifest", - 1, metadata.currentSnapshot().allManifests().size()); + 1, metadata.currentSnapshot().allManifests(table.io()).size()); } @Test diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java index 8ff43a8a4b2e..2467497eb3a2 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java @@ -106,7 +106,7 @@ public synchronized void testConcurrentFastAppends() throws IOException { }); icebergTable.refresh(); - Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests().size()); + Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests(icebergTable.io()).size()); } @Test diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index ea36fb7eacf7..bef5bbb0b04a 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -299,7 +299,7 @@ private void testSqlChangeLog(String tableName, private List findValidSnapshots(Table table) { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests().stream().anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + if (snapshot.allManifests(table.io()).stream().anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { validSnapshots.add(snapshot); } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 23169d1a59c6..97506b90ba46 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -147,7 +147,7 @@ public void setupTable() throws IOException { private List findValidSnapshots(Table table) { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests().stream().anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + if (snapshot.allManifests(table.io()).stream().anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { validSnapshots.add(snapshot); } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java index e0e03e9206ce..9aef3d4b128f 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java @@ -128,7 +128,7 @@ public void testCreateTableTxnAndAppend() { Table table = catalog.loadTable(TABLE_IDENTIFIER); Snapshot snapshot = table.currentSnapshot(); - Assert.assertTrue("Table should have one manifest file", snapshot.allManifests().size() == 1); + Assert.assertTrue("Table should have one manifest file", snapshot.allManifests(table.io()).size() == 1); } @Test diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index bcfcc8c159a1..2d987287cad9 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -225,7 +225,7 @@ public void testDropTable() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertTrue("Drop (table and data) should return true and drop the table", catalog.dropTable(TABLE_IDENTIFIER)); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java index 2e765c328602..3077dc51f6b4 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java @@ -75,7 +75,7 @@ public synchronized void testConcurrentFastAppends() { }); icebergTable.refresh(); - Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests().size()); + Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests(icebergTable.io()).size()); } @Test diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 5e444b8990f9..d0dbb8223f87 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -156,7 +156,7 @@ public void verifyStateMovesForDML() throws Exception { .describedAs("on-reference-state must not be equal on both branches") .isEqualTo(contentsAfter1Branch.getSchemaId()); // verify manifests - Assertions.assertThat(tableAfter1Main.currentSnapshot().allManifests()) + Assertions.assertThat(tableAfter1Main.currentSnapshot().allManifests(tableAfter1Main.io())) .describedAs("verify number of manifests on 'main'") .hasSize(1); @@ -178,7 +178,7 @@ public void verifyStateMovesForDML() throws Exception { .describedAs("on-reference-state must not change on %s", testCaseBranch) .isEqualTo(contentsAfter1Branch.getMetadataLocation()); // verify manifests - Assertions.assertThat(tableAfter2Main.currentSnapshot().allManifests()) + Assertions.assertThat(tableAfter2Main.currentSnapshot().allManifests(tableAfter2Main.io())) .describedAs("verify number of manifests on 'main'") .hasSize(2); } @@ -290,7 +290,7 @@ public void testDropTable() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index dbca0f00ee59..1c0d67ca9fc4 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -101,7 +101,7 @@ public void testMergeWithStaticPredicatePushDown() { "{ \"id\": 2, \"dep\": \"hardware\" }"); // remove the data file from the 'hr' partition to ensure it is not scanned - withUnavailableFiles(snapshot.addedFiles(), () -> { + withUnavailableFiles(snapshot.addedFiles(table.io()), () -> { // disable dynamic pruning and rely only on static predicate pushdown withSQLConf(ImmutableMap.of(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false"), () -> { sql("MERGE INTO %s t USING source " + diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index b04f17693e3a..dcf0a2d91e3e 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -60,7 +60,7 @@ public void testRewriteLargeManifests() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Must have 1 manifest", 1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", tableName); @@ -72,7 +72,7 @@ public void testRewriteLargeManifests() { table.refresh(); - Assert.assertEquals("Must have 4 manifests", 4, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -88,7 +88,7 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Must have 4 manifest", 4, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 4 manifest", 4, table.currentSnapshot().allManifests(table.io()).size()); List output = sql( "CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableIdent); @@ -98,7 +98,7 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { table.refresh(); - Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -110,7 +110,7 @@ public void testRewriteSmallManifestsWithoutCaching() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Must have 2 manifest", 2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); List output = sql( "CALL %s.system.rewrite_manifests(use_caching => false, table => '%s')", catalogName, tableIdent); @@ -120,7 +120,7 @@ public void testRewriteSmallManifestsWithoutCaching() { table.refresh(); - Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); } @Test @@ -132,7 +132,7 @@ public void testRewriteManifestsCaseInsensitiveArgs() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Must have 2 manifest", 2, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); List output = sql( "CALL %s.system.rewrite_manifests(usE_cAcHiNg => false, tAbLe => '%s')", catalogName, tableIdent); @@ -142,7 +142,7 @@ public void testRewriteManifestsCaseInsensitiveArgs() { table.refresh(); - Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests().size()); + Assert.assertEquals("Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); } @Test diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 75f46084124e..edc3944e69fe 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -905,7 +905,7 @@ public void testUpdateWithStaticPredicatePushdown() { Assert.assertEquals("Must have 2 files before UPDATE", "2", dataFilesCount); // remove the data file from the 'hr' partition to ensure it is not scanned - DataFile dataFile = Iterables.getOnlyElement(snapshot.addedFiles()); + DataFile dataFile = Iterables.getOnlyElement(snapshot.addedFiles(table.io())); table.io().deleteFile(dataFile.path().toString()); // disable dynamic pruning and rely only on static predicate pushdown diff --git a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index d59508b9343d..0a77b2889776 100644 --- a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedFiles()) { + for (DataFile file : table().currentSnapshot().addedFiles(table().io())) { writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); } } diff --git a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index c2a30e5d9d13..617f8fd069d7 100644 --- a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -49,7 +49,7 @@ protected void appendData() throws IOException { if (percentDeleteRow > 0) { // add pos-deletes table().refresh(); - for (DataFile file : table().currentSnapshot().addedFiles()) { + for (DataFile file : table().currentSnapshot().addedFiles(table().io())) { writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); } } diff --git a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index ade902688f82..3c57464b4fff 100644 --- a/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.2/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedFiles()) { + for (DataFile file : table().currentSnapshot().addedFiles(table().io())) { writePosDeletesWithNoise(file.path(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), 1); } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java index 209838bc87c2..1648dd26f85a 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java @@ -262,7 +262,7 @@ private List findMatchingManifests() { return ImmutableList.of(); } - return currentSnapshot.dataManifests().stream() + return currentSnapshot.dataManifests(fileIO).stream() .filter(manifest -> manifest.partitionSpecId() == spec.specId() && predicate.test(manifest)) .collect(Collectors.toList()); } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java index 91ddcd53f148..3c2432d6a2f3 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java @@ -111,7 +111,8 @@ public Offset latestOffset() { } Snapshot latestSnapshot = table.currentSnapshot(); - return new StreamingOffset(latestSnapshot.snapshotId(), Iterables.size(latestSnapshot.addedFiles()), false); + return new StreamingOffset( + latestSnapshot.snapshotId(), Iterables.size(latestSnapshot.addedFiles(table.io())), false); } @Override diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index d471c4062b6e..b278bd08a608 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -593,13 +593,13 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB.addedFiles().forEach(i -> { + snapshotB.addedFiles(table.io()).forEach(i -> { expectedDeletes.add(i.path().toString()); }); // ManifestList should be deleted too expectedDeletes.add(snapshotB.manifestListLocation()); - snapshotB.dataManifests().forEach(file -> { + snapshotB.dataManifests(table.io()).forEach(file -> { // Only the manifest of B should be deleted. if (file.snapshotId() == snapshotB.snapshotId()) { expectedDeletes.add(file.path()); @@ -668,7 +668,7 @@ public void testWithCherryPickTableSnapshot() { // Make sure no dataFiles are deleted for the B, C, D snapshot Lists.newArrayList(snapshotB, snapshotC, snapshotD).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -723,7 +723,7 @@ public void testWithExpiringStagedThenCherrypick() { // Make sure no dataFiles are deleted for the staged snapshot Lists.newArrayList(snapshotB).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -737,7 +737,7 @@ public void testWithExpiringStagedThenCherrypick() { // Make sure no dataFiles are deleted for the staged and cherry-pick Lists.newArrayList(snapshotB, snapshotD).forEach(i -> { - i.addedFiles().forEach(item -> { + i.addedFiles(table.io()).forEach(item -> { Assert.assertFalse(deletedFiles.contains(item.path().toString())); }); }); @@ -785,7 +785,7 @@ public void testExpireOlderThanWithDelete() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); rightAfterSnapshot(); @@ -795,7 +795,7 @@ public void testExpireOlderThanWithDelete() { Snapshot secondSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create replace manifest with a rewritten manifest", - 1, secondSnapshot.allManifests().size()); + 1, secondSnapshot.allManifests(table.io()).size()); table.newAppend() .appendFile(FILE_B) @@ -821,9 +821,9 @@ public void testExpireOlderThanWithDelete() { Assert.assertEquals("Should remove expired manifest lists and deleted data file", Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot.allManifests().get(0).path(), // manifest was rewritten for delete + firstSnapshot.allManifests(table.io()).get(0).path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - secondSnapshot.allManifests().get(0).path(), // manifest contained only deletes, was dropped + secondSnapshot.allManifests(table.io()).get(0).path(), // manifest contained only deletes, was dropped FILE_A.path()), // deleted deletedFiles); @@ -844,7 +844,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); rightAfterSnapshot(); @@ -854,7 +854,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Snapshot secondSnapshot = table.currentSnapshot(); Assert.assertEquals("Should replace manifest with a rewritten manifest", - 1, secondSnapshot.allManifests().size()); + 1, secondSnapshot.allManifests(table.io()).size()); table.newFastAppend() // do not merge to keep the last snapshot's manifest valid .appendFile(FILE_C) @@ -880,7 +880,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { Assert.assertEquals("Should remove expired manifest lists and deleted data file", Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot.allManifests().get(0).path(), // manifest was rewritten for delete + firstSnapshot.allManifests(table.io()).get(0).path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired FILE_A.path()), // deleted deletedFiles); @@ -902,7 +902,7 @@ public void testExpireOlderThanWithRollback() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); rightAfterSnapshot(); @@ -911,8 +911,8 @@ public void testExpireOlderThanWithRollback() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests()); - secondSnapshotManifests.removeAll(firstSnapshot.allManifests()); + Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); Assert.assertEquals("Should add one new manifest for append", 1, secondSnapshotManifests.size()); table.manageSnapshots() @@ -951,7 +951,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); Assert.assertEquals("Should create one manifest", - 1, firstSnapshot.allManifests().size()); + 1, firstSnapshot.allManifests(table.io()).size()); rightAfterSnapshot(); @@ -960,8 +960,8 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests()); - secondSnapshotManifests.removeAll(firstSnapshot.allManifests()); + Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); Assert.assertEquals("Should add one new manifest for append", 1, secondSnapshotManifests.size()); table.manageSnapshots() @@ -1047,12 +1047,12 @@ public void testExpireOlderThanWithDeleteFile() { FILE_A_EQ_DELETES.path().toString()); expectedDeletes.addAll( - thirdSnapshot.allManifests().stream() + thirdSnapshot.allManifests(table.io()).stream() .map(ManifestFile::path) .map(CharSequence::toString).collect(Collectors.toSet())); // Delete operation (fourth snapshot) generates new manifest files expectedDeletes.addAll( - fourthSnapshot.allManifests().stream() + fourthSnapshot.allManifests(table.io()).stream() .map(ManifestFile::path) .map(CharSequence::toString).collect(Collectors.toSet())); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 2bd33cc7cf1c..be1285087d38 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -302,14 +302,14 @@ public void testBinPackWithDeleteAllData() { Assert.assertEquals( "Data manifest should not have existing data file", 0, - (long) table.currentSnapshot().dataManifests().get(0).existingFilesCount()); + (long) table.currentSnapshot().dataManifests(table.io()).get(0).existingFilesCount()); Assert.assertEquals("Data manifest should have 1 delete data file", 1L, - (long) table.currentSnapshot().dataManifests().get(0).deletedFilesCount()); + (long) table.currentSnapshot().dataManifests(table.io()).get(0).deletedFilesCount()); Assert.assertEquals( "Delete manifest added row count should equal total count", total, - (long) table.currentSnapshot().deleteManifests().get(0).addedRowsCount()); + (long) table.currentSnapshot().deleteManifests(table.io()).get(0).addedRowsCount()); } @Test @@ -995,7 +995,8 @@ public void testAutoSortShuffleOutput() { .execute(); Assert.assertEquals("Should have 1 fileGroups", result.rewriteResults().size(), 1); - Assert.assertTrue("Should have written 40+ files", Iterables.size(table.currentSnapshot().addedFiles()) >= 40); + Assert.assertTrue("Should have written 40+ files", + Iterables.size(table.currentSnapshot().addedFiles(table.io())) >= 40); table.refresh(); @@ -1062,7 +1063,7 @@ public void testZOrderSort() { .execute(); Assert.assertEquals("Should have 1 fileGroups", 1, result.rewriteResults().size()); - int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedFiles()); + int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedFiles(table.io())); Assert.assertTrue("Should have written 40+ files", zOrderedFilesTotal >= 40); table.refresh(); @@ -1103,7 +1104,7 @@ public void testZOrderAllTypesSort() { .execute(); Assert.assertEquals("Should have 1 fileGroups", 1, result.rewriteResults().size()); - int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedFiles()); + int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedFiles(table.io())); Assert.assertEquals("Should have written 1 file", 1, zOrderedFilesTotal); table.refresh(); @@ -1336,7 +1337,7 @@ private List, Pair>> checkForOverlappingFiles(Table ta NestedField field = table.schema().caseInsensitiveFindField(column); Class javaClass = (Class) field.type().typeId().javaClass(); - Map> filesByPartition = Streams.stream(table.currentSnapshot().addedFiles()) + Map> filesByPartition = Streams.stream(table.currentSnapshot().addedFiles(table.io())) .collect(Collectors.groupingBy(DataFile::partition)); Stream, Pair>> overlaps = diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 40adb7d4c918..f30251e74001 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -129,7 +129,7 @@ public void testRewriteSmallManifestsNonPartitionedTable() { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); SparkActions actions = SparkActions.get(); @@ -143,7 +143,7 @@ public void testRewriteSmallManifestsNonPartitionedTable() { table.refresh(); - List newManifests = table.currentSnapshot().allManifests(); + List newManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 manifests after rewrite", 1, newManifests.size()); Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); @@ -183,7 +183,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); SparkActions actions = SparkActions.get(); @@ -207,7 +207,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { table.refresh(); // table should reflect the changes, since the commit was successful - List newManifests = table.currentSnapshot().allManifests(); + List newManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 manifests after rewrite", 1, newManifests.size()); Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); @@ -262,7 +262,7 @@ public void testRewriteSmallManifestsPartitionedTable() { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 4 manifests before rewrite", 4, manifests.size()); SparkActions actions = SparkActions.get(); @@ -284,7 +284,7 @@ public void testRewriteSmallManifestsPartitionedTable() { table.refresh(); - List newManifests = table.currentSnapshot().allManifests(); + List newManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); @@ -347,7 +347,8 @@ public void testRewriteImportedManifests() throws IOException { .stagingLocation(temp.newFolder().toString()) .execute(); - Assert.assertEquals("Action should rewrite all manifests", snapshot.allManifests(), result.rewrittenManifests()); + Assert.assertEquals("Action should rewrite all manifests", + snapshot.allManifests(table.io()), result.rewrittenManifests()); Assert.assertEquals("Action should add 1 manifest", 1, Iterables.size(result.addedManifests())); } finally { @@ -375,7 +376,7 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 manifests before rewrite", 1, manifests.size()); // set the target manifest size to a small value to force splitting records into multiple files @@ -395,7 +396,7 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { table.refresh(); - List newManifests = table.currentSnapshot().allManifests(); + List newManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); @@ -430,7 +431,7 @@ public void testRewriteManifestsWithPredicate() throws IOException { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); SparkActions actions = SparkActions.get(); @@ -447,7 +448,7 @@ public void testRewriteManifestsWithPredicate() throws IOException { table.refresh(); - List newManifests = table.currentSnapshot().allManifests(); + List newManifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); Assert.assertFalse("First manifest must be rewritten", newManifests.contains(manifests.get(0))); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index a726d58e28f2..4aed78d1e155 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -707,11 +707,11 @@ private static void assertEquals(String context, MapType map, MapData expected, } public static List dataManifests(Table table) { - return table.currentSnapshot().dataManifests(); + return table.currentSnapshot().dataManifests(table.io()); } public static List deleteManifests(Table table) { - return table.currentSnapshot().deleteManifests(); + return table.currentSnapshot().deleteManifests(table.io()); } public static Set dataFiles(Table table) { diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 1391362823e1..f6d292f89f8b 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -181,7 +181,7 @@ private void writeAndValidateWithLocations(Table table, File location, File expe } Assert.assertEquals("Both iterators should be exhausted", expectedIter.hasNext(), actualIter.hasNext()); - table.currentSnapshot().addedFiles().forEach(dataFile -> + table.currentSnapshot().addedFiles(table.io()).forEach(dataFile -> Assert.assertTrue( String.format( "File should have the parent directory %s, but has: %s.", diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index fac56d393b25..091696d58f7a 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -203,7 +203,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .mode("append") .save(tableLocation); - List files = Lists.newArrayList(icebergTable.currentSnapshot().addedFiles()); + List files = Lists.newArrayList(icebergTable.currentSnapshot().addedFiles(icebergTable.io())); Assert.assertEquals("Should have written 1 file", 1, files.size()); long fileSize = files.get(0).fileSizeInBytes(); @@ -327,7 +327,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept .mode("append") .save(tableLocation); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Must be 2 manifests", 2, manifests.size()); @@ -356,7 +356,7 @@ public void testDefaultMetadataSplitSize() throws IOException { HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); Map options = Maps.newHashMap(); - tables.create(SCHEMA, spec, options, tableLocation); + Table icebergTable = tables.create(SCHEMA, spec, options, tableLocation); List expectedRecords = Lists.newArrayList( new SimpleRecord(1, "a"), @@ -371,7 +371,7 @@ public void testDefaultMetadataSplitSize() throws IOException { int splitSize = (int) TableProperties.METADATA_SPLIT_SIZE_DEFAULT; // 32MB split size int expectedSplits = ((int) tables.load(tableLocation + "#entries") - .currentSnapshot().allManifests().get(0).length() + splitSize - 1) / splitSize; + .currentSnapshot().allManifests(icebergTable.io()).get(0).length() + splitSize - 1) / splitSize; Dataset metadataDf = spark.read() .format("iceberg") diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index eb3ca569b6a3..d055c70cea1f 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -157,9 +157,9 @@ public void testEntriesTable() throws Exception { Snapshot snapshot = table.currentSnapshot(); - Assert.assertEquals("Should only contain one manifest", 1, snapshot.allManifests().size()); + Assert.assertEquals("Should only contain one manifest", 1, snapshot.allManifests(table.io()).size()); - InputFile manifest = table.io().newInputFile(snapshot.allManifests().get(0).path()); + InputFile manifest = table.io().newInputFile(snapshot.allManifests(table.io()).get(0).path()); List expected = Lists.newArrayList(); try (CloseableIterable rows = Avro.read(manifest).project(entriesTable.schema()).build()) { // each row must inherit snapshot_id and sequence_number @@ -215,7 +215,7 @@ public void testEntriesTableDataFilePrune() throws Exception { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedFiles().iterator().next(); + DataFile file = table.currentSnapshot().addedFiles(table.io()).iterator().next(); List singleActual = rowsToJava(spark.read() .format("iceberg") @@ -242,7 +242,7 @@ public void testEntriesTableDataFilePruneMulti() throws Exception { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedFiles().iterator().next(); + DataFile file = table.currentSnapshot().addedFiles(table.io()).iterator().next(); List multiActual = rowsToJava(spark.read() .format("iceberg") @@ -270,7 +270,7 @@ public void testFilesSelectMap() throws Exception { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedFiles().iterator().next(); + DataFile file = table.currentSnapshot().addedFiles(table.io()).iterator().next(); List multiActual = rowsToJava(spark.read() .format("iceberg") @@ -317,7 +317,8 @@ public void testAllEntriesTable() throws Exception { .collectAsList(); List expected = Lists.newArrayList(); - for (ManifestFile manifest : Iterables.concat(Iterables.transform(table.snapshots(), Snapshot::allManifests))) { + for (ManifestFile manifest : Iterables.concat( + Iterables.transform(table.snapshots(), s -> s.allManifests(table.io())))) { InputFile in = table.io().newInputFile(manifest.path()); try (CloseableIterable rows = Avro.read(in).project(entriesTable.schema()).build()) { // each row must inherit snapshot_id and sequence_number @@ -393,7 +394,7 @@ public void testFilesTable() throws Exception { .collectAsList(); List expected = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().dataManifests()) { + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { InputFile in = table.io().newInputFile(manifest.path()); try (CloseableIterable rows = Avro.read(in).project(entriesTable.schema()).build()) { for (GenericData.Record record : rows) { @@ -450,7 +451,7 @@ public void testFilesTableWithSnapshotIdInheritance() throws Exception { .collectAsList(); List expected = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().dataManifests()) { + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { InputFile in = table.io().newInputFile(manifest.path()); try (CloseableIterable rows = Avro.read(in).project(entriesTable.schema()).build()) { for (GenericData.Record record : rows) { @@ -539,7 +540,7 @@ public void testFilesUnpartitionedTable() throws Exception { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile toDelete = Iterables.getOnlyElement(table.currentSnapshot().addedFiles()); + DataFile toDelete = Iterables.getOnlyElement(table.currentSnapshot().addedFiles(table.io())); // add a second file df2.select("id", "data").write() @@ -556,7 +557,7 @@ public void testFilesUnpartitionedTable() throws Exception { .collectAsList(); List expected = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().dataManifests()) { + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { InputFile in = table.io().newInputFile(manifest.path()); try (CloseableIterable rows = Avro.read(in).project(entriesTable.schema()).build()) { for (GenericData.Record record : rows) { @@ -653,7 +654,9 @@ public void testAllDataFilesTable() throws Exception { actual.sort(Comparator.comparing(o -> o.getString(1))); List expected = Lists.newArrayList(); - for (ManifestFile manifest : Iterables.concat(Iterables.transform(table.snapshots(), Snapshot::dataManifests))) { + Iterable dataManifests = Iterables.concat(Iterables.transform(table.snapshots(), + snapshot -> snapshot.dataManifests(table.io()))); + for (ManifestFile manifest : dataManifests) { InputFile in = table.io().newInputFile(manifest.path()); try (CloseableIterable rows = Avro.read(in).project(entriesTable.schema()).build()) { for (GenericData.Record record : rows) { @@ -900,7 +903,7 @@ public void testManifestsTable() { .set(TableProperties.FORMAT_VERSION, "2") .commit(); - DataFile dataFile = Iterables.getFirst(table.currentSnapshot().addedFiles(), null); + DataFile dataFile = Iterables.getFirst(table.currentSnapshot().addedFiles(table.io()), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); @@ -924,7 +927,7 @@ public void testManifestsTable() { manifestTable.schema(), "manifests")); GenericRecordBuilder summaryBuilder = new GenericRecordBuilder(AvroSchemaUtil.convert( manifestTable.schema().findType("partition_summaries.element").asStructType(), "partition_summary")); - List expected = Lists.transform(table.currentSnapshot().allManifests(), manifest -> + List expected = Lists.transform(table.currentSnapshot().allManifests(table.io()), manifest -> builder .set("content", manifest.content().id()) .set("path", manifest.path()) @@ -995,7 +998,7 @@ public void testPruneManifestsTable() { GenericRecordBuilder builder = new GenericRecordBuilder(AvroSchemaUtil.convert(projectedSchema.asStruct())); GenericRecordBuilder summaryBuilder = new GenericRecordBuilder(AvroSchemaUtil.convert( projectedSchema.findType("partition_summaries.element").asStructType(), "partition_summary")); - List expected = Lists.transform(table.currentSnapshot().allManifests(), manifest -> + List expected = Lists.transform(table.currentSnapshot().allManifests(table.io()), manifest -> builder.set("partition_spec_id", manifest.partitionSpecId()) .set("path", manifest.path()) .set("partition_summaries", Lists.transform(manifest.partitions(), partition -> @@ -1031,7 +1034,7 @@ public void testAllManifestsTable() { .set(TableProperties.FORMAT_VERSION, "2") .commit(); - DataFile dataFile = Iterables.getFirst(table.currentSnapshot().addedFiles(), null); + DataFile dataFile = Iterables.getFirst(table.currentSnapshot().addedFiles(table.io()), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); @@ -1044,11 +1047,11 @@ public void testAllManifestsTable() { .addDeletes(deleteFile) .commit(); - manifests.addAll(table.currentSnapshot().allManifests()); + manifests.addAll(table.currentSnapshot().allManifests(table.io())); table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - manifests.addAll(table.currentSnapshot().allManifests()); + manifests.addAll(table.currentSnapshot().allManifests(table.io())); List actual = spark.read() .format("iceberg") diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index f6ca5f7de9f4..cd1404766d46 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -156,7 +156,7 @@ private void checkSparkDataFile(Table table) throws IOException { table.refresh(); - List manifests = table.currentSnapshot().allManifests(); + List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 manifest", 1, manifests.size()); List dataFiles = Lists.newArrayList(); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 8bf57ba65ff1..5b158c518ae4 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -133,7 +133,7 @@ public void testBasicWrite() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); Assert.assertEquals("Result rows should match", expected, actual); - for (ManifestFile manifest : table.currentSnapshot().allManifests()) { + for (ManifestFile manifest : table.currentSnapshot().allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { // TODO: avro not support split if (!format.equals(FileFormat.AVRO)) { @@ -372,7 +372,7 @@ public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws Assert.assertEquals("Result rows should match", expected, actual); List files = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().allManifests()) { + for (ManifestFile manifest : table.currentSnapshot().allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { files.add(file); } @@ -590,7 +590,7 @@ public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType opti Assert.assertEquals("Result rows should match", expected, actual); List files = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().allManifests()) { + for (ManifestFile manifest : table.currentSnapshot().allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { files.add(file); } diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 3952019fb058..187dd4470a05 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -61,7 +61,7 @@ public void testRefreshCommand() { // Modify table outside of spark, it should be cached so Spark should see the same value after mutation Table table = validationCatalog.loadTable(tableIdent); - DataFile file = table.currentSnapshot().addedFiles().iterator().next(); + DataFile file = table.currentSnapshot().addedFiles(table.io()).iterator().next(); table.newDelete().deleteFile(file).commit(); List cachedActual = sql("SELECT * FROM %s", tableName);