You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/10 18:30:39 UTC

[GitHub] [iceberg] namrathamyske opened a new pull request, #5234: Core, API: BaseRowDelta to branch Impl

namrathamyske opened a new pull request, #5234:
URL: https://github.com/apache/iceberg/pull/5234

   Issue addressed from: https://github.com/apache/iceberg/issues/3896
   
   This PR contains the Implementation part from https://github.com/apache/iceberg/pull/4926 for only BaseRowDelta to Table operation.
   
   Continued from https://github.com/apache/iceberg/pull/5010


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r952679192


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -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());
+  }

Review Comment:
   We can use SnapshotUtil.isAncestorOf, and I think we can extract the helper to a protected method on the SnapshotProducer class because it would be used for every operation.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1399167754

   Thanks for the reviews @rdblue ! @namrathamyske I raised a PR to your branch for deprecating the old validation methods and updating rev API https://github.com/namrathamyske/iceberg/pull/21. When you get a chance please take a look! 


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083109459


##########
core/src/test/java/org/apache/iceberg/TestReplacePartitions.java:
##########
@@ -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();

Review Comment:
   Why not use the `commit` method in these tests?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "namrathamyske (via GitHub)" <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083386045


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -96,23 +97,37 @@ public RowDelta validateNoConflictingDeleteFiles() {
   }
 
   @Override
