From 9e43889cdbdd4c67d53137f3da8ffde255fd70e0 Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Tue, 31 May 2022 22:36:39 -0700 Subject: [PATCH 01/27] Append changes to snapshot branch Append changes to snapshot branch - changes after review comments remove space Adding support methods for supporting performing append/delete on branch Removing spaces Removing spaces Removing unwanted vars Adding check for null and invalid branch Adding check for null and invalid branch --- .../org/apache/iceberg/SnapshotUpdate.java | 6 ++++ .../org/apache/iceberg/SnapshotProducer.java | 7 +++++ .../org/apache/iceberg/TestFastAppend.java | 30 +++++++++++++++++++ 3 files changed, 43 insertions(+) diff --git a/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java b/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java index c1742f82ca84..8df1de705620 100644 --- a/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java +++ b/api/src/main/java/org/apache/iceberg/SnapshotUpdate.java @@ -60,4 +60,10 @@ public interface SnapshotUpdate extends PendingUpdate { * @return this for method chaining */ ThisT scanManifestsWith(ExecutorService executorService); + + /** + * Perform operations on a particular branch + * @param branch which is name of SanshotRef of type branch. + */ + ThisT toBranch(String branch); } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index b6e699a0e24a..776f97cef085 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -116,6 +116,13 @@ public ThisT scanManifestsWith(ExecutorService executorService) { return self(); } + @Override + public ThisT toBranch(String branch){ + Preconditions.checkArgument(branch != null,"branch cannot be null"); + Preconditions.checkArgument(ops.current().ref(branch) != null, "%s is not a valid ref", branch); + throw new UnsupportedOperationException("Performing operations on a branch is currently not supported"); + } + protected ExecutorService workerPool() { return this.workerPool; } diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index e0ffacd73778..6f17f651622d 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -478,4 +478,34 @@ public void testIncludedPartitionSummaryLimit() { String changedPartitions = table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); Assert.assertEquals("Should set changed partition count", "2", changedPartitions); } + + @Test(expected = UnsupportedOperationException.class) + public void testAppendToBranch() throws UnsupportedOperationException { + table.newFastAppend() + .appendFile(FILE_A) + .commit(); + + table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); + table.newDelete().toBranch("ref"); + } + + @Test(expected = IllegalArgumentException.class) + public void testAppendToNullBranch() { + table.newFastAppend() + .appendFile(FILE_A) + .commit(); + + table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); + table.newDelete().toBranch(null).deleteFile(FILE_A); + } + + @Test(expected = IllegalArgumentException.class) + public void testAppendToInValidBranch() { + table.newFastAppend() + .appendFile(FILE_A) + .commit(); + + table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); + table.newDelete().toBranch("newBranch").deleteFile(FILE_A); + } } From e6ea5f62c8e4c94dcf0a08e48c23334300e24172 Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Thu, 9 Jun 2022 15:59:52 -0700 Subject: [PATCH 02/27] Adding snapshot to branch Impl --- .../iceberg/MergingSnapshotProducer.java | 2 +- .../org/apache/iceberg/SnapshotProducer.java | 18 +++++++++++++----- .../org/apache/iceberg/TestFastAppend.java | 9 ++++++--- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index e184b50e8284..1c1b8b72541b 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -683,7 +683,7 @@ protected Map summary() { @Override public List apply(TableMetadata base) { - Snapshot current = base.currentSnapshot(); + Snapshot current = toBranch != null ? base.snapshot(base.ref(toBranch).snapshotId()) : base.currentSnapshot(); // filter any existing manifests List filtered = filterManager.filterManifests( diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 776f97cef085..2f00f9ca3b1e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -89,6 +89,8 @@ public void accept(String file) { private ExecutorService workerPool = ThreadPools.getWorkerPool(); + protected String toBranch = null; + protected SnapshotProducer(TableOperations ops) { this.ops = ops; this.base = ops.current(); @@ -120,7 +122,8 @@ public ThisT scanManifestsWith(ExecutorService executorService) { public ThisT toBranch(String branch){ Preconditions.checkArgument(branch != null,"branch cannot be null"); Preconditions.checkArgument(ops.current().ref(branch) != null, "%s is not a valid ref", branch); - throw new UnsupportedOperationException("Performing operations on a branch is currently not supported"); + this.toBranch = branch; + return self(); } protected ExecutorService workerPool() { @@ -174,8 +177,12 @@ protected void validate(TableMetadata currentMetadata) { @Override public Snapshot apply() { refresh(); - Long parentSnapshotId = base.currentSnapshot() != null ? - base.currentSnapshot().snapshotId() : null; + Long parentSnapshotId = null; + if(toBranch != null){ + parentSnapshotId = base.ref(toBranch).snapshotId(); + } else { + parentSnapshotId = base.currentSnapshot() != null ? base.currentSnapshot().snapshotId() : null; + } long sequenceNumber = base.nextSequenceNumber(); // run validations from the child operation @@ -289,6 +296,7 @@ protected TableMetadata refresh() { @Override public void commit() { // this is always set to the latest commit attempt's snapshot id. + String branch = toBranch == null ? SnapshotRef.MAIN_BRANCH : toBranch; AtomicLong newSnapshotId = new AtomicLong(-1L); try { Tasks.foreach(ops) @@ -305,11 +313,11 @@ public void commit() { TableMetadata.Builder update = TableMetadata.buildFrom(base); if (base.snapshot(newSnapshot.snapshotId()) != null) { // this is a rollback operation - update.setBranchSnapshot(newSnapshot.snapshotId(), SnapshotRef.MAIN_BRANCH); + update.setBranchSnapshot(newSnapshot.snapshotId(), branch); } else if (stageOnly) { update.addSnapshot(newSnapshot); } else { - update.setBranchSnapshot(newSnapshot, SnapshotRef.MAIN_BRANCH); + update.setBranchSnapshot(newSnapshot, branch); } TableMetadata updated = update.build(); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 6f17f651622d..df90e4a9afa6 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -479,14 +479,17 @@ public void testIncludedPartitionSummaryLimit() { Assert.assertEquals("Should set changed partition count", "2", changedPartitions); } - @Test(expected = UnsupportedOperationException.class) + @Test public void testAppendToBranch() throws UnsupportedOperationException { table.newFastAppend() .appendFile(FILE_A) .commit(); + Long currSnapshot = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); - table.newDelete().toBranch("ref"); + table.newFastAppend().toBranch("ref").appendFile(FILE_B).commit(); + Snapshot branch = table.snapshot(table.ops().current().ref("ref").snapshotId()); + Assert.assertEquals(currSnapshot, branch.parentId()); } @Test(expected = IllegalArgumentException.class) @@ -506,6 +509,6 @@ public void testAppendToInValidBranch() { .commit(); table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); - table.newDelete().toBranch("newBranch").deleteFile(FILE_A); + table.newDelete().toBranch("newBranch").deleteFile(FILE_A).commit(); } } From b43b80bc031df229d88a595bea0888d9e087f324 Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Mon, 13 Jun 2022 13:51:59 -0700 Subject: [PATCH 03/27] Adding check to see if it's a branch --- core/src/main/java/org/apache/iceberg/SnapshotProducer.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 2f00f9ca3b1e..f999feacdddd 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -122,6 +122,7 @@ public ThisT scanManifestsWith(ExecutorService executorService) { public ThisT toBranch(String branch){ Preconditions.checkArgument(branch != null,"branch cannot be null"); Preconditions.checkArgument(ops.current().ref(branch) != null, "%s is not a valid ref", branch); + Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, "%s is not a ref to type branch", branch); this.toBranch = branch; return self(); } @@ -177,11 +178,9 @@ protected void validate(TableMetadata currentMetadata) { @Override public Snapshot apply() { refresh(); - Long parentSnapshotId = null; + Long parentSnapshotId = base.currentSnapshot() != null ? base.currentSnapshot().snapshotId() : null; if(toBranch != null){ parentSnapshotId = base.ref(toBranch).snapshotId(); - } else { - parentSnapshotId = base.currentSnapshot() != null ? base.currentSnapshot().snapshotId() : null; } long sequenceNumber = base.nextSequenceNumber(); From 9cb39c2cc679a0114b6262148717620334bc8fcf Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Thu, 16 Jun 2022 13:44:14 -0700 Subject: [PATCH 04/27] Creating branch if ref absent --- core/src/main/java/org/apache/iceberg/SnapshotProducer.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f999feacdddd..f385589034a5 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -121,7 +121,11 @@ public ThisT scanManifestsWith(ExecutorService executorService) { @Override public ThisT toBranch(String branch){ Preconditions.checkArgument(branch != null,"branch cannot be null"); - Preconditions.checkArgument(ops.current().ref(branch) != null, "%s is not a valid ref", branch); + if (ops.current().ref(branch) == null) { + SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); + TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); + updatedBuilder.setRef(branch, branchRef); + } Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, "%s is not a ref to type branch", branch); this.toBranch = branch; return self(); From ba341ec4d6048047aaf40ae1bbef3b92f80b20ed Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Thu, 16 Jun 2022 13:56:24 -0700 Subject: [PATCH 05/27] Building after setting ref --- core/src/main/java/org/apache/iceberg/SnapshotProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f385589034a5..3ddcc67d1bdd 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -124,7 +124,7 @@ public ThisT toBranch(String branch){ if (ops.current().ref(branch) == null) { SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); - updatedBuilder.setRef(branch, branchRef); + updatedBuilder.setRef(branch, branchRef).build(); } Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, "%s is not a ref to type branch", branch); this.toBranch = branch; From d1ca432387506d2d4200d6bd238ec472f334a41c Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Tue, 21 Jun 2022 10:36:10 -0700 Subject: [PATCH 06/27] Setting default branch as main --- .../apache/iceberg/MergingSnapshotProducer.java | 2 +- .../org/apache/iceberg/SnapshotProducer.java | 16 +++++++--------- .../java/org/apache/iceberg/TestFastAppend.java | 12 ++++++------ 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 1c1b8b72541b..a44dad54d21a 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -683,7 +683,7 @@ protected Map summary() { @Override public List apply(TableMetadata base) { - Snapshot current = toBranch != null ? base.snapshot(base.ref(toBranch).snapshotId()) : base.currentSnapshot(); + Snapshot current = base.snapshot(base.ref(targetBranch).snapshotId()); // filter any existing manifests List filtered = filterManager.filterManifests( diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 3ddcc67d1bdd..6191decbfc38 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -89,7 +89,7 @@ public void accept(String file) { private ExecutorService workerPool = ThreadPools.getWorkerPool(); - protected String toBranch = null; + protected String targetBranch = SnapshotRef.MAIN_BRANCH; protected SnapshotProducer(TableOperations ops) { this.ops = ops; @@ -126,8 +126,9 @@ public ThisT toBranch(String branch){ TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); updatedBuilder.setRef(branch, branchRef).build(); } + Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, "%s is not a ref to type branch", branch); - this.toBranch = branch; + this.targetBranch = branch; return self(); } @@ -182,10 +183,8 @@ protected void validate(TableMetadata currentMetadata) { @Override public Snapshot apply() { refresh(); - Long parentSnapshotId = base.currentSnapshot() != null ? base.currentSnapshot().snapshotId() : null; - if(toBranch != null){ - parentSnapshotId = base.ref(toBranch).snapshotId(); - } + Long parentSnapshotId = base.ref(targetBranch) != null ? base.ref(targetBranch).snapshotId() : null; + long sequenceNumber = base.nextSequenceNumber(); // run validations from the child operation @@ -299,7 +298,6 @@ protected TableMetadata refresh() { @Override public void commit() { // this is always set to the latest commit attempt's snapshot id. - String branch = toBranch == null ? SnapshotRef.MAIN_BRANCH : toBranch; AtomicLong newSnapshotId = new AtomicLong(-1L); try { Tasks.foreach(ops) @@ -316,11 +314,11 @@ public void commit() { TableMetadata.Builder update = TableMetadata.buildFrom(base); if (base.snapshot(newSnapshot.snapshotId()) != null) { // this is a rollback operation - update.setBranchSnapshot(newSnapshot.snapshotId(), branch); + update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch); } else if (stageOnly) { update.addSnapshot(newSnapshot); } else { - update.setBranchSnapshot(newSnapshot, branch); + update.setBranchSnapshot(newSnapshot, targetBranch); } TableMetadata updated = update.build(); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index df90e4a9afa6..073b9060b91e 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -482,8 +482,8 @@ public void testIncludedPartitionSummaryLimit() { @Test public void testAppendToBranch() throws UnsupportedOperationException { table.newFastAppend() - .appendFile(FILE_A) - .commit(); + .appendFile(FILE_A) + .commit(); Long currSnapshot = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); @@ -495,8 +495,8 @@ public void testAppendToBranch() throws UnsupportedOperationException { @Test(expected = IllegalArgumentException.class) public void testAppendToNullBranch() { table.newFastAppend() - .appendFile(FILE_A) - .commit(); + .appendFile(FILE_A) + .commit(); table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); table.newDelete().toBranch(null).deleteFile(FILE_A); @@ -505,8 +505,8 @@ public void testAppendToNullBranch() { @Test(expected = IllegalArgumentException.class) public void testAppendToInValidBranch() { table.newFastAppend() - .appendFile(FILE_A) - .commit(); + .appendFile(FILE_A) + .commit(); table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); table.newDelete().toBranch("newBranch").deleteFile(FILE_A).commit(); From 0d79ef05bddf1e00d03f9eb73bdcf58638d6b37a Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Wed, 22 Jun 2022 14:04:32 -0700 Subject: [PATCH 07/27] Fetching current when no main ref is set. --- .../main/java/org/apache/iceberg/MergingSnapshotProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index a44dad54d21a..3679d52b1f8c 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -683,7 +683,7 @@ protected Map summary() { @Override public List apply(TableMetadata base) { - Snapshot current = base.snapshot(base.ref(targetBranch).snapshotId()); + Snapshot current = base.ref(targetBranch) != null ? base.snapshot(base.ref(targetBranch).snapshotId()) : base.currentSnapshot(); // filter any existing manifests List filtered = filterManager.filterManifests( From 495c0f3c086a2cb9aafd0241e4ae44d3e5da2e1d Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Tue, 28 Jun 2022 22:57:41 -0700 Subject: [PATCH 08/27] checkStyle fixes --- .../iceberg/MergingSnapshotProducer.java | 3 ++- .../org/apache/iceberg/SnapshotProducer.java | 27 +++++++++++-------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 3679d52b1f8c..8056fa592b26 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -683,7 +683,8 @@ protected Map summary() { @Override public List apply(TableMetadata base) { - Snapshot current = base.ref(targetBranch) != null ? base.snapshot(base.ref(targetBranch).snapshotId()) : base.currentSnapshot(); + Snapshot current = base.ref(targetBranch()) != null ? + base.snapshot(base.ref(targetBranch()).snapshotId()) : base.currentSnapshot(); // filter any existing manifests List filtered = filterManager.filterManifests( diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 6191decbfc38..36f831c52da9 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -89,7 +89,7 @@ public void accept(String file) { private ExecutorService workerPool = ThreadPools.getWorkerPool(); - protected String targetBranch = SnapshotRef.MAIN_BRANCH; + private String targetBranch = SnapshotRef.MAIN_BRANCH; protected SnapshotProducer(TableOperations ops) { this.ops = ops; @@ -106,6 +106,10 @@ protected SnapshotProducer(TableOperations ops) { protected abstract ThisT self(); + protected String targetBranch() { + return targetBranch; + } + @Override public ThisT stageOnly() { this.stageOnly = true; @@ -119,17 +123,18 @@ public ThisT scanManifestsWith(ExecutorService executorService) { } @Override - public ThisT toBranch(String branch){ - Preconditions.checkArgument(branch != null,"branch cannot be null"); - if (ops.current().ref(branch) == null) { - SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); - TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); - updatedBuilder.setRef(branch, branchRef).build(); - } + public ThisT toBranch(String branch) { + Preconditions.checkArgument(branch != null, "branch cannot be null"); + if (ops.current().ref(branch) == null) { + SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); + TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); + updatedBuilder.setRef(branch, branchRef).build(); + } - Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, "%s is not a ref to type branch", branch); - this.targetBranch = branch; - return self(); + Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, + "%s is not a ref to type branch", branch); + this.targetBranch = branch; + return self(); } protected ExecutorService workerPool() { From f7464ebe52611d01d6c5976403afdf982528ed7a Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Tue, 28 Jun 2022 23:51:54 -0700 Subject: [PATCH 09/27] build fixes --- core/src/main/java/org/apache/iceberg/SnapshotProducer.java | 5 +++-- core/src/test/java/org/apache/iceberg/TestFastAppend.java | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 36f831c52da9..79aa852dc240 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -128,10 +128,11 @@ public ThisT toBranch(String branch) { if (ops.current().ref(branch) == null) { SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); - updatedBuilder.setRef(branch, branchRef).build(); + updatedBuilder.setRef(branch, branchRef); + ops.commit(base, updatedBuilder.build()); } - Preconditions.checkArgument(ops.current().ref(branch).type() != SnapshotRefType.BRANCH, + Preconditions.checkArgument(ops.current().ref(branch).type().equals(SnapshotRefType.BRANCH), "%s is not a ref to type branch", branch); this.targetBranch = branch; return self(); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 073b9060b91e..6af2015fddb2 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -502,7 +502,7 @@ public void testAppendToNullBranch() { table.newDelete().toBranch(null).deleteFile(FILE_A); } - @Test(expected = IllegalArgumentException.class) + @Test public void testAppendToInValidBranch() { table.newFastAppend() .appendFile(FILE_A) @@ -510,5 +510,6 @@ public void testAppendToInValidBranch() { table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); table.newDelete().toBranch("newBranch").deleteFile(FILE_A).commit(); + Assert.assertNotNull(table.ops().current().ref("newBranch")); } } From 39de1e45835ccaf42c5e4168db6b6cddead98cd3 Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Fri, 1 Jul 2022 23:04:31 -0700 Subject: [PATCH 10/27] Validation check for branch --- .../java/org/apache/iceberg/BaseRowDelta.java | 40 ++++++++++++++----- .../java/org/apache/iceberg/TestRowDelta.java | 28 +++++++++++++ 2 files changed, 57 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index b5655fe1fb1e..846f4b1a4b0e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -19,10 +19,12 @@ package org.apache.iceberg; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.SnapshotUtil; class BaseRowDelta extends MergingSnapshotProducer implements RowDelta { private Long startingSnapshotId = null; // check all versions by default @@ -95,21 +97,37 @@ public RowDelta validateNoConflictingDeleteFiles() { return this; } + private void checkIfSnapshotIsAnAncestor(Long startingSnapshotId, Snapshot current, TableMetadata base){ + if(startingSnapshotId == null || current == null){ + return; + } + + for (Snapshot ancestor : SnapshotUtil.ancestorsOf(current.snapshotId(), base::snapshot)) { + if(ancestor.snapshotId() == startingSnapshotId){ + return; + } + + } + throw new ValidationException("Snapshot %s is not an ancestor of branch %s", startingSnapshotId, targetBranch()); + } + @Override protected void validate(TableMetadata base) { - if (base.currentSnapshot() != null) { - if (!referencedDataFiles.isEmpty()) { - validateDataFilesExist( - base, startingSnapshotId, referencedDataFiles, !validateDeletes, conflictDetectionFilter); - } + Snapshot current = base.ref(targetBranch()) != null ? + base.snapshot(base.ref(targetBranch()).snapshotId()) : base.currentSnapshot(); - if (validateNewDataFiles) { - validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter); - } + checkIfSnapshotIsAnAncestor(startingSnapshotId, current, base); + if (!referencedDataFiles.isEmpty()) { + validateDataFilesExist( + base, startingSnapshotId, referencedDataFiles, !validateDeletes, conflictDetectionFilter); + } - if (validateNewDeleteFiles) { - validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter); - } + if (validateNewDataFiles) { + validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter); + } + + if (validateNewDeleteFiles) { + validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter); } } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 4e317b18891a..5d4ba7d37ba5 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1323,4 +1323,32 @@ public void testRowDeltaCaseSensitivity() { .validateNoConflictingDeleteFiles() .commit()); } + + @Test + public void testBranchValidations(){ + table.newAppend() + .appendFile(FILE_A) + .commit(); + + Expression conflictDetectionFilter = Expressions.alwaysTrue(); + + Long firstSnapshot = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().createBranch("newBranch", firstSnapshot).commit(); + + table.newAppend() + .appendFile(FILE_B) + .commit(); + + //This commit will result in validation exception as we start validation from a snapshot which is + // not an ancestor of the branch + RowDelta rowDelta = table.newRowDelta() + .toBranch("newBranch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .conflictDetectionFilter(conflictDetectionFilter) + .validateNoConflictingDeleteFiles(); + + AssertHelpers.assertThrows("No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); + } } From 080d76e8b2e4e07754cf2aed33b77de8e0ee0abb Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Sat, 9 Jul 2022 14:18:58 -0700 Subject: [PATCH 11/27] Restricting toBranch only for BaseRowDelta --- .../java/org/apache/iceberg/BaseRowDelta.java | 21 ++++++++-- .../iceberg/MergingSnapshotProducer.java | 7 ++++ .../org/apache/iceberg/SnapshotProducer.java | 13 ++---- .../org/apache/iceberg/TestFastAppend.java | 33 --------------- .../java/org/apache/iceberg/TestRowDelta.java | 40 ++++++++++++++++++- 5 files changed, 65 insertions(+), 49 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 846f4b1a4b0e..d576603d0ec3 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -97,13 +97,26 @@ public RowDelta validateNoConflictingDeleteFiles() { return this; } - private void checkIfSnapshotIsAnAncestor(Long startingSnapshotId, Snapshot current, TableMetadata base){ - if(startingSnapshotId == null || current == null){ + @Override + public RowDelta toBranch(String branch) { + Preconditions.checkArgument(branch != null, "branch cannot be null"); + if (this.current().ref(branch) == null) { + super.createNewRef(branch); + } + + Preconditions.checkArgument(this.current().ref(branch).type().equals(SnapshotRefType.BRANCH), + "%s is not a ref to type branch", branch); + setTargetBranch(branch); + return self(); + } + + private void checkIfSnapshotIsAnAncestor(Snapshot current, TableMetadata base) { + if (this.startingSnapshotId == null || current == null) { return; } for (Snapshot ancestor : SnapshotUtil.ancestorsOf(current.snapshotId(), base::snapshot)) { - if(ancestor.snapshotId() == startingSnapshotId){ + if (ancestor.snapshotId() == this.startingSnapshotId) { return; } @@ -116,7 +129,7 @@ protected void validate(TableMetadata base) { Snapshot current = base.ref(targetBranch()) != null ? base.snapshot(base.ref(targetBranch()).snapshotId()) : base.currentSnapshot(); - checkIfSnapshotIsAnAncestor(startingSnapshotId, current, base); + checkIfSnapshotIsAnAncestor(current, base); if (!referencedDataFiles.isEmpty()) { validateDataFilesExist( base, startingSnapshotId, referencedDataFiles, !validateDeletes, conflictDetectionFilter); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 8056fa592b26..e95407c806ca 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -164,6 +164,13 @@ protected void failMissingDeletePaths() { deleteFilterManager.failMissingDeletePaths(); } + protected void createNewRef(String branch) { + SnapshotRef branchRef = SnapshotRef.branchBuilder(this.current().currentSnapshot().snapshotId()).build(); + TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(this.current()); + updatedBuilder.setRef(branch, branchRef); + ops.commit(ops.current(), updatedBuilder.build()); + } + /** * Add a filter to match files to delete. A file will be deleted if all of the rows it contains * match this or any other filter passed to this method. diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 355f25a680a1..17d4d6277f72 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -124,18 +124,11 @@ public ThisT scanManifestsWith(ExecutorService executorService) { @Override public ThisT toBranch(String branch) { - Preconditions.checkArgument(branch != null, "branch cannot be null"); - if (ops.current().ref(branch) == null) { - SnapshotRef branchRef = SnapshotRef.branchBuilder(ops.current().currentSnapshot().snapshotId()).build(); - TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); - updatedBuilder.setRef(branch, branchRef); - ops.commit(base, updatedBuilder.build()); - } + throw new UnsupportedOperationException("Performing operations on a branch is is only supported for BaseRowDelta"); + } - Preconditions.checkArgument(ops.current().ref(branch).type().equals(SnapshotRefType.BRANCH), - "%s is not a ref to type branch", branch); + protected void setTargetBranch(String branch) { this.targetBranch = branch; - return self(); } protected ExecutorService workerPool() { diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 6af2015fddb2..484f480db008 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -479,37 +479,4 @@ public void testIncludedPartitionSummaryLimit() { Assert.assertEquals("Should set changed partition count", "2", changedPartitions); } - @Test - public void testAppendToBranch() throws UnsupportedOperationException { - table.newFastAppend() - .appendFile(FILE_A) - .commit(); - - Long currSnapshot = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); - table.newFastAppend().toBranch("ref").appendFile(FILE_B).commit(); - Snapshot branch = table.snapshot(table.ops().current().ref("ref").snapshotId()); - Assert.assertEquals(currSnapshot, branch.parentId()); - } - - @Test(expected = IllegalArgumentException.class) - public void testAppendToNullBranch() { - table.newFastAppend() - .appendFile(FILE_A) - .commit(); - - table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); - table.newDelete().toBranch(null).deleteFile(FILE_A); - } - - @Test - public void testAppendToInValidBranch() { - table.newFastAppend() - .appendFile(FILE_A) - .commit(); - - table.manageSnapshots().createBranch("ref", table.currentSnapshot().snapshotId()).commit(); - table.newDelete().toBranch("newBranch").deleteFile(FILE_A).commit(); - Assert.assertNotNull(table.ops().current().ref("newBranch")); - } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 5d4ba7d37ba5..e892013d29b4 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -28,6 +28,7 @@ import org.apache.iceberg.expressions.Expressions; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.Assert; import org.junit.Test; @@ -1325,7 +1326,7 @@ public void testRowDeltaCaseSensitivity() { } @Test - public void testBranchValidations(){ + public void testBranchValidationsNotValidAncestor() { table.newAppend() .appendFile(FILE_A) .commit(); @@ -1340,7 +1341,7 @@ public void testBranchValidations(){ .appendFile(FILE_B) .commit(); - //This commit will result in validation exception as we start validation from a snapshot which is + // This commit will result in validation exception as we start validation from a snapshot which is // not an ancestor of the branch RowDelta rowDelta = table.newRowDelta() .toBranch("newBranch") @@ -1351,4 +1352,39 @@ public void testBranchValidations(){ AssertHelpers.assertThrows("No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); } + + @Test + public void testBranchValidationsValidAncestor() { + table.newAppend() + .appendFile(FILE_A) + .commit(); + + Expression conflictDetectionFilter = Expressions.alwaysTrue(); + + Long firstSnapshot = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().createBranch("newBranch", firstSnapshot).commit(); + + table.newAppend() + .appendFile(FILE_B) + .commit(); + + // This commit not result in validation exception as we start validation from a snapshot which is + // not an ancestor of the branch + table.newRowDelta() + .toBranch("newBranch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(firstSnapshot) + .conflictDetectionFilter(conflictDetectionFilter) + .validateNoConflictingDeleteFiles().commit(); + + List dataManifests = table.ops().current().snapshot(table.ops().current() + .ref("newBranch").snapshotId()).dataManifests(table.io()); + Assert.assertEquals("branch should have 1 data manifest", 1, Iterables.size(dataManifests)); + List deleteManifests = table.ops().current().snapshot(table.ops().current() + .ref("newBranch").snapshotId()).deleteManifests(table.io()); + Assert.assertEquals("branch should have 1 delete manifest", 1, Iterables.size(deleteManifests)); + List mainBranchManifests = table.currentSnapshot().dataManifests(table.io()); + Assert.assertEquals("main branch should have 2 data manifest", 2, Iterables.size(mainBranchManifests)); + } } From 1dc4f892269affd5403eb096f23cfe04e3c6f7ad Mon Sep 17 00:00:00 2001 From: nkeshavaprakash Date: Sat, 9 Jul 2022 17:19:02 -0700 Subject: [PATCH 12/27] removing new line --- core/src/test/java/org/apache/iceberg/TestFastAppend.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 484f480db008..e0ffacd73778 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -478,5 +478,4 @@ public void testIncludedPartitionSummaryLimit() { String changedPartitions = table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); Assert.assertEquals("Should set changed partition count", "2", changedPartitions); } - } From 41b2f621aaa68c3c1aa8a7ee57fd00167b9f67d8 Mon Sep 17 00:00:00 2001 From: namrathamk <85421654+namrathamk@users.noreply.github.com> Date: Tue, 23 Aug 2022 00:45:29 -0600 Subject: [PATCH 13/27] Updating BaseRowDelta as per SnapshotProducer --- .../apache/iceberg/BaseOverwriteFiles.java | 8 +- .../apache/iceberg/BaseReplacePartitions.java | 15 ++-- .../org/apache/iceberg/BaseRewriteFiles.java | 2 +- .../java/org/apache/iceberg/BaseRowDelta.java | 32 ++++++- .../iceberg/MergingSnapshotProducer.java | 90 ++++++++++++------- .../java/org/apache/iceberg/TestRowDelta.java | 75 +++++++++++++--- 6 files changed, 163 insertions(+), 59 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index a073d79e5552..77b00bcacc27 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -133,19 +133,19 @@ protected void validate(TableMetadata base, Snapshot snapshot) { } if (validateNewDataFiles) { - validateAddedDataFiles(base, startingSnapshotId, dataConflictDetectionFilter()); + validateAddedDataFiles(base, startingSnapshotId, dataConflictDetectionFilter(), snapshot); } if (validateNewDeletes) { if (rowFilter() != Expressions.alwaysFalse()) { Expression filter = conflictDetectionFilter != null ? conflictDetectionFilter : rowFilter(); - validateNoNewDeleteFiles(base, startingSnapshotId, filter); - validateDeletedDataFiles(base, startingSnapshotId, filter); + validateNoNewDeleteFiles(base, startingSnapshotId, filter, snapshot); + validateDeletedDataFiles(base, startingSnapshotId, filter, snapshot); } if (deletedDataFiles.size() > 0) { validateNoNewDeletesForDataFiles( - base, startingSnapshotId, conflictDetectionFilter, deletedDataFiles); + base, startingSnapshotId, conflictDetectionFilter, deletedDataFiles, snapshot); } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java index dd44505e9d39..b651d9d9432e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java +++ b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java @@ -83,19 +83,22 @@ public ReplacePartitions validateNoConflictingData() { public void validate(TableMetadata currentMetadata, Snapshot snapshot) { if (validateConflictingData) { if (dataSpec().isUnpartitioned()) { - validateAddedDataFiles(currentMetadata, startingSnapshotId, Expressions.alwaysTrue()); + validateAddedDataFiles( + currentMetadata, startingSnapshotId, Expressions.alwaysTrue(), snapshot); } else { - validateAddedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions); + validateAddedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions, snapshot); } } if (validateConflictingDeletes) { if (dataSpec().isUnpartitioned()) { - validateDeletedDataFiles(currentMetadata, startingSnapshotId, Expressions.alwaysTrue()); - validateNoNewDeleteFiles(currentMetadata, startingSnapshotId, Expressions.alwaysTrue()); + validateDeletedDataFiles( + currentMetadata, startingSnapshotId, Expressions.alwaysTrue(), snapshot); + validateNoNewDeleteFiles( + currentMetadata, startingSnapshotId, Expressions.alwaysTrue(), snapshot); } else { - validateDeletedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions); - validateNoNewDeleteFiles(currentMetadata, startingSnapshotId, replacedPartitions); + validateDeletedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions, snapshot); + validateNoNewDeleteFiles(currentMetadata, startingSnapshotId, replacedPartitions, snapshot); } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java index 8a3b137b2d3d..646c9ce8c6ed 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java @@ -114,7 +114,7 @@ protected void validate(TableMetadata base, Snapshot snapshot) { if (replacedDataFiles.size() > 0) { // if there are replaced data files, there cannot be any new row-level deletes for those data // files - validateNoNewDeletesForDataFiles(base, startingSnapshotId, replacedDataFiles); + validateNoNewDeletesForDataFiles(base, startingSnapshotId, replacedDataFiles, snapshot); } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 50a0e26ab368..c32ed184cff4 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.SnapshotUtil; class BaseRowDelta extends MergingSnapshotProducer implements RowDelta { private Long startingSnapshotId = null; // check all versions by default @@ -95,24 +97,46 @@ public RowDelta validateNoConflictingDeleteFiles() { return this; } + @Override + public RowDelta toBranch(String branch) { + targetBranch(branch); + return this; + } + + private void checkIfSnapshotIsAnAncestor(Snapshot snapshot, TableMetadata base) { + if (this.startingSnapshotId == null || snapshot == null) { + return; + } + + for (Snapshot ancestor : SnapshotUtil.ancestorsOf(snapshot.snapshotId(), base::snapshot)) { + if (ancestor.snapshotId() == this.startingSnapshotId) { + return; + } + } + throw new ValidationException( + "Snapshot %s is not an ancestor of %s", startingSnapshotId, snapshot.snapshotId()); + } + @Override protected void validate(TableMetadata base, Snapshot snapshot) { - if (base.currentSnapshot() != null) { + if (snapshot != null) { + checkIfSnapshotIsAnAncestor(snapshot, base); if (!referencedDataFiles.isEmpty()) { validateDataFilesExist( base, startingSnapshotId, referencedDataFiles, !validateDeletes, - conflictDetectionFilter); + conflictDetectionFilter, + snapshot); } if (validateNewDataFiles) { - validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter); + validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter, snapshot); } if (validateNewDeleteFiles) { - validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter); + validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter, snapshot); } } } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index b82244f0714f..7a9698f0d498 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -276,9 +276,9 @@ private ManifestFile copyManifest(ManifestFile manifest) { * @param partitionSet a set of partitions to filter new conflicting data files */ protected void validateAddedDataFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { CloseableIterable> conflictEntries = - addedDataFiles(base, startingSnapshotId, null, partitionSet); + addedDataFiles(base, startingSnapshotId, null, partitionSet, snapshot); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -304,9 +304,12 @@ protected void validateAddedDataFiles( * @param conflictDetectionFilter an expression used to find new conflicting data files */ protected void validateAddedDataFiles( - TableMetadata base, Long startingSnapshotId, Expression conflictDetectionFilter) { + TableMetadata base, + Long startingSnapshotId, + Expression conflictDetectionFilter, + Snapshot snapshot) { CloseableIterable> conflictEntries = - addedDataFiles(base, startingSnapshotId, conflictDetectionFilter, null); + addedDataFiles(base, startingSnapshotId, conflictDetectionFilter, null, snapshot); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -336,15 +339,20 @@ private CloseableIterable> addedDataFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, - PartitionSet partitionSet) { + PartitionSet partitionSet, + Snapshot snapshot) { // if there is no current table state, no files have been added - if (base.currentSnapshot() == null) { + if (snapshot == null) { return CloseableIterable.empty(); } Pair, Set> history = validationHistory( - base, startingSnapshotId, VALIDATE_ADDED_FILES_OPERATIONS, ManifestContent.DATA); + base, + startingSnapshotId, + VALIDATE_ADDED_FILES_OPERATIONS, + ManifestContent.DATA, + snapshot); List manifests = history.first(); Set newSnapshots = history.second(); @@ -378,9 +386,12 @@ private CloseableIterable> addedDataFiles( * @param dataFiles data files to validate have no new row deletes */ protected void validateNoNewDeletesForDataFiles( - TableMetadata base, Long startingSnapshotId, Iterable dataFiles) { + TableMetadata base, + Long startingSnapshotId, + Iterable dataFiles, + Snapshot snapshot) { validateNoNewDeletesForDataFiles( - base, startingSnapshotId, null, dataFiles, newFilesSequenceNumber != null); + base, startingSnapshotId, null, dataFiles, newFilesSequenceNumber != null, snapshot); } /** @@ -396,8 +407,10 @@ protected void validateNoNewDeletesForDataFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, - Iterable dataFiles) { - validateNoNewDeletesForDataFiles(base, startingSnapshotId, dataFilter, dataFiles, false); + Iterable dataFiles, + Snapshot snapshot) { + validateNoNewDeletesForDataFiles( + base, startingSnapshotId, dataFilter, dataFiles, false, snapshot); } /** @@ -422,13 +435,15 @@ private void validateNoNewDeletesForDataFiles( Long startingSnapshotId, Expression dataFilter, Iterable dataFiles, - boolean ignoreEqualityDeletes) { + boolean ignoreEqualityDeletes, + Snapshot snapshot) { // if there is no current table state, no files have been added if (base.currentSnapshot() == null || base.formatVersion() < 2) { return; } - DeleteFileIndex deletes = addedDeleteFiles(base, startingSnapshotId, dataFilter, null); + DeleteFileIndex deletes = + addedDeleteFiles(base, startingSnapshotId, dataFilter, null, snapshot); long startingSequenceNumber = startingSequenceNumber(base, startingSnapshotId); for (DataFile dataFile : dataFiles) { @@ -459,8 +474,9 @@ private void validateNoNewDeletesForDataFiles( * @param dataFilter an expression used to find new conflicting delete files */ protected void validateNoNewDeleteFiles( - TableMetadata base, Long startingSnapshotId, Expression dataFilter) { - DeleteFileIndex deletes = addedDeleteFiles(base, startingSnapshotId, dataFilter, null); + TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot snapshot) { + DeleteFileIndex deletes = + addedDeleteFiles(base, startingSnapshotId, dataFilter, null, snapshot); ValidationException.check( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", @@ -477,8 +493,9 @@ protected void validateNoNewDeleteFiles( * @param partitionSet a partition set used to find new conflicting delete files */ protected void validateNoNewDeleteFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { - DeleteFileIndex deletes = addedDeleteFiles(base, startingSnapshotId, null, partitionSet); + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { + DeleteFileIndex deletes = + addedDeleteFiles(base, startingSnapshotId, null, partitionSet, snapshot); ValidationException.check( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", @@ -498,9 +515,10 @@ protected DeleteFileIndex addedDeleteFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, - PartitionSet partitionSet) { + PartitionSet partitionSet, + Snapshot snapshot) { // if there is no current table state, return empty delete file index - if (base.currentSnapshot() == null || base.formatVersion() < 2) { + if (snapshot == null || base.formatVersion() < 2) { return DeleteFileIndex.builderFor(ops.io(), ImmutableList.of()) .specsById(base.specsById()) .build(); @@ -511,7 +529,8 @@ protected DeleteFileIndex addedDeleteFiles( base, startingSnapshotId, VALIDATE_ADDED_DELETE_FILES_OPERATIONS, - ManifestContent.DELETES); + ManifestContent.DELETES, + snapshot); List deleteManifests = history.first(); long startingSequenceNumber = startingSequenceNumber(base, startingSnapshotId); @@ -527,9 +546,9 @@ protected DeleteFileIndex addedDeleteFiles( * @param dataFilter an expression used to find deleted data files */ protected void validateDeletedDataFiles( - TableMetadata base, Long startingSnapshotId, Expression dataFilter) { + TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot snapshot) { CloseableIterable> conflictEntries = - deletedDataFiles(base, startingSnapshotId, dataFilter, null); + deletedDataFiles(base, startingSnapshotId, dataFilter, null, snapshot); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -555,9 +574,9 @@ protected void validateDeletedDataFiles( * @param partitionSet a partition set used to find deleted data files */ protected void validateDeletedDataFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { CloseableIterable> conflictEntries = - deletedDataFiles(base, startingSnapshotId, null, partitionSet); + deletedDataFiles(base, startingSnapshotId, null, partitionSet, snapshot); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -587,7 +606,8 @@ private CloseableIterable> deletedDataFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, - PartitionSet partitionSet) { + PartitionSet partitionSet, + Snapshot snapshot) { // if there is no current table state, no files have been deleted if (base.currentSnapshot() == null) { return CloseableIterable.empty(); @@ -595,7 +615,11 @@ private CloseableIterable> deletedDataFiles( Pair, Set> history = validationHistory( - base, startingSnapshotId, VALIDATE_DATA_FILES_EXIST_OPERATIONS, ManifestContent.DATA); + base, + startingSnapshotId, + VALIDATE_DATA_FILES_EXIST_OPERATIONS, + ManifestContent.DATA, + snapshot); List manifests = history.first(); Set newSnapshots = history.second(); @@ -661,9 +685,10 @@ protected void validateDataFilesExist( Long startingSnapshotId, CharSequenceSet requiredDataFiles, boolean skipDeletes, - Expression conflictDetectionFilter) { + Expression conflictDetectionFilter, + Snapshot snapshot) { // if there is no current table state, no files have been removed - if (base.currentSnapshot() == null) { + if (snapshot == null) { return; } @@ -673,7 +698,8 @@ protected void validateDataFilesExist( : VALIDATE_DATA_FILES_EXIST_OPERATIONS; Pair, Set> history = - validationHistory(base, startingSnapshotId, matchingOperations, ManifestContent.DATA); + validationHistory( + base, startingSnapshotId, matchingOperations, ManifestContent.DATA, snapshot); List manifests = history.first(); Set newSnapshots = history.second(); @@ -709,14 +735,14 @@ private Pair, Set> validationHistory( TableMetadata base, Long startingSnapshotId, Set matchingOperations, - ManifestContent content) { + ManifestContent content, + Snapshot snapshot) { List manifests = Lists.newArrayList(); Set newSnapshots = Sets.newHashSet(); Snapshot lastSnapshot = null; Iterable snapshots = - SnapshotUtil.ancestorsBetween( - base.currentSnapshot().snapshotId(), startingSnapshotId, base::snapshot); + SnapshotUtil.ancestorsBetween(snapshot.snapshotId(), startingSnapshotId, base::snapshot); for (Snapshot currentSnapshot : snapshots) { lastSnapshot = currentSnapshot; diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index e2929b470994..ee99576da5f1 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1414,18 +1414,69 @@ public void testRowDeltaCaseSensitivity() { } @Test - public void testRowDeltaToBranchUnsupported() { + public void testBranchValidationsNotValidAncestor() { + table.newAppend().appendFile(FILE_A).commit(); + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + table.newAppend().appendFile(FILE_B).commit(); + + // This commit will result in validation exception as we start validation from a snapshot which + // is not an ancestor of the branch + RowDelta rowDelta = + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles(); + AssertHelpers.assertThrows( - "Should reject committing row delta to branch", - UnsupportedOperationException.class, - "Cannot commit to branch someBranch: org.apache.iceberg.BaseRowDelta does not support branch commits", - () -> - table - .newRowDelta() - .caseSensitive(false) - .addRows(FILE_B) - .addDeletes(FILE_A2_DELETES) - .toBranch("someBranch") - .commit()); + "No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); + } + + @Test + public void testBranchValidationsValidAncestor() { + table.newAppend().appendFile(FILE_A).commit(); + Long ancestorSnapshot = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch("branch", ancestorSnapshot).commit(); + + // This commit not result in validation exception as we start validation from a snapshot which + // is an actual ancestor of the branch + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(ancestorSnapshot) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles() + .commit(); + + int branchSnapshot = 2; + + Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); + Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); + } + + @Test + public void testBranchConflictingDeletes() { + table.newAppend().appendFile(FILE_A).commit(); + Long ancestorSnapshot = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch("branch", ancestorSnapshot).commit(); + + // This commit not result in validation exception as we start validation from a snapshot which + // is an actual ancestor of the branch + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(ancestorSnapshot) + .conflictDetectionFilter(Expressions.equal("data", "a")) + .validateNoConflictingDeleteFiles() + .commit(); + + int branchSnapshot = 2; + + Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); + Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); } } From 49b6667566f3aa7a90e3ccd297a2a7b7e270f577 Mon Sep 17 00:00:00 2001 From: namrathamk <85421654+namrathamk@users.noreply.github.com> Date: Tue, 23 Aug 2022 00:59:23 -0600 Subject: [PATCH 14/27] Updating BaseRowDelta as per SnapshotProducer --- .../java/org/apache/iceberg/TestRowDelta.java | 81 +++++++------------ 1 file changed, 31 insertions(+), 50 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 1bf711ff8d50..1d52431249d9 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1439,64 +1439,45 @@ public void testBranchConflictingDeletes() { @Test public void testBranchValidationsNotValidAncestor() { - table.newAppend() - .appendFile(FILE_A) - .commit(); - - Expression conflictDetectionFilter = Expressions.alwaysTrue(); - - Long firstSnapshot = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().createBranch("newBranch", firstSnapshot).commit(); - - table.newAppend() - .appendFile(FILE_B) - .commit(); + table.newAppend().appendFile(FILE_A).commit(); + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + table.newAppend().appendFile(FILE_B).commit(); - // This commit will result in validation exception as we start validation from a snapshot which is - // not an ancestor of the branch - RowDelta rowDelta = table.newRowDelta() - .toBranch("newBranch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(table.currentSnapshot().snapshotId()) - .conflictDetectionFilter(conflictDetectionFilter) - .validateNoConflictingDeleteFiles(); + // This commit will result in validation exception as we start validation from a snapshot which + // is not an ancestor of the branch + RowDelta rowDelta = + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles(); - AssertHelpers.assertThrows("No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); + AssertHelpers.assertThrows( + "No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); } @Test public void testBranchValidationsValidAncestor() { - table.newAppend() - .appendFile(FILE_A) - .commit(); - - Expression conflictDetectionFilter = Expressions.alwaysTrue(); - - Long firstSnapshot = table.currentSnapshot().snapshotId(); + table.newAppend().appendFile(FILE_A).commit(); + Long ancestorSnapshot = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch("branch", ancestorSnapshot).commit(); - table.manageSnapshots().createBranch("newBranch", firstSnapshot).commit(); + // This commit not result in validation exception as we start validation from a snapshot which + // is an actual ancestor of the branch + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(ancestorSnapshot) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles() + .commit(); - table.newAppend() - .appendFile(FILE_B) - .commit(); + int branchSnapshot = 2; - // This commit not result in validation exception as we start validation from a snapshot which is - // not an ancestor of the branch - table.newRowDelta() - .toBranch("newBranch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(firstSnapshot) - .conflictDetectionFilter(conflictDetectionFilter) - .validateNoConflictingDeleteFiles().commit(); - - List dataManifests = table.ops().current().snapshot(table.ops().current() - .ref("newBranch").snapshotId()).dataManifests(table.io()); - Assert.assertEquals("branch should have 1 data manifest", 1, Iterables.size(dataManifests)); - List deleteManifests = table.ops().current().snapshot(table.ops().current() - .ref("newBranch").snapshotId()).deleteManifests(table.io()); - Assert.assertEquals("branch should have 1 delete manifest", 1, Iterables.size(deleteManifests)); - List mainBranchManifests = table.currentSnapshot().dataManifests(table.io()); - Assert.assertEquals("main branch should have 2 data manifest", 2, Iterables.size(mainBranchManifests)); + Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); + Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); } } From cfc5e67a65a504ac187eb211601edc467b22dfdd Mon Sep 17 00:00:00 2001 From: namrathamk <85421654+namrathamk@users.noreply.github.com> Date: Sun, 28 Aug 2022 11:21:42 -0600 Subject: [PATCH 15/27] Updating BaseRowDelta as per SnapshotProducer --- .../java/org/apache/iceberg/BaseRowDelta.java | 20 ++---- .../iceberg/MergingSnapshotProducer.java | 67 +++++++++---------- .../java/org/apache/iceberg/TestRowDelta.java | 45 ++++++------- 3 files changed, 58 insertions(+), 74 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index c32ed184cff4..898158fe3f94 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -103,24 +103,14 @@ public RowDelta toBranch(String branch) { return this; } - private void checkIfSnapshotIsAnAncestor(Snapshot snapshot, TableMetadata base) { - if (this.startingSnapshotId == null || snapshot == null) { - return; - } - - for (Snapshot ancestor : SnapshotUtil.ancestorsOf(snapshot.snapshotId(), base::snapshot)) { - if (ancestor.snapshotId() == this.startingSnapshotId) { - return; - } - } - throw new ValidationException( - "Snapshot %s is not an ancestor of %s", startingSnapshotId, snapshot.snapshotId()); - } - @Override protected void validate(TableMetadata base, Snapshot snapshot) { if (snapshot != null) { - checkIfSnapshotIsAnAncestor(snapshot, base); + Preconditions.checkArgument( + SnapshotUtil.isAncestorOf(snapshot.snapshotId(), startingSnapshotId, base::snapshot), + "Snapshot %s is not an ancestor of %s", + startingSnapshotId, + snapshot.snapshotId()); if (!referencedDataFiles.isEmpty()) { validateDataFilesExist( base, diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 7a9698f0d498..19cf1e79024e 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -276,9 +276,9 @@ private ManifestFile copyManifest(ManifestFile manifest) { * @param partitionSet a set of partitions to filter new conflicting data files */ protected void validateAddedDataFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { CloseableIterable> conflictEntries = - addedDataFiles(base, startingSnapshotId, null, partitionSet, snapshot); + addedDataFiles(base, startingSnapshotId, null, partitionSet, parent); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -307,9 +307,9 @@ protected void validateAddedDataFiles( TableMetadata base, Long startingSnapshotId, Expression conflictDetectionFilter, - Snapshot snapshot) { + Snapshot parent) { CloseableIterable> conflictEntries = - addedDataFiles(base, startingSnapshotId, conflictDetectionFilter, null, snapshot); + addedDataFiles(base, startingSnapshotId, conflictDetectionFilter, null, parent); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -340,9 +340,9 @@ private CloseableIterable> addedDataFiles( Long startingSnapshotId, Expression dataFilter, PartitionSet partitionSet, - Snapshot snapshot) { + Snapshot parent) { // if there is no current table state, no files have been added - if (snapshot == null) { + if (parent == null) { return CloseableIterable.empty(); } @@ -352,7 +352,7 @@ private CloseableIterable> addedDataFiles( startingSnapshotId, VALIDATE_ADDED_FILES_OPERATIONS, ManifestContent.DATA, - snapshot); + parent); List manifests = history.first(); Set newSnapshots = history.second(); @@ -386,12 +386,9 @@ private CloseableIterable> addedDataFiles( * @param dataFiles data files to validate have no new row deletes */ protected void validateNoNewDeletesForDataFiles( - TableMetadata base, - Long startingSnapshotId, - Iterable dataFiles, - Snapshot snapshot) { + TableMetadata base, Long startingSnapshotId, Iterable dataFiles, Snapshot parent) { validateNoNewDeletesForDataFiles( - base, startingSnapshotId, null, dataFiles, newFilesSequenceNumber != null, snapshot); + base, startingSnapshotId, null, dataFiles, newFilesSequenceNumber != null, parent); } /** @@ -408,9 +405,9 @@ protected void validateNoNewDeletesForDataFiles( Long startingSnapshotId, Expression dataFilter, Iterable dataFiles, - Snapshot snapshot) { + Snapshot parent) { validateNoNewDeletesForDataFiles( - base, startingSnapshotId, dataFilter, dataFiles, false, snapshot); + base, startingSnapshotId, dataFilter, dataFiles, false, parent); } /** @@ -436,14 +433,13 @@ private void validateNoNewDeletesForDataFiles( Expression dataFilter, Iterable dataFiles, boolean ignoreEqualityDeletes, - Snapshot snapshot) { + Snapshot parent) { // if there is no current table state, no files have been added if (base.currentSnapshot() == null || base.formatVersion() < 2) { return; } - DeleteFileIndex deletes = - addedDeleteFiles(base, startingSnapshotId, dataFilter, null, snapshot); + DeleteFileIndex deletes = addedDeleteFiles(base, startingSnapshotId, dataFilter, null, parent); long startingSequenceNumber = startingSequenceNumber(base, startingSnapshotId); for (DataFile dataFile : dataFiles) { @@ -474,9 +470,8 @@ private void validateNoNewDeletesForDataFiles( * @param dataFilter an expression used to find new conflicting delete files */ protected void validateNoNewDeleteFiles( - TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot snapshot) { - DeleteFileIndex deletes = - addedDeleteFiles(base, startingSnapshotId, dataFilter, null, snapshot); + TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot parent) { + DeleteFileIndex deletes = addedDeleteFiles(base, startingSnapshotId, dataFilter, null, parent); ValidationException.check( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", @@ -493,9 +488,9 @@ protected void validateNoNewDeleteFiles( * @param partitionSet a partition set used to find new conflicting delete files */ protected void validateNoNewDeleteFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { DeleteFileIndex deletes = - addedDeleteFiles(base, startingSnapshotId, null, partitionSet, snapshot); + addedDeleteFiles(base, startingSnapshotId, null, partitionSet, parent); ValidationException.check( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", @@ -516,9 +511,9 @@ protected DeleteFileIndex addedDeleteFiles( Long startingSnapshotId, Expression dataFilter, PartitionSet partitionSet, - Snapshot snapshot) { + Snapshot parent) { // if there is no current table state, return empty delete file index - if (snapshot == null || base.formatVersion() < 2) { + if (parent == null || base.formatVersion() < 2) { return DeleteFileIndex.builderFor(ops.io(), ImmutableList.of()) .specsById(base.specsById()) .build(); @@ -530,7 +525,7 @@ protected DeleteFileIndex addedDeleteFiles( startingSnapshotId, VALIDATE_ADDED_DELETE_FILES_OPERATIONS, ManifestContent.DELETES, - snapshot); + parent); List deleteManifests = history.first(); long startingSequenceNumber = startingSequenceNumber(base, startingSnapshotId); @@ -546,9 +541,9 @@ protected DeleteFileIndex addedDeleteFiles( * @param dataFilter an expression used to find deleted data files */ protected void validateDeletedDataFiles( - TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot snapshot) { + TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot parent) { CloseableIterable> conflictEntries = - deletedDataFiles(base, startingSnapshotId, dataFilter, null, snapshot); + deletedDataFiles(base, startingSnapshotId, dataFilter, null, parent); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -574,9 +569,9 @@ protected void validateDeletedDataFiles( * @param partitionSet a partition set used to find deleted data files */ protected void validateDeletedDataFiles( - TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot snapshot) { + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { CloseableIterable> conflictEntries = - deletedDataFiles(base, startingSnapshotId, null, partitionSet, snapshot); + deletedDataFiles(base, startingSnapshotId, null, partitionSet, parent); try (CloseableIterator> conflicts = conflictEntries.iterator()) { if (conflicts.hasNext()) { @@ -607,7 +602,7 @@ private CloseableIterable> deletedDataFiles( Long startingSnapshotId, Expression dataFilter, PartitionSet partitionSet, - Snapshot snapshot) { + Snapshot parent) { // if there is no current table state, no files have been deleted if (base.currentSnapshot() == null) { return CloseableIterable.empty(); @@ -619,7 +614,7 @@ private CloseableIterable> deletedDataFiles( startingSnapshotId, VALIDATE_DATA_FILES_EXIST_OPERATIONS, ManifestContent.DATA, - snapshot); + parent); List manifests = history.first(); Set newSnapshots = history.second(); @@ -686,9 +681,9 @@ protected void validateDataFilesExist( CharSequenceSet requiredDataFiles, boolean skipDeletes, Expression conflictDetectionFilter, - Snapshot snapshot) { + Snapshot parent) { // if there is no current table state, no files have been removed - if (snapshot == null) { + if (parent == null) { return; } @@ -699,7 +694,7 @@ protected void validateDataFilesExist( Pair, Set> history = validationHistory( - base, startingSnapshotId, matchingOperations, ManifestContent.DATA, snapshot); + base, startingSnapshotId, matchingOperations, ManifestContent.DATA, parent); List manifests = history.first(); Set newSnapshots = history.second(); @@ -736,13 +731,13 @@ private Pair, Set> validationHistory( Long startingSnapshotId, Set matchingOperations, ManifestContent content, - Snapshot snapshot) { + Snapshot parent) { List manifests = Lists.newArrayList(); Set newSnapshots = Sets.newHashSet(); Snapshot lastSnapshot = null; Iterable snapshots = - SnapshotUtil.ancestorsBetween(snapshot.snapshotId(), startingSnapshotId, base::snapshot); + SnapshotUtil.ancestorsBetween(parent.snapshotId(), startingSnapshotId, base::snapshot); for (Snapshot currentSnapshot : snapshots) { lastSnapshot = currentSnapshot; diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 1d52431249d9..2c44d2518157 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -36,7 +36,6 @@ import org.apache.iceberg.expressions.Expressions; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.Assert; import org.junit.Test; @@ -1423,13 +1422,13 @@ public void testBranchConflictingDeletes() { // This commit not result in validation exception as we start validation from a snapshot which // is an actual ancestor of the branch table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(ancestorSnapshot) - .conflictDetectionFilter(Expressions.equal("data", "a")) - .validateNoConflictingDeleteFiles() - .commit(); + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(ancestorSnapshot) + .conflictDetectionFilter(Expressions.equal("data", "a")) + .validateNoConflictingDeleteFiles() + .commit(); int branchSnapshot = 2; @@ -1446,16 +1445,16 @@ public void testBranchValidationsNotValidAncestor() { // This commit will result in validation exception as we start validation from a snapshot which // is not an ancestor of the branch RowDelta rowDelta = - table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(table.currentSnapshot().snapshotId()) - .conflictDetectionFilter(Expressions.alwaysTrue()) - .validateNoConflictingDeleteFiles(); + table + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles(); AssertHelpers.assertThrows( - "No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); + "No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); } @Test @@ -1467,13 +1466,13 @@ public void testBranchValidationsValidAncestor() { // This commit not result in validation exception as we start validation from a snapshot which // is an actual ancestor of the branch table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(ancestorSnapshot) - .conflictDetectionFilter(Expressions.alwaysTrue()) - .validateNoConflictingDeleteFiles() - .commit(); + .newRowDelta() + .toBranch("branch") + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(ancestorSnapshot) + .conflictDetectionFilter(Expressions.alwaysTrue()) + .validateNoConflictingDeleteFiles() + .commit(); int branchSnapshot = 2; From 70a3cf3db406e93f218ffd80c3431f6a35636974 Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Sun, 28 Aug 2022 14:52:40 -0600 Subject: [PATCH 16/27] build fixes --- .../java/org/apache/iceberg/BaseRowDelta.java | 13 ++++----- .../java/org/apache/iceberg/TestRowDelta.java | 27 +++---------------- 2 files changed, 10 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 898158fe3f94..ccc83eb423da 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -106,11 +105,13 @@ public RowDelta toBranch(String branch) { @Override protected void validate(TableMetadata base, Snapshot snapshot) { if (snapshot != null) { - Preconditions.checkArgument( - SnapshotUtil.isAncestorOf(snapshot.snapshotId(), startingSnapshotId, base::snapshot), - "Snapshot %s is not an ancestor of %s", - startingSnapshotId, - snapshot.snapshotId()); + if (startingSnapshotId != null) { + Preconditions.checkArgument( + SnapshotUtil.isAncestorOf(snapshot.snapshotId(), startingSnapshotId, base::snapshot), + "Snapshot %s is not an ancestor of %s", + startingSnapshotId, + snapshot.snapshotId()); + } if (!referencedDataFiles.isEmpty()) { validateDataFilesExist( base, diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 2c44d2518157..0e6e86628d05 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1413,29 +1413,6 @@ public void testRowDeltaCaseSensitivity() { .commit()); } - @Test - public void testBranchConflictingDeletes() { - table.newAppend().appendFile(FILE_A).commit(); - Long ancestorSnapshot = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createBranch("branch", ancestorSnapshot).commit(); - - // This commit not result in validation exception as we start validation from a snapshot which - // is an actual ancestor of the branch - table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(ancestorSnapshot) - .conflictDetectionFilter(Expressions.equal("data", "a")) - .validateNoConflictingDeleteFiles() - .commit(); - - int branchSnapshot = 2; - - Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); - Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); - } - @Test public void testBranchValidationsNotValidAncestor() { table.newAppend().appendFile(FILE_A).commit(); @@ -1454,7 +1431,9 @@ public void testBranchValidationsNotValidAncestor() { .validateNoConflictingDeleteFiles(); AssertHelpers.assertThrows( - "No matching ancestor found", ValidationException.class, () -> rowDelta.commit()); + "Snapshot 2 is not an ancestor of 1", + IllegalArgumentException.class, + () -> rowDelta.commit()); } @Test From f9b3d671fb69d9ab8b61cd7ea67db61d06803e91 Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Thu, 5 Jan 2023 16:43:36 -0800 Subject: [PATCH 17/27] fixing tests to take parameters, Snapshot summary fix --- .../java/org/apache/iceberg/BaseRowDelta.java | 14 +- .../org/apache/iceberg/SnapshotProducer.java | 21 +- .../org/apache/iceberg/TableTestBase.java | 14 + .../java/org/apache/iceberg/TestRowDelta.java | 517 ++++++++++-------- 4 files changed, 333 insertions(+), 233 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index ccc83eb423da..a8e60045ac4f 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -103,14 +103,14 @@ public RowDelta toBranch(String branch) { } @Override - protected void validate(TableMetadata base, Snapshot snapshot) { - if (snapshot != null) { + protected void validate(TableMetadata base, Snapshot parent) { + if (parent != null) { if (startingSnapshotId != null) { Preconditions.checkArgument( - SnapshotUtil.isAncestorOf(snapshot.snapshotId(), startingSnapshotId, base::snapshot), + SnapshotUtil.isAncestorOf(parent.snapshotId(), startingSnapshotId, base::snapshot), "Snapshot %s is not an ancestor of %s", startingSnapshotId, - snapshot.snapshotId()); + parent.snapshotId()); } if (!referencedDataFiles.isEmpty()) { validateDataFilesExist( @@ -119,15 +119,15 @@ protected void validate(TableMetadata base, Snapshot snapshot) { referencedDataFiles, !validateDeletes, conflictDetectionFilter, - snapshot); + parent); } if (validateNewDataFiles) { - validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter, snapshot); + validateAddedDataFiles(base, startingSnapshotId, conflictDetectionFilter, parent); } if (validateNewDeleteFiles) { - validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter, snapshot); + validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter, parent); } } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f11cad4e7984..f2b479601c8e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -267,7 +267,26 @@ private Map summary(TableMetadata previous) { } Map previousSummary; - if (previous.currentSnapshot() != null) { + if (!targetBranch.equals(SnapshotRef.MAIN_BRANCH)) { + if (previous.ref(targetBranch) != null) { + if (previous.snapshot(previous.ref(targetBranch).snapshotId()).summary() != null) { + previousSummary = previous.snapshot(previous.ref(targetBranch).snapshotId()).summary(); + } else { + previousSummary = ImmutableMap.of(); + } + } else { + // if there was no previous snapshot, default the summary to start totals at 0 + ImmutableMap.Builder summaryBuilder = ImmutableMap.builder(); + summaryBuilder + .put(SnapshotSummary.TOTAL_RECORDS_PROP, "0") + .put(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "0") + .put(SnapshotSummary.TOTAL_DATA_FILES_PROP, "0") + .put(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .put(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .put(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0"); + previousSummary = summaryBuilder.build(); + } + } else if (previous.currentSnapshot() != null) { if (previous.currentSnapshot().summary() != null) { previousSummary = previous.currentSnapshot().summary(); } else { diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index e118b4200041..929397982b5a 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -472,6 +472,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) { Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); } + void validateTableDeleteFilesWithRef(Table tbl, String ref, DeleteFile... expectedFiles) { + Set expectedFilePaths = Sets.newHashSet(); + for (DeleteFile file : expectedFiles) { + expectedFilePaths.add(file.path()); + } + Set actualFilePaths = Sets.newHashSet(); + for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { + for (DeleteFile file : task.deletes()) { + actualFilePaths.add(file.path()); + } + } + Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); + } + List paths(DataFile... dataFiles) { List paths = Lists.newArrayListWithExpectedSize(dataFiles.length); for (DataFile file : dataFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 7f9cc3dfac77..4ede90754b7c 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -39,18 +39,32 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(Parameterized.class) public class TestRowDelta extends V2TableTestBase { + + private final String branch; + + @Parameterized.Parameters(name = "branch = {0}") + public static Object[] parameters() { + return new Object[][] { + new Object[] {"main"}, new Object[] {"testBranch"}, + }; + } + + public TestRowDelta(String branch) { + this.branch = branch; + } + @Test public void testAddDeleteFile() { - table - .newRowDelta() - .addRows(FILE_A) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + SnapshotUpdate rowDelta = + table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); - Snapshot snap = table.currentSnapshot(); + commit(table, rowDelta, branch); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 1", 1, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); @@ -81,155 +95,171 @@ public void testAddDeleteFile() { @Test public void testValidateDataFilesExistDefaults() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); + + commit(table, rowDelta1, branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // overwrite FILE_A - table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2).commit(); + SnapshotUpdate rowDelta2 = table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2); + + commit(table, rowDelta2, branch); // delete FILE_B - table.newDelete().deleteFile(FILE_B).commit(); + SnapshotUpdate rowDelta3 = table.newDelete().deleteFile(FILE_B); + + commit(table, rowDelta3, branch); - long deleteSnapshotId = table.currentSnapshot().snapshotId(); + long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A is missing", ValidationException.class, "Cannot commit, missing data files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", deleteSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); - table - .newRowDelta() - .addDeletes(FILE_B_DELETES) - .validateDataFilesExist(ImmutableList.of(FILE_B.path())) - .validateFromSnapshot(validateFromSnapshotId) - .commit(); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_B_DELETES) + .validateDataFilesExist(ImmutableList.of(FILE_B.path())) + .validateFromSnapshot(validateFromSnapshotId), + branch); Assert.assertEquals( "Table should have one new delete manifest", 1, - table.currentSnapshot().deleteManifests(table.io()).size()); - ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); + ManifestFile deletes = latestSnapshot(table, branch).deleteManifests(table.io()).get(0); validateDeleteManifest( deletes, dataSeqs(4L), fileSeqs(4L), - ids(table.currentSnapshot().snapshotId()), + ids(latestSnapshot(table, branch).snapshotId()), files(FILE_B_DELETES), statuses(Status.ADDED)); } @Test public void testValidateDataFilesExistOverwrite() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // overwrite FILE_A - table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2).commit(); + commit(table, table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2), branch); - long deleteSnapshotId = table.currentSnapshot().snapshotId(); + long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A is missing", ValidationException.class, "Cannot commit, missing data files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", deleteSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); } @Test public void testValidateDataFilesExistReplacePartitions() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // overwrite FILE_A's partition - table.newReplacePartitions().addFile(FILE_A2).commit(); + commit(table, table.newReplacePartitions().addFile(FILE_A2), branch); - long deleteSnapshotId = table.currentSnapshot().snapshotId(); + long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A is missing", ValidationException.class, "Cannot commit, missing data files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", deleteSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); } @Test public void testValidateDataFilesExistFromSnapshot() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); - long appendSnapshotId = table.currentSnapshot().snapshotId(); + long appendSnapshotId = latestSnapshot(table, branch).snapshotId(); // overwrite FILE_A's partition - table.newReplacePartitions().addFile(FILE_A2).commit(); + commit(table, table.newReplacePartitions().addFile(FILE_A2), branch); // test changes to the table back to the snapshot where FILE_A was overwritten - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); - long replaceSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); + long replaceSnapshotId = latestSnapshot(table, branch).snapshotId(); // even though FILE_A was deleted, it happened before the "from" snapshot, so the validation // allows it - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit(); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch); - Snapshot snap = table.currentSnapshot(); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); @@ -266,135 +296,146 @@ public void testValidateDataFilesExistFromSnapshot() { @Test public void testValidateDataFilesExistRewrite() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // rewrite FILE_A - table.newRewrite().rewriteFiles(Sets.newHashSet(FILE_A), Sets.newHashSet(FILE_A2)).commit(); + commit( + table, + table.newRewrite().rewriteFiles(Sets.newHashSet(FILE_A), Sets.newHashSet(FILE_A2)), + branch); - long deleteSnapshotId = table.currentSnapshot().snapshotId(); + long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A is missing", ValidationException.class, "Cannot commit, missing data files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", deleteSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); } @Test public void testValidateDataFilesExistValidateDeletes() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // delete FILE_A - table.newDelete().deleteFile(FILE_A).commit(); + commit(table, table.newDelete().deleteFile(FILE_A), branch); - long deleteSnapshotId = table.currentSnapshot().snapshotId(); + long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A is missing", ValidationException.class, "Cannot commit, missing data files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateDeletedFiles() - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateDeletedFiles() + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", deleteSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); } @Test public void testValidateNoConflicts() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); // test changes to the table back to the snapshot where FILE_A and FILE_B existed - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // delete FILE_A - table.newAppend().appendFile(FILE_A2).commit(); + commit(table, table.newAppend().appendFile(FILE_A2), branch); - long appendSnapshotId = table.currentSnapshot().snapshotId(); + long appendSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail to add FILE_A_DELETES because FILE_A2 was added", ValidationException.class, "Found conflicting files", () -> - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 - .validateNoConflictingDataFiles() - .commit()); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 + .validateNoConflictingDataFiles(), + branch)); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", appendSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); Assert.assertEquals( "Table should not have any delete manifests", 0, - table.currentSnapshot().deleteManifests(table.io()).size()); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); } @Test public void testValidateNoConflictsFromSnapshot() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); - long appendSnapshotId = table.currentSnapshot().snapshotId(); + long appendSnapshotId = latestSnapshot(table, branch).snapshotId(); // delete FILE_A - table.newAppend().appendFile(FILE_A2).commit(); + commit(table, table.newAppend().appendFile(FILE_A2), branch); // even though FILE_A2 was added, it happened before the "from" snapshot, so the validation // allows it - long validateFromSnapshotId = table.currentSnapshot().snapshotId(); + long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateDeletedFiles() - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) - .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 - .validateNoConflictingDataFiles() - .commit(); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateDeletedFiles() + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())) + .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 + .validateNoConflictingDataFiles(), + branch); - Snapshot snap = table.currentSnapshot(); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); @@ -431,27 +472,29 @@ public void testValidateNoConflictsFromSnapshot() { @Test public void testOverwriteWithDeleteFile() { - table - .newRowDelta() - .addRows(FILE_A) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + commit( + table, + table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + branch); - long deltaSnapshotId = table.currentSnapshot().snapshotId(); + long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( - "Commit should produce sequence number 1", 1, table.currentSnapshot().sequenceNumber()); + "Commit should produce sequence number 1", + 1, + latestSnapshot(table, branch).sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); // overwriting by a filter will also remove delete files that match because all matching data // files are removed. - table - .newOverwrite() - .overwriteByRowFilter(Expressions.equal(Expressions.bucket("data", 16), 0)) - .commit(); + commit( + table, + table + .newOverwrite() + .overwriteByRowFilter(Expressions.equal(Expressions.bucket("data", 16), 0)), + branch); - Snapshot snap = table.currentSnapshot(); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); @@ -478,24 +521,24 @@ public void testOverwriteWithDeleteFile() { @Test public void testReplacePartitionsWithDeleteFile() { - table - .newRowDelta() - .addRows(FILE_A) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + commit( + table, + table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + branch); - long deltaSnapshotId = table.currentSnapshot().snapshotId(); + long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( - "Commit should produce sequence number 1", 1, table.currentSnapshot().sequenceNumber()); + "Commit should produce sequence number 1", + 1, + latestSnapshot(table, branch).sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); // overwriting the partition will also remove delete files that match because all matching data // files are removed. - table.newReplacePartitions().addFile(FILE_A2).commit(); + commit(table, table.newReplacePartitions().addFile(FILE_A2), branch); - Snapshot snap = table.currentSnapshot(); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); @@ -532,24 +575,24 @@ public void testReplacePartitionsWithDeleteFile() { @Test public void testDeleteByExpressionWithDeleteFile() { - table - .newRowDelta() - .addRows(FILE_A) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + commit( + table, + table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + branch); - long deltaSnapshotId = table.currentSnapshot().snapshotId(); + long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( - "Commit should produce sequence number 1", 1, table.currentSnapshot().sequenceNumber()); + "Commit should produce sequence number 1", + 1, + latestSnapshot(table, branch).sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); // deleting with a filter will also remove delete files that match because all matching data // files are removed. - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + commit(table, table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()), branch); - Snapshot snap = table.currentSnapshot(); + Snapshot snap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); @@ -576,18 +619,20 @@ public void testDeleteByExpressionWithDeleteFile() { @Test public void testDeleteDataFileWithDeleteFile() { - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).commit(); + commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); - long deltaSnapshotId = table.currentSnapshot().snapshotId(); + long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( - "Commit should produce sequence number 1", 1, table.currentSnapshot().sequenceNumber()); + "Commit should produce sequence number 1", + 1, + latestSnapshot(table, branch).sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); // deleting a specific data file will not affect a delete file - table.newDelete().deleteFile(FILE_A).commit(); + commit(table, table.newDelete().deleteFile(FILE_A), branch); - Snapshot deleteSnap = table.currentSnapshot(); + Snapshot deleteSnap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); @@ -617,9 +662,9 @@ public void testDeleteDataFileWithDeleteFile() { // to be 2, the largest known sequence number. this will cause FILE_A_DELETES to be removed // because it is too old // to apply to any data files. - table.newDelete().deleteFile("no-such-file").commit(); + commit(table, table.newDelete().deleteFile("no-such-file"), branch); - Snapshot nextSnap = table.currentSnapshot(); + Snapshot nextSnap = latestSnapshot(table, branch); Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); @@ -639,18 +684,20 @@ public void testDeleteDataFileWithDeleteFile() { @Test public void testFastAppendDoesNotRemoveStaleDeleteFiles() { - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).commit(); + commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); - long deltaSnapshotId = table.currentSnapshot().snapshotId(); + long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( - "Commit should produce sequence number 1", 1, table.currentSnapshot().sequenceNumber()); + "Commit should produce sequence number 1", + 1, + latestSnapshot(table, branch).sequenceNumber()); Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); // deleting a specific data file will not affect a delete file - table.newDelete().deleteFile(FILE_A).commit(); + commit(table, table.newDelete().deleteFile(FILE_A), branch); - Snapshot deleteSnap = table.currentSnapshot(); + Snapshot deleteSnap = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); @@ -677,9 +724,9 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { // the manifest that removed FILE_A will be dropped next merging commit, but FastAppend will not // remove it - table.newFastAppend().appendFile(FILE_B).commit(); + commit(table, table.newFastAppend().appendFile(FILE_B), branch); - Snapshot nextSnap = table.currentSnapshot(); + Snapshot nextSnap = latestSnapshot(table, branch); Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); @@ -733,7 +780,7 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { .withRecordCount(1) .build(); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); // add a data file to partition B DataFile dataFile2 = @@ -744,10 +791,10 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { .withRecordCount(1) .build(); - table.newAppend().appendFile(dataFile2).commit(); + commit(table, table.newAppend().appendFile(dataFile2), branch); // use this snapshot as the starting snapshot in rowDelta - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); // add a delete file for partition A DeleteFile deleteFile = @@ -771,21 +818,21 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { .validateNoConflictingDataFiles(); // concurrently delete the file for partition B - table.newDelete().deleteFile(dataFile2).commit(); + commit(table, table.newDelete().deleteFile(dataFile2), branch); // commit the delta for partition A - rowDelta.commit(); + commit(table, rowDelta, branch); Assert.assertEquals( "Table should have one new delete manifest", 1, - table.currentSnapshot().deleteManifests(table.io()).size()); - ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0); + latestSnapshot(table, branch).deleteManifests(table.io()).size()); + ManifestFile deletes = latestSnapshot(table, branch).deleteManifests(table.io()).get(0); validateDeleteManifest( deletes, dataSeqs(4L), fileSeqs(4L), - ids(table.currentSnapshot().snapshotId()), + ids(latestSnapshot(table, branch).snapshotId()), files(deleteFile), statuses(Status.ADDED)); } @@ -808,10 +855,10 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { .withRecordCount(1) .build(); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); // use this snapshot as the starting snapshot in rowDelta - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); // add a delete file for partition A DeleteFile deleteFile = @@ -835,13 +882,13 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { .validateNoConflictingDataFiles(); // concurrently delete the file for partition A - table.newDelete().deleteFile(dataFile1).commit(); + commit(table, table.newDelete().deleteFile(dataFile1), branch); AssertHelpers.assertThrows( "Should fail to add deletes because data file is missing", ValidationException.class, "Cannot commit, missing data files", - rowDelta::commit); + () -> commit(table, rowDelta, branch)); } @Test @@ -851,7 +898,7 @@ public void testAddDeleteFilesMultipleSpecs() { // append a partitioned data file DataFile firstSnapshotDataFile = newDataFile("data_bucket=0"); - table.newAppend().appendFile(firstSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(firstSnapshotDataFile), branch); // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); @@ -860,14 +907,14 @@ public void testAddDeleteFilesMultipleSpecs() { // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); - table.newAppend().appendFile(secondSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(secondSnapshotDataFile), branch); // evolve the spec and add a new partition field table.updateSpec().addField("data").commit(); // append a data file with the new spec DataFile thirdSnapshotDataFile = newDataFile("data=abc"); - table.newAppend().appendFile(thirdSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(thirdSnapshotDataFile), branch); Assert.assertEquals("Should have 3 specs", 3, table.specs().size()); @@ -878,15 +925,17 @@ public void testAddDeleteFilesMultipleSpecs() { DeleteFile secondDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); DeleteFile thirdDeleteFile = newDeleteFile(thirdSnapshotDataFile.specId(), "data=abc"); - table - .newRowDelta() - .addRows(dataFile) - .addDeletes(firstDeleteFile) - .addDeletes(secondDeleteFile) - .addDeletes(thirdDeleteFile) - .commit(); + commit( + table, + table + .newRowDelta() + .addRows(dataFile) + .addDeletes(firstDeleteFile) + .addDeletes(secondDeleteFile) + .addDeletes(thirdDeleteFile), + branch); - Snapshot snapshot = table.currentSnapshot(); + Snapshot snapshot = latestSnapshot(table, branch); Assert.assertEquals("Commit should produce sequence number 4", 4, snapshot.sequenceNumber()); Assert.assertEquals( "Last sequence number should be 4", 4, table.ops().current().lastSequenceNumber()); @@ -978,7 +1027,7 @@ public void testManifestMergingMultipleSpecs() { // append a partitioned data file DataFile firstSnapshotDataFile = newDataFile("data_bucket=0"); - table.newAppend().appendFile(firstSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(firstSnapshotDataFile), branch); // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); @@ -987,15 +1036,18 @@ public void testManifestMergingMultipleSpecs() { // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); - table.newAppend().appendFile(secondSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(secondSnapshotDataFile), branch); // commit two delete files to two specs in a single operation DeleteFile firstDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); DeleteFile secondDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); - table.newRowDelta().addDeletes(firstDeleteFile).addDeletes(secondDeleteFile).commit(); + commit( + table, + table.newRowDelta().addDeletes(firstDeleteFile).addDeletes(secondDeleteFile), + branch); - Snapshot thirdSnapshot = table.currentSnapshot(); + Snapshot thirdSnapshot = latestSnapshot(table, branch); // 2 appends and 1 row delta where delete files belong to different specs Assert.assertEquals( @@ -1007,9 +1059,12 @@ public void testManifestMergingMultipleSpecs() { DeleteFile thirdDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); DeleteFile fourthDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); - table.newRowDelta().addDeletes(thirdDeleteFile).addDeletes(fourthDeleteFile).commit(); + commit( + table, + table.newRowDelta().addDeletes(thirdDeleteFile).addDeletes(fourthDeleteFile), + branch); - Snapshot fourthSnapshot = table.currentSnapshot(); + Snapshot fourthSnapshot = latestSnapshot(table, branch); // make sure merging respects spec boundaries Assert.assertEquals( @@ -1044,7 +1099,7 @@ public void testManifestMergingMultipleSpecs() { public void testAbortMultipleSpecs() { // append a partitioned data file DataFile firstSnapshotDataFile = newDataFile("data_bucket=0"); - table.newAppend().appendFile(firstSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(firstSnapshotDataFile), branch); // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); @@ -1053,7 +1108,7 @@ public void testAbortMultipleSpecs() { // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); - table.newAppend().appendFile(secondSnapshotDataFile).commit(); + commit(table, table.newAppend().appendFile(secondSnapshotDataFile), branch); // prepare two delete files that belong to different specs DeleteFile firstDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); @@ -1065,6 +1120,7 @@ public void testAbortMultipleSpecs() { RowDelta rowDelta = table .newRowDelta() + .toBranch(branch) .addDeletes(firstDeleteFile) .addDeletes(secondDeleteFile) .deleteWith(deletedFiles::add) @@ -1074,13 +1130,13 @@ public void testAbortMultipleSpecs() { rowDelta.apply(); // perform a conflicting concurrent operation - table.newDelete().deleteFile(firstSnapshotDataFile).commit(); + commit(table, table.newDelete().deleteFile(firstSnapshotDataFile), branch); AssertHelpers.assertThrows( "Should fail to commit row delta", ValidationException.class, "Cannot commit, missing data files", - rowDelta::commit); + () -> commit(table, rowDelta, branch)); // we should clean up 1 manifest list and 2 delete manifests Assert.assertEquals("Should delete 3 files", 3, deletedFiles.size()); @@ -1088,9 +1144,9 @@ public void testAbortMultipleSpecs() { @Test public void testConcurrentConflictingRowDelta() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); Expression conflictDetectionFilter = Expressions.alwaysTrue(); @@ -1098,6 +1154,7 @@ public void testConcurrentConflictingRowDelta() { RowDelta rowDelta = table .newRowDelta() + .toBranch(branch) .addRows(FILE_B) .addDeletes(FILE_A_DELETES) .validateFromSnapshot(firstSnapshot.snapshotId()) @@ -1107,6 +1164,7 @@ public void testConcurrentConflictingRowDelta() { table .newRowDelta() + .toBranch(branch) .addDeletes(FILE_A_DELETES) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) @@ -1117,14 +1175,14 @@ public void testConcurrentConflictingRowDelta() { "Should reject commit", ValidationException.class, "Found new conflicting delete files", - rowDelta::commit); + () -> commit(table, rowDelta, branch)); } @Test public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); Expression conflictDetectionFilter = Expressions.alwaysTrue(); @@ -1139,6 +1197,7 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { table .newRowDelta() + .toBranch(branch) .addDeletes(FILE_A_DELETES) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) @@ -1149,7 +1208,7 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { "Should reject commit", ValidationException.class, "Found new conflicting delete files", - rowDelta::commit); + () -> commit(table, rowDelta, branch)); } @Test @@ -1170,7 +1229,7 @@ public void testConcurrentNonConflictingRowDelta() { .withRecordCount(1) .build(); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); // add a data file to partition B DataFile dataFile2 = @@ -1181,9 +1240,9 @@ public void testConcurrentNonConflictingRowDelta() { .withRecordCount(1) .build(); - table.newAppend().appendFile(dataFile2).commit(); + commit(table, table.newAppend().appendFile(dataFile2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); Expression conflictDetectionFilter = Expressions.equal("data", "a"); @@ -1201,6 +1260,7 @@ public void testConcurrentNonConflictingRowDelta() { RowDelta rowDelta = table .newRowDelta() + .toBranch(branch) .addDeletes(deleteFile1) .validateFromSnapshot(baseSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) @@ -1219,13 +1279,14 @@ public void testConcurrentNonConflictingRowDelta() { table .newRowDelta() + .toBranch(branch) .addDeletes(deleteFile2) .validateFromSnapshot(baseSnapshot.snapshotId()) .commit(); - rowDelta.commit(); + commit(table, rowDelta, branch); - validateTableDeleteFiles(table, deleteFile1, deleteFile2); + validateTableDeleteFilesWithRef(table, branch, deleteFile1, deleteFile2); } @Test @@ -1240,9 +1301,9 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe // add a data file to partition A DataFile dataFile1 = newDataFile("data=a"); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); // add an equality delete file DeleteFile deleteFile1 = @@ -1253,6 +1314,7 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe RowDelta rowDelta = table .newRowDelta() + .toBranch(branch) .addDeletes(deleteFile1) .validateFromSnapshot(baseSnapshot.snapshotId()) .validateNoConflictingDataFiles() @@ -1270,8 +1332,8 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe baseSnapshot.sequenceNumber()) .validateFromSnapshot(baseSnapshot.snapshotId()); - rowDelta.commit(); - rewriteFiles.commit(); + commit(table, rowDelta, branch); + commit(table, rewriteFiles, branch); validateTableDeleteFiles(table, deleteFile1); validateTableFiles(table, dataFile2); @@ -1290,9 +1352,9 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { // add a data file to partition A DataFile dataFile1 = newDataFile("data=a"); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); // add an equality delete file DeleteFile deleteFile1 = @@ -1303,6 +1365,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { RowDelta rowDelta = table .newRowDelta() + .toBranch(branch) .addDeletes(deleteFile1) .validateFromSnapshot(baseSnapshot.snapshotId()) .validateNoConflictingDataFiles() @@ -1320,16 +1383,16 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { baseSnapshot.sequenceNumber()) .validateFromSnapshot(baseSnapshot.snapshotId()); - rowDelta.commit(); - rewriteFiles.commit(); + commit(table, rowDelta, branch); + commit(table, rewriteFiles, branch); table.refresh(); - List dataManifests = table.currentSnapshot().dataManifests(table.io()); + List dataManifests = latestSnapshot(table, branch).dataManifests(table.io()); Assert.assertEquals("should have 1 data manifest", 1, dataManifests.size()); ManifestFile mergedDataManifest = dataManifests.get(0); Assert.assertEquals("Manifest seq number must match", 3L, mergedDataManifest.sequenceNumber()); - long currentSnapshotId = table.currentSnapshot().snapshotId(); + long currentSnapshotId = latestSnapshot(table, branch).snapshotId(); validateManifest( mergedDataManifest, @@ -1352,9 +1415,9 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() // add a data file to partition A DataFile dataFile1 = newDataFile("data=a"); - table.newAppend().appendFile(dataFile1).commit(); + commit(table, table.newAppend().appendFile(dataFile1), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); // add an position delete file DeleteFile deleteFile1 = newDeleteFile(table.spec().specId(), "data=a"); @@ -1364,6 +1427,7 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() table .newRowDelta() .addDeletes(deleteFile1) + .toBranch(branch) .validateFromSnapshot(baseSnapshot.snapshotId()) .validateNoConflictingDataFiles() .validateNoConflictingDeleteFiles(); @@ -1380,22 +1444,22 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() baseSnapshot.sequenceNumber()) .validateFromSnapshot(baseSnapshot.snapshotId()); - rowDelta.commit(); + commit(table, rowDelta, branch); AssertHelpers.assertThrows( "Should not allow any new position delete associated with the data file", ValidationException.class, "Cannot commit, found new position delete for replaced data file", - rewriteFiles::commit); + () -> commit(table, rewriteFiles, branch)); } @Test public void testRowDeltaCaseSensitivity() { - table.newAppend().appendFile(FILE_A).appendFile(FILE_A2).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_A_DELETES), branch); Expression conflictDetectionFilter = Expressions.equal(Expressions.bucket("dAtA", 16), 0); @@ -1406,6 +1470,7 @@ public void testRowDeltaCaseSensitivity() { () -> table .newRowDelta() + .toBranch(branch) .addRows(FILE_B) .addDeletes(FILE_A2_DELETES) .validateFromSnapshot(firstSnapshot.snapshotId()) @@ -1421,6 +1486,7 @@ public void testRowDeltaCaseSensitivity() { () -> table .newRowDelta() + .toBranch(branch) .caseSensitive(true) .addRows(FILE_B) .addDeletes(FILE_A2_DELETES) @@ -1438,6 +1504,7 @@ public void testRowDeltaCaseSensitivity() { () -> table .newRowDelta() + .toBranch(branch) .caseSensitive(false) .addRows(FILE_B) .addDeletes(FILE_A2_DELETES) From cd5569c1a8bebe4aef32aff3548f2745f5ced5cc Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Thu, 5 Jan 2023 16:43:36 -0800 Subject: [PATCH 18/27] fetching data files and delete files by ref --- .../org/apache/iceberg/TableTestBase.java | 13 ++++++++++ .../java/org/apache/iceberg/TestRowDelta.java | 25 +++++++++++-------- 2 files changed, 27 insertions(+), 11 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 929397982b5a..a9d82c28850b 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -458,6 +458,19 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) { Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); } + void validateTableFilesWithRef(Table tbl, String ref, DataFile... expectedFiles) { + Set expectedFilePaths = Sets.newHashSet(); + for (DataFile file : expectedFiles) { + expectedFilePaths.add(file.path()); + } + Set actualFilePaths = Sets.newHashSet(); + for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { + actualFilePaths.add(task.file().path()); + } + Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); + } + + void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DeleteFile file : expectedFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 4ede90754b7c..ebb464ee0155 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -93,7 +93,7 @@ public void testAddDeleteFile() { statuses(Status.ADDED, Status.ADDED)); } - @Test + //@Test public void testValidateDataFilesExistDefaults() { SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); @@ -161,7 +161,7 @@ public void testValidateDataFilesExistDefaults() { statuses(Status.ADDED)); } - @Test + //@Test public void testValidateDataFilesExistOverwrite() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -198,7 +198,7 @@ public void testValidateDataFilesExistOverwrite() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - @Test + //@Test public void testValidateDataFilesExistReplacePartitions() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -235,7 +235,7 @@ public void testValidateDataFilesExistReplacePartitions() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - @Test + //@Test public void testValidateDataFilesExistFromSnapshot() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -470,7 +470,7 @@ public void testValidateNoConflictsFromSnapshot() { statuses(Status.ADDED)); } - @Test + //@Test public void testOverwriteWithDeleteFile() { commit( table, @@ -519,7 +519,7 @@ public void testOverwriteWithDeleteFile() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + //@Test public void testReplacePartitionsWithDeleteFile() { commit( table, @@ -1335,8 +1335,13 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe commit(table, rowDelta, branch); commit(table, rewriteFiles, branch); - validateTableDeleteFiles(table, deleteFile1); - validateTableFiles(table, dataFile2); + if(branch == "testBranch") { + validateTableDeleteFilesWithRef(table, "testBranch", deleteFile1); + validateTableFilesWithRef(table, branch, dataFile2); + } else { + validateTableDeleteFiles(table, deleteFile1); + validateTableFiles(table, dataFile2); + } } @Test @@ -1365,7 +1370,6 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { RowDelta rowDelta = table .newRowDelta() - .toBranch(branch) .addDeletes(deleteFile1) .validateFromSnapshot(baseSnapshot.snapshotId()) .validateNoConflictingDataFiles() @@ -1403,7 +1407,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { statuses(Status.ADDED, Status.DELETED)); } - @Test + //@Test public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() { // change the spec to be partitioned by data table @@ -1427,7 +1431,6 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() table .newRowDelta() .addDeletes(deleteFile1) - .toBranch(branch) .validateFromSnapshot(baseSnapshot.snapshotId()) .validateNoConflictingDataFiles() .validateNoConflictingDeleteFiles(); From 3d5659bb82cb9f241eefa13c41a2e8c6002287c6 Mon Sep 17 00:00:00 2001 From: namrathamyske Date: Sat, 7 Jan 2023 22:36:30 -0800 Subject: [PATCH 19/27] spotless fix --- .../java/org/apache/iceberg/TableTestBase.java | 1 - .../java/org/apache/iceberg/TestRowDelta.java | 16 ++++++++-------- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index a9d82c28850b..219df92d74a9 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -470,7 +470,6 @@ void validateTableFilesWithRef(Table tbl, String ref, DataFile... expectedFiles) Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); } - void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DeleteFile file : expectedFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index ebb464ee0155..186dba209dac 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -93,7 +93,7 @@ public void testAddDeleteFile() { statuses(Status.ADDED, Status.ADDED)); } - //@Test + // @Test public void testValidateDataFilesExistDefaults() { SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); @@ -161,7 +161,7 @@ public void testValidateDataFilesExistDefaults() { statuses(Status.ADDED)); } - //@Test + // @Test public void testValidateDataFilesExistOverwrite() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -198,7 +198,7 @@ public void testValidateDataFilesExistOverwrite() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - //@Test + // @Test public void testValidateDataFilesExistReplacePartitions() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -235,7 +235,7 @@ public void testValidateDataFilesExistReplacePartitions() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - //@Test + // @Test public void testValidateDataFilesExistFromSnapshot() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -470,7 +470,7 @@ public void testValidateNoConflictsFromSnapshot() { statuses(Status.ADDED)); } - //@Test + // @Test public void testOverwriteWithDeleteFile() { commit( table, @@ -519,7 +519,7 @@ public void testOverwriteWithDeleteFile() { statuses(Status.DELETED, Status.EXISTING)); } - //@Test + // @Test public void testReplacePartitionsWithDeleteFile() { commit( table, @@ -1335,7 +1335,7 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe commit(table, rowDelta, branch); commit(table, rewriteFiles, branch); - if(branch == "testBranch") { + if (branch == "testBranch") { validateTableDeleteFilesWithRef(table, "testBranch", deleteFile1); validateTableFilesWithRef(table, branch, dataFile2); } else { @@ -1407,7 +1407,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { statuses(Status.ADDED, Status.DELETED)); } - //@Test + // @Test public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() { // change the spec to be partitioned by data table From 0028bcd4d37e720c6cf63ce388f9ffb37cbc8168 Mon Sep 17 00:00:00 2001 From: namrathamyske Date: Wed, 11 Jan 2023 14:57:31 -0800 Subject: [PATCH 20/27] Test case for overwrite and rewrite --- .../apache/iceberg/BaseOverwriteFiles.java | 6 + .../apache/iceberg/BaseReplacePartitions.java | 6 + .../org/apache/iceberg/BaseRewriteFiles.java | 6 + .../iceberg/MergingSnapshotProducer.java | 2 +- .../org/apache/iceberg/TestOverwrite.java | 111 +++++----- .../org/apache/iceberg/TestRewriteFiles.java | 194 +++++++++--------- .../java/org/apache/iceberg/TestRowDelta.java | 12 +- .../iceberg/io/TestFileWriterFactory.java | 45 +++- 8 files changed, 222 insertions(+), 160 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index 77b00bcacc27..55bbad7750bf 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -104,6 +104,12 @@ public OverwriteFiles validateNoConflictingDeletes() { return this; } + @Override + public BaseOverwriteFiles toBranch(String branch) { + targetBranch(branch); + return this; + } + @Override protected void validate(TableMetadata base, Snapshot snapshot) { if (validateAddedFilesMatchOverwriteFilter) { diff --git a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java index b651d9d9432e..327a276a87cf 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java +++ b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java @@ -79,6 +79,12 @@ public ReplacePartitions validateNoConflictingData() { return this; } + @Override + public BaseReplacePartitions toBranch(String branch) { + targetBranch(branch); + return this; + } + @Override public void validate(TableMetadata currentMetadata, Snapshot snapshot) { if (validateConflictingData) { diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java index 646c9ce8c6ed..05a446a14cb5 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java @@ -109,6 +109,12 @@ public RewriteFiles validateFromSnapshot(long snapshotId) { return this; } + @Override + public BaseRewriteFiles toBranch(String branch) { + targetBranch(branch); + return this; + } + @Override protected void validate(TableMetadata base, Snapshot snapshot) { if (replacedDataFiles.size() > 0) { diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 9feac2d88166..e58456c650fb 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -436,7 +436,7 @@ private void validateNoNewDeletesForDataFiles( boolean ignoreEqualityDeletes, Snapshot parent) { // if there is no current table state, no files have been added - if (base.currentSnapshot() == null || base.formatVersion() < 2) { + if (parent == null || base.formatVersion() < 2) { return; } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 082f6396bde7..7df76961f3c3 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -102,13 +102,21 @@ public class TestOverwrite extends TableTestBase { )) .build(); - @Parameterized.Parameters(name = "formatVersion = {0}") + private final String branch; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") public static Object[] parameters() { - return new Object[] {1, 2}; + return new Object[][] { + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"} + }; } - public TestOverwrite(int formatVersion) { + public TestOverwrite(int formatVersion, String branch) { super(formatVersion); + this.branch = branch; } private static ByteBuffer longToBuffer(long value) { @@ -125,26 +133,26 @@ public void createTestTable() throws IOException { this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_BY_DATE, formatVersion); - table.newAppend().appendFile(FILE_0_TO_4).appendFile(FILE_5_TO_9).commit(); + commit(table, table.newAppend().appendFile(FILE_0_TO_4).appendFile(FILE_5_TO_9), branch); } @Test public void testOverwriteWithoutAppend() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); - table.newOverwrite().overwriteByRowFilter(equal("date", "2018-06-08")).commit(); + commit(table, table.newOverwrite().overwriteByRowFilter(equal("date", "2018-06-08")), branch); - long overwriteId = table.currentSnapshot().snapshotId(); + long overwriteId = latestSnapshot(table, branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals( "Table should have one manifest", 1, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(overwriteId, baseId), files(FILE_0_TO_4, FILE_5_TO_9), statuses(Status.DELETED, Status.EXISTING)); @@ -153,7 +161,8 @@ public void testOverwriteWithoutAppend() { @Test public void testOverwriteFailsDelete() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); OverwriteFiles overwrite = table @@ -164,40 +173,43 @@ public void testOverwriteFailsDelete() { "Should reject commit with file not matching delete expression", ValidationException.class, "Cannot delete file where some, but not all, rows match filter", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( - "Should not create a new snapshot", baseId, table.currentSnapshot().snapshotId()); + "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); } @Test public void testOverwriteWithAppendOutsideOfDelete() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); - table - .newOverwrite() - .overwriteByRowFilter(equal("date", "2018-06-08")) - .addFile(FILE_10_TO_14) // in 2018-06-09, NOT in 2018-06-08 - .commit(); + commit( + table, + table + .newOverwrite() + .overwriteByRowFilter(equal("date", "2018-06-08")) + .addFile(FILE_10_TO_14), + branch); // in 2018-06-09, NOT in 2018-06-08 - long overwriteId = table.currentSnapshot().snapshotId(); + long overwriteId = latestSnapshot(table, branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals( "Table should have 2 manifests", 2, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(overwriteId), files(FILE_10_TO_14), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(overwriteId, baseId), files(FILE_0_TO_4, FILE_5_TO_9), statuses(Status.DELETED, Status.EXISTING)); @@ -209,24 +221,27 @@ public void testOverwriteWithMergedAppendOutsideOfDelete() { table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); - table - .newOverwrite() - .overwriteByRowFilter(equal("date", "2018-06-08")) - .addFile(FILE_10_TO_14) // in 2018-06-09, NOT in 2018-06-08 - .commit(); + commit( + table, + table + .newOverwrite() + .overwriteByRowFilter(equal("date", "2018-06-08")) + .addFile(FILE_10_TO_14), + branch); // in 2018-06-09, NOT in 2018-06-08 - long overwriteId = table.currentSnapshot().snapshotId(); + long overwriteId = latestSnapshot(table, branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); Assert.assertEquals( "Table should have one merged manifest", 1, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(overwriteId, overwriteId, baseId), files(FILE_10_TO_14, FILE_0_TO_4, FILE_5_TO_9), statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); @@ -238,7 +253,8 @@ public void testValidatedOverwriteWithAppendOutsideOfDelete() { table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); OverwriteFiles overwrite = table @@ -251,16 +267,17 @@ public void testValidatedOverwriteWithAppendOutsideOfDelete() { "Should reject commit with file not matching delete expression", ValidationException.class, "Cannot append file with rows that do not match filter", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( - "Should not create a new snapshot", baseId, table.currentSnapshot().snapshotId()); + "Should not create a new snapshot", baseId, latestSnapshot(table, branch).snapshotId()); } @Test public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); OverwriteFiles overwrite = table @@ -273,16 +290,17 @@ public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { "Should reject commit with file not matching delete expression", ValidationException.class, "Cannot append file with rows that do not match filter", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( - "Should not create a new snapshot", baseId, table.currentSnapshot().snapshotId()); + "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); } @Test public void testValidatedOverwriteWithAppendSuccess() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = + latestSnapshot(base, branch) == null ? -1 : latestSnapshot(base, branch).snapshotId(); OverwriteFiles overwrite = table @@ -295,22 +313,9 @@ public void testValidatedOverwriteWithAppendSuccess() { "Should reject commit with file not matching delete expression", ValidationException.class, "Cannot append file with rows that do not match filter", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( - "Should not create a new snapshot", baseId, table.currentSnapshot().snapshotId()); - } - - @Test - public void testOverwriteToBranchUnsupported() { - AssertHelpers.assertThrows( - "Cannot commit to branch someBranch: org.apache.iceberg.BaseOverwriteFiles does not support branch commits", - UnsupportedOperationException.class, - () -> - table - .newOverwrite() - .overwriteByRowFilter(and(equal("date", "2018-06-09"), lessThan("id", 20))) - .addFile(FILE_10_TO_14) - .toBranch("someBranch")); + "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 4af7fc887a46..b708559eebf7 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -38,13 +38,22 @@ @RunWith(Parameterized.class) public class TestRewriteFiles extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") + + private final String branch; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") public static Object[] parameters() { - return new Object[] {1, 2}; + return new Object[][] { + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"} + }; } - public TestRewriteFiles(int formatVersion) { + public TestRewriteFiles(int formatVersion, String branch) { super(formatVersion); + this.branch = branch; } @Test @@ -58,21 +67,20 @@ public void testEmptyTable() { "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)).commit()); + () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch)); AssertHelpers.assertThrows( "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a-deletes.parquet", () -> - table + commit(table, table .newRewrite() .rewriteFiles( ImmutableSet.of(), ImmutableSet.of(FILE_A_DELETES), ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_B_DELETES)) - .commit()); + ImmutableSet.of(FILE_B_DELETES)), branch)); } @Test @@ -83,35 +91,33 @@ public void testAddOnly() { "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()).apply()); + () -> apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Delete files to add must be empty because there's no delete file to be rewritten", () -> - table - .newRewrite() + apply( + table.newRewrite() .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(), ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)) - .apply()); + ImmutableSet.of(FILE_A_DELETES)), branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Delete files to add must be empty because there's no delete file to be rewritten", () -> - table + apply(table .newRewrite() .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(), ImmutableSet.of(FILE_B), - ImmutableSet.of(FILE_B_DELETES)) - .apply()); + ImmutableSet.of(FILE_B_DELETES)), branch)); } @Test @@ -122,53 +128,51 @@ public void testDeleteOnly() { "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", - () -> table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)).apply()); + () -> apply(table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", () -> - table + apply(table .newRewrite() .rewriteFiles( ImmutableSet.of(), ImmutableSet.of(), ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)) - .apply()); + ImmutableSet.of(FILE_A_DELETES)), branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", () -> - table + apply(table .newRewrite() .rewriteFiles( ImmutableSet.of(), ImmutableSet.of(), ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_A_DELETES)) - .apply()); + ImmutableSet.of(FILE_A_DELETES)), branch)); } @Test public void testDeleteWithDuplicateEntriesInManifest() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table.newAppend().appendFile(FILE_A).appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); - long baseSnapshotId = base.currentSnapshot().snapshotId(); + long baseSnapshotId = latestSnapshot(base, branch).snapshotId(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - base.currentSnapshot().allManifests(table.io()).size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); + latestSnapshot(base, branch).allManifests(table.io()).size()); + ManifestFile initialManifest = latestSnapshot(base, branch).allManifests(table.io()).get(0); Snapshot pending = - table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)).apply(); + apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)), branch); Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -194,18 +198,18 @@ public void testDeleteWithDuplicateEntriesInManifest() { public void testAddAndDelete() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); - long baseSnapshotId = base.currentSnapshot().snapshotId(); + long baseSnapshotId = latestSnapshot(table, branch).snapshotId(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - base.currentSnapshot().allManifests(table.io()).size()); - ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); + latestSnapshot(table, branch).allManifests(table.io()).size()); + ManifestFile initialManifest = latestSnapshot(table, branch).allManifests(table.io()).get(0); Snapshot pending = - table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)).apply(); + apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)), branch); Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -233,17 +237,16 @@ public void testRewriteDataAndDeleteFiles() { "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table + commit(table, table .newRowDelta() .addRows(FILE_A) .addRows(FILE_B) .addRows(FILE_C) .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + .addDeletes(FILE_B_DELETES), branch); TableMetadata base = readMetadata(); - Snapshot baseSnap = base.currentSnapshot(); + Snapshot baseSnap = latestSnapshot(base, branch); long baseSnapshotId = baseSnap.snapshotId(); Assert.assertEquals( "Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); @@ -264,15 +267,14 @@ public void testRewriteDataAndDeleteFiles() { // Rewrite the files. Snapshot pending = - table + apply(table .newRewrite() - .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_A_DELETES), ImmutableSet.of(FILE_D), - ImmutableSet.of()) - .apply(); + ImmutableSet.of()), branch); Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -307,17 +309,16 @@ public void testRewriteDataAndAssignOldSequenceNumber() { "Sequence number is only supported in iceberg format v2. ", formatVersion > 1); Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table + commit(table, table .newRowDelta() .addRows(FILE_A) .addRows(FILE_B) .addRows(FILE_C) .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + .addDeletes(FILE_B_DELETES), branch); TableMetadata base = readMetadata(); - Snapshot baseSnap = base.currentSnapshot(); + Snapshot baseSnap = latestSnapshot(base, branch); long baseSnapshotId = baseSnap.snapshotId(); Assert.assertEquals( "Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); @@ -337,13 +338,12 @@ public void testRewriteDataAndAssignOldSequenceNumber() { statuses(ADDED, ADDED)); // Rewrite the files. - long oldSequenceNumber = table.currentSnapshot().sequenceNumber(); + long oldSequenceNumber = latestSnapshot(table, branch).sequenceNumber(); Snapshot pending = - table + apply(table .newRewrite() - .validateFromSnapshot(table.currentSnapshot().snapshotId()) - .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber) - .apply(); + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) + .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber), branch); Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -384,13 +384,13 @@ public void testRewriteDataAndAssignOldSequenceNumber() { @Test public void testFailure() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); table.ops().failCommits(5); RewriteFiles rewrite = table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -403,7 +403,7 @@ public void testFailure() { "Should retry 4 times and throw last failure", CommitFailedException.class, "Injected failure", - rewrite::commit); + () -> commit(table, rewrite, branch)); Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); @@ -417,28 +417,27 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - table + commit(table, table .newRowDelta() .addRows(FILE_A) .addRows(FILE_B) .addRows(FILE_C) .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + .addDeletes(FILE_B_DELETES), branch); - long baseSnapshotId = readMetadata().currentSnapshot().snapshotId(); + long baseSnapshotId = latestSnapshot(readMetadata(), branch).snapshotId(); table.ops().failCommits(5); RewriteFiles rewrite = table .newRewrite() - .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_A_DELETES, FILE_B_DELETES), ImmutableSet.of(FILE_D), ImmutableSet.of()); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -481,13 +480,13 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { @Test public void testRecovery() { - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); table.ops().failCommits(3); RewriteFiles rewrite = table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -496,7 +495,7 @@ public void testRecovery() { validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_B), statuses(ADDED)); validateManifestEntries(manifest2, ids(pending.snapshotId()), files(FILE_A), statuses(DELETED)); - rewrite.commit(); + commit(table, rewrite, branch); Assert.assertTrue("Should reuse the manifest for appends", new File(manifest1.path()).exists()); Assert.assertTrue( @@ -505,7 +504,7 @@ public void testRecovery() { TableMetadata metadata = readMetadata(); Assert.assertTrue( "Should commit the manifest for append", - metadata.currentSnapshot().allManifests(table.io()).contains(manifest2)); + latestSnapshot(metadata, branch).allManifests(table.io()).contains(manifest2)); // 2 manifests added by rewrite and 1 original manifest should be found. Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); @@ -516,28 +515,27 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - table + commit(table, table .newRowDelta() .addRows(FILE_A) .addRows(FILE_B) .addRows(FILE_C) .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) - .commit(); + .addDeletes(FILE_B_DELETES), branch); - long baseSnapshotId = readMetadata().currentSnapshot().snapshotId(); + long baseSnapshotId = latestSnapshot(readMetadata(), branch).snapshotId(); table.ops().failCommits(3); RewriteFiles rewrite = table .newRewrite() - .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_A_DELETES, FILE_B_DELETES), ImmutableSet.of(FILE_D), ImmutableSet.of()); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -560,7 +558,7 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { files(FILE_A_DELETES, FILE_B_DELETES), statuses(DELETED, DELETED)); - rewrite.commit(); + commit(table, rewrite, branch); Assert.assertTrue("Should reuse new manifest", new File(manifest1.path()).exists()); Assert.assertTrue("Should reuse new manifest", new File(manifest2.path()).exists()); @@ -570,7 +568,7 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); Assert.assertEquals( "Should committed the manifests", - metadata.currentSnapshot().allManifests(table.io()), + latestSnapshot(metadata, branch).allManifests(table.io()), committedManifests); // As commit success all the manifests added with rewrite should be available. @@ -582,10 +580,10 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - table.newRowDelta().addRows(FILE_A2).addDeletes(FILE_A2_DELETES).commit(); + commit(table, table.newRowDelta().addRows(FILE_A2).addDeletes(FILE_A2_DELETES), branch); TableMetadata metadata = readMetadata(); - long baseSnapshotId = metadata.currentSnapshot().snapshotId(); + long baseSnapshotId = latestSnapshot(metadata, branch).snapshotId(); // Apply and commit the rewrite transaction. RewriteFiles rewrite = @@ -594,7 +592,7 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { .rewriteFiles( ImmutableSet.of(), ImmutableSet.of(FILE_A2_DELETES), ImmutableSet.of(), ImmutableSet.of(FILE_B_DELETES)); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -619,7 +617,7 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { files(FILE_A2_DELETES), statuses(DELETED)); - rewrite.commit(); + commit(table, rewrite, branch); Assert.assertTrue("Should reuse new manifest", new File(manifest1.path()).exists()); Assert.assertTrue("Should reuse new manifest", new File(manifest2.path()).exists()); @@ -629,7 +627,7 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); Assert.assertEquals( "Should committed the manifests", - metadata.currentSnapshot().allManifests(table.io()), + latestSnapshot(metadata, branch).allManifests(table.io()), committedManifests); // As commit success all the manifests added with rewrite should be available. @@ -641,17 +639,17 @@ public void testRemoveAllDeletes() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).commit(); + commit(table , table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); // Apply and commit the rewrite transaction. RewriteFiles rewrite = table .newRewrite() - .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) .rewriteFiles( ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_A_DELETES), ImmutableSet.of(), ImmutableSet.of()); - Snapshot pending = rewrite.apply(); + Snapshot pending = apply(rewrite, branch); Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); @@ -667,7 +665,7 @@ public void testRemoveAllDeletes() { files(FILE_A_DELETES), statuses(DELETED)); - rewrite.commit(); + commit(table, rewrite, branch); Assert.assertTrue("Should reuse the new manifest", new File(manifest1.path()).exists()); Assert.assertTrue("Should reuse the new manifest", new File(manifest2.path()).exists()); @@ -676,7 +674,7 @@ public void testRemoveAllDeletes() { List committedManifests = Lists.newArrayList(manifest1, manifest2); Assert.assertTrue( "Should committed the manifests", - metadata.currentSnapshot().allManifests(table.io()).containsAll(committedManifests)); + latestSnapshot(metadata, branch).allManifests(table.io()).containsAll(committedManifests)); // As commit success all the manifests added with rewrite should be available. Assert.assertEquals("4 manifests should exist", 4, listManifestFiles().size()); @@ -686,19 +684,19 @@ public void testRemoveAllDeletes() { public void testDeleteNonExistentFile() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - base.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(base, branch).allManifests(table.io()).size()); AssertHelpers.assertThrows( "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-c.parquet", - () -> table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)).commit()); + () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)), branch)); Assert.assertEquals("Only 1 manifests should exist", 1, listManifestFiles().size()); } @@ -707,16 +705,16 @@ public void testDeleteNonExistentFile() { public void testAlreadyDeletedFile() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); TableMetadata base = readMetadata(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - base.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(base, branch).allManifests(table.io()).size()); RewriteFiles rewrite = table.newRewrite(); - Snapshot pending = rewrite.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)).apply(); + Snapshot pending = apply(rewrite.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch); Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); @@ -727,17 +725,17 @@ public void testAlreadyDeletedFile() { validateManifestEntries( pending.allManifests(table.io()).get(1), - ids(pendingId, base.currentSnapshot().snapshotId()), + ids(pendingId, latestSnapshot(table, branch).snapshotId()), files(FILE_A), statuses(DELETED)); - rewrite.commit(); + commit(table, rewrite, branch); AssertHelpers.assertThrows( "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)).commit()); + () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)), branch)); Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); } @@ -746,30 +744,28 @@ public void testAlreadyDeletedFile() { public void testNewDeleteFile() { Assume.assumeTrue("Delete files are only supported in v2", formatVersion > 1); - table.newAppend().appendFile(FILE_A).commit(); + commit(table, table.newAppend().appendFile(FILE_A), branch); - long snapshotBeforeDeletes = table.currentSnapshot().snapshotId(); + long snapshotBeforeDeletes = latestSnapshot(table, branch).snapshotId(); - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_A_DELETES), branch); - long snapshotAfterDeletes = table.currentSnapshot().snapshotId(); + long snapshotAfterDeletes = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should fail because deletes were added after the starting snapshot", ValidationException.class, "Cannot commit, found new delete for replaced data file", () -> - table + apply(table .newRewrite() .validateFromSnapshot(snapshotBeforeDeletes) - .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)) - .apply()); + .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), branch)); // the rewrite should be valid when validating from the snapshot after the deletes - table + apply(table .newRewrite() .validateFromSnapshot(snapshotAfterDeletes) - .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)) - .apply(); + .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), branch); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 186dba209dac..e7c1d47b0954 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -93,7 +93,7 @@ public void testAddDeleteFile() { statuses(Status.ADDED, Status.ADDED)); } - // @Test + @Test public void testValidateDataFilesExistDefaults() { SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); @@ -161,7 +161,7 @@ public void testValidateDataFilesExistDefaults() { statuses(Status.ADDED)); } - // @Test + @Test public void testValidateDataFilesExistOverwrite() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -198,7 +198,7 @@ public void testValidateDataFilesExistOverwrite() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - // @Test + @Test public void testValidateDataFilesExistReplacePartitions() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -470,7 +470,7 @@ public void testValidateNoConflictsFromSnapshot() { statuses(Status.ADDED)); } - // @Test + @Test public void testOverwriteWithDeleteFile() { commit( table, @@ -519,7 +519,7 @@ public void testOverwriteWithDeleteFile() { statuses(Status.DELETED, Status.EXISTING)); } - // @Test + @Test public void testReplacePartitionsWithDeleteFile() { commit( table, @@ -1407,7 +1407,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { statuses(Status.ADDED, Status.DELETED)); } - // @Test + @Test public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() { // change the spec to be partitioned by data table diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index eff918b145f7..5dd9f286ff10 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -34,6 +34,7 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.orc.GenericOrcReader; @@ -136,7 +137,7 @@ public void testEqualityDeleteWriter() throws IOException { DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition); // commit the written data file - table.newRowDelta().addRows(dataFile).commit(); + table.newRowDelta().toBranch("testBranch").addRows(dataFile).commit(); // write an equality delete file List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(3, "bbb"), toRow(5, "ccc")); @@ -159,6 +160,48 @@ public void testEqualityDeleteWriter() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } + @Test + public void testEqualityDeletesWithBranch() throws IOException { + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + Schema equalityDeleteRowSchema = table.schema().select("id"); + FileWriterFactory writerFactory = + newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema); + + // write a data file + DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition); + + // commit the written data file + table.newRowDelta().toBranch("testBranch").addRows(dataFile).commit(); + + // write an equality delete file + List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(3, "bbb"), toRow(5, "ccc")); + DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); + + // verify the written delete file + GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema); + List expectedDeletes = + ImmutableList.of( + deleteRecord.copy("id", 1), deleteRecord.copy("id", 3), deleteRecord.copy("id", 5)); + InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); + List actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile); + Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + + // commit the written delete file + table.newRowDelta().toBranch("testBranch").addDeletes(deleteFile).commit(); + + // verify the delete file is applied correctly + List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table) + .useSnapshot(table.snapshot("testBranch").snapshotId()) + .select("*") + .build()) { + reader.forEach(set::add); + } + Assert.assertEquals("Records should match", toSet(expectedRows), set); + } + @Test public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { Assume.assumeFalse("Table must start unpartitioned", partitioned); From 7173dcbf53ed4ae89ed69fd2135f91f1a555ff64 Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Sat, 14 Jan 2023 13:15:02 -0800 Subject: [PATCH 21/27] Removing unnecessary testcases --- .../org/apache/iceberg/TableTestBase.java | 6 +- .../org/apache/iceberg/TestRewriteFiles.java | 251 +++++++++++------- .../java/org/apache/iceberg/TestRowDelta.java | 57 +--- 3 files changed, 157 insertions(+), 157 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 219df92d74a9..7df2fe1dd86f 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -458,7 +458,7 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) { Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); } - void validateTableFilesWithRef(Table tbl, String ref, DataFile... expectedFiles) { + void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { expectedFilePaths.add(file.path()); @@ -484,13 +484,13 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) { Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); } - void validateTableDeleteFilesWithRef(Table tbl, String ref, DeleteFile... expectedFiles) { + void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DeleteFile file : expectedFiles) { expectedFilePaths.add(file.path()); } Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { + for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) { for (DeleteFile file : task.deletes()) { actualFilePaths.add(file.path()); } diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index b708559eebf7..5f126678fe3a 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -44,10 +44,10 @@ public class TestRewriteFiles extends TableTestBase { @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") public static Object[] parameters() { return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"} }; } @@ -61,26 +61,33 @@ public void testEmptyTable() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + Assert.assertNull("Should not have a current snapshot", base.ref(branch)); AssertHelpers.assertThrows( "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch)); + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), + branch)); AssertHelpers.assertThrows( "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a-deletes.parquet", () -> - commit(table, table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES), - ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_B_DELETES)), branch)); + commit( + table, + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES), + ImmutableSet.of(FILE_A), + ImmutableSet.of(FILE_B_DELETES)), + branch)); } @Test @@ -91,33 +98,40 @@ public void testAddOnly() { "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), branch)); + () -> + apply( + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), + branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Delete files to add must be empty because there's no delete file to be rewritten", () -> - apply( - table.newRewrite() - .rewriteFiles( - ImmutableSet.of(FILE_A), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)), branch)); + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(FILE_A), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES)), + branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Delete files to add must be empty because there's no delete file to be rewritten", () -> - apply(table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(FILE_A), - ImmutableSet.of(), - ImmutableSet.of(FILE_B), - ImmutableSet.of(FILE_B_DELETES)), branch)); + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(FILE_A), + ImmutableSet.of(), + ImmutableSet.of(FILE_B), + ImmutableSet.of(FILE_B_DELETES)), + branch)); } @Test @@ -128,47 +142,55 @@ public void testDeleteOnly() { "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", - () -> apply(table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), branch)); + () -> + apply( + table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), + branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", () -> - apply(table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)), branch)); + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES)), + branch)); AssertHelpers.assertThrows( "Expected an exception", IllegalArgumentException.class, "Files to delete cannot be null or empty", () -> - apply(table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_A_DELETES)), branch)); + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A), + ImmutableSet.of(FILE_A_DELETES)), + branch)); } @Test public void testDeleteWithDuplicateEntriesInManifest() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A).appendFile(FILE_B), branch); + commit( + table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); long baseSnapshotId = latestSnapshot(base, branch).snapshotId(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - latestSnapshot(base, branch).allManifests(table.io()).size()); + latestSnapshot(base, branch).allManifests(table.io()).size()); ManifestFile initialManifest = latestSnapshot(base, branch).allManifests(table.io()).get(0); Snapshot pending = @@ -201,11 +223,11 @@ public void testAddAndDelete() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); - long baseSnapshotId = latestSnapshot(table, branch).snapshotId(); + long baseSnapshotId = latestSnapshot(base, branch).snapshotId(); Assert.assertEquals( "Should create 1 manifest for initial write", 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); ManifestFile initialManifest = latestSnapshot(table, branch).allManifests(table.io()).get(0); Snapshot pending = @@ -237,13 +259,16 @@ public void testRewriteDataAndDeleteFiles() { "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - commit(table, table - .newRowDelta() - .addRows(FILE_A) - .addRows(FILE_B) - .addRows(FILE_C) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES), branch); + commit( + table, + table + .newRowDelta() + .addRows(FILE_A) + .addRows(FILE_B) + .addRows(FILE_C) + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES), + branch); TableMetadata base = readMetadata(); Snapshot baseSnap = latestSnapshot(base, branch); @@ -267,14 +292,16 @@ public void testRewriteDataAndDeleteFiles() { // Rewrite the files. Snapshot pending = - apply(table - .newRewrite() - .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) - .rewriteFiles( - ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_A_DELETES), - ImmutableSet.of(FILE_D), - ImmutableSet.of()), branch); + apply( + table + .newRewrite() + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) + .rewriteFiles( + ImmutableSet.of(FILE_A), + ImmutableSet.of(FILE_A_DELETES), + ImmutableSet.of(FILE_D), + ImmutableSet.of()), + branch); Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -309,13 +336,16 @@ public void testRewriteDataAndAssignOldSequenceNumber() { "Sequence number is only supported in iceberg format v2. ", formatVersion > 1); Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - commit(table, table - .newRowDelta() - .addRows(FILE_A) - .addRows(FILE_B) - .addRows(FILE_C) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES), branch); + commit( + table, + table + .newRowDelta() + .addRows(FILE_A) + .addRows(FILE_B) + .addRows(FILE_C) + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES), + branch); TableMetadata base = readMetadata(); Snapshot baseSnap = latestSnapshot(base, branch); @@ -340,10 +370,12 @@ public void testRewriteDataAndAssignOldSequenceNumber() { // Rewrite the files. long oldSequenceNumber = latestSnapshot(table, branch).sequenceNumber(); Snapshot pending = - apply(table - .newRewrite() - .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) - .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber), branch); + apply( + table + .newRewrite() + .validateFromSnapshot(latestSnapshot(table, branch).snapshotId()) + .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber), + branch); Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); Assert.assertFalse( @@ -403,7 +435,7 @@ public void testFailure() { "Should retry 4 times and throw last failure", CommitFailedException.class, "Injected failure", - () -> commit(table, rewrite, branch)); + () -> commit(table, rewrite, branch)); Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); @@ -417,13 +449,16 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - commit(table, table - .newRowDelta() - .addRows(FILE_A) - .addRows(FILE_B) - .addRows(FILE_C) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES), branch); + commit( + table, + table + .newRowDelta() + .addRows(FILE_A) + .addRows(FILE_B) + .addRows(FILE_C) + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES), + branch); long baseSnapshotId = latestSnapshot(readMetadata(), branch).snapshotId(); table.ops().failCommits(5); @@ -515,13 +550,16 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - commit(table, table - .newRowDelta() - .addRows(FILE_A) - .addRows(FILE_B) - .addRows(FILE_C) - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES), branch); + commit( + table, + table + .newRowDelta() + .addRows(FILE_A) + .addRows(FILE_B) + .addRows(FILE_C) + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES), + branch); long baseSnapshotId = latestSnapshot(readMetadata(), branch).snapshotId(); table.ops().failCommits(3); @@ -639,7 +677,7 @@ public void testRemoveAllDeletes() { Assume.assumeTrue( "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - commit(table , table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); + commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); // Apply and commit the rewrite transaction. RewriteFiles rewrite = @@ -696,7 +734,11 @@ public void testDeleteNonExistentFile() { "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-c.parquet", - () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)), branch)); + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)), + branch)); Assert.assertEquals("Only 1 manifests should exist", 1, listManifestFiles().size()); } @@ -714,7 +756,8 @@ public void testAlreadyDeletedFile() { latestSnapshot(base, branch).allManifests(table.io()).size()); RewriteFiles rewrite = table.newRewrite(); - Snapshot pending = apply(rewrite.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch); + Snapshot pending = + apply(rewrite.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch); Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); @@ -735,7 +778,11 @@ public void testAlreadyDeletedFile() { "Expected an exception", ValidationException.class, "Missing required files to delete: /path/to/data-a.parquet", - () -> commit(table, table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)), branch)); + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)), + branch)); Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); } @@ -757,15 +804,19 @@ public void testNewDeleteFile() { ValidationException.class, "Cannot commit, found new delete for replaced data file", () -> - apply(table - .newRewrite() - .validateFromSnapshot(snapshotBeforeDeletes) - .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), branch)); + apply( + table + .newRewrite() + .validateFromSnapshot(snapshotBeforeDeletes) + .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), + branch)); // the rewrite should be valid when validating from the snapshot after the deletes - apply(table - .newRewrite() - .validateFromSnapshot(snapshotAfterDeletes) - .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), branch); + apply( + table + .newRewrite() + .validateFromSnapshot(snapshotAfterDeletes) + .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), + branch); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index e7c1d47b0954..f85b00cce799 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1286,7 +1286,7 @@ public void testConcurrentNonConflictingRowDelta() { commit(table, rowDelta, branch); - validateTableDeleteFilesWithRef(table, branch, deleteFile1, deleteFile2); + validateBranchDeleteFiles(table, branch, deleteFile1, deleteFile2); } @Test @@ -1335,13 +1335,8 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe commit(table, rowDelta, branch); commit(table, rewriteFiles, branch); - if (branch == "testBranch") { - validateTableDeleteFilesWithRef(table, "testBranch", deleteFile1); - validateTableFilesWithRef(table, branch, dataFile2); - } else { - validateTableDeleteFiles(table, deleteFile1); - validateTableFiles(table, dataFile2); - } + validateBranchDeleteFiles(table, branch, deleteFile1); + validateBranchFiles(table, branch, dataFile2); } @Test @@ -1517,50 +1512,4 @@ public void testRowDeltaCaseSensitivity() { .validateNoConflictingDeleteFiles() .commit()); } - - @Test - public void testBranchValidationsNotValidAncestor() { - table.newAppend().appendFile(FILE_A).commit(); - table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); - table.newAppend().appendFile(FILE_B).commit(); - - // This commit will result in validation exception as we start validation from a snapshot which - // is not an ancestor of the branch - RowDelta rowDelta = - table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(table.currentSnapshot().snapshotId()) - .conflictDetectionFilter(Expressions.alwaysTrue()) - .validateNoConflictingDeleteFiles(); - - AssertHelpers.assertThrows( - "Snapshot 2 is not an ancestor of 1", - IllegalArgumentException.class, - () -> rowDelta.commit()); - } - - @Test - public void testBranchValidationsValidAncestor() { - table.newAppend().appendFile(FILE_A).commit(); - Long ancestorSnapshot = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createBranch("branch", ancestorSnapshot).commit(); - - // This commit not result in validation exception as we start validation from a snapshot which - // is an actual ancestor of the branch - table - .newRowDelta() - .toBranch("branch") - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(ancestorSnapshot) - .conflictDetectionFilter(Expressions.alwaysTrue()) - .validateNoConflictingDeleteFiles() - .commit(); - - int branchSnapshot = 2; - - Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); - Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); - } } From 4f64b09e795874cbeb005503d44a3b2138e50015 Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Sun, 15 Jan 2023 13:16:24 -0800 Subject: [PATCH 22/27] Build fixes --- .palantir/revapi.yml | 239 ++++++++++++++++++ .../org/apache/iceberg/SnapshotProducer.java | 26 +- .../iceberg/io/TestFileWriterFactory.java | 45 +--- 3 files changed, 244 insertions(+), 66 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 781cf369dff9..4f3cf83e4367 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -261,6 +261,235 @@ acceptedBreaks: \ T) throws java.io.IOException, com.fasterxml.jackson.core.JacksonException\ \ @ org.apache.iceberg.rest.RESTSerializers.UpdateRequirementDeserializer" justification: "False positive - JacksonException is a subclass of IOException" + - code: "java.method.inheritedMovedToClass" + old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + new: "method org.apache.iceberg.BaseOverwriteFiles org.apache.iceberg.BaseOverwriteFiles::toBranch(java.lang.String)" + justification: "Introducing branch snapshot operations on BaseOverwrite" + - code: "java.method.inheritedMovedToClass" + old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + new: "method org.apache.iceberg.BaseReplacePartitions org.apache.iceberg.BaseReplacePartitions::toBranch(java.lang.String)" + justification: "Introducing branch snapshot operations for BaseReplacePartitions" + - code: "java.method.numberOfParametersChanged" + old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ + \ @ org.apache.iceberg.StreamingDelete" + new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ + \ @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" + justification: "Passing parent snapshot for supporting operations on branch" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ + \ @ org.apache.iceberg.StreamingDelete" + new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ + \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ + \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" + justification: "Passing parent snapshot for supporting operations on branch" - code: "java.method.removed" old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::filterDeleted(org.apache.iceberg.io.CloseableIterable,\ \ java.util.function.Predicate)" @@ -349,6 +578,16 @@ acceptedBreaks: old: "method void org.apache.iceberg.SnapshotProducer::validate(org.apache.iceberg.TableMetadata)\ \ @ org.apache.iceberg.StreamingDelete" justification: "Removing deprecations for 1.2.0" + - code: "java.method.returnTypeChangedCovariantly" + old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + new: "method org.apache.iceberg.BaseOverwriteFiles org.apache.iceberg.BaseOverwriteFiles::toBranch(java.lang.String)" + justification: "Introducing branch snapshot operations on BaseOverwrite" + - code: "java.method.returnTypeChangedCovariantly" + old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ + \ @ org.apache.iceberg.BaseReplacePartitions" + new: "method org.apache.iceberg.BaseReplacePartitions org.apache.iceberg.BaseReplacePartitions::toBranch(java.lang.String)" + justification: "Introducing branch snapshot operations for BaseReplacePartitions" org.apache.iceberg:iceberg-orc: - code: "java.method.removed" old: "method org.apache.iceberg.orc.ORC.WriteBuilder org.apache.iceberg.orc.ORC.WriteBuilder::config(java.lang.String,\ diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f2b479601c8e..8dc175f16de8 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -267,28 +267,10 @@ private Map summary(TableMetadata previous) { } Map previousSummary; - if (!targetBranch.equals(SnapshotRef.MAIN_BRANCH)) { - if (previous.ref(targetBranch) != null) { - if (previous.snapshot(previous.ref(targetBranch).snapshotId()).summary() != null) { - previousSummary = previous.snapshot(previous.ref(targetBranch).snapshotId()).summary(); - } else { - previousSummary = ImmutableMap.of(); - } - } else { - // if there was no previous snapshot, default the summary to start totals at 0 - ImmutableMap.Builder summaryBuilder = ImmutableMap.builder(); - summaryBuilder - .put(SnapshotSummary.TOTAL_RECORDS_PROP, "0") - .put(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "0") - .put(SnapshotSummary.TOTAL_DATA_FILES_PROP, "0") - .put(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") - .put(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") - .put(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0"); - previousSummary = summaryBuilder.build(); - } - } else if (previous.currentSnapshot() != null) { - if (previous.currentSnapshot().summary() != null) { - previousSummary = previous.currentSnapshot().summary(); + SnapshotRef previousSnapshot = previous.ref(targetBranch); + if (previousSnapshot != null) { + if (previous.snapshot(previousSnapshot.snapshotId()).summary() != null) { + previousSummary = previous.snapshot(previousSnapshot.snapshotId()).summary(); } else { // previous snapshot had no summary, use an empty summary previousSummary = ImmutableMap.of(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index 5dd9f286ff10..eff918b145f7 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -34,7 +34,6 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.orc.GenericOrcReader; @@ -137,7 +136,7 @@ public void testEqualityDeleteWriter() throws IOException { DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition); // commit the written data file - table.newRowDelta().toBranch("testBranch").addRows(dataFile).commit(); + table.newRowDelta().addRows(dataFile).commit(); // write an equality delete file List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(3, "bbb"), toRow(5, "ccc")); @@ -160,48 +159,6 @@ public void testEqualityDeleteWriter() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test - public void testEqualityDeletesWithBranch() throws IOException { - List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); - Schema equalityDeleteRowSchema = table.schema().select("id"); - FileWriterFactory writerFactory = - newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema); - - // write a data file - DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition); - - // commit the written data file - table.newRowDelta().toBranch("testBranch").addRows(dataFile).commit(); - - // write an equality delete file - List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(3, "bbb"), toRow(5, "ccc")); - DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); - - // verify the written delete file - GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema); - List expectedDeletes = - ImmutableList.of( - deleteRecord.copy("id", 1), deleteRecord.copy("id", 3), deleteRecord.copy("id", 5)); - InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); - List actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); - - // commit the written delete file - table.newRowDelta().toBranch("testBranch").addDeletes(deleteFile).commit(); - - // verify the delete file is applied correctly - List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table) - .useSnapshot(table.snapshot("testBranch").snapshotId()) - .select("*") - .build()) { - reader.forEach(set::add); - } - Assert.assertEquals("Records should match", toSet(expectedRows), set); - } - @Test public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { Assume.assumeFalse("Table must start unpartitioned", partitioned); From 90f23abcd38f9b039e179fab8c8c9c6b0c5d17e6 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 17 Jan 2023 12:39:23 -0800 Subject: [PATCH 23/27] Core: Add tests for replace partitions with branching, fix check for empty table --- .../iceberg/MergingSnapshotProducer.java | 2 +- .../apache/iceberg/TestReplacePartitions.java | 286 ++++++++++-------- 2 files changed, 165 insertions(+), 123 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index e58456c650fb..7e212c16401b 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -605,7 +605,7 @@ private CloseableIterable> deletedDataFiles( PartitionSet partitionSet, Snapshot parent) { // if there is no current table state, no files have been deleted - if (base.currentSnapshot() == null) { + if (parent == null) { return CloseableIterable.empty(); } diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index 6a23596aee06..dc74099028cd 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -70,40 +70,48 @@ public class TestReplacePartitions extends TableTestBase { .withRecordCount(1) .build(); - @Parameterized.Parameters(name = "formatVersion = {0}") + private final String branch; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") public static Object[] parameters() { - return new Object[] {1, 2}; + return new Object[][] { + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"} + }; } - public TestReplacePartitions(int formatVersion) { + public TestReplacePartitions(int formatVersion, String branch) { super(formatVersion); + this.branch = branch; } @Test public void testReplaceOnePartition() { - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); - table.newReplacePartitions().addFile(FILE_E).commit(); + table.newReplacePartitions().addFile(FILE_E).toBranch(branch).commit(); - long replaceId = readMetadata().currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals( "Table should have 2 manifests", 2, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(replaceId), files(FILE_E), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(replaceId, baseId), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); @@ -114,20 +122,22 @@ public void testReplaceAndMergeOnePartition() { // ensure the overwrite results in a merge table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); - table.newReplacePartitions().addFile(FILE_E).commit(); + table.newReplacePartitions().addFile(FILE_E).toBranch(branch).commit(); - long replaceId = readMetadata().currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(table, branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals( - "Table should have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + "Table should have 1 manifest", + 1, + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(replaceId, replaceId, baseId), files(FILE_E, FILE_A, FILE_B), statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); @@ -145,33 +155,34 @@ public void testReplaceWithUnpartitionedTable() throws IOException { Assert.assertEquals( "Table version should be 0", 0, (long) TestTables.metadataVersion("unpartitioned")); - unpartitioned.newAppend().appendFile(FILE_A).commit(); - + unpartitioned.newAppend().appendFile(FILE_A).toBranch(branch).commit(); // make sure the data was successfully added Assert.assertEquals( "Table version should be 1", 1, (long) TestTables.metadataVersion("unpartitioned")); - validateSnapshot(null, TestTables.readMetadata("unpartitioned").currentSnapshot(), FILE_A); + validateSnapshot( + null, latestSnapshot(TestTables.readMetadata("unpartitioned"), branch), FILE_A); - unpartitioned.newReplacePartitions().addFile(FILE_B).commit(); + ReplacePartitions replacePartitions = unpartitioned.newReplacePartitions().addFile(FILE_B); + commit(table, replacePartitions, branch); Assert.assertEquals( "Table version should be 2", 2, (long) TestTables.metadataVersion("unpartitioned")); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(replaceMetadata, branch).snapshotId(); Assert.assertEquals( "Table should have 2 manifests", 2, - replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).size()); + latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).size()); validateManifestEntries( - replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(0), + latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).get(0), ids(replaceId), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(1), + latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).get(1), ids(replaceId), files(FILE_A), statuses(Status.DELETED)); @@ -192,27 +203,30 @@ public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { Assert.assertEquals( "Table version should be 1", 1, (long) TestTables.metadataVersion("unpartitioned")); - unpartitioned.newAppend().appendFile(FILE_A).commit(); + AppendFiles appendFiles = unpartitioned.newAppend().appendFile(FILE_A); + commit(table, appendFiles, branch); // make sure the data was successfully added Assert.assertEquals( "Table version should be 2", 2, (long) TestTables.metadataVersion("unpartitioned")); - validateSnapshot(null, TestTables.readMetadata("unpartitioned").currentSnapshot(), FILE_A); + validateSnapshot( + null, latestSnapshot(TestTables.readMetadata("unpartitioned"), branch), FILE_A); - unpartitioned.newReplacePartitions().addFile(FILE_B).commit(); + ReplacePartitions replacePartitions = unpartitioned.newReplacePartitions().addFile(FILE_B); + commit(table, replacePartitions, branch); Assert.assertEquals( "Table version should be 3", 3, (long) TestTables.metadataVersion("unpartitioned")); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(replaceMetadata, branch).snapshotId(); Assert.assertEquals( "Table should have 1 manifest", 1, - replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).size()); + latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).size()); validateManifestEntries( - replaceMetadata.currentSnapshot().allManifests(unpartitioned.io()).get(0), + latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).get(0), ids(replaceId, replaceId), files(FILE_B, FILE_A), statuses(Status.ADDED, Status.DELETED)); @@ -220,13 +234,18 @@ public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { @Test public void testValidationFailure() { - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); ReplacePartitions replace = - table.newReplacePartitions().addFile(FILE_F).addFile(FILE_G).validateAppendOnly(); + table + .newReplacePartitions() + .addFile(FILE_F) + .addFile(FILE_G) + .toBranch(branch) + .validateAppendOnly(); AssertHelpers.assertThrows( "Should reject commit with file not matching delete expression", @@ -235,34 +254,36 @@ public void testValidationFailure() { replace::commit); Assert.assertEquals( - "Should not create a new snapshot", baseId, readMetadata().currentSnapshot().snapshotId()); + "Should not create a new snapshot", + baseId, + latestSnapshot(readMetadata(), branch).snapshotId()); } @Test public void testValidationSuccess() { - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); - table.newReplacePartitions().addFile(FILE_G).validateAppendOnly().commit(); + table.newReplacePartitions().addFile(FILE_G).validateAppendOnly().toBranch(branch).commit(); - long replaceId = readMetadata().currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); Assert.assertEquals( "Table should have 2 manifests", 2, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); // manifest is not merged because it is less than the minimum validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(replaceId), files(FILE_G), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(baseId, baseId), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); @@ -270,7 +291,7 @@ public void testValidationSuccess() { @Test public void testValidationNotInvoked() { - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); @@ -278,25 +299,29 @@ public void testValidationNotInvoked() { table .newReplacePartitions() .addFile(FILE_E) - .validateFromSnapshot(base.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(base, branch).snapshotId()) + .toBranch(branch) .commit(); table .newReplacePartitions() .addFile(FILE_A) // Replaces FILE_E which becomes Deleted .addFile(FILE_B) - .validateFromSnapshot(base.currentSnapshot().snapshotId()) + .validateFromSnapshot(latestSnapshot(base, branch).snapshotId()) + .toBranch(branch) .commit(); - long replaceId = readMetadata().currentSnapshot().snapshotId(); + long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( - "Table should have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + "Table should have 2 manifest", + 2, + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(replaceId, replaceId), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(replaceId), files(FILE_E), statuses(Status.DELETED)); @@ -304,7 +329,7 @@ public void testValidationNotInvoked() { @Test public void testValidateWithDefaultSnapshotId() { - table.newReplacePartitions().addFile(FILE_A).commit(); + table.newReplacePartitions().addFile(FILE_A).toBranch(branch).commit(); // Concurrent Replace Partitions should fail with ValidationException ReplacePartitions replace = table.newReplacePartitions(); @@ -319,18 +344,19 @@ public void testValidateWithDefaultSnapshotId() { .addFile(FILE_B) .validateNoConflictingData() .validateNoConflictingDeletes() + .toBranch(branch) .commit()); } @Test public void testConcurrentReplaceConflict() { - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); // Concurrent Replace Partitions should fail with ValidationException - table.newReplacePartitions().addFile(FILE_A).commit(); + table.newReplacePartitions().addFile(FILE_A).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -345,40 +371,42 @@ public void testConcurrentReplaceConflict() { .addFile(FILE_B) .validateNoConflictingData() .validateNoConflictingDeletes() + .toBranch(branch) .commit()); } @Test public void testConcurrentReplaceNoConflict() { - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long id1 = base.currentSnapshot().snapshotId(); + long id1 = latestSnapshot(base, branch).snapshotId(); // Concurrent Replace Partitions should not fail if concerning different partitions - table.newReplacePartitions().addFile(FILE_A).commit(); - long id2 = readMetadata().currentSnapshot().snapshotId(); + table.newReplacePartitions().addFile(FILE_A).toBranch(branch).commit(); + long id2 = latestSnapshot(readMetadata(), branch).snapshotId(); table .newReplacePartitions() .validateFromSnapshot(id1) .validateNoConflictingData() .validateNoConflictingDeletes() + .toBranch(branch) .addFile(FILE_B) .commit(); - long id3 = readMetadata().currentSnapshot().snapshotId(); + long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( "Table should have 2 manifests", 2, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(id2), files(FILE_A), statuses(Status.ADDED)); @@ -389,13 +417,13 @@ public void testConcurrentReplaceConflictNonPartitioned() { Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceBaseId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceBaseId = latestSnapshot(replaceMetadata, branch).snapshotId(); // Concurrent ReplacePartitions should fail with ValidationException - unpartitioned.newReplacePartitions().addFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newReplacePartitions().addFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -409,18 +437,19 @@ public void testConcurrentReplaceConflictNonPartitioned() { .validateNoConflictingData() .validateNoConflictingDeletes() .addFile(FILE_UNPARTITIONED_A) + .toBranch(branch) .commit()); } @Test public void testAppendReplaceConflict() { - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); // Concurrent Append and ReplacePartition should fail with ValidationException - table.newFastAppend().appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -435,46 +464,48 @@ public void testAppendReplaceConflict() { .validateNoConflictingDeletes() .addFile(FILE_A) .addFile(FILE_B) + .toBranch(branch) .commit()); } @Test public void testAppendReplaceNoConflict() { - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long id1 = base.currentSnapshot().snapshotId(); + long id1 = latestSnapshot(base, branch).snapshotId(); // Concurrent Append and ReplacePartition should not conflict if concerning different partitions - table.newFastAppend().appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).toBranch(branch).commit(); - long id2 = readMetadata().currentSnapshot().snapshotId(); + long id2 = latestSnapshot(readMetadata(), branch).snapshotId(); table .newReplacePartitions() .validateFromSnapshot(id1) .validateNoConflictingData() .validateNoConflictingDeletes() + .toBranch(branch) .addFile(FILE_A) .commit(); - long id3 = readMetadata().currentSnapshot().snapshotId(); + long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( "Table should have 3 manifests", 3, - table.currentSnapshot().allManifests(table.io()).size()); + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), files(FILE_A), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(id2), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(2), + latestSnapshot(table, branch).allManifests(table.io()).get(2), ids(id3), files(FILE_A), statuses(Status.DELETED)); @@ -485,13 +516,13 @@ public void testAppendReplaceConflictNonPartitioned() { Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceBaseId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceBaseId = latestSnapshot(replaceMetadata, branch).snapshotId(); // Concurrent Append and ReplacePartitions should fail with ValidationException - unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -505,19 +536,25 @@ public void testAppendReplaceConflictNonPartitioned() { .validateNoConflictingData() .validateNoConflictingDeletes() .addFile(FILE_UNPARTITIONED_A) + .toBranch(branch) .commit()); } @Test public void testDeleteReplaceConflict() { Assume.assumeTrue(formatVersion == 2); - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); // Concurrent Delete and ReplacePartition should fail with ValidationException - table.newRowDelta().addDeletes(FILE_A_DELETES).validateFromSnapshot(baseId).commit(); + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(baseId) + .toBranch(branch) + .commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -530,6 +567,7 @@ public void testDeleteReplaceConflict() { .validateFromSnapshot(baseId) .validateNoConflictingData() .validateNoConflictingDeletes() + .toBranch(branch) .addFile(FILE_A) .commit()); } @@ -541,13 +579,13 @@ public void testDeleteReplaceConflictNonPartitioned() { Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - unpartitioned.newAppend().appendFile(FILE_A).commit(); + unpartitioned.newAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceBaseId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceBaseId = latestSnapshot(replaceMetadata, branch).snapshotId(); // Concurrent Delete and ReplacePartitions should fail with ValidationException - unpartitioned.newRowDelta().addDeletes(FILE_UNPARTITIONED_A_DELETES).commit(); + unpartitioned.newRowDelta().addDeletes(FILE_UNPARTITIONED_A_DELETES).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -561,14 +599,15 @@ public void testDeleteReplaceConflictNonPartitioned() { .validateNoConflictingData() .validateNoConflictingDeletes() .addFile(FILE_UNPARTITIONED_A) + .toBranch(branch) .commit()); } @Test public void testDeleteReplaceNoConflict() { Assume.assumeTrue(formatVersion == 2); - table.newFastAppend().appendFile(FILE_A).commit(); - long id1 = readMetadata().currentSnapshot().snapshotId(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); + long id1 = latestSnapshot(readMetadata(), branch).snapshotId(); // Concurrent Delta and ReplacePartition should not conflict if concerning different partitions table @@ -578,8 +617,9 @@ public void testDeleteReplaceNoConflict() { .validateNoConflictingDataFiles() .validateNoConflictingDeleteFiles() .validateFromSnapshot(id1) + .toBranch(branch) .commit(); - long id2 = readMetadata().currentSnapshot().snapshotId(); + long id2 = latestSnapshot(readMetadata(), branch).snapshotId(); table .newReplacePartitions() @@ -587,23 +627,26 @@ public void testDeleteReplaceNoConflict() { .validateNoConflictingDeletes() .validateFromSnapshot(id1) .addFile(FILE_B) + .toBranch(branch) .commit(); - long id3 = readMetadata().currentSnapshot().snapshotId(); + long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( - "Table should have 3 manifest", 3, table.currentSnapshot().allManifests(table.io()).size()); + "Table should have 3 manifest", + 3, + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(id1), files(FILE_A), statuses(Status.ADDED)); validateDeleteManifest( - table.currentSnapshot().allManifests(table.io()).get(2), + latestSnapshot(table, branch).allManifests(table.io()).get(2), dataSeqs(2L), fileSeqs(2L), ids(id2), @@ -614,13 +657,13 @@ public void testDeleteReplaceNoConflict() { @Test public void testOverwriteReplaceConflict() { Assume.assumeTrue(formatVersion == 2); - table.newFastAppend().appendFile(FILE_A).commit(); + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); // Concurrent Overwrite and ReplacePartition should fail with ValidationException - table.newOverwrite().deleteFile(FILE_A).commit(); + table.newOverwrite().deleteFile(FILE_A).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -634,20 +677,21 @@ public void testOverwriteReplaceConflict() { .validateNoConflictingData() .validateNoConflictingDeletes() .addFile(FILE_A) + .toBranch(branch) .commit()); } @Test public void testOverwriteReplaceNoConflict() { Assume.assumeTrue(formatVersion == 2); - table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).toBranch(branch).commit(); TableMetadata base = readMetadata(); - long baseId = base.currentSnapshot().snapshotId(); + long baseId = latestSnapshot(base, branch).snapshotId(); // Concurrent Overwrite and ReplacePartition should not fail with if concerning different // partitions - table.newOverwrite().deleteFile(FILE_A).commit(); + table.newOverwrite().deleteFile(FILE_A).toBranch(branch).commit(); table .newReplacePartitions() @@ -655,19 +699,22 @@ public void testOverwriteReplaceNoConflict() { .validateNoConflictingDeletes() .validateFromSnapshot(baseId) .addFile(FILE_B) + .toBranch(branch) .commit(); - long finalId = readMetadata().currentSnapshot().snapshotId(); + long finalId = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( - "Table should have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + "Table should have 2 manifest", + 2, + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(finalId), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(finalId), files(FILE_B), statuses(Status.DELETED)); @@ -681,13 +728,13 @@ public void testOverwriteReplaceConflictNonPartitioned() { TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); - long replaceBaseId = replaceMetadata.currentSnapshot().snapshotId(); + long replaceBaseId = latestSnapshot(replaceMetadata, branch).snapshotId(); // Concurrent Overwrite and ReplacePartitions should fail with ValidationException - unpartitioned.newOverwrite().deleteFile(FILE_UNPARTITIONED_A).commit(); + unpartitioned.newOverwrite().deleteFile(FILE_UNPARTITIONED_A).toBranch(branch).commit(); AssertHelpers.assertThrows( "Should reject commit with file matching partitions replaced", @@ -701,39 +748,43 @@ public void testOverwriteReplaceConflictNonPartitioned() { .validateNoConflictingData() .validateNoConflictingDeletes() .addFile(FILE_UNPARTITIONED_A) + .toBranch(branch) .commit()); } @Test public void testValidateOnlyDeletes() { - table.newAppend().appendFile(FILE_A).commit(); - long baseId = readMetadata().currentSnapshot().snapshotId(); + table.newAppend().appendFile(FILE_A).toBranch(branch).commit(); + long baseId = latestSnapshot(readMetadata(), branch).snapshotId(); // Snapshot Isolation mode: appends do not conflict with replace - table.newAppend().appendFile(FILE_B).commit(); + table.newAppend().appendFile(FILE_B).toBranch(branch).commit(); table .newReplacePartitions() .validateFromSnapshot(baseId) .validateNoConflictingDeletes() .addFile(FILE_B) + .toBranch(branch) .commit(); - long finalId = readMetadata().currentSnapshot().snapshotId(); + long finalId = latestSnapshot(readMetadata(), branch).snapshotId(); Assert.assertEquals( - "Table should have 3 manifest", 3, table.currentSnapshot().allManifests(table.io()).size()); + "Table should have 3 manifest", + 3, + latestSnapshot(table, branch).allManifests(table.io()).size()); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(0), + latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(finalId), files(FILE_B), statuses(Status.ADDED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(1), + latestSnapshot(table, branch).allManifests(table.io()).get(1), ids(finalId), files(FILE_B), statuses(Status.DELETED)); validateManifestEntries( - table.currentSnapshot().allManifests(table.io()).get(2), + latestSnapshot(table, branch).allManifests(table.io()).get(2), ids(baseId), files(FILE_A), statuses(Status.ADDED)); @@ -743,13 +794,4 @@ public void testValidateOnlyDeletes() { public void testEmptyPartitionPathWithUnpartitionedTable() { DataFiles.builder(PartitionSpec.unpartitioned()).withPartitionPath(""); } - - @Test - public void testReplacePartitionsOnBranchUnsupported() { - AssertHelpers.assertThrows( - "Should reject committing rewrite manifests to branch", - UnsupportedOperationException.class, - "Cannot commit to branch someBranch: org.apache.iceberg.BaseReplacePartitions does not support branch commits", - () -> table.newReplacePartitions().addFile(FILE_UNPARTITIONED_A).toBranch("someBranch")); - } } From 66e1850b7d45dc5a05b979388e0d6e1c4dbf89ac Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Tue, 17 Jan 2023 20:07:39 -0800 Subject: [PATCH 24/27] overwrite validation tests --- .../org/apache/iceberg/TableTestBase.java | 14 - .../iceberg/TestOverwriteWithValidation.java | 486 ++++++++++-------- 2 files changed, 263 insertions(+), 237 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 7df2fe1dd86f..0914f1d77370 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -470,20 +470,6 @@ void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); } - void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DeleteFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().planFiles()) { - for (DeleteFile file : task.deletes()) { - actualFilePaths.add(file.path()); - } - } - Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); - } - void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DeleteFile file : expectedFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java index acc11b6d8e82..0b7ed030252a 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java @@ -177,13 +177,21 @@ public class TestOverwriteWithValidation extends TableTestBase { private static final Expression EXPRESSION_DAY_2_ANOTHER_ID_RANGE = greaterThanOrEqual("id", 10L); - @Parameterized.Parameters(name = "formatVersion = {0}") + private final String branch; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") public static Object[] parameters() { - return new Object[] {1, 2}; + return new Object[][] { + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"} + }; } - public TestOverwriteWithValidation(int formatVersion) { + public TestOverwriteWithValidation(int formatVersion, String branch) { super(formatVersion); + this.branch = branch; } private static ByteBuffer longToBuffer(long value) { @@ -202,112 +210,120 @@ public void before() throws IOException { @Test public void testOverwriteEmptyTableNotValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newOverwrite().addFile(FILE_DAY_2_MODIFIED).commit(); + commit(table, table.newOverwrite().addFile(FILE_DAY_2_MODIFIED), branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteEmptyTableStrictValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table - .newOverwrite() - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(alwaysTrue()) - .validateNoConflictingData() - .commit(); + commit( + table, + table + .newOverwrite() + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(alwaysTrue()) + .validateNoConflictingData(), + branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteEmptyTableValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table - .newOverwrite() - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(EXPRESSION_DAY_2) - .validateNoConflictingData() - .commit(); + commit( + table, + table + .newOverwrite() + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(EXPRESSION_DAY_2) + .validateNoConflictingData(), + branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteTableNotValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); - table.newOverwrite().deleteFile(FILE_DAY_2).addFile(FILE_DAY_2_MODIFIED).commit(); + commit(table, table.newOverwrite().deleteFile(FILE_DAY_2).addFile(FILE_DAY_2_MODIFIED), branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteTableStrictValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); - table - .newOverwrite() - .deleteFile(FILE_DAY_2) - .addFile(FILE_DAY_2_MODIFIED) - .validateFromSnapshot(baseSnapshot.snapshotId()) - .conflictDetectionFilter(alwaysTrue()) - .validateNoConflictingData() - .commit(); + commit( + table, + table + .newOverwrite() + .deleteFile(FILE_DAY_2) + .addFile(FILE_DAY_2_MODIFIED) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .conflictDetectionFilter(alwaysTrue()) + .validateNoConflictingData(), + branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteTableValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); - table - .newOverwrite() - .deleteFile(FILE_DAY_2) - .addFile(FILE_DAY_2_MODIFIED) - .validateFromSnapshot(baseSnapshot.snapshotId()) - .conflictDetectionFilter(EXPRESSION_DAY_2) - .validateNoConflictingData() - .commit(); + commit( + table, + table + .newOverwrite() + .deleteFile(FILE_DAY_2) + .addFile(FILE_DAY_2_MODIFIED) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .conflictDetectionFilter(EXPRESSION_DAY_2) + .validateNoConflictingData(), + branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteCompatibleAdditionNotValidated() { - table.newAppend().appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); - validateSnapshot(null, table.currentSnapshot(), FILE_DAY_2); + validateSnapshot(null, latestSnapshot(table, branch), FILE_DAY_2); OverwriteFiles overwrite = table.newOverwrite().deleteFile(FILE_DAY_2).addFile(FILE_DAY_2_MODIFIED); - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteCompatibleAdditionStrictValidated() { - table.newAppend().appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_2); OverwriteFiles overwrite = @@ -319,26 +335,26 @@ public void testOverwriteCompatibleAdditionStrictValidated() { .conflictDetectionFilter(alwaysTrue()) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_1).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Found conflicting files", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteCompatibleAdditionValidated() { - table.newAppend().appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_2); OverwriteFiles overwrite = @@ -350,18 +366,18 @@ public void testOverwriteCompatibleAdditionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteCompatibleDeletionValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); OverwriteFiles overwrite = @@ -373,18 +389,18 @@ public void testOverwriteCompatibleDeletionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newDelete().deleteFile(FILE_DAY_1).commit(); + commit(table, table.newDelete().deleteFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteIncompatibleAdditionValidated() { - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1); OverwriteFiles overwrite = @@ -395,26 +411,26 @@ public void testOverwriteIncompatibleAdditionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_2).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Found conflicting files", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteIncompatibleDeletionValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); OverwriteFiles overwrite = @@ -426,26 +442,26 @@ public void testOverwriteIncompatibleDeletionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newDelete().deleteFile(FILE_DAY_2).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + commit(table, table.newDelete().deleteFile(FILE_DAY_2), branch); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Missing required files to delete:", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteCompatibleRewriteAllowed() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); OverwriteFiles overwrite = @@ -457,23 +473,25 @@ public void testOverwriteCompatibleRewriteAllowed() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table - .newRewrite() - .rewriteFiles(ImmutableSet.of(FILE_DAY_2), ImmutableSet.of(FILE_DAY_2)) - .commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + commit( + table, + table.newRewrite().rewriteFiles(ImmutableSet.of(FILE_DAY_2), ImmutableSet.of(FILE_DAY_2)), + branch); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - overwrite.commit(); + commit(table, overwrite, branch); Assert.assertNotEquals( - "Should successfully commit", committedSnapshotId, table.currentSnapshot().snapshotId()); + "Should successfully commit", + committedSnapshotId, + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteCompatibleExpirationAdditionValidated() { - table.newAppend().appendFile(FILE_DAY_2).commit(); // id 1 + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); // id 1 - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_2); OverwriteFiles overwrite = @@ -485,20 +503,20 @@ public void testOverwriteCompatibleExpirationAdditionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_1).commit(); // id 2 + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); // id 2 table.expireSnapshots().expireSnapshotId(1L).commit(); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteCompatibleExpirationDeletionValidated() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); // id 1 + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); // id 1 - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); validateSnapshot(null, baseSnapshot, FILE_DAY_1, FILE_DAY_2); OverwriteFiles overwrite = @@ -510,20 +528,20 @@ public void testOverwriteCompatibleExpirationDeletionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newDelete().deleteFile(FILE_DAY_1).commit(); // id 2 + commit(table, table.newDelete().deleteFile(FILE_DAY_1), branch); // id 2 table.expireSnapshots().expireSnapshotId(1L).commit(); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteIncompatibleExpirationValidated() { - table.newAppend().appendFile(FILE_DAY_1).commit(); // id 1 + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); // id 1 - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table .newOverwrite() @@ -532,28 +550,28 @@ public void testOverwriteIncompatibleExpirationValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_2).commit(); // id 2 + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); // id 2 - table.newDelete().deleteFile(FILE_DAY_1).commit(); // id 3 + commit(table, table.newDelete().deleteFile(FILE_DAY_1), branch); // id 3 table.expireSnapshots().expireSnapshotId(2L).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Cannot determine history", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteIncompatibleBaseExpirationEmptyTableValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); OverwriteFiles overwrite = table @@ -562,28 +580,28 @@ public void testOverwriteIncompatibleBaseExpirationEmptyTableValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_2).commit(); // id 1 + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); // id 1 - table.newDelete().deleteFile(FILE_DAY_1).commit(); // id 2 + commit(table, table.newDelete().deleteFile(FILE_DAY_1), branch); // id 2 table.expireSnapshots().expireSnapshotId(1L).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Cannot determine history", - overwrite::commit); + () -> commit(table, overwrite, branch)); Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testOverwriteAnotherRangeValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); OverwriteFiles overwrite = table @@ -592,16 +610,16 @@ public void testOverwriteAnotherRangeValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2_ID_RANGE) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testOverwriteAnotherRangeWithinPartitionValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); Expression conflictDetectionFilter = and(EXPRESSION_DAY_2, EXPRESSION_DAY_2_ID_RANGE); OverwriteFiles overwrite = @@ -611,20 +629,20 @@ public void testOverwriteAnotherRangeWithinPartitionValidated() { .conflictDetectionFilter(conflictDetectionFilter) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_2_ANOTHER_RANGE).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_2_ANOTHER_RANGE), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_2_ANOTHER_RANGE, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_ANOTHER_RANGE, FILE_DAY_2_MODIFIED); } @Test public void testTransactionCompatibleAdditionValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); - Snapshot baseSnapshot = table.currentSnapshot(); + Snapshot baseSnapshot = latestSnapshot(table, branch); Transaction txn = table.newTransaction(); OverwriteFiles overwrite = @@ -635,21 +653,21 @@ public void testTransactionCompatibleAdditionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); txn.commitTransaction(); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } @Test public void testTransactionIncompatibleAdditionValidated() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); Transaction txn = table.newTransaction(); - txn.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, txn.newAppend().appendFile(FILE_DAY_1), branch); OverwriteFiles overwrite = txn.newOverwrite() @@ -657,10 +675,10 @@ public void testTransactionIncompatibleAdditionValidated() { .conflictDetectionFilter(EXPRESSION_DAY_2) .validateNoConflictingData(); - table.newAppend().appendFile(FILE_DAY_2).commit(); - long committedSnapshotId = table.currentSnapshot().snapshotId(); + commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); + long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - overwrite.commit(); + commit(table, overwrite, branch); AssertHelpers.assertThrows( "Should reject commit", @@ -671,18 +689,18 @@ public void testTransactionIncompatibleAdditionValidated() { Assert.assertEquals( "Should not create a new snapshot", committedSnapshotId, - table.currentSnapshot().snapshotId()); + latestSnapshot(table, branch).snapshotId()); } @Test public void testConcurrentConflictingPositionDeletes() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -694,21 +712,24 @@ public void testConcurrentConflictingPositionDeletes() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); AssertHelpers.assertThrows( - "Should reject commit", ValidationException.class, "found new delete", overwrite::commit); + "Should reject commit", + ValidationException.class, + "found new delete", + () -> commit(table, overwrite, branch)); } @Test public void testConcurrentConflictingPositionDeletesOverwriteByFilter() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -720,22 +741,22 @@ public void testConcurrentConflictingPositionDeletesOverwriteByFilter() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Found new conflicting delete", - overwrite::commit); + () -> commit(table, overwrite, branch)); } @Test public void testConcurrentConflictingDataFileDeleteOverwriteByFilter() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -746,22 +767,22 @@ public void testConcurrentConflictingDataFileDeleteOverwriteByFilter() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newOverwrite().deleteFile(FILE_DAY_2).commit(); + commit(table, table.newOverwrite().deleteFile(FILE_DAY_2), branch); AssertHelpers.assertThrows( "Should reject commit", ValidationException.class, "Found conflicting deleted files", - overwrite::commit); + () -> commit(table, overwrite, branch)); } @Test public void testConcurrentNonConflictingDataFileDeleteOverwriteByFilter() { - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -772,22 +793,22 @@ public void testConcurrentNonConflictingDataFileDeleteOverwriteByFilter() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newOverwrite().deleteFile(FILE_DAY_1).commit(); + commit(table, table.newOverwrite().deleteFile(FILE_DAY_1), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_2_MODIFIED); + validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } @Test public void testConcurrentNonConflictingPositionDeletes() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -799,23 +820,23 @@ public void testConcurrentNonConflictingPositionDeletes() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); - validateTableDeleteFiles(table, FILE_DAY_1_POS_DELETES); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } @Test public void testConcurrentNonConflictingPositionDeletesOverwriteByFilter() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -827,23 +848,23 @@ public void testConcurrentNonConflictingPositionDeletesOverwriteByFilter() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); - validateTableDeleteFiles(table, FILE_DAY_1_POS_DELETES); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } @Test public void testConcurrentConflictingEqualityDeletes() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -855,21 +876,27 @@ public void testConcurrentConflictingEqualityDeletes() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_2_EQ_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_EQ_DELETES), branch); AssertHelpers.assertThrows( - "Should reject commit", ValidationException.class, "found new delete", overwrite::commit); + "Should reject commit", + ValidationException.class, + "found new delete", + () -> commit(table, overwrite, branch)); } @Test public void testConcurrentNonConflictingEqualityDeletes() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_2).appendFile(FILE_DAY_2_ANOTHER_RANGE).commit(); + commit( + table, + table.newAppend().appendFile(FILE_DAY_2).appendFile(FILE_DAY_2_ANOTHER_RANGE), + branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -881,23 +908,23 @@ public void testConcurrentNonConflictingEqualityDeletes() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_2_ANOTHER_RANGE, FILE_DAY_2_MODIFIED); - validateTableDeleteFiles(table, FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES); + validateBranchFiles(table, branch, FILE_DAY_2_ANOTHER_RANGE, FILE_DAY_2_MODIFIED); + validateBranchDeleteFiles(table, branch, FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES); } @Test public void testOverwriteByFilterInheritsConflictDetectionFilter() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - Snapshot firstSnapshot = table.currentSnapshot(); + Snapshot firstSnapshot = latestSnapshot(table, branch); OverwriteFiles overwrite = table @@ -909,19 +936,19 @@ public void testOverwriteByFilterInheritsConflictDetectionFilter() { .validateNoConflictingData() .validateNoConflictingDeletes(); - table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES).commit(); + commit(table, table.newRowDelta().addDeletes(FILE_DAY_1_POS_DELETES), branch); - overwrite.commit(); + commit(table, overwrite, branch); - validateTableFiles(table, FILE_DAY_1, FILE_DAY_2_MODIFIED); - validateTableDeleteFiles(table, FILE_DAY_1_POS_DELETES); + validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); + validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } @Test public void testOverwriteCaseSensitivity() { - table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); - table.newAppend().appendFile(FILE_DAY_1).commit(); + commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); Expression rowFilter = equal("dAtE", "2018-06-09"); @@ -930,25 +957,29 @@ public void testOverwriteCaseSensitivity() { ValidationException.class, "Cannot find field 'dAtE'", () -> - table - .newOverwrite() - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData() - .commit()); + commit( + table, + table + .newOverwrite() + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)); AssertHelpers.assertThrows( "Should fail with case sensitive binding", ValidationException.class, "Cannot find field 'dAtE'", () -> - table - .newOverwrite() - .caseSensitive(true) - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData() - .commit()); + commit( + table, + table + .newOverwrite() + .caseSensitive(true) + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)); // binding should succeed and trigger the validation AssertHelpers.assertThrows( @@ -956,36 +987,45 @@ public void testOverwriteCaseSensitivity() { ValidationException.class, "Found conflicting files", () -> - table - .newOverwrite() - .caseSensitive(false) - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData() - .commit()); + commit( + table, + table + .newOverwrite() + .caseSensitive(false) + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)); } @Test public void testMetadataOnlyDeleteWithPositionDeletes() { Assume.assumeTrue(formatVersion == 2); - Assert.assertNull("Should be empty table", table.currentSnapshot()); + Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); - table.newAppend().appendFile(FILE_DAY_2).appendFile(FILE_DAY_2_ANOTHER_RANGE).commit(); + commit( + table, + table.newAppend().appendFile(FILE_DAY_2).appendFile(FILE_DAY_2_ANOTHER_RANGE), + branch); - table - .newRowDelta() - .addDeletes(FILE_DAY_2_POS_DELETES) - .addDeletes(FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES) - .commit(); + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_DAY_2_POS_DELETES) + .addDeletes(FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES), + branch); - table - .newOverwrite() - .overwriteByRowFilter(EXPRESSION_DAY_2_ANOTHER_ID_RANGE) - .addFile(FILE_DAY_2_MODIFIED) - .commit(); + commit( + table, + table + .newOverwrite() + .overwriteByRowFilter(EXPRESSION_DAY_2_ANOTHER_ID_RANGE) + .addFile(FILE_DAY_2_MODIFIED), + branch); - validateTableFiles(table, FILE_DAY_2, FILE_DAY_2_MODIFIED); - validateTableDeleteFiles(table, FILE_DAY_2_POS_DELETES); + validateBranchFiles(table, branch, FILE_DAY_2, FILE_DAY_2_MODIFIED); + validateBranchDeleteFiles(table, branch, FILE_DAY_2_POS_DELETES); } } From fc8780c690aec00c331693ddef65746d4dceb743 Mon Sep 17 00:00:00 2001 From: Namratha Mysore Keshavaprakash Date: Tue, 17 Jan 2023 20:16:58 -0800 Subject: [PATCH 25/27] variable name change --- .../main/java/org/apache/iceberg/SnapshotProducer.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 8dc175f16de8..4a7aa746315f 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -267,10 +267,10 @@ private Map summary(TableMetadata previous) { } Map previousSummary; - SnapshotRef previousSnapshot = previous.ref(targetBranch); - if (previousSnapshot != null) { - if (previous.snapshot(previousSnapshot.snapshotId()).summary() != null) { - previousSummary = previous.snapshot(previousSnapshot.snapshotId()).summary(); + SnapshotRef previousBranchHead = previous.ref(targetBranch); + if (previousBranchHead != null) { + if (previous.snapshot(previousBranchHead.snapshotId()).summary() != null) { + previousSummary = previous.snapshot(previousBranchHead.snapshotId()).summary(); } else { // previous snapshot had no summary, use an empty summary previousSummary = ImmutableMap.of(); From 6b6aefcb6a838fe8eb47ea2762bb88757b22dd03 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 20 Jan 2023 14:57:04 -0800 Subject: [PATCH 26/27] Uncomment test. --- core/src/test/java/org/apache/iceberg/TestRowDelta.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index f85b00cce799..e0db24df228e 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -235,7 +235,7 @@ public void testValidateDataFilesExistReplacePartitions() { latestSnapshot(table, branch).deleteManifests(table.io()).size()); } - // @Test + @Test public void testValidateDataFilesExistFromSnapshot() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); From 1a98e5439b8b18382e2fce2d1e981b8272b53871 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 20 Jan 2023 19:32:47 -0800 Subject: [PATCH 27/27] Core: Deprecate validation methods which don't accept parent snapshot, update revapi --- .palantir/revapi.yml | 229 ------------------ .../iceberg/MergingSnapshotProducer.java | 194 ++++++++++++++- 2 files changed, 193 insertions(+), 230 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 9c9aca02e478..6c2628cc0026 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -261,235 +261,6 @@ acceptedBreaks: \ T) throws java.io.IOException, com.fasterxml.jackson.core.JacksonException\ \ @ org.apache.iceberg.rest.RESTSerializers.UpdateRequirementDeserializer" justification: "False positive - JacksonException is a subclass of IOException" - - code: "java.method.inheritedMovedToClass" - old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - new: "method org.apache.iceberg.BaseOverwriteFiles org.apache.iceberg.BaseOverwriteFiles::toBranch(java.lang.String)" - justification: "Introducing branch snapshot operations on BaseOverwrite" - - code: "java.method.inheritedMovedToClass" - old: "method ThisT org.apache.iceberg.SnapshotUpdate::toBranch(java.lang.String)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - new: "method org.apache.iceberg.BaseReplacePartitions org.apache.iceberg.BaseReplacePartitions::toBranch(java.lang.String)" - justification: "Introducing branch snapshot operations for BaseReplacePartitions" - - code: "java.method.numberOfParametersChanged" - old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet)\ - \ @ org.apache.iceberg.StreamingDelete" - new: "method org.apache.iceberg.DeleteFileIndex org.apache.iceberg.MergingSnapshotProducer::addedDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.util.PartitionSet,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateAddedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression)\ - \ @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDataFilesExist(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.CharSequenceSet, boolean, org.apache.iceberg.expressions.Expression,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateDeletedDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeleteFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.util.PartitionSet, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable) @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, java.lang.Iterable, org.apache.iceberg.Snapshot)\ - \ @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ - \ @ org.apache.iceberg.BaseOverwriteFiles" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseOverwriteFiles" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ - \ @ org.apache.iceberg.BaseReplacePartitions" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.BaseReplacePartitions" - justification: "Passing parent snapshot for supporting operations on branch" - - code: "java.method.numberOfParametersChanged" - old: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable)\ - \ @ org.apache.iceberg.StreamingDelete" - new: "method void org.apache.iceberg.MergingSnapshotProducer::validateNoNewDeletesForDataFiles(org.apache.iceberg.TableMetadata,\ - \ java.lang.Long, org.apache.iceberg.expressions.Expression, java.lang.Iterable,\ - \ org.apache.iceberg.Snapshot) @ org.apache.iceberg.StreamingDelete" - justification: "Passing parent snapshot for supporting operations on branch" - code: "java.method.removed" old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::filterDeleted(org.apache.iceberg.io.CloseableIterable,\ \ java.util.function.Predicate)" diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 7e212c16401b..57df521b512a 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -268,6 +268,20 @@ private ManifestFile copyManifest(ManifestFile manifest) { appendedManifestsSummary); } + /** + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param partitionSet a set of partitions to filter new conflicting data files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateAddedDataFiles(TableMetadata, Long, PartitionSet, + * Snapshot)} instead + */ + @Deprecated + protected void validateAddedDataFiles( + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { + validateAddedDataFiles(base, startingSnapshotId, partitionSet, base.currentSnapshot()); + } + /** * Validates that no files matching given partitions have been added to the table since a starting * snapshot. @@ -275,6 +289,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param partitionSet a set of partitions to filter new conflicting data files + * @param parent ending snapshot on the lineage being validated */ protected void validateAddedDataFiles( TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { @@ -296,6 +311,21 @@ protected void validateAddedDataFiles( } } + /** + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param conflictDetectionFilter an expression used to find new conflicting data files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateAddedDataFiles(TableMetadata, Long, Expression, Snapshot)} + * instead + */ + @Deprecated + protected void validateAddedDataFiles( + TableMetadata base, Long startingSnapshotId, Expression conflictDetectionFilter) { + validateAddedDataFiles( + base, startingSnapshotId, conflictDetectionFilter, base.currentSnapshot()); + } + /** * Validates that no files matching a filter have been added to the table since a starting * snapshot. @@ -328,13 +358,14 @@ protected void validateAddedDataFiles( } /** - * Returns an iterable of files matching a filter have been added to the table since a starting + * Returns an iterable of files matching a filter have been added to a branch since a starting * snapshot. * * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter an expression used to find new data files * @param partitionSet a set of partitions to find new data files + * @param parent ending snapshot of the branch */ private CloseableIterable> addedDataFiles( TableMetadata base, @@ -385,6 +416,7 @@ private CloseableIterable> addedDataFiles( * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFiles data files to validate have no new row deletes + * @param parent ending snapshot on the branch being validated */ protected void validateNoNewDeletesForDataFiles( TableMetadata base, Long startingSnapshotId, Iterable dataFiles, Snapshot parent) { @@ -392,6 +424,28 @@ protected void validateNoNewDeletesForDataFiles( base, startingSnapshotId, null, dataFiles, newFilesSequenceNumber != null, parent); } + /** + * Validates that no new delete files that must be applied to the given data files have been added + * to the table since a starting snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param dataFiles data files to validate have no new row deletes + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateNoNewDeletesForDataFiles} instead + */ + @Deprecated + protected void validateNoNewDeletesForDataFiles( + TableMetadata base, Long startingSnapshotId, Iterable dataFiles) { + validateNoNewDeletesForDataFiles( + base, + startingSnapshotId, + null, + dataFiles, + newFilesSequenceNumber != null, + base.currentSnapshot()); + } + /** * Validates that no new delete files that must be applied to the given data files have been added * to the table since a starting snapshot. @@ -400,6 +454,28 @@ protected void validateNoNewDeletesForDataFiles( * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter a data filter * @param dataFiles data files to validate have no new row deletes + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateNoNewDeletesForDataFiles} instead + */ + @Deprecated + protected void validateNoNewDeletesForDataFiles( + TableMetadata base, + Long startingSnapshotId, + Expression dataFilter, + Iterable dataFiles) { + validateNoNewDeletesForDataFiles( + base, startingSnapshotId, dataFilter, dataFiles, base.currentSnapshot()); + } + + /** + * Validates that no new delete files that must be applied to the given data files have been added + * to the table since a starting snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param dataFilter a data filter + * @param dataFiles data files to validate have no new row deletes + * @param parent ending snapshot on the branch being validated */ protected void validateNoNewDeletesForDataFiles( TableMetadata base, @@ -427,6 +503,7 @@ protected void validateNoNewDeletesForDataFiles( * @param dataFilter a data filter * @param dataFiles data files to validate have no new row deletes * @param ignoreEqualityDeletes whether equality deletes should be ignored in validation + * @param parent ending snapshot on the branch being validated */ private void validateNoNewDeletesForDataFiles( TableMetadata base, @@ -469,6 +546,23 @@ private void validateNoNewDeletesForDataFiles( * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter an expression used to find new conflicting delete files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateNoNewDeleteFiles} instead + */ + @Deprecated + protected void validateNoNewDeleteFiles( + TableMetadata base, Long startingSnapshotId, Expression dataFilter) { + validateNoNewDeleteFiles(base, startingSnapshotId, dataFilter, base.currentSnapshot()); + } + + /** + * Validates that no delete files matching a filter have been added to the table since a starting + * snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param dataFilter an expression used to find new conflicting delete files + * @param parent ending snapshot on the branch being validated */ protected void validateNoNewDeleteFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot parent) { @@ -487,6 +581,24 @@ protected void validateNoNewDeleteFiles( * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param partitionSet a partition set used to find new conflicting delete files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateNoNewDeleteFiles(TableMetadata, Long, PartitionSet, + * Snapshot)} instead + */ + @Deprecated + protected void validateNoNewDeleteFiles( + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { + validateNoNewDeleteFiles(base, startingSnapshotId, partitionSet, base.currentSnapshot()); + } + + /** + * Validates that no delete files matching a partition set have been added to the table since a + * starting snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param partitionSet a partition set used to find new conflicting delete files + * @param parent ending snapshot on the branch being validated */ protected void validateNoNewDeleteFiles( TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { @@ -506,6 +618,28 @@ protected void validateNoNewDeleteFiles( * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter an expression used to find delete files * @param partitionSet a partition set used to find delete files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#addedDeleteFiles(TableMetadata, Long, Expression, PartitionSet, + * Snapshot)} instead + */ + @Deprecated + protected DeleteFileIndex addedDeleteFiles( + TableMetadata base, + Long startingSnapshotId, + Expression dataFilter, + PartitionSet partitionSet) { + return addedDeleteFiles( + base, startingSnapshotId, dataFilter, partitionSet, base.currentSnapshot()); + } + + /** + * Returns matching delete files have been added to the table since a starting snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param dataFilter an expression used to find delete files + * @param partitionSet a partition set used to find delete files + * @param parent parent snapshot of the branch */ protected DeleteFileIndex addedDeleteFiles( TableMetadata base, @@ -540,6 +674,24 @@ protected DeleteFileIndex addedDeleteFiles( * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter an expression used to find deleted data files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateDeletedDataFiles(TableMetadata, Long, Expression, + * Snapshot)} instead + */ + @Deprecated + protected void validateDeletedDataFiles( + TableMetadata base, Long startingSnapshotId, Expression dataFilter) { + validateDeletedDataFiles(base, startingSnapshotId, dataFilter, base.currentSnapshot()); + } + + /** + * Validates that no files matching a filter have been deleted from the table since a starting + * snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param dataFilter an expression used to find deleted data files + * @param parent ending snapshot on the branch being validated */ protected void validateDeletedDataFiles( TableMetadata base, Long startingSnapshotId, Expression dataFilter, Snapshot parent) { @@ -568,6 +720,24 @@ protected void validateDeletedDataFiles( * @param base table metadata to validate * @param startingSnapshotId id of the snapshot current at the start of the operation * @param partitionSet a partition set used to find deleted data files + * @deprecated will be removed in 1.3.0; use {@link + * MergingSnapshotProducer#validateNoNewDeleteFiles(TableMetadata, Long, PartitionSet, + * Snapshot)} instead + */ + @Deprecated + protected void validateDeletedDataFiles( + TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet) { + validateDeletedDataFiles(base, startingSnapshotId, partitionSet, base.currentSnapshot()); + } + + /** + * Validates that no files matching a filter have been deleted from the table since a starting + * snapshot. + * + * @param base table metadata to validate + * @param startingSnapshotId id of the snapshot current at the start of the operation + * @param partitionSet a partition set used to find deleted data files + * @param parent ending snapshot on the branch being validated */ protected void validateDeletedDataFiles( TableMetadata base, Long startingSnapshotId, PartitionSet partitionSet, Snapshot parent) { @@ -597,6 +767,7 @@ protected void validateDeletedDataFiles( * @param startingSnapshotId id of the snapshot current at the start of the operation * @param dataFilter an expression used to find deleted data files * @param partitionSet a set of partitions to find deleted data files + * @param parent ending snapshot on the branch being validated */ private CloseableIterable> deletedDataFiles( TableMetadata base, @@ -675,6 +846,27 @@ private DeleteFileIndex buildDeleteFileIndex( return builder.build(); } + @Deprecated + /** + * @deprecated will be removed in 1.3.0 use {@link + * MergingSnapshotProducer#validateDataFilesExist(TableMetadata, Long, CharSequenceSet, + * boolean, Expression, Snapshot)} instead. + */ + protected void validateDataFilesExist( + TableMetadata base, + Long startingSnapshotId, + CharSequenceSet requiredDataFiles, + boolean skipDeletes, + Expression conflictDetectionFilter) { + validateDataFilesExist( + base, + startingSnapshotId, + requiredDataFiles, + skipDeletes, + conflictDetectionFilter, + base.currentSnapshot()); + } + @SuppressWarnings("CollectionUndefinedEquality") protected void validateDataFilesExist( TableMetadata base,