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 2023/01/02 16:10:14 UTC

[GitHub] [iceberg] fqaiser94 opened a new pull request, #6513: API: Add Commit Condition Check WIP

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

   This is a draft pull request to support the following proposal: 
   
   If I can get buy-in in the linked issue that this is an actual problem worth solving and if the general approach in the PR makes sense, I would be happy to figure out the remaining details to take this draft pull request to the finish 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] fqaiser94 commented on pull request #6513: API,Core: Support Conditional Commits

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

   All comments have been addressed. 
   This is now ready for a second round of reviews. 


-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {

Review Comment:
   When I initially started working on this PR, I wasn't sure if I would be able to implement those methods for every `PendingUpdate`. Now that I understand most of the codebase, I don't see this as an issue anymore. As a result, I've moved the `addValidation` and `validateCurrent` methods inside the `PendingUpdate` interface and implemented them for every `PendingUpdate` as per your suggestion. 



-- 
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 #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {

Review Comment:
   Are there pending updates for which we don't want to be able to validate the table state? It seems to me that we probably want to just add this to `PendingUpdate`. We can add a default implementation that throws `UnsupportedOperationException` if we don't want to implement this for some reason.



-- 
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] fqaiser94 commented on pull request #6513: API: Support Conditional Commits

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

   @stevenzwu sorry, I've [responded](https://github.com/apache/iceberg/issues/6514#issuecomment-1451250830) in the issue now, let's continue the conversation there. 


-- 
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 #6513: API: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   This already implement `self()`.



-- 
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