-  protected void validate(TableMetadata base, Snapshot snapshot) {
-    if (base.currentSnapshot() != null) {
+  public RowDelta toBranch(String branch) {
+    targetBranch(branch);
+    return this;
+  }
+
+  @Override
+  protected void validate(TableMetadata base, Snapshot parent) {
+    if (parent != null) {
+      if (startingSnapshotId != null) {
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(parent.snapshotId(), startingSnapshotId, base::snapshot),
+            "Snapshot %s is not an ancestor of %s",

Review Comment:
   @amogh-jahagirdar Is this even required anymore ? We are anyway doing ancestor check in https://github.com/apache/iceberg/blob/a1d994c131dc5b5f42e1693e92e058493382593e/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java#L742



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083103566


##########
core/src/test/java/org/apache/iceberg/TestOverwrite.java:
##########
@@ -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();

Review Comment:
   Minor: prefer to keep ternary expressions as simple as possible. If you're calling a method twice with the same arguments, you generally want to use a variable:
   
   ```java
   Snapshot latest = latestSnapshot(base, branch);
   long baseId = latest == null ? -1 : latest.snapshotId();
   ```
   
   It's also a bit odd that the current snapshot is always defined, but here we handle the case where the branch does not yet exist. Why is that? Are we committing to main and then implicitly branching somewhere?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r953201851


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  public void testBranchConflictingDeletes() {

Review Comment:
   I think that this class either needs to copy all of the tests and validate against a branch (like `testValidateDataFilesExistDefaults`), or be parameterized. I prefer parameterized.
   
   If the tests are parameterized for `main` and `some_branch`, you could refactor all of the `commit` calls to something like this:
   
   ```
     public void commitOperation(SnapshotProducer producer) {
       if (branch.equals("main")) {
         producer.commit();
       } else {
         producer.toBranch(branch).commit();
       }
     }
   ```



##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  public void testBranchConflictingDeletes() {

Review Comment:
   I think that this class either needs to copy all of the tests and validate against a branch (like `testValidateDataFilesExistDefaults`), or be parameterized. I prefer parameterized.
   
   If the tests are parameterized for `main` and `some_branch`, you could refactor all of the `commit` calls to something like this:
   
   ```java
     public void commitOperation(SnapshotProducer producer) {
       if (branch.equals("main")) {
         producer.commit();
       } else {
         producer.toBranch(branch).commit();
       }
     }
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r953200438


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -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) {

Review Comment:
   I think that these should all be called `parent` so it is clear that this is the parent of the snapshot to be committed.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r952232778


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -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());
+  }

Review Comment:
   We can use SnapshotUtil.isAncestorOf for this.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue merged PR #5234:
URL: https://github.com/apache/iceberg/pull/5234


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1073044093


##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -472,6 +484,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
     Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DeleteFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) {
+      for (DeleteFile file : task.deletes()) {
+        actualFilePaths.add(file.path());
+      }
+    }
+    Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Will go ahead edit the whole class too.



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -472,6 +484,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
     Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DeleteFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) {
+      for (DeleteFile file : task.deletes()) {
+        actualFilePaths.add(file.path());
+      }
+    }
+    Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Will go ahead edit the whole test class too.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1067520428


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1449,18 +1519,48 @@ 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());
+        "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);

Review Comment:
   Sorry not getting why we need this test case, won't it be covered by the others? 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r953200130


##########
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);

Review Comment:
   Can we name this `parent` instead of `snapshot`? Snapshot is not very clear because there are a lot of snapshots.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r952688281


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -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) {

Review Comment:
   Would it make sense to pass in just the snapshot id, and call it endingSnapshotId? It also feels more readable to put the parameter next to startingSnapshotId because it's like we're passing in a start and end in the branch lineage. 
   
   For reference: https://github.com/apache/iceberg/pull/5595/files#diff-f2d4265e007483f23a69703b88348743aecf99cc98395cd1730de3900a609eb6R339



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083099568


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -96,23 +97,37 @@ public RowDelta validateNoConflictingDeleteFiles() {
   }
 
   @Override
-  protected void validate(TableMetadata base, Snapshot snapshot) {
-    if (base.currentSnapshot() != null) {
+  public RowDelta toBranch(String branch) {
+    targetBranch(branch);
+    return this;
+  }
+
+  @Override
+  protected void validate(TableMetadata base, Snapshot parent) {
+    if (parent != null) {
+      if (startingSnapshotId != null) {
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(parent.snapshotId(), startingSnapshotId, base::snapshot),
+            "Snapshot %s is not an ancestor of %s",
+            startingSnapshotId,
+            parent.snapshotId());
+      }

Review Comment:
   Nit: whitespace after this block.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083117789


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -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);
 
-    long deleteSnapshotId = table.currentSnapshot().snapshotId();
+    commit(table, rowDelta3, branch);
+
+    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
+  // @Test

Review Comment:
   This passes locally. I'm going to assume this was an accident and restore the test to get CI running it.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1073043781


##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -458,6 +458,18 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) {
     Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DataFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) {
+      actualFilePaths.add(task.file().path());
+    }
+    Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Hi, `validateTableFiles` has 30 usages in in TestOverwriteWithValidation



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -472,6 +484,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
     Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DeleteFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) {
+      for (DeleteFile file : task.deletes()) {
+        actualFilePaths.add(file.path());
+      }
+    }
+    Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   validateTableDeleteFiles has other 5 usages in TestOverwriteWithValidation



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1067542176


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1449,18 +1519,48 @@ 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());
+        "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);

Review Comment:
   Cool, will be removing these.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1080673261


##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -458,13 +458,25 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) {
     Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
   }
 
-  void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
+  void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DataFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();

Review Comment:
   Nit: block spacing.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083112330


##########
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);

Review Comment:
   Odd spacing here. I'd expect the commit to come after rowDelta insteadd of being combined with the latestSnapshot call.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamk commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamk commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r953530411


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  public void testBranchConflictingDeletes() {

Review Comment:
   My bad, this test is still a work in progress. Making more changes.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r952679192


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -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());
+  }

Review Comment:
   We can use SnapshotUtil.isAncestorOf, and I think we can extract the helper to a protected method on the SnapshotProducer class because it would be valid for every operation.



##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  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();

Review Comment:
   Let's double check the style, I think we just need to run spotless here.



##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -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) {

Review Comment:
   Would it make sense to pass in just the snapshot id, and call it endingSnapshotId? It also feels more readable to put the parameter next to startingSnapshotId. For reference: https://github.com/apache/iceberg/pull/5595/files#diff-f2d4265e007483f23a69703b88348743aecf99cc98395cd1730de3900a609eb6R339



##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  public void testBranchConflictingDeletes() {

Review Comment:
   This test looks like we're testing a case where there aren't conflicting deletes right?



##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1414,18 +1415,69 @@ public void testRowDeltaCaseSensitivity() {
   }
 
   @Test
-  public void testRowDeltaToBranchUnsupported() {
-    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",
-        () ->
+  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();
+    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()
-                .caseSensitive(false)
-                .addRows(FILE_B)
-                .addDeletes(FILE_A2_DELETES)
-                .toBranch("someBranch")
-                .commit());
+                    .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());
+  }
+
+  @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();

Review Comment:
   Same, spotless



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083113239


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -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);
 
-    long deleteSnapshotId = table.currentSnapshot().snapshotId();
+    commit(table, rowDelta3, branch);
+
+    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
+  // @Test

Review Comment:
   @amogh-jahagirdar, why is this test disabled?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083103875


##########
core/src/test/java/org/apache/iceberg/TestOverwrite.java:
##########
@@ -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

Review Comment:
   Minor: Can the comment remain on the `FILE_10_TO_14` line?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1384706303

   Great work @namrathamyske, just some minor comments but at least the core of it around the tests looks good to me. When you get a chance to update the ReplacePartition tests that would be great, since we can then see all the CI tests pass (right now it looks to be failing since the ReplacePartition tests haven't been updated yet). 


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1080673051


##########
.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"

Review Comment:
   When the review is closer to approval, we should see how many of these breaking changes can be solved. For example, this one about moving the method we may be able to take care of by overriding rather than replacing the method.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083191910


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -397,8 +405,10 @@ protected void validateNoNewDeletesForDataFiles(
       TableMetadata base,
       Long startingSnapshotId,
       Expression dataFilter,
-      Iterable<DataFile> dataFiles) {
-    validateNoNewDeletesForDataFiles(base, startingSnapshotId, dataFilter, dataFiles, false);
+      Iterable<DataFile> dataFiles,
+      Snapshot parent) {

Review Comment:
   Looks like this doesn't update Javadoc...



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "namrathamyske (via GitHub)" <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083386045


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -96,23 +97,37 @@ public RowDelta validateNoConflictingDeleteFiles() {
   }
 
   @Override
-  protected void validate(TableMetadata base, Snapshot snapshot) {
-    if (base.currentSnapshot() != null) {
+  public RowDelta toBranch(String branch) {
+    targetBranch(branch);
+    return this;
+  }
+
+  @Override
+  protected void validate(TableMetadata base, Snapshot parent) {
+    if (parent != null) {
+      if (startingSnapshotId != null) {
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(parent.snapshotId(), startingSnapshotId, base::snapshot),
+            "Snapshot %s is not an ancestor of %s",

Review Comment:
   @amogh-jahagirdar Is this ancestor check even required anymore ? We are anyway doing ancestor check in https://github.com/apache/iceberg/blob/a1d994c131dc5b5f42e1693e92e058493382593e/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java#L742



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1067483646


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -429,29 +470,31 @@ public void testValidateNoConflictsFromSnapshot() {
         statuses(Status.ADDED));
   }
 
-  @Test
+  // @Test

Review Comment:
   Any reason this test is commented out? 



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -472,6 +484,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
     Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateTableDeleteFilesWithRef(Table tbl, String ref, DeleteFile... expectedFiles) {

Review Comment:
   validateBranchDeleteFiles?



##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1270,11 +1332,16 @@ 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);
+    if (branch == "testBranch") {

Review Comment:
   equals() instead of ==



##########
core/src/main/java/org/apache/iceberg/SnapshotProducer.java:
##########
@@ -267,7 +267,26 @@ private Map<String, String> summary(TableMetadata previous) {
     }
 
     Map<String, String> 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();
+        }

Review Comment:
   Could we move the value of previous.ref(targetBranch) to it's own variable for readability?



##########
core/src/main/java/org/apache/iceberg/SnapshotProducer.java:
##########
@@ -267,7 +267,26 @@ private Map<String, String> summary(TableMetadata previous) {
     }
 
     Map<String, String> 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();
+        }

Review Comment:
   Also I think we could probably simplify all the if/else logic here. the branch will either be main or not, and in both cases it's the same logic. Regardless of main or not main: There is either not a prev snapshot in which case default to 0, or the prev snapshot had no summary use an empty summary. or the prev snapshot had a summary so use it.



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -458,6 +458,18 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) {
     Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateTableFilesWithRef(Table tbl, String ref, DataFile... expectedFiles) {

Review Comment:
   validateBranchDataFiles?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1179781697

   @amogh-jahagirdar @rdblue As per https://github.com/apache/iceberg/pull/4926#issuecomment-1161499158 I have changed validation checks when we commit to a branch. Right now I have implemented for BaseRowDelta usecase. My future plan is to extend for others.
   
   My thought process is every-time we check for validations, the start snapshot should start from an ancestor of the branch. If not we can throw a validation exception. Start snapshot should ideally never go before the snapshot with which branch was created. But we don't store the commit for which the branch was created in metadata. So i am quite confused how to go ahead. Any thoughts appreciated!


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r952246303


##########
core/src/main/java/org/apache/iceberg/SnapshotProducer.java:
##########
@@ -349,30 +354,34 @@ public void commit() {
               base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
               2.0 /* exponential */)
           .onlyRetryOn(CommitFailedException.class)
-          .run(taskOps -> {
-            Snapshot newSnapshot = apply();
-            newSnapshotId.set(newSnapshot.snapshotId());
-            TableMetadata.Builder update = TableMetadata.buildFrom(base);
-            if (base.snapshot(newSnapshot.snapshotId()) != null) {
-              // this is a rollback operation
-              update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch);
-            } else if (stageOnly) {
-              update.addSnapshot(newSnapshot);
-            } else {
-              update.setBranchSnapshot(newSnapshot, targetBranch);
-            }
-
-            TableMetadata updated = update.build();
-            if (updated.changes().isEmpty()) {
-              // do not commit if the metadata has not changed. for example, this may happen when setting the current
-              // snapshot to an ID that is already current. note that this check uses identity.
-              return;
-            }
-
-            // if the table UUID is missing, add it here. the UUID will be re-created each time this operation retries
-            // to ensure that if a concurrent operation assigns the UUID, this operation will not fail.
-            taskOps.commit(base, updated.withUUID());
-          });
+          .run(
+              taskOps -> {
+                Snapshot newSnapshot = apply();
+                newSnapshotId.set(newSnapshot.snapshotId());
+                TableMetadata.Builder update = TableMetadata.buildFrom(base);
+                if (base.snapshot(newSnapshot.snapshotId()) != null) {
+                  // this is a rollback operation
+                  update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch);
+                } else if (stageOnly) {
+                  update.addSnapshot(newSnapshot);
+                } else {
+                  update.setBranchSnapshot(newSnapshot, targetBranch);
+                }
+
+                TableMetadata updated = update.build();
+                if (updated.changes().isEmpty()) {
+                  // do not commit if the metadata has not changed. for example, this may happen
+                  // when setting the current
+                  // snapshot to an ID that is already current. note that this check uses identity.
+                  return;
+                }
+
+                // if the table UUID is missing, add it here. the UUID will be re-created each time
+                // this operation retries
+                // to ensure that if a concurrent operation assigns the UUID, this operation will
+                // not fail.
+                taskOps.commit(base, updated.withUUID());
+              });

Review Comment:
   I think we can undo this.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1073043781


##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -458,6 +458,18 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) {
     Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DataFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) {
+      actualFilePaths.add(task.file().path());
+    }
+    Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Hi, `validateTableFiles` has 30 usages in in TestOverwriteWithValidation



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1073952053


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -337,15 +340,20 @@ private CloseableIterable<ManifestEntry<DataFile>> addedDataFiles(
       TableMetadata base,
       Long startingSnapshotId,
       Expression dataFilter,
-      PartitionSet partitionSet) {
+      PartitionSet partitionSet,
+      Snapshot parent) {
     // if there is no current table state, no files have been added

Review Comment:
   My bad, missed this when I raised the change to your branch. I think we should update this comment to say "If there is no parent, no files have been added"



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1067501650


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1270,11 +1332,16 @@ 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);
+    if (branch == "testBranch") {

Review Comment:
   Also just realized we don't need to differentiate the two cases here, I think we could just have a single method which takes in a branch (and the branch could just be main or not, it doesn't really matter)



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1067482077


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -1270,11 +1332,16 @@ 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);
+    if (branch == "testBranch") {

Review Comment:
   equals() instead of == ?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083101029


##########
core/src/main/java/org/apache/iceberg/BaseRowDelta.java:
##########
@@ -96,23 +97,37 @@ public RowDelta validateNoConflictingDeleteFiles() {
   }
 
   @Override
-  protected void validate(TableMetadata base, Snapshot snapshot) {
-    if (base.currentSnapshot() != null) {
+  public RowDelta toBranch(String branch) {
+    targetBranch(branch);
+    return this;
+  }
+
+  @Override
+  protected void validate(TableMetadata base, Snapshot parent) {
+    if (parent != null) {
+      if (startingSnapshotId != null) {
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(parent.snapshotId(), startingSnapshotId, base::snapshot),
+            "Snapshot %s is not an ancestor of %s",

Review Comment:
   I think a bit more context would be helpful if you ever encounter this error. `"Cannot validate changes: starting snapshot %s is not an ancestor of parent snapshot %s (branch %s)"`



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083195038


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -460,8 +471,8 @@ 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 parent) {

Review Comment:
   I think a lot of the `revapi` problems are probably related to adding `parent` to these methods. We should add the older version back that passes `base.currentSnapshot()` and mark it deprecated, to be removed in 1.3.0. That will take care of at least some of the breaking changes.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1400570837

   Thanks, @namrathamyske and @amogh-jahagirdar! Great to have this in.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1400600384

   Thanks for the reviews @rdblue! and thanks @namrathamyske for the contribution!  


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on pull request #5234: Core, API: BaseRowDelta to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1229549029

   In PR https://github.com/apache/iceberg/pull/5618 MergeAppend and TableTestBase commit and apply methods are implemented. Will make use of those methods after its merged. 


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1073952408


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -337,15 +340,20 @@ private CloseableIterable<ManifestEntry<DataFile>> addedDataFiles(
       TableMetadata base,
       Long startingSnapshotId,
       Expression dataFilter,
-      PartitionSet partitionSet) {
+      PartitionSet partitionSet,
+      Snapshot parent) {
     // if there is no current table state, no files have been added
-    if (base.currentSnapshot() == null) {
+    if (parent == null) {

Review Comment:
   My bad, missed this when I raised the change to your branch. I think we should update this comment to say "If there is no parent, no files have been added"



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1384537219

   @jackye1995 @amogh-jahagirdar Finished for the above producer implementations. I am yet to commit tests for BaseReplacePartitions. Let me know what you think!


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083118880


##########
core/src/test/java/org/apache/iceberg/TestRowDelta.java:
##########
@@ -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);
 
-    long deleteSnapshotId = table.currentSnapshot().snapshotId();
+    commit(table, rowDelta3, branch);
+
+    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
+  // @Test

Review Comment:
   yeah shouldn't be disabled, thanks! 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1083099208


##########
core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java:
##########
@@ -79,23 +79,32 @@ 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) {
       if (dataSpec().isUnpartitioned()) {
-        validateAddedDataFiles(currentMetadata, startingSnapshotId, Expressions.alwaysTrue());
+        validateAddedDataFiles(
+            currentMetadata, startingSnapshotId, Expressions.alwaysTrue(), snapshot);
       } else {
-        validateAddedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions);
+        validateAddedDataFiles(currentMetadata, startingSnapshotId, replacedPartitions, snapshot);

Review Comment:
   Minor: I think it makes more sense to use `parent` rather than `snapshot` for the parent snapshot variable. Not a blocker, but it makes it easier to understand what's going on.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#discussion_r1071651640


##########
core/src/main/java/org/apache/iceberg/SnapshotProducer.java:
##########
@@ -267,9 +267,10 @@ private Map<String, String> summary(TableMetadata previous) {
     }
 
     Map<String, String> previousSummary;
-    if (previous.currentSnapshot() != null) {
-      if (previous.currentSnapshot().summary() != null) {
-        previousSummary = previous.currentSnapshot().summary();
+    SnapshotRef previousSnapshot = previous.ref(targetBranch);

Review Comment:
   Nit: Maybe a better name instead of previousSnapshot is previousBranchHead? 



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -472,6 +484,20 @@ void validateTableDeleteFiles(Table tbl, DeleteFile... expectedFiles) {
     Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DeleteFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) {
+      for (DeleteFile file : task.deletes()) {
+        actualFilePaths.add(file.path());
+      }
+    }
+    Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Same as above after this change do we still need the other validateTableDeleteFiles?



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -458,6 +458,18 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) {
     Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
   }
 
+  void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) {
+    Set<CharSequence> expectedFilePaths = Sets.newHashSet();
+    for (DataFile file : expectedFiles) {
+      expectedFilePaths.add(file.path());
+    }
+    Set<CharSequence> actualFilePaths = Sets.newHashSet();
+    for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) {
+      actualFilePaths.add(task.file().path());
+    }
+    Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths);
+  }

Review Comment:
   Do we still need the other validateTableFiles after this change? I don't think we should.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1386049263

   Raised a PR to @namrathamyske branch for replace partition and a bug fix for validation during overwrite https://github.com/namrathamyske/iceberg/pull/20/files


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] namrathamyske commented on pull request #5234: Core, API: BaseRowDelta, BaseOverwrite,BaseReplacePartitions, BaseRewrite to branch Impl

Posted by GitBox <gi...@apache.org>.
namrathamyske commented on PR #5234:
URL: https://github.com/apache/iceberg/pull/5234#issuecomment-1386396949

   @rdblue Appreciate your feedback on this!


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org