This is an automated email from the ASF dual-hosted git repository.
aokolnychyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/master by this push:
new 482f420 Split Snapshot.manifests into dataManifests and
deleteManifests (#1080)
482f420 is described below
commit 482f420141486f5b7b02bd62dc14cdb94b9a9cca
Author: Ryan Blue <[email protected]>
AuthorDate: Tue Jun 2 12:43:03 2020 -0700
Split Snapshot.manifests into dataManifests and deleteManifests (#1080)
This replaces all calls to Snapshot.manifests with calls to one of 3 new
methods:
* `Snapshot.allManifests` returns both delete and data manifests
* `Snapshot.deleteManifests` returns only delete manifests
* `Snapshot.dataManifests` returns only data manifests
Existing references mostly use either `allManifests` or `dataManifests`,
depending on the context. For example, tests with assertions for the number of
manifests use `allManifests` because the test cases should validate there are
no new delete manifests, but other tests that validate rewritten manifests are
deleted use `dataManifests` because only data manifests are rewritten and
deleted.
This tries to make minimal changes that preserve the current behavior.
Operations are not updated to support delete manifests (rewrite still only
rewrites data manifests), but will carry through the list of delete manifests
correctly.
---
api/src/main/java/org/apache/iceberg/Snapshot.java | 22 +++-
.../java/org/apache/iceberg/AllDataFilesTable.java | 6 +-
.../java/org/apache/iceberg/AllEntriesTable.java | 19 +++-
.../java/org/apache/iceberg/AllManifestsTable.java | 2 +-
.../org/apache/iceberg/BaseMetastoreCatalog.java | 3 +-
.../org/apache/iceberg/BaseRewriteManifests.java | 7 +-
.../main/java/org/apache/iceberg/BaseSnapshot.java | 50 +++++++--
.../java/org/apache/iceberg/BaseTransaction.java | 2 +-
.../java/org/apache/iceberg/DataFilesTable.java | 2 +-
.../java/org/apache/iceberg/DataTableScan.java | 4 +-
.../main/java/org/apache/iceberg/FastAppend.java | 3 +-
.../main/java/org/apache/iceberg/FileHistory.java | 119 ---------------------
.../main/java/org/apache/iceberg/FindFiles.java | 2 +-
.../apache/iceberg/IncrementalDataTableScan.java | 2 +-
.../org/apache/iceberg/ManifestEntriesTable.java | 5 +-
.../java/org/apache/iceberg/ManifestsTable.java | 2 +-
.../apache/iceberg/MergingSnapshotProducer.java | 6 +-
.../java/org/apache/iceberg/RemoveSnapshots.java | 17 ++-
.../main/java/org/apache/iceberg/ScanSummary.java | 2 +-
.../java/org/apache/iceberg/SnapshotParser.java | 4 +-
.../java/org/apache/iceberg/SnapshotProducer.java | 2 +-
.../java/org/apache/iceberg/TableTestBase.java | 8 +-
.../java/org/apache/iceberg/TestDeleteFiles.java | 8 +-
.../apache/iceberg/TestEntriesMetadataTable.java | 6 +-
.../java/org/apache/iceberg/TestFastAppend.java | 32 +++---
.../org/apache/iceberg/TestManifestCleanup.java | 18 ++--
.../java/org/apache/iceberg/TestMergeAppend.java | 115 ++++++++++----------
.../java/org/apache/iceberg/TestOverwrite.java | 14 +--
.../org/apache/iceberg/TestRemoveSnapshots.java | 4 +-
.../org/apache/iceberg/TestReplacePartitions.java | 26 ++---
.../java/org/apache/iceberg/TestRewriteFiles.java | 48 ++++-----
.../org/apache/iceberg/TestRewriteManifests.java | 90 ++++++++--------
.../java/org/apache/iceberg/TestSnapshotJson.java | 8 +-
.../java/org/apache/iceberg/TestTableMetadata.java | 12 +--
.../apache/iceberg/TestTimestampPartitions.java | 4 +-
.../java/org/apache/iceberg/TestTransaction.java | 68 ++++++------
.../java/org/apache/iceberg/TestWapWorkflow.java | 24 ++---
.../apache/iceberg/hadoop/TestHadoopCommits.java | 4 +-
.../iceberg/hive/HiveCreateReplaceTableTest.java | 2 +-
.../org/apache/iceberg/hive/HiveTableTest.java | 2 +-
.../iceberg/hive/TestHiveTableConcurrency.java | 2 +-
.../iceberg/actions/RewriteManifestsAction.java | 2 +-
.../actions/TestRewriteManifestsAction.java | 18 ++--
.../apache/iceberg/spark/TestSparkDataFile.java | 2 +-
.../spark/source/TestDataSourceOptions.java | 4 +-
.../spark/source/TestIcebergSourceTablesBase.java | 20 ++--
.../iceberg/spark/source/TestSparkDataWrite.java | 6 +-
47 files changed, 395 insertions(+), 433 deletions(-)
diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java
b/api/src/main/java/org/apache/iceberg/Snapshot.java
index 1a7376f..47a6d69 100644
--- a/api/src/main/java/org/apache/iceberg/Snapshot.java
+++ b/api/src/main/java/org/apache/iceberg/Snapshot.java
@@ -64,13 +64,25 @@ public interface Snapshot {
long timestampMillis();
/**
- * Return the location of all manifests in this snapshot.
- * <p>
- * The current table is made of the union of the data files in these
manifests.
+ * Return all {@link ManifestFile} instances for either data or delete
manifests in this snapshot.
+ *
+ * @return a list of ManifestFile
+ */
+ List<ManifestFile> allManifests();
+
+ /**
+ * Return a {@link ManifestFile} for each data manifest in this snapshot.
+ *
+ * @return a list of ManifestFile
+ */
+ List<ManifestFile> dataManifests();
+
+ /**
+ * Return a {@link ManifestFile} for each delete manifest in this snapshot.
*
- * @return a list of fully-qualified manifest locations
+ * @return a list of ManifestFile
*/
- List<ManifestFile> manifests();
+ List<ManifestFile> deleteManifests();
/**
* Return the name of the {@link DataOperations data operation} that
produced this snapshot.
diff --git a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java
b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java
index 9e41a57..c6f4927 100644
--- a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java
+++ b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java
@@ -119,7 +119,7 @@ public class AllDataFilesTable extends BaseMetadataTable {
@Override
protected CloseableIterable<FileScanTask> planFiles(
TableOperations ops, Snapshot snapshot, Expression rowFilter, boolean
caseSensitive, boolean colStats) {
- CloseableIterable<ManifestFile> manifests =
allManifestFiles(ops.current().snapshots());
+ CloseableIterable<ManifestFile> manifests =
allDataManifestFiles(ops.current().snapshots());
String schemaString = SchemaParser.toJson(schema());
String specString =
PartitionSpecParser.toJson(PartitionSpec.unpartitioned());
ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(rowFilter);
@@ -133,9 +133,9 @@ public class AllDataFilesTable extends BaseMetadataTable {
}
}
- static CloseableIterable<ManifestFile> allManifestFiles(List<Snapshot>
snapshots) {
+ private static CloseableIterable<ManifestFile>
allDataManifestFiles(List<Snapshot> snapshots) {
try (CloseableIterable<ManifestFile> iterable = new ParallelIterable<>(
- Iterables.transform(snapshots, Snapshot::manifests),
ThreadPools.getWorkerPool())) {
+ Iterables.transform(snapshots, Snapshot::dataManifests),
ThreadPools.getWorkerPool())) {
return CloseableIterable.withNoopClose(Sets.newHashSet(iterable));
} catch (IOException e) {
throw new RuntimeIOException(e, "Failed to close parallel iterable");
diff --git a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java
b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java
index 54644b5..3d84373 100644
--- a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java
+++ b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java
@@ -19,16 +19,22 @@
package org.apache.iceberg;
+import java.io.IOException;
import java.util.Collection;
+import java.util.List;
+import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.ResidualEvaluator;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.ParallelIterable;
+import org.apache.iceberg.util.ThreadPools;
/**
- * A {@link Table} implementation that exposes a table's manifest entries as
rows.
+ * A {@link Table} implementation that exposes a table's manifest entries as
rows, for both delete and data files.
* <p>
* WARNING: this table exposes internal details, like files that have been
deleted. For a table of the live data files,
* use {@link DataFilesTable}.
@@ -99,7 +105,7 @@ public class AllEntriesTable extends BaseMetadataTable {
@Override
protected CloseableIterable<FileScanTask> planFiles(
TableOperations ops, Snapshot snapshot, Expression rowFilter, boolean
caseSensitive, boolean colStats) {
- CloseableIterable<ManifestFile> manifests =
AllDataFilesTable.allManifestFiles(ops.current().snapshots());
+ CloseableIterable<ManifestFile> manifests =
allManifestFiles(ops.current().snapshots());
Schema fileSchema = new
Schema(schema().findType("data_file").asStructType().fields());
String schemaString = SchemaParser.toJson(schema());
String specString =
PartitionSpecParser.toJson(PartitionSpec.unpartitioned());
@@ -109,4 +115,13 @@ public class AllEntriesTable extends BaseMetadataTable {
ops.io(), manifest, fileSchema, schemaString, specString,
residuals));
}
}
+
+ private static CloseableIterable<ManifestFile>
allManifestFiles(List<Snapshot> snapshots) {
+ try (CloseableIterable<ManifestFile> iterable = new ParallelIterable<>(
+ Iterables.transform(snapshots, Snapshot::allManifests),
ThreadPools.getWorkerPool())) {
+ return CloseableIterable.withNoopClose(Sets.newHashSet(iterable));
+ } catch (IOException e) {
+ throw new RuntimeIOException(e, "Failed to close parallel iterable");
+ }
+ }
}
diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java
b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java
index 88929a2..e337ebd 100644
--- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java
+++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java
@@ -136,7 +136,7 @@ public class AllManifestsTable extends BaseMetadataTable {
} else {
return StaticDataTask.of(
ops.io().newInputFile(ops.current().file().location()),
- snap.manifests(),
+ snap.allManifests(),
manifest -> ManifestsTable.manifestFileToRow(table().spec(),
manifest));
}
}));
diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
index 8d0951e..cabf648 100644
--- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
+++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
@@ -220,7 +220,8 @@ public abstract class BaseMetastoreCatalog implements
Catalog {
Set<String> manifestListsToDelete = Sets.newHashSet();
Set<ManifestFile> manifestsToDelete = Sets.newHashSet();
for (Snapshot snapshot : metadata.snapshots()) {
- manifestsToDelete.addAll(snapshot.manifests());
+ // add all manifests to the delete set because both data and delete
files should be removed
+ Iterables.addAll(manifestsToDelete, snapshot.allManifests());
// 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/BaseRewriteManifests.java
b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
index 24c3c2d..1993c79 100644
--- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
+++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
@@ -20,7 +20,6 @@
package org.apache.iceberg;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
@@ -161,7 +160,7 @@ public class BaseRewriteManifests extends
SnapshotProducer<RewriteManifests> imp
@Override
public List<ManifestFile> apply(TableMetadata base) {
- List<ManifestFile> currentManifests = base.currentSnapshot().manifests();
+ List<ManifestFile> currentManifests =
base.currentSnapshot().dataManifests();
Set<ManifestFile> currentManifestSet =
ImmutableSet.copyOf(currentManifests);
validateDeletedManifests(currentManifestSet);
@@ -174,15 +173,15 @@ public class BaseRewriteManifests extends
SnapshotProducer<RewriteManifests> imp
validateFilesCounts();
- // TODO: add sequence numbers here
Iterable<ManifestFile> newManifestsWithMetadata = Iterables.transform(
Iterables.concat(newManifests, addedManifests,
rewrittenAddedManifests),
manifest ->
GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build());
// put new manifests at the beginning
- List<ManifestFile> apply = new ArrayList<>();
+ List<ManifestFile> apply = Lists.newArrayList();
Iterables.addAll(apply, newManifestsWithMetadata);
apply.addAll(keptManifests);
+ apply.addAll(base.currentSnapshot().deleteManifests());
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 4da4b37..f50f616 100644
--- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java
+++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java
@@ -46,7 +46,9 @@ class BaseSnapshot implements Snapshot {
private final Map<String, String> summary;
// lazily initialized
- private List<ManifestFile> manifests = null;
+ private List<ManifestFile> allManifests = null;
+ private List<ManifestFile> dataManifests = null;
+ private List<ManifestFile> deleteManifests = null;
private List<DataFile> cachedAdds = null;
private List<DataFile> cachedDeletes = null;
@@ -85,9 +87,9 @@ class BaseSnapshot implements Snapshot {
long timestampMillis,
String operation,
Map<String, String> summary,
- List<ManifestFile> manifests) {
+ List<ManifestFile> dataManifests) {
this(io, INITIAL_SEQUENCE_NUMBER, snapshotId, parentId, timestampMillis,
operation, summary, (InputFile) null);
- this.manifests = manifests;
+ this.allManifests = dataManifests;
}
@Override
@@ -120,14 +122,42 @@ class BaseSnapshot implements Snapshot {
return summary;
}
- @Override
- public List<ManifestFile> manifests() {
- if (manifests == null) {
+ private void cacheManifests() {
+ if (allManifests == null) {
// if manifests isn't set, then the snapshotFile is set and should be
read to get the list
- this.manifests = ManifestLists.read(manifestList);
+ this.allManifests = ManifestLists.read(manifestList);
+ }
+
+ if (dataManifests == null) {
+ this.dataManifests = ImmutableList.copyOf(Iterables.filter(allManifests,
+ manifest -> manifest.content() == ManifestContent.DATA));
+ this.deleteManifests =
ImmutableList.copyOf(Iterables.filter(allManifests,
+ manifest -> manifest.content() == ManifestContent.DELETES));
+ }
+ }
+
+ @Override
+ public List<ManifestFile> allManifests() {
+ if (allManifests == null) {
+ cacheManifests();
}
+ return allManifests;
+ }
- return manifests;
+ @Override
+ public List<ManifestFile> dataManifests() {
+ if (dataManifests == null) {
+ cacheManifests();
+ }
+ return dataManifests;
+ }
+
+ @Override
+ public List<ManifestFile> deleteManifests() {
+ if (deleteManifests == null) {
+ cacheManifests();
+ }
+ return deleteManifests;
}
@Override
@@ -156,7 +186,7 @@ class BaseSnapshot implements Snapshot {
ImmutableList.Builder<DataFile> deletes = ImmutableList.builder();
// read only manifests that were created by this snapshot
- Iterable<ManifestFile> changedManifests = Iterables.filter(manifests(),
+ Iterable<ManifestFile> changedManifests = Iterables.filter(dataManifests(),
manifest -> Objects.equal(manifest.snapshotId(), snapshotId));
try (CloseableIterable<ManifestEntry<DataFile>> entries = new
ManifestGroup(io, changedManifests)
.ignoreExisting()
@@ -189,7 +219,7 @@ class BaseSnapshot implements Snapshot {
.add("timestamp_ms", timestampMillis)
.add("operation", operation)
.add("summary", summary)
- .add("manifests", manifests())
+ .add("manifest-list", manifestList.location())
.toString();
}
}
diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java
b/core/src/main/java/org/apache/iceberg/BaseTransaction.java
index 81b7960..dace2c1 100644
--- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java
+++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java
@@ -395,7 +395,7 @@ class BaseTransaction implements Transaction {
Snapshot snap = ops.current().snapshot(snapshotId);
if (snap != null) {
committedFiles.add(snap.manifestListLocation());
- snap.manifests()
+ snap.allManifests()
.forEach(manifest -> committedFiles.add(manifest.path()));
} else {
return null;
diff --git a/core/src/main/java/org/apache/iceberg/DataFilesTable.java
b/core/src/main/java/org/apache/iceberg/DataFilesTable.java
index 0c010cf..dcfc70f 100644
--- a/core/src/main/java/org/apache/iceberg/DataFilesTable.java
+++ b/core/src/main/java/org/apache/iceberg/DataFilesTable.java
@@ -101,7 +101,7 @@ public class DataFilesTable extends BaseMetadataTable {
@Override
protected CloseableIterable<FileScanTask> planFiles(
TableOperations ops, Snapshot snapshot, Expression rowFilter, boolean
caseSensitive, boolean colStats) {
- CloseableIterable<ManifestFile> manifests =
CloseableIterable.withNoopClose(snapshot.manifests());
+ CloseableIterable<ManifestFile> manifests =
CloseableIterable.withNoopClose(snapshot.dataManifests());
String schemaString = SchemaParser.toJson(schema());
String specString =
PartitionSpecParser.toJson(PartitionSpec.unpartitioned());
ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(rowFilter);
diff --git a/core/src/main/java/org/apache/iceberg/DataTableScan.java
b/core/src/main/java/org/apache/iceberg/DataTableScan.java
index ce78104..b098bdd 100644
--- a/core/src/main/java/org/apache/iceberg/DataTableScan.java
+++ b/core/src/main/java/org/apache/iceberg/DataTableScan.java
@@ -79,14 +79,14 @@ public class DataTableScan extends BaseTableScan {
@Override
public CloseableIterable<FileScanTask> planFiles(TableOperations ops,
Snapshot snapshot,
Expression rowFilter,
boolean caseSensitive, boolean colStats) {
- ManifestGroup manifestGroup = new ManifestGroup(ops.io(),
snapshot.manifests())
+ ManifestGroup manifestGroup = new ManifestGroup(ops.io(),
snapshot.dataManifests())
.caseSensitive(caseSensitive)
.select(colStats ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS)
.filterData(rowFilter)
.specsById(ops.current().specsById())
.ignoreDeleted();
- if (PLAN_SCANS_WITH_WORKER_POOL && snapshot.manifests().size() > 1) {
+ if (PLAN_SCANS_WITH_WORKER_POOL && snapshot.dataManifests().size() > 1) {
manifestGroup = manifestGroup.planWith(ThreadPools.getWorkerPool());
}
diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java
b/core/src/main/java/org/apache/iceberg/FastAppend.java
index 7f7e62f..b370a18 100644
--- a/core/src/main/java/org/apache/iceberg/FastAppend.java
+++ b/core/src/main/java/org/apache/iceberg/FastAppend.java
@@ -131,14 +131,13 @@ class FastAppend extends SnapshotProducer<AppendFiles>
implements AppendFiles {
throw new RuntimeIOException(e, "Failed to write manifest");
}
- // TODO: add sequence numbers here
Iterable<ManifestFile> appendManifestsWithMetadata = Iterables.transform(
Iterables.concat(appendManifests, rewrittenAppendManifests),
manifest ->
GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build());
Iterables.addAll(newManifests, appendManifestsWithMetadata);
if (base.currentSnapshot() != null) {
- newManifests.addAll(base.currentSnapshot().manifests());
+ newManifests.addAll(base.currentSnapshot().allManifests());
}
return newManifests;
diff --git a/core/src/main/java/org/apache/iceberg/FileHistory.java
b/core/src/main/java/org/apache/iceberg/FileHistory.java
deleted file mode 100644
index b6645f7..0000000
--- a/core/src/main/java/org/apache/iceberg/FileHistory.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iceberg;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Set;
-import org.apache.iceberg.exceptions.RuntimeIOException;
-import org.apache.iceberg.expressions.Literal;
-import org.apache.iceberg.io.CloseableIterable;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
-import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
-import org.apache.iceberg.relocated.com.google.common.collect.Sets;
-import org.apache.iceberg.types.Types;
-import org.apache.iceberg.util.CharSequenceWrapper;
-
-public class FileHistory {
- private static final ImmutableList<String> HISTORY_COLUMNS =
ImmutableList.of("file_path");
-
- private FileHistory() {
- }
-
- public static Builder table(Table table) {
- return new Builder(table);
- }
-
- public static class Builder {
- private final Table table;
- private final Set<CharSequenceWrapper> locations = Sets.newHashSet();
- private Long startTime = null;
- private Long endTime = null;
-
- public Builder(Table table) {
- this.table = table;
- }
-
- public Builder location(String location) {
- locations.add(CharSequenceWrapper.wrap(location));
- return this;
- }
-
- public Builder after(String timestamp) {
- Literal<Long> tsLiteral =
Literal.of(timestamp).to(Types.TimestampType.withoutZone());
- this.startTime = tsLiteral.value() / 1000;
- return this;
- }
-
- public Builder after(long timestampMillis) {
- this.startTime = timestampMillis;
- return this;
- }
-
- public Builder before(String timestamp) {
- Literal<Long> tsLiteral =
Literal.of(timestamp).to(Types.TimestampType.withoutZone());
- this.endTime = tsLiteral.value() / 1000;
- return this;
- }
-
- public Builder before(long timestampMillis) {
- this.endTime = timestampMillis;
- return this;
- }
-
- @SuppressWarnings("unchecked")
- public Iterable<ManifestEntry<?>> build() {
- Iterable<Snapshot> snapshots = table.snapshots();
-
- if (startTime != null) {
- snapshots = Iterables.filter(snapshots, snap -> snap.timestampMillis()
>= startTime);
- }
-
- if (endTime != null) {
- snapshots = Iterables.filter(snapshots, snap -> snap.timestampMillis()
<= endTime);
- }
-
- // only use manifests that were added in the matching snapshots
- Set<Long> matchingIds = Sets.newHashSet(Iterables.transform(snapshots,
Snapshot::snapshotId));
- Iterable<ManifestFile> manifests = Iterables.filter(
- Iterables.concat(Iterables.transform(snapshots,
Snapshot::manifests)),
- manifest -> manifest.snapshotId() == null ||
matchingIds.contains(manifest.snapshotId()));
-
- // a manifest group will only read each manifest once
- ManifestGroup group = new ManifestGroup(((HasTableOperations)
table).operations().io(), manifests);
-
- List<ManifestEntry<?>> results = Lists.newArrayList();
- try (CloseableIterable<ManifestEntry<DataFile>> entries =
group.select(HISTORY_COLUMNS).entries()) {
- // TODO: replace this with an IN predicate
- CharSequenceWrapper locationWrapper = CharSequenceWrapper.wrap(null);
- for (ManifestEntry<?> entry : entries) {
- if (entry != null &&
locations.contains(locationWrapper.set(entry.file().path()))) {
- results.add(entry.copy());
- }
- }
- } catch (IOException e) {
- throw new RuntimeIOException(e);
- }
-
- return results;
- }
- }
-}
diff --git a/core/src/main/java/org/apache/iceberg/FindFiles.java
b/core/src/main/java/org/apache/iceberg/FindFiles.java
index 94e6f8a..03d24df 100644
--- a/core/src/main/java/org/apache/iceberg/FindFiles.java
+++ b/core/src/main/java/org/apache/iceberg/FindFiles.java
@@ -197,7 +197,7 @@ public class FindFiles {
}
// when snapshot is not null
- CloseableIterable<ManifestEntry<DataFile>> entries = new
ManifestGroup(ops.io(), snapshot.manifests())
+ CloseableIterable<ManifestEntry<DataFile>> entries = new
ManifestGroup(ops.io(), snapshot.dataManifests())
.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 98be406..0d4065b 100644
--- a/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java
+++ b/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java
@@ -84,7 +84,7 @@ class IncrementalDataTableScan extends DataTableScan {
Set<Long> snapshotIds = Sets.newHashSet(Iterables.transform(snapshots,
Snapshot::snapshotId));
Set<ManifestFile> manifests = FluentIterable
.from(snapshots)
- .transformAndConcat(s -> s.manifests())
+ .transformAndConcat(s -> s.dataManifests())
.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 061230f..f2536ec 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java
@@ -30,7 +30,7 @@ import
org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.TypeUtil;
/**
- * A {@link Table} implementation that exposes a table's manifest entries as
rows.
+ * A {@link Table} implementation that exposes a table's manifest entries as
rows, for both delete and data files.
* <p>
* WARNING: this table exposes internal details, like files that have been
deleted. For a table of the live data files,
* use {@link DataFilesTable}.
@@ -101,7 +101,8 @@ public class ManifestEntriesTable extends BaseMetadataTable
{
@Override
protected CloseableIterable<FileScanTask> planFiles(
TableOperations ops, Snapshot snapshot, Expression rowFilter, boolean
caseSensitive, boolean colStats) {
- CloseableIterable<ManifestFile> manifests =
CloseableIterable.withNoopClose(snapshot.manifests());
+ // return entries from both data and delete manifests
+ CloseableIterable<ManifestFile> manifests =
CloseableIterable.withNoopClose(snapshot.allManifests());
Schema fileSchema = new
Schema(schema().findType("data_file").asStructType().fields());
String schemaString = SchemaParser.toJson(schema());
String specString =
PartitionSpecParser.toJson(PartitionSpec.unpartitioned());
diff --git a/core/src/main/java/org/apache/iceberg/ManifestsTable.java
b/core/src/main/java/org/apache/iceberg/ManifestsTable.java
index 7d95556..a9b1354 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestsTable.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestsTable.java
@@ -77,7 +77,7 @@ public class ManifestsTable extends BaseMetadataTable {
String manifestListLocation = scan.snapshot().manifestListLocation();
return StaticDataTask.of(
ops.io().newInputFile(manifestListLocation != null ?
manifestListLocation : ops.current().file().location()),
- scan.snapshot().manifests(),
+ scan.snapshot().allManifests(),
manifest -> 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 f8fbe90..7a6ab8f 100644
--- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
+++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
@@ -283,7 +283,7 @@ abstract class MergingSnapshotProducer<ThisT> extends
SnapshotProducer<ThisT> {
// filter any existing manifests
List<ManifestFile> filtered;
if (current != null) {
- List<ManifestFile> manifests = current.manifests();
+ List<ManifestFile> manifests = current.dataManifests();
filtered = Arrays.asList(filterManifests(metricsEvaluator, manifests));
} else {
filtered = ImmutableList.of();
@@ -306,6 +306,10 @@ abstract class MergingSnapshotProducer<ThisT> extends
SnapshotProducer<ThisT> {
Iterables.addAll(manifests, unmergedManifests);
}
+ if (current != null) {
+ manifests.addAll(current.deleteManifests());
+ }
+
ValidationException.check(!failMissingDeletePaths ||
deletedFiles.containsAll(deletePaths),
"Missing required files to delete: %s",
COMMA.join(Iterables.transform(Iterables.filter(deletePaths,
diff --git a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
index f927728..5f33bfd 100644
--- a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
+++ b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
@@ -22,6 +22,7 @@ package org.apache.iceberg;
import java.io.IOException;
import java.util.Date;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
@@ -30,6 +31,7 @@ import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.base.Joiner;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
@@ -349,7 +351,7 @@ class RemoveSnapshots implements ExpireSnapshots {
.onFailure((item, exc) -> LOG.warn("Failed to get deleted files: this
may cause orphaned data files", exc))
.run(manifest -> {
// the manifest has deletes, scan it to find files to delete
- try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(),
ops.current().specsById())) {
+ try (BaseManifestReader<?, ?> reader = openManifest(manifest,
ops.io(), ops.current().specsById())) {
for (ManifestEntry<?> entry : reader.entries()) {
// if the snapshot ID of the DELETE entry is no longer valid,
the data can be deleted
if (entry.status() == ManifestEntry.Status.DELETED &&
@@ -398,7 +400,18 @@ class RemoveSnapshots implements ExpireSnapshots {
.build();
} else {
- return CloseableIterable.withNoopClose(snapshot.manifests());
+ return CloseableIterable.withNoopClose(snapshot.allManifests());
+ }
+ }
+
+ private static BaseManifestReader<?, ?> openManifest(ManifestFile manifest,
FileIO io,
+ Map<Integer,
PartitionSpec> specsById) {
+ switch (manifest.content()) {
+ case DATA:
+ return ManifestFiles.read(manifest, io, specsById);
+ case DELETES:
+ return ManifestFiles.readDeleteManifest(manifest, io, specsById);
}
+ throw new UnsupportedOperationException("Cannot read unknown manifest
type: " + manifest.content());
}
}
diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java
b/core/src/main/java/org/apache/iceberg/ScanSummary.java
index f93f785..8591e00 100644
--- a/core/src/main/java/org/apache/iceberg/ScanSummary.java
+++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java
@@ -159,7 +159,7 @@ public class ScanSummary {
removeTimeFilters(filters, Expressions.rewriteNot(scan.filter()));
Expression rowFilter = joinFilters(filters);
- Iterable<ManifestFile> manifests = table.currentSnapshot().manifests();
+ Iterable<ManifestFile> manifests =
table.currentSnapshot().dataManifests();
boolean filterByTimestamp = !timeFilters.isEmpty();
Set<Long> snapshotsInTimeRange = Sets.newHashSet();
diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java
b/core/src/main/java/org/apache/iceberg/SnapshotParser.java
index b5a4f7f..9852cd8 100644
--- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java
+++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java
@@ -79,9 +79,9 @@ public class SnapshotParser {
// write just the location. manifests should not be embedded in JSON
along with a list
generator.writeStringField(MANIFEST_LIST, manifestList);
} else {
- // embed the manifest list in the JSON
+ // embed the manifest list in the JSON, v1 only
generator.writeArrayFieldStart(MANIFESTS);
- for (ManifestFile file : snapshot.manifests()) {
+ for (ManifestFile file : snapshot.allManifests()) {
generator.writeString(file.path());
}
generator.writeEndArray();
diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
index e4d8adc..0daf5f7 100644
--- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
+++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
@@ -278,7 +278,7 @@ abstract class SnapshotProducer<ThisT> implements
SnapshotUpdate<ThisT> {
// 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.manifests()));
+ cleanUncommitted(Sets.newHashSet(saved.allManifests()));
// also clean up unused manifest lists created by multiple attempts
for (String manifestList : manifestLists) {
if (!saved.manifestListLocation().equals(manifestList)) {
diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java
b/core/src/test/java/org/apache/iceberg/TableTestBase.java
index 8de5e4b..e115675 100644
--- a/core/src/test/java/org/apache/iceberg/TableTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java
@@ -28,6 +28,7 @@ import java.util.stream.LongStream;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
@@ -224,10 +225,13 @@ public class TableTestBase {
}
void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber,
DataFile... newFiles) {
- List<ManifestFile> oldManifests = old != null ? old.manifests() :
ImmutableList.of();
+ Assert.assertEquals("Should not change delete manifests",
+ old != null ? Sets.newHashSet(old.deleteManifests()) :
ImmutableSet.of(),
+ Sets.newHashSet(snap.deleteManifests()));
+ List<ManifestFile> oldManifests = old != null ? old.dataManifests() :
ImmutableList.of();
// copy the manifests to a modifiable list and remove the existing
manifests
- List<ManifestFile> newManifests = Lists.newArrayList(snap.manifests());
+ List<ManifestFile> newManifests = Lists.newArrayList(snap.dataManifests());
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/TestDeleteFiles.java
b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
index bdc5537..026a490 100644
--- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
@@ -57,8 +57,8 @@ public class TestDeleteFiles extends TableTestBase {
Assert.assertEquals("Metadata should be at version 2", 2L, (long)
version());
Snapshot delete = readMetadata().currentSnapshot();
- Assert.assertEquals("Should have 1 manifest", 1,
delete.manifests().size());
- validateManifestEntries(delete.manifests().get(0),
+ Assert.assertEquals("Should have 1 manifest", 1,
delete.allManifests().size());
+ validateManifestEntries(delete.allManifests().get(0),
ids(delete.snapshotId(), append.snapshotId(), append.snapshotId()),
files(FILE_A, FILE_B, FILE_C),
statuses(Status.DELETED, Status.EXISTING, Status.EXISTING));
@@ -69,8 +69,8 @@ public class TestDeleteFiles extends TableTestBase {
Assert.assertEquals("Metadata should be at version 3", 3L, (long)
version());
Snapshot delete2 = readMetadata().currentSnapshot();
- Assert.assertEquals("Should have 1 manifest", 1,
delete2.manifests().size());
- validateManifestEntries(delete2.manifests().get(0),
+ Assert.assertEquals("Should have 1 manifest", 1,
delete2.allManifests().size());
+ validateManifestEntries(delete2.allManifests().get(0),
ids(delete2.snapshotId(), append.snapshotId()),
files(FILE_B, FILE_C),
statuses(Status.DELETED, Status.EXISTING));
diff --git
a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java
b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java
index 81362b9..4d252c2 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 class TestEntriesMetadataTable extends TableTestBase {
FileScanTask file = Iterables.getOnlyElement(scan.planFiles());
Assert.assertEquals("Data file should be the table's manifest",
- Iterables.getOnlyElement(table.currentSnapshot().manifests()).path(),
file.file().path());
+
Iterables.getOnlyElement(table.currentSnapshot().allManifests()).path(),
file.file().path());
Assert.assertEquals("Should contain 2 data file records", 2,
file.file().recordCount());
}
@@ -124,10 +124,10 @@ public class TestEntriesMetadataTable extends
TableTestBase {
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().manifests().size());
+ Assert.assertEquals(1,
entriesTable.currentSnapshot().allManifests().size());
int expectedSplits =
- ((int) entriesTable.currentSnapshot().manifests().get(0).length() +
splitSize - 1) / splitSize;
+ ((int) entriesTable.currentSnapshot().allManifests().get(0).length() +
splitSize - 1) / splitSize;
TableScan scan = entriesTable.newScan();
diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java
b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
index 1cc478d..7d48922 100644
--- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
@@ -115,11 +115,11 @@ public class TestFastAppend extends TableTestBase {
long commitId = snap.snapshotId();
- validateManifest(snap.manifests().get(0),
+ validateManifest(snap.allManifests().get(0),
seqs(1, 1),
ids(commitId, commitId),
files(FILE_C, FILE_D));
- validateManifest(snap.manifests().get(1),
+ validateManifest(snap.allManifests().get(1),
seqs(1, 1),
ids(commitId, commitId),
files(FILE_A, FILE_B));
@@ -139,7 +139,7 @@ public class TestFastAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertNotNull("Should have a current snapshot",
base.currentSnapshot());
- List<ManifestFile> v2manifests = base.currentSnapshot().manifests();
+ List<ManifestFile> v2manifests = base.currentSnapshot().allManifests();
Assert.assertEquals("Should have one existing manifest", 1,
v2manifests.size());
// prepare a new append
@@ -165,7 +165,7 @@ public class TestFastAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertNotNull("Should have a current snapshot",
base.currentSnapshot());
- List<ManifestFile> v3manifests = base.currentSnapshot().manifests();
+ List<ManifestFile> v3manifests = base.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 existing manifests", 2,
v3manifests.size());
// prepare a new append
@@ -195,7 +195,7 @@ public class TestFastAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertNotNull("Should have a current snapshot",
base.currentSnapshot());
- List<ManifestFile> v2manifests = base.currentSnapshot().manifests();
+ List<ManifestFile> v2manifests = base.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 existing manifest", 1,
v2manifests.size());
// commit from the stale table
@@ -222,7 +222,7 @@ public class TestFastAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertNotNull("Should have a current snapshot",
base.currentSnapshot());
- List<ManifestFile> v2manifests = base.currentSnapshot().manifests();
+ List<ManifestFile> v2manifests = base.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 existing manifest", 1,
v2manifests.size());
append.commit();
@@ -232,10 +232,10 @@ public class TestFastAppend extends TableTestBase {
// apply was called before the conflicting commit, but the commit was
still consistent
validateSnapshot(base.currentSnapshot(), committed.currentSnapshot(),
FILE_D);
- List<ManifestFile> committedManifests =
Lists.newArrayList(committed.currentSnapshot().manifests());
- committedManifests.removeAll(base.currentSnapshot().manifests());
+ List<ManifestFile> committedManifests =
Lists.newArrayList(committed.currentSnapshot().allManifests());
+ committedManifests.removeAll(base.currentSnapshot().allManifests());
Assert.assertEquals("Should reused manifest created by apply",
- pending.manifests().get(0), committedManifests.get(0));
+ pending.allManifests().get(0), committedManifests.get(0));
}
@Test
@@ -246,7 +246,7 @@ public class TestFastAppend extends TableTestBase {
AppendFiles append = table.newFastAppend().appendFile(FILE_B);
Snapshot pending = append.apply();
- ManifestFile newManifest = pending.manifests().get(0);
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
AssertHelpers.assertThrows("Should retry 4 times and throw last failure",
@@ -264,7 +264,7 @@ public class TestFastAppend extends TableTestBase {
ManifestFile manifest = writeManifest(FILE_A, FILE_B);
AppendFiles append = table.newFastAppend().appendManifest(manifest);
Snapshot pending = append.apply();
- ManifestFile newManifest = pending.manifests().get(0);
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
AssertHelpers.assertThrows("Should retry 4 times and throw last failure",
@@ -283,7 +283,7 @@ public class TestFastAppend extends TableTestBase {
AppendFiles append = table.newFastAppend().appendFile(FILE_B);
Snapshot pending = append.apply();
- ManifestFile newManifest = pending.manifests().get(0);
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
append.commit();
@@ -293,7 +293,7 @@ public class TestFastAppend extends TableTestBase {
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().manifests().contains(newManifest));
+ metadata.currentSnapshot().allManifests().contains(newManifest));
}
@Test
@@ -306,7 +306,7 @@ public class TestFastAppend extends TableTestBase {
AppendFiles append = table.newFastAppend().appendFile(FILE_B);
Snapshot pending = append.apply();
- ManifestFile newManifest = pending.manifests().get(0);
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
append.commit();
@@ -316,7 +316,7 @@ public class TestFastAppend extends TableTestBase {
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().manifests().contains(newManifest));
+ metadata.currentSnapshot().allManifests().contains(newManifest));
}
@Test
@@ -336,7 +336,7 @@ public class TestFastAppend extends TableTestBase {
.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
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 acb08bb..47f3354 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java
@@ -50,19 +50,19 @@ public class TestManifestCleanup extends TableTestBase {
.commit();
Assert.assertEquals("Table should have one append manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
table.newDelete()
.deleteFromRowFilter(Expressions.alwaysTrue())
.commit();
Assert.assertEquals("Table should have one delete manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
table.newAppend().commit();
Assert.assertEquals("Table should have no manifests",
- 0, table.currentSnapshot().manifests().size());
+ 0, table.currentSnapshot().allManifests().size());
}
@Test
@@ -77,7 +77,7 @@ public class TestManifestCleanup extends TableTestBase {
Snapshot s1 = table.currentSnapshot();
Assert.assertEquals("Table should have one append manifest",
- 1, s1.manifests().size());
+ 1, s1.allManifests().size());
table.newDelete()
.deleteFile(FILE_B)
@@ -85,13 +85,13 @@ public class TestManifestCleanup extends TableTestBase {
Snapshot s2 = table.currentSnapshot();
Assert.assertEquals("Table should have one mixed manifest",
- 1, s2.manifests().size());
+ 1, s2.allManifests().size());
table.newAppend().commit();
Snapshot s3 = table.currentSnapshot();
Assert.assertEquals("Table should have the same manifests",
- s2.manifests(), s3.manifests());
+ s2.allManifests(), s3.allManifests());
}
@Test
@@ -105,7 +105,7 @@ public class TestManifestCleanup extends TableTestBase {
.commit();
Assert.assertEquals("Table should have one append manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
table.newOverwrite()
.overwriteByRowFilter(Expressions.alwaysTrue())
@@ -114,7 +114,7 @@ public class TestManifestCleanup extends TableTestBase {
.commit();
Assert.assertEquals("Table should have one delete manifest and one append
manifest",
- 2, table.currentSnapshot().manifests().size());
+ 2, table.currentSnapshot().allManifests().size());
table.newOverwrite()
.overwriteByRowFilter(Expressions.alwaysTrue())
@@ -123,6 +123,6 @@ public class TestManifestCleanup extends TableTestBase {
.commit();
Assert.assertEquals("Table should have one delete manifest and one append
manifest",
- 2, table.currentSnapshot().manifests().size());
+ 2, table.currentSnapshot().allManifests().size());
}
}
diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
index a393adf..bc6c6a3 100644
--- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
@@ -62,11 +62,11 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, pending.manifests().size());
+ 1, pending.allManifests().size());
long pendingId = pending.snapshotId();
- validateManifest(pending.manifests().get(0), ids(pendingId, pendingId),
files(FILE_A, FILE_B));
+ validateManifest(pending.allManifests().get(0), ids(pendingId, pendingId),
files(FILE_A, FILE_B));
}
@Test
@@ -104,10 +104,10 @@ public class TestMergeAppend extends TableTestBase {
long pendingId = pending.snapshotId();
- validateManifest(pending.manifests().get(0),
+ validateManifest(pending.allManifests().get(0),
ids(pendingId, pendingId),
files(FILE_C, FILE_D));
- validateManifest(pending.manifests().get(1),
+ validateManifest(pending.allManifests().get(1),
ids(pendingId, pendingId),
files(FILE_A, FILE_B));
}
@@ -131,8 +131,8 @@ public class TestMergeAppend extends TableTestBase {
long pendingId = pending.snapshotId();
- Assert.assertEquals("Should create 1 merged manifest", 1,
pending.manifests().size());
- validateManifest(pending.manifests().get(0),
+ Assert.assertEquals("Should create 1 merged manifest", 1,
pending.allManifests().size());
+ validateManifest(pending.allManifests().get(0),
ids(pendingId, pendingId, pendingId, pendingId),
files(FILE_C, FILE_D, FILE_A, FILE_B));
}
@@ -152,8 +152,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
long baseId = base.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
Snapshot pending = table.newAppend()
.appendFile(FILE_C)
@@ -161,8 +161,8 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should contain 1 merged manifest for second write",
- 1, pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ 1, pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertNotEquals("Should not contain manifest from initial write",
initialManifest, newManifest);
@@ -194,7 +194,7 @@ public class TestMergeAppend extends TableTestBase {
.commit();
Assert.assertEquals("Should contain 2 merged manifest for first write",
- 2, readMetadata().currentSnapshot().manifests().size());
+ 2, readMetadata().currentSnapshot().allManifests().size());
table.newAppend()
.appendManifest(manifest)
@@ -203,7 +203,7 @@ public class TestMergeAppend extends TableTestBase {
.commit();
Assert.assertEquals("Should contain 3 merged manifest for second write",
- 3, readMetadata().currentSnapshot().manifests().size());
+ 3, readMetadata().currentSnapshot().allManifests().size());
// validate that the metadata summary is correct when using appendManifest
Assert.assertEquals("Summary metadata should include 3 added files",
@@ -228,7 +228,7 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should contain 3 merged manifest after 1st write
write",
- 3, pending.manifests().size());
+ 3, pending.allManifests().size());
// validate that the metadata summary is correct when using appendManifest
Assert.assertEquals("Summary metadata should include 4 added files",
@@ -250,8 +250,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
long baseId = base.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
table.newDelete()
.deleteFile(FILE_A)
@@ -260,8 +260,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata delete = readMetadata();
long deleteId = delete.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 filtered manifest for delete",
- 1, delete.currentSnapshot().manifests().size());
- ManifestFile deleteManifest = delete.currentSnapshot().manifests().get(0);
+ 1, delete.currentSnapshot().allManifests().size());
+ ManifestFile deleteManifest =
delete.currentSnapshot().allManifests().get(0);
validateManifestEntries(deleteManifest,
ids(deleteId, baseId),
@@ -274,8 +274,8 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should contain 1 merged manifest for second write",
- 1, pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ 1, pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertNotEquals("Should not contain manifest from initial write",
initialManifest, newManifest);
@@ -306,7 +306,7 @@ public class TestMergeAppend extends TableTestBase {
long idFileB = readMetadata().currentSnapshot().snapshotId();
Assert.assertEquals("Should have 2 manifests from setup writes",
- 2, readMetadata().currentSnapshot().manifests().size());
+ 2, readMetadata().currentSnapshot().allManifests().size());
table.newAppend()
.appendFile(FILE_C)
@@ -315,16 +315,16 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should have 3 unmerged manifests",
- 3, base.currentSnapshot().manifests().size());
- Set<ManifestFile> unmerged =
Sets.newHashSet(base.currentSnapshot().manifests());
+ 3, base.currentSnapshot().allManifests().size());
+ Set<ManifestFile> unmerged =
Sets.newHashSet(base.currentSnapshot().allManifests());
Snapshot pending = table.newAppend()
.appendFile(FILE_D)
.apply();
Assert.assertEquals("Should contain 1 merged manifest after the 4th write",
- 1, pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ 1, pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertFalse("Should not contain previous manifests",
unmerged.contains(newManifest));
long pendingId = pending.snapshotId();
@@ -351,8 +351,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
long baseId = base.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
Snapshot pending = table.newAppend()
.appendFile(FILE_C)
@@ -360,15 +360,15 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should contain 2 unmerged manifests after second
write",
- 2, pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ 2, pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertNotEquals("Should not contain manifest from initial write",
initialManifest, newManifest);
long pendingId = pending.snapshotId();
validateManifest(newManifest, ids(pendingId, pendingId), files(FILE_C,
FILE_D));
- validateManifest(pending.manifests().get(1), ids(baseId, baseId),
files(initialManifest));
+ validateManifest(pending.allManifests().get(1), ids(baseId, baseId),
files(initialManifest));
}
@Test
@@ -380,8 +380,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
// build the new spec using the table's schema, which uses fresh IDs
PartitionSpec newSpec = PartitionSpec.builderFor(base.schema())
@@ -402,13 +402,13 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should use 2 manifest files",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
// new manifest comes first
- validateManifest(pending.manifests().get(0), ids(pending.snapshotId()),
files(newFileC));
+ validateManifest(pending.allManifests().get(0), ids(pending.snapshotId()),
files(newFileC));
Assert.assertEquals("Second manifest should be the initial manifest with
the old spec",
- initialManifest, pending.manifests().get(1));
+ initialManifest, pending.allManifests().get(1));
}
@Test
@@ -426,8 +426,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should contain 2 manifests",
- 2, base.currentSnapshot().manifests().size());
- ManifestFile manifest = base.currentSnapshot().manifests().get(0);
+ 2, base.currentSnapshot().allManifests().size());
+ ManifestFile manifest = base.currentSnapshot().allManifests().get(0);
// build the new spec using the table's schema, which uses fresh IDs
PartitionSpec newSpec = PartitionSpec.builderFor(base.schema())
@@ -448,12 +448,12 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should use 2 manifest files",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
Assert.assertFalse("First manifest should not be in the new snapshot",
- pending.manifests().contains(manifest));
+ pending.allManifests().contains(manifest));
- validateManifest(pending.manifests().get(0), ids(pending.snapshotId()),
files(newFileC));
- validateManifest(pending.manifests().get(1), ids(id2, id1), files(FILE_B,
FILE_A));
+ validateManifest(pending.allManifests().get(0), ids(pending.snapshotId()),
files(newFileC));
+ validateManifest(pending.allManifests().get(1), ids(id2, id1),
files(FILE_B, FILE_A));
}
@Test
@@ -467,15 +467,15 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
long baseId = base.currentSnapshot().snapshotId();
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
table.ops().failCommits(5);
AppendFiles append = table.newAppend().appendFile(FILE_B);
Snapshot pending = append.apply();
- Assert.assertEquals("Should merge to 1 manifest", 1,
pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ Assert.assertEquals("Should merge to 1 manifest", 1,
pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
validateManifest(newManifest,
@@ -497,7 +497,7 @@ public class TestMergeAppend extends TableTestBase {
ManifestFile manifest = writeManifest(FILE_A, FILE_B);
AppendFiles append = table.newAppend().appendManifest(manifest);
Snapshot pending = append.apply();
- ManifestFile newManifest = pending.manifests().get(0);
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
AssertHelpers.assertThrows("Should retry 4 times and throw last failure",
@@ -517,15 +517,15 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
long baseId = base.currentSnapshot().snapshotId();
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
table.ops().failCommits(3);
AppendFiles append = table.newAppend().appendFile(FILE_B);
Snapshot pending = append.apply();
- Assert.assertEquals("Should merge to 1 manifest", 1,
pending.manifests().size());
- ManifestFile newManifest = pending.manifests().get(0);
+ Assert.assertEquals("Should merge to 1 manifest", 1,
pending.allManifests().size());
+ ManifestFile newManifest = pending.allManifests().get(0);
Assert.assertTrue("Should create new manifest", new
File(newManifest.path()).exists());
validateManifest(newManifest,
@@ -537,7 +537,7 @@ public class TestMergeAppend extends TableTestBase {
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().manifests());
+ Lists.newArrayList(newManifest),
metadata.currentSnapshot().allManifests());
}
@Test
@@ -558,7 +558,7 @@ public class TestMergeAppend extends TableTestBase {
Snapshot snapshot = table.currentSnapshot();
long snapshotId = snapshot.snapshotId();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 committed manifest", 1,
manifests.size());
validateManifestEntries(manifests.get(0),
@@ -606,7 +606,7 @@ public class TestMergeAppend extends TableTestBase {
Snapshot snapshot = table.currentSnapshot();
- Assert.assertEquals("Manifests should be merged into 1", 1,
snapshot.manifests().size());
+ Assert.assertEquals("Manifests should be merged into 1", 1,
snapshot.allManifests().size());
Assert.assertFalse("Merged append manifest should be deleted", new
File(manifest2.path()).exists());
}
@@ -715,8 +715,8 @@ public class TestMergeAppend extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
// build the new spec using the table's schema, which uses fresh IDs
PartitionSpec newSpec = PartitionSpec.builderFor(base.schema())
@@ -736,16 +736,17 @@ public class TestMergeAppend extends TableTestBase {
.apply();
Assert.assertEquals("Should use 2 manifest files",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
// new manifest comes first
- validateManifest(pending.manifests().get(0), ids(pending.snapshotId()),
files(newFile));
+ validateManifest(pending.allManifests().get(0), ids(pending.snapshotId()),
files(newFile));
Assert.assertEquals("Second manifest should be the initial manifest with
the old spec",
- initialManifest, pending.manifests().get(1));
+ initialManifest, pending.allManifests().get(1));
// field ids of manifest entries in two manifests with different specs of
the same source field should be different
- ManifestEntry<DataFile> entry =
ManifestFiles.read(pending.manifests().get(0),
FILE_IO).entries().iterator().next();
+ ManifestEntry<DataFile> entry =
ManifestFiles.read(pending.allManifests().get(0), FILE_IO)
+ .entries().iterator().next();
Types.NestedField field = ((PartitionData)
entry.file().partition()).getPartitionType().fields().get(0);
Assert.assertEquals(1000, field.fieldId());
Assert.assertEquals("id_bucket", field.name());
@@ -753,7 +754,7 @@ public class TestMergeAppend extends TableTestBase {
Assert.assertEquals(1001, field.fieldId());
Assert.assertEquals("data_bucket", field.name());
- entry = ManifestFiles.read(pending.manifests().get(1),
FILE_IO).entries().iterator().next();
+ entry = ManifestFiles.read(pending.allManifests().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/TestOverwrite.java
b/core/src/test/java/org/apache/iceberg/TestOverwrite.java
index 4c379e9..a8c86e8 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 class TestOverwrite extends TableTestBase {
Assert.assertNotEquals("Should create a new snapshot", baseId,
overwriteId);
Assert.assertEquals("Table should have one manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(0),
ids(overwriteId, baseId),
files(FILE_0_TO_4, FILE_5_TO_9),
statuses(Status.DELETED, Status.EXISTING));
@@ -174,15 +174,15 @@ public class TestOverwrite extends TableTestBase {
Assert.assertNotEquals("Should create a new snapshot", baseId,
overwriteId);
Assert.assertEquals("Table should have 2 manifests",
- 2, table.currentSnapshot().manifests().size());
+ 2, table.currentSnapshot().allManifests().size());
// manifest is not merged because it is less than the minimum
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(0),
ids(overwriteId),
files(FILE_10_TO_14),
statuses(Status.ADDED));
- validateManifestEntries(table.currentSnapshot().manifests().get(1),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(1),
ids(overwriteId, baseId),
files(FILE_0_TO_4, FILE_5_TO_9),
statuses(Status.DELETED, Status.EXISTING));
@@ -205,9 +205,9 @@ public class TestOverwrite extends TableTestBase {
Assert.assertNotEquals("Should create a new snapshot", baseId,
overwriteId);
Assert.assertEquals("Table should have one merged manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().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 da6e394..49f0c4a 100644
--- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
+++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
@@ -381,7 +381,7 @@ public class TestRemoveSnapshots extends TableTestBase {
t4 = System.currentTimeMillis();
}
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
ManifestFile newManifest = writeManifest(
"manifest-file-1.avro",
@@ -449,7 +449,7 @@ public class TestRemoveSnapshots extends TableTestBase {
// ManifestList should be deleted too
expectedDeletes.add(snapshotB.manifestListLocation());
- snapshotB.manifests().forEach(file -> {
+ snapshotB.dataManifests().forEach(file -> {
//Only the manifest of B should be deleted.
if (file.snapshotId() == snapshotB.snapshotId()) {
expectedDeletes.add(file.path());
diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
index 811cc7e..d2da367 100644
--- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
+++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
@@ -81,15 +81,15 @@ public class TestReplacePartitions extends TableTestBase {
long replaceId = readMetadata().currentSnapshot().snapshotId();
Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId);
Assert.assertEquals("Table should have 2 manifests",
- 2, table.currentSnapshot().manifests().size());
+ 2, table.currentSnapshot().allManifests().size());
// manifest is not merged because it is less than the minimum
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(0),
ids(replaceId),
files(FILE_E),
statuses(Status.ADDED));
- validateManifestEntries(table.currentSnapshot().manifests().get(1),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(1),
ids(replaceId, baseId),
files(FILE_A, FILE_B),
statuses(Status.DELETED, Status.EXISTING));
@@ -115,9 +115,9 @@ public class TestReplacePartitions extends TableTestBase {
long replaceId = readMetadata().currentSnapshot().snapshotId();
Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId);
Assert.assertEquals("Table should have 1 manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(0),
ids(replaceId, replaceId, baseId),
files(FILE_E, FILE_A, FILE_B),
statuses(Status.ADDED, Status.DELETED, Status.EXISTING));
@@ -153,12 +153,12 @@ public class TestReplacePartitions extends TableTestBase {
long replaceId = replaceMetadata.currentSnapshot().snapshotId();
Assert.assertEquals("Table should have 2 manifests",
- 2, replaceMetadata.currentSnapshot().manifests().size());
+ 2, replaceMetadata.currentSnapshot().allManifests().size());
-
validateManifestEntries(replaceMetadata.currentSnapshot().manifests().get(0),
+
validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(0),
ids(replaceId), files(FILE_B), statuses(Status.ADDED));
-
validateManifestEntries(replaceMetadata.currentSnapshot().manifests().get(1),
+
validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(1),
ids(replaceId), files(FILE_A), statuses(Status.DELETED));
}
@@ -195,9 +195,9 @@ public class TestReplacePartitions extends TableTestBase {
long replaceId = replaceMetadata.currentSnapshot().snapshotId();
Assert.assertEquals("Table should have 1 manifest",
- 1, replaceMetadata.currentSnapshot().manifests().size());
+ 1, replaceMetadata.currentSnapshot().allManifests().size());
-
validateManifestEntries(replaceMetadata.currentSnapshot().manifests().get(0),
+
validateManifestEntries(replaceMetadata.currentSnapshot().allManifests().get(0),
ids(replaceId, replaceId), files(FILE_B, FILE_A),
statuses(Status.ADDED, Status.DELETED));
}
@@ -242,15 +242,15 @@ public class TestReplacePartitions extends TableTestBase {
long replaceId = readMetadata().currentSnapshot().snapshotId();
Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId);
Assert.assertEquals("Table should have 2 manifests",
- 2, table.currentSnapshot().manifests().size());
+ 2, table.currentSnapshot().allManifests().size());
// manifest is not merged because it is less than the minimum
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(0),
ids(replaceId),
files(FILE_G),
statuses(Status.ADDED));
- validateManifestEntries(table.currentSnapshot().manifests().get(1),
+ validateManifestEntries(table.currentSnapshot().allManifests().get(1),
ids(baseId, baseId),
files(FILE_A, FILE_B),
statuses(Status.ADDED, 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 b4b64e7..ad2015f 100644
--- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
@@ -99,26 +99,26 @@ public class TestRewriteFiles extends TableTestBase {
TableMetadata base = readMetadata();
long baseSnapshotId = base.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
Snapshot pending = table.newRewrite()
.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C))
.apply();
Assert.assertEquals("Should contain 2 manifest",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
Assert.assertFalse("Should not contain manifest from initial write",
- pending.manifests().contains(initialManifest));
+ pending.allManifests().contains(initialManifest));
long pendingId = pending.snapshotId();
- validateManifestEntries(pending.manifests().get(0),
+ validateManifestEntries(pending.allManifests().get(0),
ids(pendingId),
files(FILE_C),
statuses(ADDED));
- validateManifestEntries(pending.manifests().get(1),
+ validateManifestEntries(pending.allManifests().get(1),
ids(pendingId, pendingId, baseSnapshotId),
files(FILE_A, FILE_A, FILE_B),
statuses(DELETED, DELETED, EXISTING));
@@ -139,26 +139,26 @@ public class TestRewriteFiles extends TableTestBase {
TableMetadata base = readMetadata();
long baseSnapshotId = base.currentSnapshot().snapshotId();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
Snapshot pending = table.newRewrite()
.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C))
.apply();
Assert.assertEquals("Should contain 2 manifest",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
Assert.assertFalse("Should not contain manifest from initial write",
- pending.manifests().contains(initialManifest));
+ pending.allManifests().contains(initialManifest));
long pendingId = pending.snapshotId();
- validateManifestEntries(pending.manifests().get(0),
+ validateManifestEntries(pending.allManifests().get(0),
ids(pendingId),
files(FILE_C),
statuses(ADDED));
- validateManifestEntries(pending.manifests().get(1),
+ validateManifestEntries(pending.allManifests().get(1),
ids(pendingId, baseSnapshotId),
files(FILE_A, FILE_B),
statuses(DELETED, EXISTING));
@@ -179,9 +179,9 @@ public class TestRewriteFiles extends TableTestBase {
.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B));
Snapshot pending = rewrite.apply();
- Assert.assertEquals("Should produce 2 manifests", 2,
pending.manifests().size());
- ManifestFile manifest1 = pending.manifests().get(0);
- ManifestFile manifest2 = pending.manifests().get(1);
+ Assert.assertEquals("Should produce 2 manifests", 2,
pending.allManifests().size());
+ ManifestFile manifest1 = pending.allManifests().get(0);
+ ManifestFile manifest2 = pending.allManifests().get(1);
validateManifestEntries(manifest1,
ids(pending.snapshotId()), files(FILE_B), statuses(ADDED));
@@ -209,9 +209,9 @@ public class TestRewriteFiles extends TableTestBase {
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.manifests().size());
- ManifestFile manifest1 = pending.manifests().get(0);
- ManifestFile manifest2 = pending.manifests().get(1);
+ Assert.assertEquals("Should produce 2 manifests", 2,
pending.allManifests().size());
+ ManifestFile manifest1 = pending.allManifests().get(0);
+ ManifestFile manifest2 = pending.allManifests().get(1);
validateManifestEntries(manifest1,
ids(pending.snapshotId()), files(FILE_B), statuses(ADDED));
@@ -225,7 +225,7 @@ public class TestRewriteFiles extends TableTestBase {
TableMetadata metadata = readMetadata();
Assert.assertTrue("Should commit the manifest for append",
- metadata.currentSnapshot().manifests().contains(manifest2));
+ metadata.currentSnapshot().allManifests().contains(manifest2));
// 2 manifests added by rewrite and 1 original manifest should be found.
Assert.assertEquals("Only 3 manifests should exist", 3,
listManifestFiles().size());
@@ -242,7 +242,7 @@ public class TestRewriteFiles extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
+ 1, base.currentSnapshot().allManifests().size());
AssertHelpers.assertThrows("Expected an exception",
ValidationException.class,
@@ -264,7 +264,7 @@ public class TestRewriteFiles extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
+ 1, base.currentSnapshot().allManifests().size());
RewriteFiles rewrite = table.newRewrite();
Snapshot pending = rewrite
@@ -272,16 +272,16 @@ public class TestRewriteFiles extends TableTestBase {
.apply();
Assert.assertEquals("Should contain 2 manifest",
- 2, pending.manifests().size());
+ 2, pending.allManifests().size());
long pendingId = pending.snapshotId();
- validateManifestEntries(pending.manifests().get(0),
+ validateManifestEntries(pending.allManifests().get(0),
ids(pendingId),
files(FILE_B),
statuses(ADDED));
- validateManifestEntries(pending.manifests().get(1),
+ validateManifestEntries(pending.allManifests().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 cb64793..74a46de 100644
--- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java
+++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java
@@ -69,13 +69,13 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendId = table.currentSnapshot().snapshotId();
- Assert.assertEquals(1, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(1, table.currentSnapshot().allManifests().size());
table.rewriteManifests()
.clusterBy(file -> "")
.commit();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(1, manifests.size());
validateManifestEntries(manifests.get(0),
@@ -104,13 +104,13 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long fileAppendId = table.currentSnapshot().snapshotId();
- Assert.assertEquals(2, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(2, table.currentSnapshot().allManifests().size());
table.rewriteManifests()
.clusterBy(file -> "")
.commit();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Manifests must be merged into 1", 1,
manifests.size());
// get the correct file order
@@ -141,7 +141,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendId = table.currentSnapshot().snapshotId();
- Assert.assertEquals(1, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(1, table.currentSnapshot().allManifests().size());
// cluster by path will split the manifest into two
@@ -149,7 +149,7 @@ public class TestRewriteManifests extends TableTestBase {
.clusterBy(file -> file.path())
.commit();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(2, manifests.size());
manifests.sort(Comparator.comparing(ManifestFile::path));
@@ -176,7 +176,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendIdB = table.currentSnapshot().snapshotId();
- Assert.assertEquals(2, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(2, table.currentSnapshot().allManifests().size());
// cluster by constant will combine manifests into one
@@ -184,7 +184,7 @@ public class TestRewriteManifests extends TableTestBase {
.clusterBy(file -> "file")
.commit();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(1, manifests.size());
// get the file order correct
@@ -225,7 +225,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendIdC = table.currentSnapshot().snapshotId();
- Assert.assertEquals(3, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(3, table.currentSnapshot().allManifests().size());
//keep the file A manifest, combine the other two
@@ -240,7 +240,7 @@ public class TestRewriteManifests extends TableTestBase {
})
.commit();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(2, manifests.size());
// get the file order correct
@@ -275,14 +275,14 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendId = table.currentSnapshot().snapshotId();
- Assert.assertEquals(1, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(1, table.currentSnapshot().allManifests().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<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(2, manifests.size());
manifests.sort(Comparator.comparing(ManifestFile::path));
@@ -324,13 +324,13 @@ public class TestRewriteManifests extends TableTestBase {
})
.commit();
- Assert.assertEquals(2, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(2, table.currentSnapshot().allManifests().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<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(1, manifests.size());
// get the file order correct
@@ -370,7 +370,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
long appendIdB = table.currentSnapshot().snapshotId();
- Assert.assertEquals(2, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(2, table.currentSnapshot().allManifests().size());
// commit the rewrite manifests in progress
rewrite.commit();
@@ -378,7 +378,7 @@ public class TestRewriteManifests extends TableTestBase {
// 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<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(2, manifests.size());
validateManifestEntries(manifests.get(0),
@@ -408,13 +408,13 @@ public class TestRewriteManifests extends TableTestBase {
.clusterBy(file -> "file")
.commit();
- Assert.assertEquals(1, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(1, table.currentSnapshot().allManifests().size());
// commit the append in progress
append.commit();
long appendIdB = table.currentSnapshot().snapshotId();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals(2, manifests.size());
// last append should be the first in the list
@@ -439,7 +439,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -463,7 +463,7 @@ public class TestRewriteManifests extends TableTestBase {
rewriteManifests.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(3, manifests.size());
validateSummary(snapshot, 1, 1, 2, 0);
@@ -501,7 +501,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -525,7 +525,7 @@ public class TestRewriteManifests extends TableTestBase {
rewriteManifests.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(3, manifests.size());
validateSummary(snapshot, 1, 1, 2, 0);
@@ -565,8 +565,8 @@ public class TestRewriteManifests extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
int initialPartitionSpecId = initialManifest.partitionSpecId();
@@ -598,13 +598,13 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Assert.assertEquals("Should use 3 manifest files",
- 3, table.currentSnapshot().manifests().size());
+ 3, table.currentSnapshot().allManifests().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<ManifestFile> manifestFiles = table.currentSnapshot().manifests();
+ List<ManifestFile> manifestFiles = table.currentSnapshot().allManifests();
Assert.assertEquals("Rewrite manifest should produce 2 manifest files",
2, manifestFiles.size());
@@ -635,8 +635,8 @@ public class TestRewriteManifests extends TableTestBase {
TableMetadata base = readMetadata();
Assert.assertEquals("Should create 1 manifest for initial write",
- 1, base.currentSnapshot().manifests().size());
- ManifestFile initialManifest = base.currentSnapshot().manifests().get(0);
+ 1, base.currentSnapshot().allManifests().size());
+ ManifestFile initialManifest =
base.currentSnapshot().allManifests().get(0);
int initialPartitionSpecId = initialManifest.partitionSpecId();
// build the new spec using the table's schema, which uses fresh IDs
@@ -667,7 +667,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Assert.assertEquals("Rewrite manifests should produce 3 manifest files",
- 3, table.currentSnapshot().manifests().size());
+ 3, table.currentSnapshot().allManifests().size());
// cluster by constant will combine manifests into one but small target
size will create one per entry
BaseRewriteManifests rewriteManifests = spy((BaseRewriteManifests)
table.rewriteManifests());
@@ -675,7 +675,7 @@ public class TestRewriteManifests extends TableTestBase {
// rewriteManifests should produce 4 manifestFiles, because of
targetByteSize=1
rewriteManifests.clusterBy(dataFile -> "file").commit();
- List<ManifestFile> manifestFiles = table.currentSnapshot().manifests();
+ List<ManifestFile> manifestFiles = table.currentSnapshot().allManifests();
Assert.assertEquals("Should use 4 manifest files",
4, manifestFiles.size());
@@ -707,7 +707,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -729,12 +729,12 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot secondSnapshot = table.currentSnapshot();
- Assert.assertEquals(2, table.currentSnapshot().manifests().size());
+ Assert.assertEquals(2, table.currentSnapshot().allManifests().size());
rewriteManifests.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(3, manifests.size());
validateSummary(snapshot, 1, 1, 2, 0);
@@ -772,7 +772,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -802,7 +802,7 @@ public class TestRewriteManifests extends TableTestBase {
rewriteManifests.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(3, manifests.size());
validateSummary(snapshot, 1, 1, 2, 0);
@@ -836,7 +836,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -870,7 +870,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -908,7 +908,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(3, manifests.size());
validateSummary(snapshot, 3, 1, 2, 2);
@@ -939,7 +939,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -974,7 +974,7 @@ public class TestRewriteManifests extends TableTestBase {
rewriteManifests.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(2, manifests.size());
validateSummary(snapshot, 3, 0, 2, 1);
@@ -1001,7 +1001,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot snapshot = table.currentSnapshot();
- List<ManifestFile> manifests = snapshot.manifests();
+ List<ManifestFile> manifests = snapshot.allManifests();
Assert.assertEquals(1, manifests.size());
ManifestFile manifest = manifests.get(0);
@@ -1047,7 +1047,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -1056,7 +1056,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot secondSnapshot = table.currentSnapshot();
- List<ManifestFile> secondSnapshotManifests = secondSnapshot.manifests();
+ List<ManifestFile> secondSnapshotManifests = secondSnapshot.allManifests();
Assert.assertEquals(2, secondSnapshotManifests.size());
ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0);
@@ -1094,7 +1094,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot firstSnapshot = table.currentSnapshot();
- List<ManifestFile> firstSnapshotManifests = firstSnapshot.manifests();
+ List<ManifestFile> firstSnapshotManifests = firstSnapshot.allManifests();
Assert.assertEquals(1, firstSnapshotManifests.size());
ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0);
@@ -1103,7 +1103,7 @@ public class TestRewriteManifests extends TableTestBase {
.commit();
Snapshot secondSnapshot = table.currentSnapshot();
- List<ManifestFile> secondSnapshotManifests = secondSnapshot.manifests();
+ List<ManifestFile> secondSnapshotManifests = secondSnapshot.allManifests();
Assert.assertEquals(2, secondSnapshotManifests.size());
ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0);
diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java
b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java
index be383e8..8e03bec 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 class TestSnapshotJson {
Assert.assertEquals("Snapshot ID should match",
expected.snapshotId(), snapshot.snapshotId());
Assert.assertEquals("Files should match",
- expected.manifests(), snapshot.manifests());
+ expected.allManifests(), snapshot.allManifests());
Assert.assertNull("Operation should be null", snapshot.operation());
Assert.assertNull("Summary should be null", snapshot.summary());
}
@@ -78,7 +78,7 @@ public class TestSnapshotJson {
Assert.assertEquals("Manifest list should match",
expected.manifestListLocation(), snapshot.manifestListLocation());
Assert.assertEquals("Files should match",
- expected.manifests(), snapshot.manifests());
+ expected.allManifests(), snapshot.allManifests());
Assert.assertEquals("Operation should match",
expected.operation(), snapshot.operation());
Assert.assertEquals("Summary should match",
@@ -107,7 +107,7 @@ public class TestSnapshotJson {
ops.io(), id, parentId, expected.timestampMillis(), null, null,
manifests);
Assert.assertEquals("Files should match in memory list",
- inMemory.manifests(), expected.manifests());
+ inMemory.allManifests(), expected.allManifests());
String json = SnapshotParser.toJson(expected);
Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json);
@@ -123,7 +123,7 @@ public class TestSnapshotJson {
Assert.assertEquals("Manifest list should match",
expected.manifestListLocation(), snapshot.manifestListLocation());
Assert.assertEquals("Files should match",
- expected.manifests(), snapshot.manifests());
+ expected.allManifests(), snapshot.allManifests());
Assert.assertNull("Operation should be null", snapshot.operation());
Assert.assertNull("Summary should be null", snapshot.summary());
}
diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
index fc030f1..205a79e 100644
--- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
+++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
@@ -129,12 +129,12 @@ public class TestTableMetadata {
Assert.assertEquals("Parent snapshot ID should match",
(Long) previousSnapshotId, metadata.currentSnapshot().parentId());
Assert.assertEquals("Current snapshot files should match",
- currentSnapshot.manifests(), metadata.currentSnapshot().manifests());
+ currentSnapshot.allManifests(),
metadata.currentSnapshot().allManifests());
Assert.assertEquals("Previous snapshot ID should match",
previousSnapshotId,
metadata.snapshot(previousSnapshotId).snapshotId());
Assert.assertEquals("Previous snapshot files should match",
- previousSnapshot.manifests(),
- metadata.snapshot(previousSnapshotId).manifests());
+ previousSnapshot.allManifests(),
+ metadata.snapshot(previousSnapshotId).allManifests());
}
@Test
@@ -226,12 +226,12 @@ public class TestTableMetadata {
Assert.assertEquals("Parent snapshot ID should match",
(Long) previousSnapshotId, metadata.currentSnapshot().parentId());
Assert.assertEquals("Current snapshot files should match",
- currentSnapshot.manifests(), metadata.currentSnapshot().manifests());
+ currentSnapshot.allManifests(),
metadata.currentSnapshot().allManifests());
Assert.assertEquals("Previous snapshot ID should match",
previousSnapshotId,
metadata.snapshot(previousSnapshotId).snapshotId());
Assert.assertEquals("Previous snapshot files should match",
- previousSnapshot.manifests(),
- metadata.snapshot(previousSnapshotId).manifests());
+ previousSnapshot.allManifests(),
+ metadata.snapshot(previousSnapshotId).allManifests());
Assert.assertEquals("Snapshot logs should match",
expected.previousFiles(), metadata.previousFiles());
}
diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
index 2343b7c..c719375 100644
--- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
+++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
@@ -72,8 +72,8 @@ public class TestTimestampPartitions extends TableTestBase {
.appendFile(dataFile)
.commit();
long id = table.currentSnapshot().snapshotId();
- Assert.assertEquals(table.currentSnapshot().manifests().size(), 1);
- validateManifestEntries(table.currentSnapshot().manifests().get(0),
+ Assert.assertEquals(table.currentSnapshot().allManifests().size(), 1);
+ validateManifestEntries(table.currentSnapshot().allManifests().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 bceaa73..2182608 100644
--- a/core/src/test/java/org/apache/iceberg/TestTransaction.java
+++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java
@@ -126,16 +126,16 @@ public class TestTransaction extends TableTestBase {
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().manifests().size());
+ 1, readMetadata().currentSnapshot().allManifests().size());
Assert.assertEquals("Table snapshot should be the delete snapshot",
deleteSnapshot, readMetadata().currentSnapshot());
-
validateManifestEntries(readMetadata().currentSnapshot().manifests().get(0),
+
validateManifestEntries(readMetadata().currentSnapshot().allManifests().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).manifests().get(0),
+
validateManifestEntries(readMetadata().snapshots().get(0).allManifests().get(0),
ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()),
files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED));
}
@@ -177,16 +177,16 @@ public class TestTransaction extends TableTestBase {
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().manifests().size());
+ 1, readMetadata().currentSnapshot().allManifests().size());
Assert.assertEquals("Table snapshot should be the delete snapshot",
deleteSnapshot, readMetadata().currentSnapshot());
-
validateManifestEntries(readMetadata().currentSnapshot().manifests().get(0),
+
validateManifestEntries(readMetadata().currentSnapshot().allManifests().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).manifests().get(0),
+
validateManifestEntries(readMetadata().snapshots().get(0).allManifests().get(0),
ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()),
files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED));
}
@@ -294,7 +294,7 @@ public class TestTransaction extends TableTestBase {
.appendFile(FILE_B)
.commit();
- Set<ManifestFile> appendManifests =
Sets.newHashSet(txn.table().currentSnapshot().manifests());
+ Set<ManifestFile> appendManifests =
Sets.newHashSet(txn.table().currentSnapshot().allManifests());
Assert.assertSame("Base metadata should not change when commit is created",
base, readMetadata());
@@ -308,7 +308,7 @@ public class TestTransaction extends TableTestBase {
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().manifests()));
+ appendManifests,
Sets.newHashSet(table.currentSnapshot().allManifests()));
}
@Test
@@ -333,7 +333,7 @@ public class TestTransaction extends TableTestBase {
.appendFile(FILE_B)
.commit();
- Set<ManifestFile> appendManifests =
Sets.newHashSet(txn.table().currentSnapshot().manifests());
+ Set<ManifestFile> appendManifests =
Sets.newHashSet(txn.table().currentSnapshot().allManifests());
Assert.assertSame("Base metadata should not change when commit is created",
base, readMetadata());
@@ -347,7 +347,7 @@ public class TestTransaction extends TableTestBase {
Assert.assertEquals("Table should be on version 2 after real append", 2,
(int) version());
- Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().manifests());
+ Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().allManifests());
txn.commitTransaction();
@@ -358,7 +358,7 @@ public class TestTransaction extends TableTestBase {
expectedManifests.addAll(conflictAppendManifests);
Assert.assertEquals("Should reuse manifests from initial append commit and
conflicting append",
- expectedManifests,
Sets.newHashSet(table.currentSnapshot().manifests()));
+ expectedManifests,
Sets.newHashSet(table.currentSnapshot().allManifests()));
}
@Test
@@ -389,8 +389,8 @@ public class TestTransaction extends TableTestBase {
.commit();
Assert.assertEquals("Append should create one manifest",
- 1, txn.table().currentSnapshot().manifests().size());
- ManifestFile appendManifest =
txn.table().currentSnapshot().manifests().get(0);
+ 1, txn.table().currentSnapshot().allManifests().size());
+ ManifestFile appendManifest =
txn.table().currentSnapshot().allManifests().get(0);
Assert.assertSame("Base metadata should not change when commit is created",
base, readMetadata());
@@ -404,7 +404,7 @@ public class TestTransaction extends TableTestBase {
Assert.assertEquals("Table should be on version 2 after real append", 2,
(int) version());
- Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().manifests());
+ Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().allManifests());
txn.commitTransaction();
@@ -415,9 +415,9 @@ public class TestTransaction extends TableTestBase {
previousManifests.addAll(conflictAppendManifests);
Assert.assertEquals("Should merge both commit manifests into a single
manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
Assert.assertFalse("Should merge both commit manifests into a new
manifest",
-
previousManifests.contains(table.currentSnapshot().manifests().get(0)));
+
previousManifests.contains(table.currentSnapshot().allManifests().get(0)));
Assert.assertFalse("Append manifest should be deleted", new
File(appendManifest.path()).exists());
}
@@ -446,8 +446,8 @@ public class TestTransaction extends TableTestBase {
.commit();
Assert.assertEquals("Append should create one manifest",
- 1, txn.table().currentSnapshot().manifests().size());
- ManifestFile appendManifest =
txn.table().currentSnapshot().manifests().get(0);
+ 1, txn.table().currentSnapshot().allManifests().size());
+ ManifestFile appendManifest =
txn.table().currentSnapshot().allManifests().get(0);
Assert.assertSame("Base metadata should not change when commit is created",
base, readMetadata());
@@ -461,7 +461,7 @@ public class TestTransaction extends TableTestBase {
Assert.assertEquals("Table should be on version 2 after real append", 2,
(int) version());
- Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().manifests());
+ Set<ManifestFile> conflictAppendManifests =
Sets.newHashSet(table.currentSnapshot().allManifests());
txn.commitTransaction();
@@ -472,9 +472,9 @@ public class TestTransaction extends TableTestBase {
previousManifests.addAll(conflictAppendManifests);
Assert.assertEquals("Should merge both commit manifests into a single
manifest",
- 1, table.currentSnapshot().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
Assert.assertFalse("Should merge both commit manifests into a new
manifest",
-
previousManifests.contains(table.currentSnapshot().manifests().get(0)));
+
previousManifests.contains(table.currentSnapshot().allManifests().get(0)));
Assert.assertFalse("Append manifest should be deleted", new
File(appendManifest.path()).exists());
}
@@ -495,8 +495,8 @@ public class TestTransaction extends TableTestBase {
.commit();
Assert.assertEquals("Table should be on version 2 after append", 2, (int)
version());
- Assert.assertEquals("Append should create one manifest", 1,
table.currentSnapshot().manifests().size());
- ManifestFile v1manifest = table.currentSnapshot().manifests().get(0);
+ Assert.assertEquals("Append should create one manifest", 1,
table.currentSnapshot().allManifests().size());
+ ManifestFile v1manifest = table.currentSnapshot().allManifests().get(0);
TableMetadata base = readMetadata();
@@ -518,8 +518,9 @@ public class TestTransaction extends TableTestBase {
Assert.assertSame("Base metadata should not change when commit is
created", base, readMetadata());
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().manifests().size());
- ManifestFile mergedManifest =
txn.table().currentSnapshot().manifests().get(0);
+ Assert.assertEquals("Append should have one merged manifest",
+ 1, txn.table().currentSnapshot().allManifests().size());
+ ManifestFile mergedManifest =
txn.table().currentSnapshot().allManifests().get(0);
// find the initial copy of the appended manifest
String copiedAppendManifest = Iterables.getOnlyElement(Iterables.filter(
@@ -549,7 +550,7 @@ public class TestTransaction extends TableTestBase {
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().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
}
@Test
@@ -569,7 +570,7 @@ public class TestTransaction extends TableTestBase {
.commit();
Assert.assertEquals("Table should be on version 2 after append", 2, (int)
version());
- Assert.assertEquals("Append should create one manifest", 1,
table.currentSnapshot().manifests().size());
+ Assert.assertEquals("Append should create one manifest", 1,
table.currentSnapshot().allManifests().size());
TableMetadata base = readMetadata();
@@ -583,8 +584,9 @@ public class TestTransaction extends TableTestBase {
Assert.assertSame("Base metadata should not change when commit is
created", base, readMetadata());
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().manifests().size());
- ManifestFile mergedManifest =
txn.table().currentSnapshot().manifests().get(0);
+ Assert.assertEquals("Append should have one merged manifest",
+ 1, txn.table().currentSnapshot().allManifests().size());
+ ManifestFile mergedManifest =
txn.table().currentSnapshot().allManifests().get(0);
// cause the transaction commit to fail and retry
table.newAppend()
@@ -606,7 +608,7 @@ public class TestTransaction extends TableTestBase {
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().manifests().size());
+ 1, table.currentSnapshot().allManifests().size());
}
@Test
@@ -638,7 +640,7 @@ public class TestTransaction extends TableTestBase {
txn.commitTransaction();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Expected 2 manifests", 2, manifests.size());
}
@@ -661,7 +663,7 @@ public class TestTransaction extends TableTestBase {
.commit();
long secondSnapshotId = table.currentSnapshot().snapshotId();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 manifests after 2 appends", 2,
manifests.size());
ManifestFile newManifest = writeManifest(
@@ -685,7 +687,7 @@ public class TestTransaction extends TableTestBase {
Assert.assertTrue("Append manifest should not be deleted", new
File(newManifest.path()).exists());
- List<ManifestFile> finalManifests = table.currentSnapshot().manifests();
+ List<ManifestFile> finalManifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 final manifest", 1,
finalManifests.size());
validateManifestEntries(finalManifests.get(0),
diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
index b26a5fe..7a7754f 100644
--- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
+++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
@@ -143,7 +143,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 2,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 2,
@@ -171,7 +171,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 1,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 3,
@@ -217,7 +217,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 1,
@@ -337,7 +337,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 2,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 2,
@@ -401,7 +401,7 @@ public class TestWapWorkflow extends TableTestBase {
parentSnapshot.snapshotId() + 1,
base.currentSnapshot().snapshotId());
Assert.assertEquals("Should contain manifests for both files", 2,
- base.currentSnapshot().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Parent snapshot id should change to latest snapshot
before commit",
@@ -420,7 +420,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 3,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Parent snapshot id should change to latest snapshot
before commit",
@@ -485,7 +485,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 2,
base.snapshotLog().size());
@@ -501,7 +501,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Parent snapshot id should point to same snapshot",
@@ -521,7 +521,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Parent snapshot id should point to same snapshot",
@@ -578,7 +578,7 @@ public class TestWapWorkflow extends TableTestBase {
parentSnapshot.snapshotId() + 1,
base.currentSnapshot().snapshotId());
Assert.assertEquals("Should contain manifests for both files", 2,
- base.currentSnapshot().manifests().size());
+ base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should not contain redundant append due to retry", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Parent snapshot id should change to latest snapshot
before commit",
@@ -625,7 +625,7 @@ public class TestWapWorkflow extends TableTestBase {
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().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 2,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 2,
@@ -680,7 +680,7 @@ public class TestWapWorkflow extends TableTestBase {
base = readMetadata();
Assert.assertEquals("Should have three snapshots", 3,
base.snapshots().size());
- Assert.assertEquals("Should contain manifests for both files", 2,
base.currentSnapshot().manifests().size());
+ Assert.assertEquals("Should contain manifests for both files", 2,
base.currentSnapshot().allManifests().size());
Assert.assertEquals("Should contain append from last commit", 1,
Iterables.size(base.currentSnapshot().addedFiles()));
Assert.assertEquals("Snapshot log should indicate number of snapshots
committed", 2,
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 6728724..25200be 100644
--- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java
+++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java
@@ -283,7 +283,7 @@ public class TestHadoopCommits extends HadoopTableTestBase {
TableMetadata metadata = readMetadataVersion(3);
Assert.assertEquals("Current snapshot should contain 2 manifests",
- 2, metadata.currentSnapshot().manifests().size());
+ 2, metadata.currentSnapshot().allManifests().size());
}
@Test
@@ -306,7 +306,7 @@ public class TestHadoopCommits extends HadoopTableTestBase {
TableMetadata metadata = readMetadataVersion(5);
Assert.assertEquals("Current snapshot should contain 1 merged manifest",
- 1, metadata.currentSnapshot().manifests().size());
+ 1, metadata.currentSnapshot().allManifests().size());
}
@Test
diff --git
a/hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
b/hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
index bb3fb82..b714605 100644
--- a/hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
+++ b/hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
@@ -128,7 +128,7 @@ public class HiveCreateReplaceTableTest extends
HiveMetastoreTest {
Table table = catalog.loadTable(TABLE_IDENTIFIER);
Snapshot snapshot = table.currentSnapshot();
- Assert.assertTrue("Table should have one manifest file",
snapshot.manifests().size() == 1);
+ Assert.assertTrue("Table should have one manifest file",
snapshot.allManifests().size() == 1);
}
@Test
diff --git a/hive/src/test/java/org/apache/iceberg/hive/HiveTableTest.java
b/hive/src/test/java/org/apache/iceberg/hive/HiveTableTest.java
index 3adc561..9168e90 100644
--- a/hive/src/test/java/org/apache/iceberg/hive/HiveTableTest.java
+++ b/hive/src/test/java/org/apache/iceberg/hive/HiveTableTest.java
@@ -197,7 +197,7 @@ public class HiveTableTest extends HiveTableBaseTest {
String manifestListLocation =
table.currentSnapshot().manifestListLocation().replace("file:", "");
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertTrue("Drop (table and data) should return true and drop the
table",
catalog.dropTable(TABLE_IDENTIFIER));
diff --git
a/hive/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java
b/hive/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java
index 88a70f8..a4e1c5b 100644
--- a/hive/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java
+++ b/hive/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java
@@ -75,7 +75,7 @@ public class TestHiveTableConcurrency extends
HiveTableBaseTest {
});
icebergTable.refresh();
- Assert.assertEquals(20, icebergTable.currentSnapshot().manifests().size());
+ Assert.assertEquals(20,
icebergTable.currentSnapshot().allManifests().size());
}
@Test
diff --git
a/spark/src/main/java/org/apache/iceberg/actions/RewriteManifestsAction.java
b/spark/src/main/java/org/apache/iceberg/actions/RewriteManifestsAction.java
index 7999c87..787852a 100644
--- a/spark/src/main/java/org/apache/iceberg/actions/RewriteManifestsAction.java
+++ b/spark/src/main/java/org/apache/iceberg/actions/RewriteManifestsAction.java
@@ -286,7 +286,7 @@ public class RewriteManifestsAction
return ImmutableList.of();
}
- return currentSnapshot.manifests().stream()
+ return currentSnapshot.dataManifests().stream()
.filter(manifest -> manifest.partitionSpecId() == spec.specId() &&
predicate.test(manifest))
.collect(Collectors.toList());
}
diff --git
a/spark/src/test/java/org/apache/iceberg/actions/TestRewriteManifestsAction.java
b/spark/src/test/java/org/apache/iceberg/actions/TestRewriteManifestsAction.java
index f7466ce..b3877dd 100644
---
a/spark/src/test/java/org/apache/iceberg/actions/TestRewriteManifestsAction.java
+++
b/spark/src/test/java/org/apache/iceberg/actions/TestRewriteManifestsAction.java
@@ -143,7 +143,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 manifests before rewrite", 2,
manifests.size());
Actions actions = Actions.forTable(table);
@@ -157,7 +157,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> newManifests = table.currentSnapshot().manifests();
+ List<ManifestFile> newManifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 manifests after rewrite", 1,
newManifests.size());
Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount());
@@ -212,7 +212,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 4 manifests before rewrite", 4,
manifests.size());
Actions actions = Actions.forTable(table);
@@ -234,7 +234,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> newManifests = table.currentSnapshot().manifests();
+ List<ManifestFile> newManifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 manifests after rewrite", 2,
newManifests.size());
Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount());
@@ -297,7 +297,7 @@ public class TestRewriteManifestsAction {
.stagingLocation(temp.newFolder().toString())
.execute();
- Assert.assertEquals("Action should rewrite all manifests",
snapshot.manifests(), result.deletedManifests());
+ Assert.assertEquals("Action should rewrite all manifests",
snapshot.allManifests(), result.deletedManifests());
Assert.assertEquals("Action should add 1 manifest", 1,
result.addedManifests().size());
} finally {
@@ -325,7 +325,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
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
@@ -345,7 +345,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> newManifests = table.currentSnapshot().manifests();
+ List<ManifestFile> newManifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 manifests after rewrite", 2,
newManifests.size());
Dataset<Row> resultDF = spark.read().format("iceberg").load(tableLocation);
@@ -380,7 +380,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 2 manifests before rewrite", 2,
manifests.size());
Actions actions = Actions.forTable(table);
@@ -397,7 +397,7 @@ public class TestRewriteManifestsAction {
table.refresh();
- List<ManifestFile> newManifests = table.currentSnapshot().manifests();
+ List<ManifestFile> newManifests = table.currentSnapshot().allManifests();
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/src/test/java/org/apache/iceberg/spark/TestSparkDataFile.java
b/spark/src/test/java/org/apache/iceberg/spark/TestSparkDataFile.java
index b22b294..47c9caf 100644
--- a/spark/src/test/java/org/apache/iceberg/spark/TestSparkDataFile.java
+++ b/spark/src/test/java/org/apache/iceberg/spark/TestSparkDataFile.java
@@ -153,7 +153,7 @@ public class TestSparkDataFile {
table.refresh();
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Should have 1 manifest", 1, manifests.size());
List<DataFile> dataFiles = Lists.newArrayList();
diff --git
a/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
index fd6290a..5dda74d 100644
---
a/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
+++
b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
@@ -311,7 +311,7 @@ public class TestDataSourceOptions {
.mode("append")
.save(tableLocation);
- List<ManifestFile> manifests = table.currentSnapshot().manifests();
+ List<ManifestFile> manifests = table.currentSnapshot().allManifests();
Assert.assertEquals("Must be 2 manifests", 2, manifests.size());
@@ -355,7 +355,7 @@ public class TestDataSourceOptions {
int splitSize = (int) TableProperties.METADATA_SPLIT_SIZE_DEFAULT; // 32MB
split size
int expectedSplits = ((int) tables.load(tableLocation + "#entries")
- .currentSnapshot().manifests().get(0).length() + splitSize - 1) /
splitSize;
+ .currentSnapshot().allManifests().get(0).length() + splitSize - 1) /
splitSize;
Dataset<Row> metadataDf = spark.read()
.format("iceberg")
diff --git
a/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index 3bfa0cd..e17836c 100644
---
a/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++
b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -119,9 +119,9 @@ public abstract class TestIcebergSourceTablesBase {
Snapshot snapshot = table.currentSnapshot();
- Assert.assertEquals("Should only contain one manifest", 1,
snapshot.manifests().size());
+ Assert.assertEquals("Should only contain one manifest", 1,
snapshot.allManifests().size());
- InputFile manifest =
table.io().newInputFile(snapshot.manifests().get(0).path());
+ InputFile manifest =
table.io().newInputFile(snapshot.allManifests().get(0).path());
List<GenericData.Record> expected = Lists.newArrayList();
try (CloseableIterable<GenericData.Record> rows =
Avro.read(manifest).project(entriesTable.schema()).build()) {
// each row must inherit snapshot_id and sequence_number
@@ -171,7 +171,7 @@ public abstract class TestIcebergSourceTablesBase {
.collectAsList();
List<GenericData.Record> expected = Lists.newArrayList();
- for (ManifestFile manifest :
Iterables.concat(Iterables.transform(table.snapshots(), Snapshot::manifests))) {
+ for (ManifestFile manifest :
Iterables.concat(Iterables.transform(table.snapshots(),
Snapshot::allManifests))) {
InputFile in = table.io().newInputFile(manifest.path());
try (CloseableIterable<GenericData.Record> rows =
Avro.read(in).project(entriesTable.schema()).build()) {
// each row must inherit snapshot_id and sequence_number
@@ -223,7 +223,7 @@ public abstract class TestIcebergSourceTablesBase {
.collectAsList();
List<GenericData.Record> expected = Lists.newArrayList();
- for (ManifestFile manifest : table.currentSnapshot().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().dataManifests()) {
InputFile in = table.io().newInputFile(manifest.path());
try (CloseableIterable<GenericData.Record> rows =
Avro.read(in).project(entriesTable.schema()).build()) {
for (GenericData.Record record : rows) {
@@ -275,7 +275,7 @@ public abstract class TestIcebergSourceTablesBase {
.collectAsList();
List<GenericData.Record> expected = Lists.newArrayList();
- for (ManifestFile manifest : table.currentSnapshot().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().dataManifests()) {
InputFile in = table.io().newInputFile(manifest.path());
try (CloseableIterable<GenericData.Record> rows =
Avro.read(in).project(entriesTable.schema()).build()) {
for (GenericData.Record record : rows) {
@@ -376,7 +376,7 @@ public abstract class TestIcebergSourceTablesBase {
.collectAsList();
List<GenericData.Record> expected = Lists.newArrayList();
- for (ManifestFile manifest : table.currentSnapshot().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().dataManifests()) {
InputFile in = table.io().newInputFile(manifest.path());
try (CloseableIterable<GenericData.Record> rows =
Avro.read(in).project(entriesTable.schema()).build()) {
for (GenericData.Record record : rows) {
@@ -473,7 +473,7 @@ public abstract class TestIcebergSourceTablesBase {
actual.sort(Comparator.comparing(o -> o.getString(1)));
List<GenericData.Record> expected = Lists.newArrayList();
- for (ManifestFile manifest :
Iterables.concat(Iterables.transform(table.snapshots(), Snapshot::manifests))) {
+ for (ManifestFile manifest :
Iterables.concat(Iterables.transform(table.snapshots(),
Snapshot::dataManifests))) {
InputFile in = table.io().newInputFile(manifest.path());
try (CloseableIterable<GenericData.Record> rows =
Avro.read(in).project(entriesTable.schema()).build()) {
for (GenericData.Record record : rows) {
@@ -661,7 +661,7 @@ public abstract class TestIcebergSourceTablesBase {
manifestTable.schema(), "manifests"));
GenericRecordBuilder summaryBuilder = new
GenericRecordBuilder(AvroSchemaUtil.convert(
manifestTable.schema().findType("partition_summaries.element").asStructType(),
"partition_summary"));
- List<GenericData.Record> expected =
Lists.transform(table.currentSnapshot().manifests(), manifest ->
+ List<GenericData.Record> expected =
Lists.transform(table.currentSnapshot().allManifests(), manifest ->
builder.set("path", manifest.path())
.set("length", manifest.length())
.set("partition_spec_id", manifest.partitionSpecId())
@@ -697,11 +697,11 @@ public abstract class TestIcebergSourceTablesBase {
.mode("append")
.save(loadLocation(tableIdentifier));
- manifests.addAll(table.currentSnapshot().manifests());
+ manifests.addAll(table.currentSnapshot().allManifests());
table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit();
- manifests.addAll(table.currentSnapshot().manifests());
+ manifests.addAll(table.currentSnapshot().allManifests());
List<Row> actual = spark.read()
.format("iceberg")
diff --git
a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java
b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java
index 8a51a7a..a5fae08 100644
---
a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java
+++
b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java
@@ -120,7 +120,7 @@ public class TestSparkDataWrite {
List<SimpleRecord> 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().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().allManifests()) {
for (DataFile file : ManifestFiles.read(manifest, table.io())) {
// TODO: avro not support split
if (!format.equals(FileFormat.AVRO)) {
@@ -315,7 +315,7 @@ public class TestSparkDataWrite {
Assert.assertEquals("Result rows should match", expected, actual);
List<DataFile> files = Lists.newArrayList();
- for (ManifestFile manifest : table.currentSnapshot().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().allManifests()) {
for (DataFile file : ManifestFiles.read(manifest, table.io())) {
files.add(file);
}
@@ -364,7 +364,7 @@ public class TestSparkDataWrite {
Assert.assertEquals("Result rows should match", expected, actual);
List<DataFile> files = Lists.newArrayList();
- for (ManifestFile manifest : table.currentSnapshot().manifests()) {
+ for (ManifestFile manifest : table.currentSnapshot().allManifests()) {
for (DataFile file : ManifestFiles.read(manifest, table.io())) {
files.add(file);
}