Re: [PR] API,Core: Support Conditional Commits [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/TestCustomValidations.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestCustomValidations extends TableTestBase {
+
+  @Parameterized.Parameters(name = "formatVersion = {0}")
+  public static Object[] parameters() {
+    return new Object[] {1, 2};
+  }
+
+  public TestCustomValidations(int formatVersion) {
+    super(formatVersion);
+  }
+
+  private <E> E setupTableAndEnv(Supplier<E> setupEnv) throws Exception {
+    cleanupTables();
+    setupTable();
+    return setupEnv.get();
+  }
+
+  private <E, T> void testValidationPasses(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+    pendingUpdate.validateCurrentTable(currentTable -> true, "Never fails.");
+    pendingUpdate.commit();
+
+    assertSuccess.accept(env);
+  }
+
+  private <E, T> void testValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the given predicate returns false",
+        ValidationException.class,
+        "Test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> false, "Test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testFirstValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the first predicate returns false",
+        ValidationException.class,
+        "First test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "First test returned: %b", false);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> true, "Second test returned: %b", true);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testSecondValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the second predicate returns false",
+        ValidationException.class,
+        "Second test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> true, "First test returned: %b", true);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "Second test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationFailsDueToConcurrentCommit(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    String customWatermarkKey = "custom_watermark";
+    String currentCustomWatermarkValue = "1";
+    String nextCustomWatermarkValue = "2";
+    table.updateProperties().set(customWatermarkKey, currentCustomWatermarkValue).commit();
+
+    PendingUpdate<T> uncommitted = pendingUpdateSupplier.apply(env);
+    String failMessage = "Test failed";
+    uncommitted.validateCurrentTable(
+        currentTable ->
+            Objects.equals(
+                currentTable.properties().get(customWatermarkKey), currentCustomWatermarkValue),
+        failMessage);
+
+    // concurrent update to the table which advances our custom_watermark value
+    table.updateProperties().set(customWatermarkKey, nextCustomWatermarkValue).commit();
+
+    if (uncommitted instanceof UpdateSchema
+        || uncommitted instanceof UpdatePartitionSpec
+        || uncommitted instanceof UpdateSnapshotReferencesOperation) {
+      // The implementations of the above interfaces do not refresh to get the latest
+      // TableMetadata before calling the underlying table's commit method.
+      // As a result, no ValidationException is thrown because they do not see the concurrent
+      // modifications until the underlying table's commit method is called which is when they
+      // detect the TableMetadata is out-of-date and the commit attempt fails at that point.
+      // Either way, we are able to ensure that we never commit to the table unless we are assured
+      // that the validations hold for the current version of the table.
+      AssertHelpers.assertThrows(
+          "Should throw a CommitFailedException on commit due to concurrent update causing metadata to become stale.",
+          CommitFailedException.class,
+          "Cannot commit changes based on stale metadata",
+          uncommitted::commit);
+    } else {
+      AssertHelpers.assertThrows(
+          "Should throw a ValidationException on commit due to concurrent update causing the given predicate to return false",
+          ValidationException.class,
+          failMessage,
+          uncommitted::commit);
+    }
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testModifyingTableInsideValidationThrowsException(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Any attempts to modify a table inside a validation should throw an exception",
+        java.lang.UnsupportedOperationException.class,
+        "Cannot modify a static table",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> {
+                // illegal action
+                currentTable.updateProperties().set("custom_watermark", "2").commit();
+                return true;
+              },
+              "Test failed.");
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationBehaviours(
+      Supplier<E> setup,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess,
+      Consumer<E> assertFailure)
+      throws Exception {
+    testValidationPasses(setup, pendingUpdateSupplier, assertSuccess);
+    testValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testFirstValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testSecondValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testValidationFailsDueToConcurrentCommit(setup, pendingUpdateSupplier, assertFailure);
+    testModifyingTableInsideValidationThrowsException(setup, pendingUpdateSupplier, assertFailure);
+  }
+
+  private <T> void testValidationBehaviours(
+      Supplier<PendingUpdate<T>> pendingUpdateSupplier,
+      Runnable assertSuccess,
+      Runnable assertFailure)
+      throws Exception {
+    testValidationBehaviours(
+        () -> null,
+        (__) -> pendingUpdateSupplier.get(),
+        (__) -> assertSuccess.run(),
+        (__) -> assertFailure.run());
+  }
+
+  @Test
+  public void testCherryPickOperation() throws Exception {
+    class Setup {
+      final long firstSnapshotId;
+      final long overwriteSnapshotId;
+
+      Setup(long firstSnapshotId, long overwriteSnapshotId) {
+        this.firstSnapshotId = firstSnapshotId;
+        this.overwriteSnapshotId = overwriteSnapshotId;
+      }
+    }
+
+    testValidationBehaviours(

Review Comment:
   I appreciate the politely-worded comment but I am going to push back a little here. 
   
   In general, I agree with having clear and readable tests over deduplicated code and in fact, I originally wrote the tests without any "deduplication" but quickly found that maintaining them was a massive burden. I "deduplicated" the code to ease this and didn't personally feel the readability was compromised that badly but naturally, as the author of the code I have obvious bias :D 
   
   I'll leave this as unresolved for now to see if anyone else finds the tests extremely unreadable, in which case I'll can make the requested 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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
.palantir/revapi.yml:
##########
@@ -261,6 +266,17 @@ 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.numberOfParametersChanged"
+      old: "method void org.apache.iceberg.SetLocation::<init>(org.apache.iceberg.TableOperations)"
+      new: "method void org.apache.iceberg.SetLocation::<init>(java.lang.String, org.apache.iceberg.TableOperations)"
+      justification: "SetLocation should only be constructed by Iceberg code. Hence\
+        \ the change of constructor params shouldn't affect users."
+    - code: "java.method.numberOfParametersChanged"
+      old: "method void org.apache.iceberg.SetStatistics::<init>(org.apache.iceberg.TableOperations)"
+      new: "method void org.apache.iceberg.SetStatistics::<init>(java.lang.String,\
+        \ org.apache.iceberg.TableOperations)"
+      justification: "SetStatistics should only be constructed by Iceberg code. Hence\
+        \ the change of constructor params shouldn't affect users."

Review Comment:
   Fixed. 



-- 
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 #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/ValidatablePendingUpdate.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+
+public interface ValidatablePendingUpdate<ThisT, T> extends PendingUpdate<T> {

Review Comment:
   I'd probably include these in `PendingUpdate`.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/NoOpOperation.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+/**
+ * Performs a no-op commit.
+ *
+ * <p>This update is not exposed though the Table API. Instead, it is a package-private part of the
+ * Transaction API intended for use in {@link ManageSnapshots}.
+ */
+class NoOpOperation extends BaseValidatablePendingUpdate<NoOpOperation, Void>
+    implements ValidatablePendingUpdate<NoOpOperation, Void> {
+
+  private final String tableName;
+  private final TableOperations ops;
+
+  NoOpOperation(String tableName, TableOperations ops) {
+    this.tableName = tableName;
+    this.ops = ops;
+  }
+
+  @Override
+  public Void apply() {
+    return null;
+  }
+
+  @Override
+  public void commit() {
+    validateCurrent(SerializableTable.copyOf(new BaseTable(ops, tableName)));

Review Comment:
   > Also, I think it would be better to use a BaseTable with an underlying StaticTableOperations.
   
   Great point, done. 
   
   > It doesn't seem worth adding table name to all of the operation constructors just to construct this BaseTable. What about just passing null? The table is only used for validation.
   
   Technically, it's possible for a user to access the table name inside a validation context e.g. 
   ```
   .addValidation(
     currentTable -> Objects.equals(currentTable.name(), "abc"), 
     "Table name doesn't match our expected value of `abc`")
   ```
   This validation will obviously fail if we pass in `null`. I realize this is an extremely unlikely use-case,  however since it's pretty easy for us to pass in the correct table name, I'd rather do the "right" thing than pass around `null` values. 
   
   Those are just my thoughts. If you feel strongly about it, just let me know and I can make the change 👍 
   
   



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   Sorry, this is the only comment I'm confused about. 
   
   Are you saying I should change the return type of the the `validateCurrentTable` method from this: 
   ```
   void validateCurrentTable(Predicate<Table> test, String message, Object... args)
   ```
   to this: 
   ```
   ThisT validateCurrentTable(Predicate<Table> test, String message, Object... args)
   ```
   ?
   
   I can do that, but this means I have to the change type signature of PendingUpdate from `PendingUpdate<T>` to `PendingUpdate<ThisT, T>`. 
   Every interface that extends this interface and every class that implements this interface will have a breaking change. 
   Is that acceptable? 
   
   Sample rev-api output:
   
   ```
   * What went wrong:
   Execution failed for task ':iceberg-api:revapi'.
   > There were Java public API/ABI breaks reported by revapi:
     
     java.class.noLongerImplementsInterface: Class no longer implements interface 'org.apache.iceberg.PendingUpdate<java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.noLongerImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.nowImplementsInterface: Class now implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateProperties, java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: NON_BREAKING, BINARY: NON_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.nowImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.superTypeTypeParametersChanged: Super type's type parameters changed from 'org.apache.iceberg.PendingUpdate<java.util.Map<java.lang.String, java.lang.String>>' to 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateProperties, java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: POTENTIALLY_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.superTypeTypeParametersChanged" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.noLongerImplementsInterface: Class no longer implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.noLongerImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.nowImplementsInterface: Class now implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateSchema, org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: NON_BREAKING, BINARY: NON_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.nowImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.superTypeTypeParametersChanged: Super type's type parameters changed from 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.Schema>' to 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateSchema, org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: POTENTIALLY_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.superTypeTypeParametersChanged" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
   ```



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;

Review Comment:
   Fixed. 
   
   The reason I had to do this originally was because the compiler kept throwing the following warning: 
   ```
   /Users/fq/src/iceberg/core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:55: error: [FormatStringAnnotation] Variables used as format strings that are not local variables must be compile time constants.
   ```
   It doesn't like the lambda I think: 
   ```
   validations.forEach(
           validation ->
               ValidationException.check(
                   validation.predicate.test(currentTable), validation.message, validation.args));
   ```
   
   Unfortunately, I haven't been able to find a way to convince the compiler, so as a compromise I've had to to include a `@SuppressWarnings("FormatStringAnnotation")` to get around this issue. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,24 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param failMessage The message that will be included in the {@link ValidationException} that
+   *     will be thrown if the test returns false.
+   */
+  @FormatMethod
+  void validateCurrentTable(Predicate<Table> test, String failMessage);

Review Comment:
   🤦 Fixed. 



-- 
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 #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,24 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param failMessage The message that will be included in the {@link ValidationException} that
+   *     will be thrown if the test returns false.
+   */
+  @FormatMethod
+  void validateCurrentTable(Predicate<Table> test, String failMessage);

Review Comment:
   This only has the failure message, not args for that failure message.



-- 
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] fqaiser94 commented on pull request #6513: API,Core: Support Conditional Commits

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

   Sorry @nastra @jackye1995, I didn't mean to remove you both as reviewers.
   For some reason, the UI won't even let me add you folks back as reviewers! 😵‍💫 


-- 
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] nastra commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,28 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param message The message that will be included in the {@link ValidationException} that will
+   *     be thrown if the test returns false.
+   * @param args The args that will be included in the {@link ValidationException} that will be
+   *     thrown if the test returns false.
+   */
+  @FormatMethod
+  default void validateCurrentTable(Predicate<Table> test, String message, Object... args) {

Review Comment:
   ```suggestion
     default void validateCurrentTable(Predicate<Table> predicate, String message, Object... args) {
   ```



##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final String message;
+    private final Object[] args;
+
+    @FormatMethod
+    Validation(Predicate<Table> predicate, String message, Object... args) {
+      this.predicate = predicate;
+      this.message = message;
+      this.args = args;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(
+      Predicate<Table> predicate, String message, Object... args) {
+    this.validations.add(new Validation(predicate, message, args));
+  }
+
+  @SuppressWarnings("FormatStringAnnotation")
+  protected final void validateCurrentTableMetadata(TableMetadata base) {

Review Comment:
   why not just name this `validate(..)`? The RevAPI failure you were seeing is because your branch is on quite an old Iceberg version. The method it was conflicting with was removed in https://github.com/apache/iceberg/commit/5a9eb3c20a3867d6ca5ba0d4bea87e1760bab84c



##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -41,13 +65,15 @@
   T apply();
 
   /**
-   * Apply the pending changes and commit.
+   * Apply the pending changes, validate the current version of the table, and commit.
    *
    * <p>Changes are committed by calling the underlying table's commit method.
    *
    * <p>Once the commit is successful, the updated table will be refreshed.
    *
    * @throws ValidationException If the update cannot be applied to the current table metadata.
+   * @throws UnsupportedOperationException If any predicate supplied through {@link

Review Comment:
   where is this verified?



##########
core/src/test/java/org/apache/iceberg/TestCustomValidations.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestCustomValidations extends TableTestBase {
+
+  @Parameterized.Parameters(name = "formatVersion = {0}")

Review Comment:
   do we really need to test with different format versions?



##########
core/src/test/java/org/apache/iceberg/TestCustomValidations.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestCustomValidations extends TableTestBase {
+
+  @Parameterized.Parameters(name = "formatVersion = {0}")
+  public static Object[] parameters() {
+    return new Object[] {1, 2};
+  }
+
+  public TestCustomValidations(int formatVersion) {
+    super(formatVersion);
+  }
+
+  private <E> E setupTableAndEnv(Supplier<E> setupEnv) throws Exception {
+    cleanupTables();
+    setupTable();
+    return setupEnv.get();
+  }
+
+  private <E, T> void testValidationPasses(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+    pendingUpdate.validateCurrentTable(currentTable -> true, "Never fails.");
+    pendingUpdate.commit();
+
+    assertSuccess.accept(env);
+  }
+
+  private <E, T> void testValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the given predicate returns false",
+        ValidationException.class,
+        "Test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> false, "Test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testFirstValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the first predicate returns false",
+        ValidationException.class,
+        "First test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "First test returned: %b", false);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> true, "Second test returned: %b", true);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testSecondValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the second predicate returns false",
+        ValidationException.class,
+        "Second test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> true, "First test returned: %b", true);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "Second test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationFailsDueToConcurrentCommit(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    String customWatermarkKey = "custom_watermark";
+    String currentCustomWatermarkValue = "1";
+    String nextCustomWatermarkValue = "2";
+    table.updateProperties().set(customWatermarkKey, currentCustomWatermarkValue).commit();
+
+    PendingUpdate<T> uncommitted = pendingUpdateSupplier.apply(env);
+    String failMessage = "Test failed";
+    uncommitted.validateCurrentTable(
+        currentTable ->
+            Objects.equals(
+                currentTable.properties().get(customWatermarkKey), currentCustomWatermarkValue),
+        failMessage);
+
+    // concurrent update to the table which advances our custom_watermark value
+    table.updateProperties().set(customWatermarkKey, nextCustomWatermarkValue).commit();
+
+    if (uncommitted instanceof UpdateSchema
+        || uncommitted instanceof UpdatePartitionSpec
+        || uncommitted instanceof UpdateSnapshotReferencesOperation) {
+      // The implementations of the above interfaces do not refresh to get the latest
+      // TableMetadata before calling the underlying table's commit method.
+      // As a result, no ValidationException is thrown because they do not see the concurrent
+      // modifications until the underlying table's commit method is called which is when they
+      // detect the TableMetadata is out-of-date and the commit attempt fails at that point.
+      // Either way, we are able to ensure that we never commit to the table unless we are assured
+      // that the validations hold for the current version of the table.
+      AssertHelpers.assertThrows(
+          "Should throw a CommitFailedException on commit due to concurrent update causing metadata to become stale.",
+          CommitFailedException.class,
+          "Cannot commit changes based on stale metadata",
+          uncommitted::commit);
+    } else {
+      AssertHelpers.assertThrows(
+          "Should throw a ValidationException on commit due to concurrent update causing the given predicate to return false",
+          ValidationException.class,
+          failMessage,
+          uncommitted::commit);
+    }
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testModifyingTableInsideValidationThrowsException(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Any attempts to modify a table inside a validation should throw an exception",
+        java.lang.UnsupportedOperationException.class,
+        "Cannot modify a static table",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> {
+                // illegal action
+                currentTable.updateProperties().set("custom_watermark", "2").commit();
+                return true;
+              },
+              "Test failed.");
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationBehaviours(
+      Supplier<E> setup,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess,
+      Consumer<E> assertFailure)
+      throws Exception {
+    testValidationPasses(setup, pendingUpdateSupplier, assertSuccess);
+    testValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testFirstValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testSecondValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testValidationFailsDueToConcurrentCommit(setup, pendingUpdateSupplier, assertFailure);
+    testModifyingTableInsideValidationThrowsException(setup, pendingUpdateSupplier, assertFailure);
+  }
+
+  private <T> void testValidationBehaviours(
+      Supplier<PendingUpdate<T>> pendingUpdateSupplier,
+      Runnable assertSuccess,
+      Runnable assertFailure)
+      throws Exception {
+    testValidationBehaviours(
+        () -> null,
+        (__) -> pendingUpdateSupplier.get(),
+        (__) -> assertSuccess.run(),
+        (__) -> assertFailure.run());
+  }
+
+  @Test
+  public void testCherryPickOperation() throws Exception {
+    class Setup {
+      final long firstSnapshotId;
+      final long overwriteSnapshotId;
+
+      Setup(long firstSnapshotId, long overwriteSnapshotId) {
+        this.firstSnapshotId = firstSnapshotId;
+        this.overwriteSnapshotId = overwriteSnapshotId;
+      }
+    }
+
+    testValidationBehaviours(

Review Comment:
   I understand that we don't want to repeat code, but reading through this test makes it quite difficult to reason about what's exactly being tested and expected (because we're just calling a bunch of lambdas).
   
   I'm curious what others think, but I would vote for having clear and readable than completely de-duplicated code.
   
   



##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,28 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param message The message that will be included in the {@link ValidationException} that will
+   *     be thrown if the test returns false.
+   * @param args The args that will be included in the {@link ValidationException} that will be
+   *     thrown if the test returns false.
+   */
+  @FormatMethod
+  default void validateCurrentTable(Predicate<Table> test, String message, Object... args) {

Review Comment:
   having `validateCurrentTable(...)` leads to an awkward/non-fluent API. What about having `commitIf(Validation)` that would add the given validation and call `commit()` afterwards?



##########
core/src/test/java/org/apache/iceberg/TestTransaction.java:
##########
@@ -771,4 +773,88 @@ public void testSimpleTransactionNotDeletingMetadataOnUnknownSate() throws IOExc
     Assert.assertTrue("Manifest file should exist", new File(manifests.get(0).path()).exists());
     Assert.assertEquals("Should have 2 files in metadata", 2, countAllMetadataFiles(tableDir));
   }
+
+  @Test
+  public void testValidationPasses() {
+    Assert.assertEquals("Table should be on version 0", 0, (int) version());
+
+    Transaction txn = table.newTransaction();
+    AppendFiles appendFiles = txn.newAppend().appendFile(FILE_A);
+    appendFiles.validateCurrentTable(currentTable -> true, "Custom validation failed.");
+    appendFiles.commit();
+    txn.commitTransaction();
+
+    Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version());
+  }
+
+  @Test
+  public void testValidationFails() {
+    Assert.assertEquals("Table should be on version 0", 0, (int) version());
+
+    Transaction txn = table.newTransaction();
+    AssertHelpers.assertThrows(

Review Comment:
   usages of this class are deprecated now. Please use `Assertions.assertThatThrownBy(..).isInstanceOf(..).hasMessage(..)`



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,28 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param message The message that will be included in the {@link ValidationException} that will
+   *     be thrown if the test returns false.
+   * @param args The args that will be included in the {@link ValidationException} that will be
+   *     thrown if the test returns false.
+   */
+  @FormatMethod
+  default void validateCurrentTable(Predicate<Table> test, String message, Object... args) {

Review Comment:
   This is very close to what I originally proposed ;) 
   Although I would prefer if the `commitIf` method accepted a list of validations rather than a single validation as that would allows users to specify a unique message per failed predicate (which can be useful in recovering from the specific `ValidationException`). To be concrete, this is what I would reccommend: 
   ```
   public interface PendingUpdate<T> {
     void commitIf(List<Validation> validations)
   }
   
   public class Validation {
     private final Predicate<Table> predicate;
     private final String message;
     private final Object[] args;
   
     @FormatMethod
     Validation(Predicate<Table> predicate, String message, Object... args) {
       this.predicate = predicate;
       this.message = message;
       this.args = args;
     }
   
     @SuppressWarnings("FormatStringAnnotation")
     void check(Table currentTable) {
       ValidationException.check(predicate.test(currentTable), message, args);
     }
   }
   ```
   
   You can see a working implemention of this idea in one of my latest (reverted) commits here: https://github.com/apache/iceberg/pull/6513/commits/dd94de91beb28bdfa19128df54e92a929850f2f3
   I think it's worth reconsidering in light of some of the recent [comments](https://github.com/apache/iceberg/pull/6513#discussion_r1129872213) as this could be a good compromise. 
   However @rdblue reccommended the current API design so I'd love to get his thoughts before making this change. 



-- 
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 #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {
+    this.validations.add(new Validation(predicate, exception));
+    return returnThis();
+  }
+
+  @Override
+  public final void validateCurrent(Table current) {
+    validations.forEach(
+        validation -> {
+          if (!validation.predicate.test(current)) {
+            throw validation.exception;
+          }
+        });
+  }
+
+  protected abstract ThisT returnThis();

Review Comment:
   We name this method `self()` in other places.



-- 
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 #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {

Review Comment:
   It seems odd to pass an exception instance. Can we pass a format string and values instead? Basically, it would look like `Precondition.checkArgument` but with a predicate as the first arg rather than a boolean.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,28 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param message The message that will be included in the {@link ValidationException} that will
+   *     be thrown if the test returns false.
+   * @param args The args that will be included in the {@link ValidationException} that will be
+   *     thrown if the test returns false.
+   */
+  @FormatMethod
+  default void validateCurrentTable(Predicate<Table> test, String message, Object... args) {

Review Comment:
   This is very close to what I originally proposed ;) 
   Although I would prefer if the `commitIf` method accepted a list of validations rather than a single validation as that would allows users to specify a unique message per failed predicate (which can be useful in recovering from the specific `ValidationException`). To be concrete, this is what I would reccommend: 
   ```
   public interface PendingUpdate<T> {
   	void commitIf(List<Validation> validations)
   }
   
   public class Validation {
     private final Predicate<Table> predicate;
     private final String message;
     private final Object[] args;
   
     @FormatMethod
     Validation(Predicate<Table> predicate, String message, Object... args) {
       this.predicate = predicate;
       this.message = message;
       this.args = args;
     }
   
     @SuppressWarnings("FormatStringAnnotation")
     void check(Table currentTable) {
       ValidationException.check(predicate.test(currentTable), message, args);
     }
   }
   ```
   
   You can see a working implemention of this idea in one of my latest (reverted) commits here: https://github.com/apache/iceberg/pull/6513/commits/dd94de91beb28bdfa19128df54e92a929850f2f3
   I think it's worth reconsidering in light of some of the recent [comments](https://github.com/apache/iceberg/pull/6513#discussion_r1129872213) as this could be a good compromise. 
   However @rdblue reccommended the current API design so I'd love to get his thoughts before making this change. 



-- 
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] fqaiser94 commented on pull request #6513: API,Core: Support Conditional Commits

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

   I took a little break from this PR because reviews were moving a little slowly and it seemed like this feature wasn't considered a high priority. I have since had/seen a couple of conversations with people interested in this feature and affirmed it's value so I'm thinking now might be a good time to try reviving this PR. 
   
   I've rebased the changes on top of latest master and addressed all of the existing comments. 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 #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;

Review Comment:
   Why does this store the exception? If you're forcing the exception to be a `ValidationException` and passing args to produce a formatted message, then there is no need to store the exception instance.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   This was a little tricky but I have managed to convince Java's type system that `self` and `returnThis` have the same return type, so we no longer need to specify a `returnThis` method for `SnapshotProducer` inheritors and in fact I can rename the `returnThis` method to just `self` without any type conflicts. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   Changed my mind about this. 
   I've changed the`validateCurrentTable` to now return `void` instead of `ThisT`. 
   This makes the code changes much simpler as we no longer need this `self` method nor do we need to do any java-type-system-gymnastics.  
   The only downside is that method chaining is no longer possible after calling `validateCurrentTable` method which I don't consider to be a big deal. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   Changed my mind about this. 
   I've changed the`validateCurrentTable` to now return `void` instead of `ThisT`. 
   This makes the code changes much simpler as we no longer need this `self` method nor do we need any of the java-type-system-gymnastics I was doing previously.  
   The only downside is that method chaining is no longer possible after calling `validateCurrentTable` method which I don't consider to be a big deal. 



-- 
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] nastra commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final String message;
+    private final Object[] args;
+
+    @FormatMethod
+    Validation(Predicate<Table> predicate, String message, Object... args) {
+      this.predicate = predicate;
+      this.message = message;
+      this.args = args;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(
+      Predicate<Table> predicate, String message, Object... args) {
+    this.validations.add(new Validation(predicate, message, args));
+  }
+
+  @SuppressWarnings("FormatStringAnnotation")
+  protected final void validateCurrentTableMetadata(TableMetadata base) {

Review Comment:
   I think it would make more sense to name this `validate(..)` maybe? RevAPI is probably getting confused because we deprecated and then removed the method in https://github.com/apache/iceberg/commit/5a9eb3c20a3867d6ca5ba0d4bea87e1760bab84c#diff-736caed551a388d34b08f223954ae7ecb2fdac9d90a4098ceedd95207d7efd4dR323-R325



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/test/java/org/apache/iceberg/TestCustomValidations.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestCustomValidations extends TableTestBase {
+
+  @Parameterized.Parameters(name = "formatVersion = {0}")

Review Comment:
   I copied this because I saw other test suites doing this. 
   I agree, I don't think there is a particular need to test against different format versions for this suite of tests. 
   Fixed. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits - WIP

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


##########
api/src/main/java/org/apache/iceberg/ValidatablePendingUpdate.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+
+public interface ValidatablePendingUpdate<ThisT, T> extends PendingUpdate<T> {
+
+  /**
+   * A user provided test for determining whether it is safe to commit a {@link PendingUpdate} to a
+   * given table. This will be used in addition to whatever validations iceberg performs internally.
+   * For example this could be used for providing a restriction to only commit if a given Snapshot
+   * property is present in the current version of the table.
+   *
+   * @param test The predicate which will be used to determine if it is safe to commit to this
+   *     table.
+   * @param exception The {@link ValidationException} to throw if the test fails.
+   * @return this for method chaining
+   */
+  ThisT addValidation(Predicate<Table> test, ValidationException exception);
+
+  /**
+   * Called prior to commit with the current version of the table.
+   *
+   * <p>It is guaranteed that a {@link ValidatablePendingUpdate} will only ever be committed as long
+   * as this method does not throw an exception against the current version of the table.
+   *
+   * @param current The current version of the table. This is the same as the version of the table
+   *     over which the changes are being applied as part of this {@link PendingUpdate}. This may be
+   *     null if there is no table i.e. the table is being created as part of this {@link
+   *     PendingUpdate}. Note: attempting to modify this table in any way will throw an exception as
+   *     this table is read-only.
+   * @throws ValidationException If the pending changes cannot be applied to the current metadata

Review Comment:
   Chose specifically to throw a `ValidationException` as they make the most sense contextually and they don't cause retries (IMO it's pretty unlikely a validation would pass after a retry).



##########
build.gradle:
##########
@@ -698,6 +698,11 @@ project(':iceberg-dell') {
     implementation project(':iceberg-common')
     implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
     compileOnly 'com.emc.ecs:object-client-bundle'
+    compileOnly("org.apache.hadoop:hadoop-common") {
+      exclude group: 'commons-beanutils'
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    }

Review Comment:
   Without this change, I see the following test failure: 
   ```
   org.apache.iceberg.dell.ecs.TestEcsTableOperations > testConcurrentCommit FAILED
       java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configurable
           at java.base/java.lang.ClassLoader.defineClass1(Native Method)
           at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1017)
           at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174)
           at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:800)
           at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:698)
           at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:621)
           at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579)
           at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
           at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522)
           at org.apache.iceberg.SerializableTable.fileIO(SerializableTable.java:112)
           at org.apache.iceberg.SerializableTable.<init>(SerializableTable.java:82)
           at org.apache.iceberg.SerializableTable.copyOf(SerializableTable.java:98)
           at org.apache.iceberg.PropertiesUpdate.lambda$commit$0(PropertiesUpdate.java:121)
           at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
           at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
           at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
           at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
           at org.apache.iceberg.PropertiesUpdate.commit(PropertiesUpdate.java:117)
           at org.apache.iceberg.dell.ecs.TestEcsTableOperations.testConcurrentCommit(TestEcsTableOperations.java:55)
           Caused by:
           java.lang.ClassNotFoundException: org.apache.hadoop.conf.Configurable
               at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:581)
               at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
               at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522)
               ... 19 more
   ```



##########
api/src/main/java/org/apache/iceberg/ValidatablePendingUpdate.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+
+public interface ValidatablePendingUpdate<ThisT, T> extends PendingUpdate<T> {
+
+  /**
+   * A user provided test for determining whether it is safe to commit a {@link PendingUpdate} to a
+   * given table. This will be used in addition to whatever validations iceberg performs internally.
+   * For example this could be used for providing a restriction to only commit if a given Snapshot
+   * property is present in the current version of the table.
+   *
+   * @param test The predicate which will be used to determine if it is safe to commit to this
+   *     table.
+   * @param exception The {@link ValidationException} to throw if the test fails.
+   * @return this for method chaining
+   */
+  ThisT addValidation(Predicate<Table> test, ValidationException exception);
+
+  /**
+   * Called prior to commit with the current version of the table.
+   *
+   * <p>It is guaranteed that a {@link ValidatablePendingUpdate} will only ever be committed as long
+   * as this method does not throw an exception against the current version of the table.
+   *
+   * @param current The current version of the table. This is the same as the version of the table
+   *     over which the changes are being applied as part of this {@link PendingUpdate}. This may be
+   *     null if there is no table i.e. the table is being created as part of this {@link
+   *     PendingUpdate}. Note: attempting to modify this table in any way will throw an exception as
+   *     this table is read-only.
+   * @throws ValidationException If the pending changes cannot be applied to the current metadata
+   */
+  void validateCurrent(Table current);

Review Comment:
   I would preferred to name this `validate` rather than `validateCurrent` but the `./gradlew :iceberg-core:revapi` task reports some breaking changes: 
   
   ```
   > There were Java public API/ABI breaks reported by revapi:
     
     java.method.parameterTypeChanged: The type of the parameter changed from 'org.apache.iceberg.TableMetadata' to 'org.apache.iceberg.Table'.
     
     old: parameter void org.apache.iceberg.SnapshotProducer<ThisT>::validate(===org.apache.iceberg.TableMetadata===) @ org.apache.iceberg.BaseOverwriteFiles
     new: parameter void org.apache.iceberg.BaseValidatablePendingUpdate<ThisT, T>::validate(===org.apache.iceberg.Table===) @ org.apache.iceberg.BaseOverwriteFiles
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: BREAKING
       ----------------------------------------------------------------------------------------------------
     java.method.parameterTypeChanged: The type of the parameter changed from 'org.apache.iceberg.TableMetadata' to 'org.apache.iceberg.Table'.
     
     old: parameter void org.apache.iceberg.BaseReplacePartitions::validate(===org.apache.iceberg.TableMetadata===)
     new: parameter void org.apache.iceberg.BaseValidatablePendingUpdate<ThisT, T>::validate(===org.apache.iceberg.Table===) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: BREAKING
     ----------------------------------------------------------------------------------------------------
     java.method.parameterTypeChanged: The type of the parameter changed from 'org.apache.iceberg.TableMetadata' to 'org.apache.iceberg.Table'.
     
     old: parameter void org.apache.iceberg.SnapshotProducer<ThisT>::validate(===org.apache.iceberg.TableMetadata===) @ org.apache.iceberg.BaseRewriteManifests
     new: parameter void org.apache.iceberg.BaseValidatablePendingUpdate<ThisT, T>::validate(===org.apache.iceberg.Table===) @ org.apache.iceberg.BaseRewriteManifests
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: BREAKING
     ----------------------------------------------------------------------------------------------------
     java.method.parameterTypeChanged: The type of the parameter changed from 'org.apache.iceberg.TableMetadata' to 'org.apache.iceberg.Table'.
     
     old: parameter void org.apache.iceberg.SnapshotProducer<ThisT>::validate(===org.apache.iceberg.TableMetadata===) @ org.apache.iceberg.StreamingDelete
     new: parameter void org.apache.iceberg.BaseValidatablePendingUpdate<ThisT, T>::validate(===org.apache.iceberg.Table===) @ org.apache.iceberg.StreamingDelete
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: BREAKING
   ```
   
   I don't know how serious these are but the simplest solution for me right now was to just call our new method something different. Open to other suggestions. 
   
   If we decide to stick with `validateCurrent`, we may consider naming `addValidation` to `addCurrentValidation`



##########
api/src/main/java/org/apache/iceberg/ValidatablePendingUpdate.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+
+public interface ValidatablePendingUpdate<ThisT, T> extends PendingUpdate<T> {
+
+  /**
+   * A user provided test for determining whether it is safe to commit a {@link PendingUpdate} to a
+   * given table. This will be used in addition to whatever validations iceberg performs internally.
+   * For example this could be used for providing a restriction to only commit if a given Snapshot
+   * property is present in the current version of the table.
+   *
+   * @param test The predicate which will be used to determine if it is safe to commit to this
+   *     table.
+   * @param exception The {@link ValidationException} to throw if the test fails.
+   * @return this for method chaining
+   */
+  ThisT addValidation(Predicate<Table> test, ValidationException exception);
+
+  /**
+   * Called prior to commit with the current version of the table.
+   *
+   * <p>It is guaranteed that a {@link ValidatablePendingUpdate} will only ever be committed as long
+   * as this method does not throw an exception against the current version of the table.
+   *
+   * @param current The current version of the table. This is the same as the version of the table
+   *     over which the changes are being applied as part of this {@link PendingUpdate}. This may be
+   *     null if there is no table i.e. the table is being created as part of this {@link
+   *     PendingUpdate}. Note: attempting to modify this table in any way will throw an exception as
+   *     this table is read-only.
+   * @throws ValidationException If the pending changes cannot be applied to the current metadata

Review Comment:
   This is a pretty important detail; implementations have to make sure that they call the `validateCurrent` method on the version of the table that is being used as the `TableMetadata base` argument for the `taskOps.commit` method. Put another way, we want to avoid situations like this (in pseudo-code): 
   ```
   validateCurrent(baseTable);
   baseTable = current.refresh(); // <- refresh might pull in a new version of the table which may or may not pass validateCurrent!
   taskOps.commit(baseTable, updated);
   ```



##########
api/src/main/java/org/apache/iceberg/ValidatablePendingUpdate.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+
+public interface ValidatablePendingUpdate<ThisT, T> extends PendingUpdate<T> {

Review Comment:
   I chose to create a new interface instead of adding these methods directly to the `PendingUpdate` interface: 
   - The `validateCurrent` method could be added to the `PendingUpdate` interface directly, I would be fine with that. 
   - `addValidation` doesn't feel like it belongs on the `PendingUpdate` interface, hence the new interface for it. 
   
   Happy to take opinions on this design decision. 



##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {
+    this.validations.add(new Validation(predicate, exception));
+    return returnThis();
+  }
+
+  @Override
+  public final void validateCurrent(Table current) {
+    validations.forEach(
+        validation -> {
+          if (!validation.predicate.test(current)) {
+            throw validation.exception;
+          }
+        });
+  }
+
+  protected abstract ThisT returnThis();

Review Comment:
   This has to be implemented for every class that extends this `abstract class`.
   It's pretty easy to implement: `return this;`
   A bit boilerplate-y but I felt the trade-off was worth it to enable method chaining. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {

Review Comment:
   Ha, I struggled with the `@FormatMethod` annotation when I tried to do this originally. Fixed now. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   Sorry, this is the only comment I'm confused about. 
   
   Are you saying I should change the return type of the the `validateCurrentTable` method from this: 
   ```
   void validateCurrentTable(Predicate<Table> test, String message, Object... args)
   ```
   to this: 
   ```
   ThisT validateCurrentTable(Predicate<Table> test, String message, Object... args)
   ```
   ?
   
   I can do that, but this means I have to the change type signature of PendingUpdate from `PendingUpdate<T>` to `PendingUpdate<ThisT, T>`. 
   Every interface that extends this interface and every class that implements this interface will have a breaking change. 
   Is that acceptable? 
   
   If it helps, this is the commit I would need to basically revert: https://github.com/apache/iceberg/pull/6513/commits/455ee74f86d6b1d1a0e9d5c88dcc3a8fc2eb5782
   Also, here is a sample rev-api output of the breaking changes (actual report is much larger):
   
   ```
   * What went wrong:
   Execution failed for task ':iceberg-api:revapi'.
   > There were Java public API/ABI breaks reported by revapi:
     
     java.class.noLongerImplementsInterface: Class no longer implements interface 'org.apache.iceberg.PendingUpdate<java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.noLongerImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.nowImplementsInterface: Class now implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateProperties, java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: NON_BREAKING, BINARY: NON_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.nowImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.superTypeTypeParametersChanged: Super type's type parameters changed from 'org.apache.iceberg.PendingUpdate<java.util.Map<java.lang.String, java.lang.String>>' to 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateProperties, java.util.Map<java.lang.String, java.lang.String>>'.
     
     old: interface org.apache.iceberg.UpdateProperties
     new: interface org.apache.iceberg.UpdateProperties
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: POTENTIALLY_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.superTypeTypeParametersChanged" \
             --old "interface org.apache.iceberg.UpdateProperties" \
             --new "interface org.apache.iceberg.UpdateProperties"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.noLongerImplementsInterface: Class no longer implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.noLongerImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.nowImplementsInterface: Class now implements interface 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateSchema, org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: NON_BREAKING, BINARY: NON_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.nowImplementsInterface" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.class.superTypeTypeParametersChanged: Super type's type parameters changed from 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.Schema>' to 'org.apache.iceberg.PendingUpdate<org.apache.iceberg.UpdateSchema, org.apache.iceberg.Schema>'.
     
     old: interface org.apache.iceberg.UpdateSchema
     new: interface org.apache.iceberg.UpdateSchema
     
     SOURCE: POTENTIALLY_BREAKING, BINARY: POTENTIALLY_BREAKING
     
     From old archive: iceberg-api-1.1.0.jar
     From new archive: iceberg-api-1.2.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.class.superTypeTypeParametersChanged" \
             --old "interface org.apache.iceberg.UpdateSchema" \
             --new "interface org.apache.iceberg.UpdateSchema"
       * All breaks in this project:
           ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
   ```



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
build.gradle:
##########
@@ -698,6 +698,11 @@ project(':iceberg-dell') {
     implementation project(':iceberg-common')
     implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
     compileOnly 'com.emc.ecs:object-client-bundle'
+    compileOnly("org.apache.hadoop:hadoop-common") {
+      exclude group: 'commons-beanutils'
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    }

Review Comment:
   You're right, this is no longer needed, removed these 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 a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
.palantir/revapi.yml:
##########
@@ -261,6 +266,17 @@ 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.numberOfParametersChanged"
+      old: "method void org.apache.iceberg.SetLocation::<init>(org.apache.iceberg.TableOperations)"
+      new: "method void org.apache.iceberg.SetLocation::<init>(java.lang.String, org.apache.iceberg.TableOperations)"
+      justification: "SetLocation should only be constructed by Iceberg code. Hence\
+        \ the change of constructor params shouldn't affect users."
+    - code: "java.method.numberOfParametersChanged"
+      old: "method void org.apache.iceberg.SetStatistics::<init>(org.apache.iceberg.TableOperations)"
+      new: "method void org.apache.iceberg.SetStatistics::<init>(java.lang.String,\
+        \ org.apache.iceberg.TableOperations)"
+      justification: "SetStatistics should only be constructed by Iceberg code. Hence\
+        \ the change of constructor params shouldn't affect users."

Review Comment:
   There should not be revapi exceptions. If you add to an interface, then add a default implementation.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final String message;
+    private final Object[] args;
+
+    @FormatMethod
+    Validation(Predicate<Table> predicate, String message, Object... args) {
+      this.predicate = predicate;
+      this.message = message;
+      this.args = args;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(
+      Predicate<Table> predicate, String message, Object... args) {
+    this.validations.add(new Validation(predicate, message, args));
+  }
+
+  @SuppressWarnings("FormatStringAnnotation")
+  protected final void validateCurrentTableMetadata(TableMetadata base) {

Review Comment:
   After rebasing on top of latest master (again), this was indeed possible. Fixed. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/test/java/org/apache/iceberg/TestCustomValidations.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestCustomValidations extends TableTestBase {
+
+  @Parameterized.Parameters(name = "formatVersion = {0}")
+  public static Object[] parameters() {
+    return new Object[] {1, 2};
+  }
+
+  public TestCustomValidations(int formatVersion) {
+    super(formatVersion);
+  }
+
+  private <E> E setupTableAndEnv(Supplier<E> setupEnv) throws Exception {
+    cleanupTables();
+    setupTable();
+    return setupEnv.get();
+  }
+
+  private <E, T> void testValidationPasses(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+    pendingUpdate.validateCurrentTable(currentTable -> true, "Never fails.");
+    pendingUpdate.commit();
+
+    assertSuccess.accept(env);
+  }
+
+  private <E, T> void testValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the given predicate returns false",
+        ValidationException.class,
+        "Test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> false, "Test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testFirstValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the first predicate returns false",
+        ValidationException.class,
+        "First test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "First test returned: %b", false);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> true, "Second test returned: %b", true);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testSecondValidationFails(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Should throw a ValidationException if the second predicate returns false",
+        ValidationException.class,
+        "Second test returned: false",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(currentTable -> true, "First test returned: %b", true);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> false, "Second test returned: %b", false);
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationFailsDueToConcurrentCommit(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    String customWatermarkKey = "custom_watermark";
+    String currentCustomWatermarkValue = "1";
+    String nextCustomWatermarkValue = "2";
+    table.updateProperties().set(customWatermarkKey, currentCustomWatermarkValue).commit();
+
+    PendingUpdate<T> uncommitted = pendingUpdateSupplier.apply(env);
+    String failMessage = "Test failed";
+    uncommitted.validateCurrentTable(
+        currentTable ->
+            Objects.equals(
+                currentTable.properties().get(customWatermarkKey), currentCustomWatermarkValue),
+        failMessage);
+
+    // concurrent update to the table which advances our custom_watermark value
+    table.updateProperties().set(customWatermarkKey, nextCustomWatermarkValue).commit();
+
+    if (uncommitted instanceof UpdateSchema
+        || uncommitted instanceof UpdatePartitionSpec
+        || uncommitted instanceof UpdateSnapshotReferencesOperation) {
+      // The implementations of the above interfaces do not refresh to get the latest
+      // TableMetadata before calling the underlying table's commit method.
+      // As a result, no ValidationException is thrown because they do not see the concurrent
+      // modifications until the underlying table's commit method is called which is when they
+      // detect the TableMetadata is out-of-date and the commit attempt fails at that point.
+      // Either way, we are able to ensure that we never commit to the table unless we are assured
+      // that the validations hold for the current version of the table.
+      AssertHelpers.assertThrows(
+          "Should throw a CommitFailedException on commit due to concurrent update causing metadata to become stale.",
+          CommitFailedException.class,
+          "Cannot commit changes based on stale metadata",
+          uncommitted::commit);
+    } else {
+      AssertHelpers.assertThrows(
+          "Should throw a ValidationException on commit due to concurrent update causing the given predicate to return false",
+          ValidationException.class,
+          failMessage,
+          uncommitted::commit);
+    }
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testModifyingTableInsideValidationThrowsException(
+      Supplier<E> setupEnv,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertFailure)
+      throws Exception {
+    E env = setupTableAndEnv(setupEnv);
+
+    AssertHelpers.assertThrows(
+        "Any attempts to modify a table inside a validation should throw an exception",
+        java.lang.UnsupportedOperationException.class,
+        "Cannot modify a static table",
+        () -> {
+          PendingUpdate<T> pendingUpdate = pendingUpdateSupplier.apply(env);
+          pendingUpdate.validateCurrentTable(
+              currentTable -> {
+                // illegal action
+                currentTable.updateProperties().set("custom_watermark", "2").commit();
+                return true;
+              },
+              "Test failed.");
+          pendingUpdate.commit();
+        });
+
+    assertFailure.accept(env);
+  }
+
+  private <E, T> void testValidationBehaviours(
+      Supplier<E> setup,
+      Function<E, PendingUpdate<T>> pendingUpdateSupplier,
+      Consumer<E> assertSuccess,
+      Consumer<E> assertFailure)
+      throws Exception {
+    testValidationPasses(setup, pendingUpdateSupplier, assertSuccess);
+    testValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testFirstValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testSecondValidationFails(setup, pendingUpdateSupplier, assertFailure);
+    testValidationFailsDueToConcurrentCommit(setup, pendingUpdateSupplier, assertFailure);
+    testModifyingTableInsideValidationThrowsException(setup, pendingUpdateSupplier, assertFailure);
+  }
+
+  private <T> void testValidationBehaviours(
+      Supplier<PendingUpdate<T>> pendingUpdateSupplier,
+      Runnable assertSuccess,
+      Runnable assertFailure)
+      throws Exception {
+    testValidationBehaviours(
+        () -> null,
+        (__) -> pendingUpdateSupplier.get(),
+        (__) -> assertSuccess.run(),
+        (__) -> assertFailure.run());
+  }
+
+  @Test
+  public void testCherryPickOperation() throws Exception {
+    class Setup {
+      final long firstSnapshotId;
+      final long overwriteSnapshotId;
+
+      Setup(long firstSnapshotId, long overwriteSnapshotId) {
+        this.firstSnapshotId = firstSnapshotId;
+        this.overwriteSnapshotId = overwriteSnapshotId;
+      }
+    }
+
+    testValidationBehaviours(

Review Comment:
   I appreciate the politely-worded comment but I am going to push back a little here. 
   
   I did not write de-duplicated code for the sake of de-duplicated code here. In general, I agree with having clear and readable tests over deduplicated code and in fact, I originally wrote the tests without any "deduplication" but quickly found that maintaining them was a massive burden. I "deduplicated" the code to ease this and didn't personally feel the readability was compromised that badly. 
   
   I'll leave this as unresolved for. now to see if anyone else finds the tests extremely unreadable, in which case I'll make the requested change. 



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final String message;
+    private final Object[] args;
+
+    @FormatMethod
+    Validation(Predicate<Table> predicate, String message, Object... args) {
+      this.predicate = predicate;
+      this.message = message;
+      this.args = args;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(
+      Predicate<Table> predicate, String message, Object... args) {
+    this.validations.add(new Validation(predicate, message, args));
+  }
+
+  @SuppressWarnings("FormatStringAnnotation")
+  protected final void validateCurrentTableMetadata(TableMetadata base) {

Review Comment:
   We still can't do this even after rebasing on latest master: 
   ```
   Execution failed for task ':iceberg-core:revapi'.
   > There were Java public API/ABI breaks reported by revapi:
     
     java.method.visibilityReduced: visibility reduced
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.method.visibilityReduced" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.method.movedToSuperClass: The method used to be declared in class 'org.apache.iceberg.BaseReplacePartitions' but is now declared in superclass 'org.apache.iceberg.BasePendingUpdate<T extends java.lang.Object>'.
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: EQUIVALENT, BINARY: EQUIVALENT
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.method.movedToSuperClass" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.element.noLongerDeprecated: The element is no longer deprecated.
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: EQUIVALENT, BINARY: EQUIVALENT
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.element.noLongerDeprecated" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
   ```



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/test/java/org/apache/iceberg/TestTransaction.java:
##########
@@ -771,4 +773,88 @@ public void testSimpleTransactionNotDeletingMetadataOnUnknownSate() throws IOExc
     Assert.assertTrue("Manifest file should exist", new File(manifests.get(0).path()).exists());
     Assert.assertEquals("Should have 2 files in metadata", 2, countAllMetadataFiles(tableDir));
   }
+
+  @Test
+  public void testValidationPasses() {
+    Assert.assertEquals("Table should be on version 0", 0, (int) version());
+
+    Transaction txn = table.newTransaction();
+    AppendFiles appendFiles = txn.newAppend().appendFile(FILE_A);
+    appendFiles.validateCurrentTable(currentTable -> true, "Custom validation failed.");
+    appendFiles.commit();
+    txn.commitTransaction();
+
+    Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version());
+  }
+
+  @Test
+  public void testValidationFails() {
+    Assert.assertEquals("Table should be on version 0", 0, (int) version());
+
+    Transaction txn = table.newTransaction();
+    AssertHelpers.assertThrows(

Review Comment:
   Fixed. 



-- 
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 #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java:
##########
@@ -165,4 +165,9 @@ private Expression dataConflictDetectionFilter() {
       return Expressions.alwaysTrue();
     }
   }
+
+  @Override
+  protected SnapshotUpdate<OverwriteFiles> returnThis() {

Review Comment:
   I don't think this is a good idea. This should use the existing `self()` method and use a similar signature as the others configuration methods.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;

Review Comment:
   Fixed. 
   
   The reason I had to do this originally was because the compiler kept throwing the following warning: 
   ```
   /Users/fq/src/iceberg/core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:55: error: [FormatStringAnnotation] Variables used as format strings that are not local variables must be compile time constants.
   ```
   Unfortunately, I haven't been able to find a way to convince the compiler, so as a compromise I've had to to include a `@SuppressWarnings("FormatStringAnnotation")` to get around this issue. 



-- 
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 #6513: API: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/NoOpOperation.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+/**
+ * Performs a no-op commit.
+ *
+ * <p>This update is not exposed though the Table API. Instead, it is a package-private part of the
+ * Transaction API intended for use in {@link ManageSnapshots}.
+ */
+class NoOpOperation extends BaseValidatablePendingUpdate<NoOpOperation, Void>
+    implements ValidatablePendingUpdate<NoOpOperation, Void> {
+
+  private final String tableName;
+  private final TableOperations ops;
+
+  NoOpOperation(String tableName, TableOperations ops) {
+    this.tableName = tableName;
+    this.ops = ops;
+  }
+
+  @Override
+  public Void apply() {
+    return null;
+  }
+
+  @Override
+  public void commit() {
+    validateCurrent(SerializableTable.copyOf(new BaseTable(ops, tableName)));

Review Comment:
   It doesn't seem worth adding table name to all of the operation constructors just to construct this `BaseTable`. What about just passing null? The table is only used for validation.
   
   Also, I think it would be better to use a `BaseTable` with an underlying `StaticTableOperations`. You'll need to add a constructor that allows you to pass in the `TableMetadata`.



-- 
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 #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {

Review Comment:
   Are there pending updates for which we don't want to be able to validate the table state?



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -29,6 +31,28 @@
  */
 public interface PendingUpdate<T> {
 
+  /**
+   * Accepts a predicate which will be used to validate whether it is safe to commit the pending
+   * changes to the current version of the table at commit time.
+   *
+   * <p>For example, this method could be used to ensure the pending changes are only committed if a
+   * given snapshot property is present in the current version of the table.
+   *
+   * <p>This method can be called multiple times to add multiple predicates if necessary.
+   *
+   * @param test The predicate which will be used to validate whether it is safe to commit the
+   *     pending changes to the current version of the table at commit time. Any attempts to modify
+   *     the table passed to the predicate will throw an exception as this table is read-only.
+   * @param message The message that will be included in the {@link ValidationException} that will
+   *     be thrown if the test returns false.
+   * @param args The args that will be included in the {@link ValidationException} that will be
+   *     thrown if the test returns false.
+   */
+  @FormatMethod
+  default void validateCurrentTable(Predicate<Table> test, String message, Object... args) {

Review Comment:
   I agree the `commitIf` API is less-awkward/more-fluent. I've made the change now. Fixed. 



-- 
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] nastra commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/PendingUpdate.java:
##########
@@ -41,13 +65,15 @@
   T apply();
 
   /**
-   * Apply the pending changes and commit.
+   * Apply the pending changes, validate the current version of the table, and commit.
    *
    * <p>Changes are committed by calling the underlying table's commit method.
    *
    * <p>Once the commit is successful, the updated table will be refreshed.
    *
    * @throws ValidationException If the update cannot be applied to the current table metadata.
+   * @throws UnsupportedOperationException If any predicate supplied through {@link

Review Comment:
   where is this verified?



-- 
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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {
+    this.validations.add(new Validation(predicate, exception));
+    return returnThis();
+  }
+
+  @Override
+  public final void validateCurrent(Table current) {
+    validations.forEach(
+        validation -> {
+          if (!validation.predicate.test(current)) {
+            throw validation.exception;
+          }
+        });
+  }
+
+  protected abstract ThisT returnThis();

Review Comment:
   ~I noticed this. Unfortunately, I can't call it the same thing because it causes conflicts when both methods are in scope.~
   ~Best I can do is call it `returnSelf()`.~ 
   ~Let me know if you have a better idea, my java skills aren't the greatest :)~
   
   Never mind, I found a way to do 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] fqaiser94 commented on a diff in pull request #6513: API: Support Conditional Commits

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


##########
api/src/main/java/org/apache/iceberg/BaseValidatablePendingUpdate.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BaseValidatablePendingUpdate<ThisT, T>
+    implements ValidatablePendingUpdate<ThisT, T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  public ThisT addValidation(Predicate<Table> predicate, ValidationException exception) {
+    this.validations.add(new Validation(predicate, exception));
+    return returnThis();
+  }
+
+  @Override
+  public final void validateCurrent(Table current) {
+    validations.forEach(
+        validation -> {
+          if (!validation.predicate.test(current)) {
+            throw validation.exception;
+          }
+        });
+  }
+
+  protected abstract ThisT returnThis();

Review Comment:
   I noticed this. Unfortunately, I can't call it the same thing because it causes conflicts when both methods are in scope. 
   Best I can do is call it `returnSelf()`. 
   Let me know if you have a better idea, my java skills aren't the greatest :) 



-- 
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] stevenzwu commented on pull request #6513: API: Support Conditional Commits

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

   @fqaiser94 I added a comment to the issue regarding the motivation use cases: https://github.com/apache/iceberg/issues/6514#issuecomment-1444388008. 


-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(Predicate<Table> predicate, String failMessage) {
+    this.validations.add(new Validation(predicate, new ValidationException(failMessage)));
+  }
+
+  protected final void validateCurrentTableMetadata(String tableName, TableMetadata base) {
+    Table currentTable = new BaseTable(new StaticTableOperations(base), tableName);

Review Comment:
   OK, `tableName` is always `null` now. 



-- 
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 #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final ValidationException exception;
+
+    Validation(Predicate<Table> predicate, ValidationException exception) {
+      this.predicate = predicate;
+      this.exception = exception;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(Predicate<Table> predicate, String failMessage) {
+    this.validations.add(new Validation(predicate, new ValidationException(failMessage)));
+  }
+
+  protected final void validateCurrentTableMetadata(String tableName, TableMetadata base) {
+    Table currentTable = new BaseTable(new StaticTableOperations(base), tableName);

Review Comment:
   I'd rather not have all the changes to pass the table name in here. That makes this touch a lot of files needlessly.



-- 
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 #6513: API,Core: Support Conditional Commits

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


##########
build.gradle:
##########
@@ -698,6 +698,11 @@ project(':iceberg-dell') {
     implementation project(':iceberg-common')
     implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
     compileOnly 'com.emc.ecs:object-client-bundle'
+    compileOnly("org.apache.hadoop:hadoop-common") {
+      exclude group: 'commons-beanutils'
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    }

Review Comment:
   Is this still needed? I don't know why these changes would use `SerializableTable` now.



-- 
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] fqaiser94 commented on a diff in pull request #6513: API,Core: Support Conditional Commits

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


##########
core/src/main/java/org/apache/iceberg/BasePendingUpdate.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import com.google.errorprone.annotations.FormatMethod;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+abstract class BasePendingUpdate<T> implements PendingUpdate<T> {
+
+  private static class Validation {
+    private final Predicate<Table> predicate;
+    private final String message;
+    private final Object[] args;
+
+    @FormatMethod
+    Validation(Predicate<Table> predicate, String message, Object... args) {
+      this.predicate = predicate;
+      this.message = message;
+      this.args = args;
+    }
+  }
+
+  private final List<Validation> validations = Lists.newArrayList();
+
+  @Override
+  @FormatMethod
+  public final void validateCurrentTable(
+      Predicate<Table> predicate, String message, Object... args) {
+    this.validations.add(new Validation(predicate, message, args));
+  }
+
+  @SuppressWarnings("FormatStringAnnotation")
+  protected final void validateCurrentTableMetadata(TableMetadata base) {

Review Comment:
   We still can't do this even after rebasing on latest master, revapi still sees it as a breaking change: 
   ```
   Execution failed for task ':iceberg-core:revapi'.
   > There were Java public API/ABI breaks reported by revapi:
     
     java.method.visibilityReduced: visibility reduced
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: BREAKING, BINARY: BREAKING
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.method.visibilityReduced" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.method.movedToSuperClass: The method used to be declared in class 'org.apache.iceberg.BaseReplacePartitions' but is now declared in superclass 'org.apache.iceberg.BasePendingUpdate<T extends java.lang.Object>'.
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: EQUIVALENT, BINARY: EQUIVALENT
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.method.movedToSuperClass" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
     java.element.noLongerDeprecated: The element is no longer deprecated.
     
     old: method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)
     new: method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions
     
     SOURCE: EQUIVALENT, BINARY: EQUIVALENT
     
     From old archive: iceberg-core-1.1.0.jar
     From new archive: iceberg-core-1.3.0-SNAPSHOT.jar
     
     If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for:
     
       * Just this break:
           ./gradlew :iceberg-core:revapiAcceptBreak --justification "{why this break is ok}" \
             --code "java.element.noLongerDeprecated" \
             --old "method void org.apache.iceberg.BaseReplacePartitions::validate(org.apache.iceberg.TableMetadata)" \
             --new "method void org.apache.iceberg.BasePendingUpdate<T>::validate(org.apache.iceberg.TableMetadata) @ org.apache.iceberg.BaseReplacePartitions"
       * All breaks in this project:
           ./gradlew :iceberg-core:revapiAcceptAllBreaks --justification "{why this break is ok}"
       * All breaks in all projects:
           ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}"
     ----------------------------------------------------------------------------------------------------
   ```



-- 
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