You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "nastra (via GitHub)" <gi...@apache.org> on 2023/05/09 14:15:18 UTC

[GitHub] [iceberg] nastra opened a new pull request, #7569: Core: Add REST API for committing changes against multiple tables

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

   (no comment)


-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitTransactionRequest'
+        required: true
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:

Review Comment:
   Does this make sense with multiple tables? The route exists. Maybe this is a bad request?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,29 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:
+      type: object
+      required:
+        - identifier
+        - requirements
+        - updates
+      properties:
+        identifier:
+          $ref: '#/components/schemas/TableIdentifier'
+        requirements:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableRequirement'
+        updates:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableUpdate'
+
+    CommitTransactionRequest:
+      type: array

Review Comment:
   good point, this makes a lot of sense. I've updated this



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1742,4 +1753,129 @@ public void testCatalogTokenRefreshDisabledWithCredential() {
             eq(catalogHeaders),
             any());
   }
+
+  @Test
+  public void multipleDiffsAgainstSingleTable() {
+    Namespace namespace = Namespace.of("namespace");
+    TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable");
+
+    Transaction transaction = catalog().buildTable(identifier, SCHEMA).create().newTransaction();
+
+    UpdateSchema updateSchema =
+        transaction.updateSchema().addColumn("new_col", Types.LongType.get());
+    Schema expectedSchema = updateSchema.apply();
+    updateSchema.commit();
+
+    UpdatePartitionSpec updateSpec =
+        transaction.updateSpec().addField("shard", Expressions.bucket("id", 16));
+    PartitionSpec expectedSpec = updateSpec.apply();
+    updateSpec.commit();
+
+    TableCommit tableCommit =
+        ImmutableTableCommit.builder()
+            .identifier(identifier)
+            .base(((BaseTransaction) transaction).startMetadata())
+            .changes(((BaseTransaction) transaction).currentMetadata().changes())
+            .build();
+
+    restCatalog.commitTransaction(ImmutableList.of(tableCommit));
+
+    Table loaded = catalog().loadTable(identifier);
+    assertThat(loaded.schema().asStruct()).isEqualTo(expectedSchema.asStruct());
+    assertThat(loaded.spec().fields()).isEqualTo(expectedSpec.fields());
+  }
+
+  @Test
+  public void multipleDiffsAgainstMultipleTables() {
+    Namespace namespace = Namespace.of("multiDiffNamespace");
+    TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1");
+    TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2");
+
+    Transaction transaction1 = catalog().buildTable(identifier1, SCHEMA).create().newTransaction();
+    Transaction transaction2 = catalog().buildTable(identifier2, SCHEMA).create().newTransaction();

Review Comment:
   The pattern of creating tables and starting transactions at the same time comes up here, where it isn't clear further on whether the transactions are for the same table or different tables.
   
   It would help to name these `t1transaction` and `t2transaction` to make that clear. And separating the create code for `t1` and `t2` would also help.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -95,6 +110,12 @@ public Builder(TableMetadata base, boolean isReplace) {
       this.isReplace = isReplace;
     }
 
+    public Builder withIdentifier(TableIdentifier ident) {

Review Comment:
   `forTable` sounds good



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,32 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {
+    List<Transaction> transactions = Lists.newArrayList();
+
+    for (CommitTransactionRequest.CommitTableRequest tableChange : request.tableChanges()) {
+      Table table = catalog.loadTable(tableChange.identifier());
+      if (table instanceof BaseTable) {
+        UpdateTableRequest updateTableRequest =
+            new UpdateTableRequest(tableChange.requirements(), tableChange.updates());
+
+        Transaction transaction =
+            Transactions.newTransaction(
+                tableChange.identifier().toString(), ((BaseTable) table).operations());
+        transactions.add(transaction);
+
+        BaseTransaction.TransactionTable txTable =
+            (BaseTransaction.TransactionTable) transaction.table();
+
+        // this performs validations and makes temporary commits that are in-memory
+        commit(txTable.operations(), updateTableRequest);
+      } else {
+        throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable");
+      }
+    }
+
+    // only commit if validations passed previously
+    transactions.forEach(Transaction::commitTransaction);

Review Comment:
   @singhpk234 the intention here was to have a very simple approach that does only validate the requirements and nothing else. Hence we don't have a way of providing true atomicity here, which is left to the implementation details of a server. I've also added a comment to make this clear.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,30 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:

Review Comment:
   the existing `CommitTableRequest` doesn't have an `identifier` field unfortunately



-- 
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 pull request #7569: Core: Add REST API for committing changes against multiple tables

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

   @rdblue I've opened https://github.com/apache/iceberg/pull/7741 that contains the REST-specific 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] nastra commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -84,6 +98,7 @@ public static class Builder {
     private final List<MetadataUpdate> updates = Lists.newArrayList();
     private final Set<String> changedRefs = Sets.newHashSet();
     private final boolean isReplace;
+    private TableIdentifier identifier;

Review Comment:
   done



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,37 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  /**
+   * This is a very simplistic approach that only validates the requirements for each table and does
+   * not do any other conflict detection. Therefore, it does not guarantee true transactional
+   * atomicity, which is left to the implementation details of a REST server.
+   */
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {
+    List<Transaction> transactions = Lists.newArrayList();
+
+    for (CommitTransactionRequest.CommitTableRequest tableChange : request.tableChanges()) {
+      Table table = catalog.loadTable(tableChange.identifier());
+      if (table instanceof BaseTable) {
+        UpdateTableRequest updateTableRequest =
+            new UpdateTableRequest(tableChange.requirements(), tableChange.updates());
+
+        Transaction transaction =
+            Transactions.newTransaction(
+                tableChange.identifier().toString(), ((BaseTable) table).operations());
+        transactions.add(transaction);
+
+        BaseTransaction.TransactionTable txTable =
+            (BaseTransaction.TransactionTable) transaction.table();
+
+        // this performs validations and makes temporary commits that are in-memory
+        commit(txTable.operations(), updateTableRequest);

Review Comment:
   I can see why you'd want to use a transaction to get the bulk of the work done before committing, but I don't think this actually works because you're committing the transactions sequentially below.
   
   I think to test we need to pass the `commitTransaction` call through and implement it either for an `InMemoryCatalog` or JDBC.



-- 
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] singhpk234 commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,32 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {
+    List<Transaction> transactions = Lists.newArrayList();
+
+    for (CommitTransactionRequest.CommitTableRequest tableChange : request.tableChanges()) {
+      Table table = catalog.loadTable(tableChange.identifier());
+      if (table instanceof BaseTable) {
+        UpdateTableRequest updateTableRequest =
+            new UpdateTableRequest(tableChange.requirements(), tableChange.updates());
+
+        Transaction transaction =
+            Transactions.newTransaction(
+                tableChange.identifier().toString(), ((BaseTable) table).operations());
+        transactions.add(transaction);
+
+        BaseTransaction.TransactionTable txTable =
+            (BaseTransaction.TransactionTable) transaction.table();
+
+        // this performs validations and makes temporary commits that are in-memory
+        commit(txTable.operations(), updateTableRequest);
+      } else {
+        throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable");
+      }
+    }
+
+    // only commit if validations passed previously
+    transactions.forEach(Transaction::commitTransaction);

Review Comment:
   [doubt] let's say we had transactions {T1, T2, T3}  whose validations passed but then there was an update which made T2 conflict with current state, so when we actually commit T1 is applied but T2 can't now we need to rollback T1, how are we handling this case  ? 
   
   Is there a write up for this ?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1742,4 +1753,129 @@ public void testCatalogTokenRefreshDisabledWithCredential() {
             eq(catalogHeaders),
             any());
   }
+
+  @Test
+  public void multipleDiffsAgainstSingleTable() {
+    Namespace namespace = Namespace.of("namespace");
+    TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable");
+
+    Transaction transaction = catalog().buildTable(identifier, SCHEMA).create().newTransaction();

Review Comment:
   I think it is easier to read if you separate actions on different lines. Here, I would expect a line where the table is created and a line that starts a transaction.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1742,4 +1753,129 @@ public void testCatalogTokenRefreshDisabledWithCredential() {
             eq(catalogHeaders),
             any());
   }
+
+  @Test
+  public void multipleDiffsAgainstSingleTable() {

Review Comment:
   This isn't testing multiple changes. It's testing just one set. The fact that the `Transaction` accumulates changes across table operations isn't really material to the test.
   
   I'd change this to `testSingleTable`.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -95,6 +110,12 @@ public Builder(TableMetadata base, boolean isReplace) {
       this.isReplace = isReplace;
     }
 
+    public Builder withIdentifier(TableIdentifier ident) {

Review Comment:
   How about `forTable` or something that signals that this is a way to identify the table to 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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.UpdateRequirement;
+import org.apache.iceberg.UpdateRequirements;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  List<UpdateRequirement> requirements();
+
+  List<MetadataUpdate> updates();
+
+  static TableCommit create(TableIdentifier identifier, TableMetadata base, TableMetadata updated) {

Review Comment:
   Javadoc?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,30 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:

Review Comment:
   I think we could refactor to work around this. Could we just use a list of `CommitTableRequest`?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.catalog;
+
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {

Review Comment:
   Why include `TableMetadata` rather than both requirements and updates?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.catalog;
+
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {

Review Comment:
   that would be ideal, but currently `UpdateRequirement` is part of the REST package and defined inside `UpdateTableRequest`. We would have to extract that out from there, which I wanted to avoid in this PR



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  TableMetadata base();

Review Comment:
   Looks like this is the primary blocker, so I recommend we get started on the refactor and get some of the other pieces of this PR (like REST updates and request objects) in.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -35,21 +35,22 @@
 public class UpdateTableRequest implements RESTRequest {
 
   private TableIdentifier identifier;
-  private List<UpdateRequirement> requirements;
+  private List<org.apache.iceberg.UpdateRequirement> requirements;
   private List<MetadataUpdate> updates;
 
   public UpdateTableRequest() {
     // needed for Jackson deserialization
   }
 
-  public UpdateTableRequest(List<UpdateRequirement> requirements, List<MetadataUpdate> updates) {
+  public UpdateTableRequest(
+      List<org.apache.iceberg.UpdateRequirement> requirements, List<MetadataUpdate> updates) {
     this.requirements = requirements;
     this.updates = updates;
   }
 
-  UpdateTableRequest(
+  public UpdateTableRequest(

Review Comment:
   Maybe we should have a `create` method like `TableCommit`? Or one that accepts a `TableCommit` and returns the correct request? That would be simpler than a builder.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -62,6 +62,11 @@ public <T> T withContext(SessionContext context, Function<Catalog, T> task) {
     return task.apply(asCatalog(context));
   }
 
+  public void multiTableCommit(SessionContext context, List<TableCommit> commits) {

Review Comment:
   `commitTables` sounds better, let's go with that for 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] nastra commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.

Review Comment:
   good point, I guess we can remove that whole paragraph then



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:

Review Comment:
   that makes actually sense as well. I'll update this accordingly



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/BaseTransaction.java:
##########
@@ -107,6 +107,10 @@ public String tableName() {
   }
 
   public TableMetadata startMetadata() {
+    return base;
+  }
+
+  public TableMetadata currentMetadata() {

Review Comment:
   Can we focus on the REST catalog pieces and not transaction pieces right now? I think there's plenty of work in updating the REST spec and adding the 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] nastra commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -916,4 +920,31 @@ private Cache<TableOperations, FileIO> newFileIOCloser() {
                 })
         .build();
   }
+
+  @Override
+  public void commitTransaction(SessionContext context, List<TableCommit> commits) {
+    ImmutableCommitTransactionRequest.Builder builder = ImmutableCommitTransactionRequest.builder();
+
+    for (TableCommit commit : commits) {
+      UpdateTableRequest.Builder updateTableBuilder = UpdateTableRequest.builderFor(commit.base());
+      commit.changes().forEach(updateTableBuilder::update);
+      UpdateTableRequest updateTableRequest = updateTableBuilder.build();

Review Comment:
   yes I agree, we'd have to pull them out of the REST layer and make them work for non-REST catalogs 



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -916,4 +920,31 @@ private Cache<TableOperations, FileIO> newFileIOCloser() {
                 })
         .build();
   }
+
+  @Override
+  public void commitTransaction(SessionContext context, List<TableCommit> commits) {
+    ImmutableCommitTransactionRequest.Builder builder = ImmutableCommitTransactionRequest.builder();
+
+    for (TableCommit commit : commits) {
+      UpdateTableRequest.Builder updateTableBuilder = UpdateTableRequest.builderFor(commit.base());
+      commit.changes().forEach(updateTableBuilder::update);
+      UpdateTableRequest updateTableRequest = updateTableBuilder.build();

Review Comment:
   Looks like we may need to refactor how we produce requirements. This would not work for non-REST catalogs.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitTransactionRequest'
+        required: true
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:

Review Comment:
   The only case where I thought it might make sense to throw a 404 is when the server gets a diff for a table that doesn't exist (anymore). Alternatively we could handle this case via a general `CommitFailedException`, but `CommitFailedException` is retryable



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,32 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {
+    List<Transaction> transactions = Lists.newArrayList();
+
+    for (CommitTransactionRequest.CommitTableRequest tableChange : request.tableChanges()) {
+      Table table = catalog.loadTable(tableChange.identifier());
+      if (table instanceof BaseTable) {
+        UpdateTableRequest updateTableRequest =
+            new UpdateTableRequest(tableChange.requirements(), tableChange.updates());
+
+        Transaction transaction =
+            Transactions.newTransaction(
+                tableChange.identifier().toString(), ((BaseTable) table).operations());
+        transactions.add(transaction);
+
+        BaseTransaction.TransactionTable txTable =
+            (BaseTransaction.TransactionTable) transaction.table();
+
+        // this performs validations and makes temporary commits that are in-memory
+        commit(txTable.operations(), updateTableRequest);
+      } else {
+        throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable");
+      }
+    }
+
+    // only commit if validations passed previously
+    transactions.forEach(Transaction::commitTransaction);

Review Comment:
   @singhpk234 the intention here was to have a very simple approach that does only validate the requirements and nothing else. Hence we don't have a way of providing true atomicity here, which is left to the implementation details of a server. I've also added a comment to make this clear.
   The main idea of this PR was to only include the REST-specific changes from https://github.com/apache/iceberg/pull/6948



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -62,6 +62,11 @@ public <T> T withContext(SessionContext context, Function<Catalog, T> task) {
     return task.apply(asCatalog(context));
   }
 
+  public void multiTableCommit(SessionContext context, List<TableCommit> commits) {

Review Comment:
   This doesn't really fit with the phrasing of the other methods in this class, like `loadTable`. What about something like `commitTables`? That isn't great, but `multiTableCommit` uses "commit" as a noun rather than a verb.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/BaseTransaction.java:
##########
@@ -107,6 +107,10 @@ public String tableName() {
   }
 
   public TableMetadata startMetadata() {
+    return base;
+  }
+
+  public TableMetadata currentMetadata() {

Review Comment:
   I'm fine with this being here. I was thinking about how to write the tests and this is an easy way to stack up multiple 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] nastra commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -35,21 +35,22 @@
 public class UpdateTableRequest implements RESTRequest {
 
   private TableIdentifier identifier;
-  private List<UpdateRequirement> requirements;
+  private List<org.apache.iceberg.UpdateRequirement> requirements;
   private List<MetadataUpdate> updates;
 
   public UpdateTableRequest() {
     // needed for Jackson deserialization
   }
 
-  public UpdateTableRequest(List<UpdateRequirement> requirements, List<MetadataUpdate> updates) {
+  public UpdateTableRequest(
+      List<org.apache.iceberg.UpdateRequirement> requirements, List<MetadataUpdate> updates) {
     this.requirements = requirements;
     this.updates = updates;
   }
 
-  UpdateTableRequest(
+  public UpdateTableRequest(

Review Comment:
   added in https://github.com/apache/iceberg/pull/7867



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -62,6 +62,11 @@ public <T> T withContext(SessionContext context, Function<Catalog, T> task) {
     return task.apply(asCatalog(context));
   }
 
+  public void commitTransaction(SessionContext context, List<TableCommit> commits) {

Review Comment:
   generally speaking, the plan was to add this to `SessionCatalog` and `Catalog` (which are both in the `api` module), but given that we currently have `TableMetadata` (which is in the `core` module) in `TableCommit`, this turned out to be difficult.
   
   I've changed this up now so that it's only defined in `RESTSessionCatalog` and `RESTCatalog`. I think putting this method there is probably best for now and later we can move them to interfaces.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1742,4 +1753,129 @@ public void testCatalogTokenRefreshDisabledWithCredential() {
             eq(catalogHeaders),
             any());
   }
+
+  @Test
+  public void multipleDiffsAgainstSingleTable() {
+    Namespace namespace = Namespace.of("namespace");
+    TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable");
+
+    Transaction transaction = catalog().buildTable(identifier, SCHEMA).create().newTransaction();
+
+    UpdateSchema updateSchema =
+        transaction.updateSchema().addColumn("new_col", Types.LongType.get());
+    Schema expectedSchema = updateSchema.apply();
+    updateSchema.commit();
+
+    UpdatePartitionSpec updateSpec =
+        transaction.updateSpec().addField("shard", Expressions.bucket("id", 16));
+    PartitionSpec expectedSpec = updateSpec.apply();
+    updateSpec.commit();
+
+    TableCommit tableCommit =
+        ImmutableTableCommit.builder()
+            .identifier(identifier)
+            .base(((BaseTransaction) transaction).startMetadata())
+            .changes(((BaseTransaction) transaction).currentMetadata().changes())
+            .build();
+
+    restCatalog.commitTransaction(ImmutableList.of(tableCommit));
+
+    Table loaded = catalog().loadTable(identifier);
+    assertThat(loaded.schema().asStruct()).isEqualTo(expectedSchema.asStruct());
+    assertThat(loaded.spec().fields()).isEqualTo(expectedSpec.fields());
+  }
+
+  @Test
+  public void multipleDiffsAgainstMultipleTables() {
+    Namespace namespace = Namespace.of("multiDiffNamespace");
+    TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1");
+    TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2");
+
+    Transaction transaction1 = catalog().buildTable(identifier1, SCHEMA).create().newTransaction();
+    Transaction transaction2 = catalog().buildTable(identifier2, SCHEMA).create().newTransaction();
+
+    UpdateSchema updateSchema =
+        transaction1.updateSchema().addColumn("new_col", Types.LongType.get());
+    Schema expectedSchema = updateSchema.apply();
+    updateSchema.commit();
+
+    UpdateSchema updateSchema2 =
+        transaction2.updateSchema().addColumn("new_col2", Types.LongType.get());
+    Schema expectedSchema2 = updateSchema2.apply();
+    updateSchema2.commit();
+
+    TableCommit tableCommit1 =
+        ImmutableTableCommit.builder()
+            .identifier(identifier1)
+            .base(((BaseTransaction) transaction1).startMetadata())
+            .changes(((BaseTransaction) transaction1).currentMetadata().changes())
+            .build();
+
+    TableCommit tableCommit2 =
+        ImmutableTableCommit.builder()
+            .identifier(identifier2)
+            .base(((BaseTransaction) transaction2).startMetadata())
+            .changes(((BaseTransaction) transaction2).currentMetadata().changes())
+            .build();
+
+    restCatalog.commitTransaction(ImmutableList.of(tableCommit1, tableCommit2));

Review Comment:
   To make things like this easier, we tend to add convenience methods, like `commitTransaction(TableCommit... commits)` with a default that creates the `ImmutableList`. That makes tests a bit easier to read and write.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  TableMetadata base();
+
+  List<MetadataUpdate> changes();

Review Comment:
   The `UpdateTableRequest` uses `updates` instead of `changes`. Can we use the same here for consistency?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,32 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {
+    List<Transaction> transactions = Lists.newArrayList();
+
+    for (CommitTransactionRequest.CommitTableRequest tableChange : request.tableChanges()) {
+      Table table = catalog.loadTable(tableChange.identifier());
+      if (table instanceof BaseTable) {
+        UpdateTableRequest updateTableRequest =
+            new UpdateTableRequest(tableChange.requirements(), tableChange.updates());
+
+        Transaction transaction =
+            Transactions.newTransaction(
+                tableChange.identifier().toString(), ((BaseTable) table).operations());
+        transactions.add(transaction);
+
+        BaseTransaction.TransactionTable txTable =
+            (BaseTransaction.TransactionTable) transaction.table();
+
+        // this performs validations and makes temporary commits that are in-memory
+        commit(txTable.operations(), updateTableRequest);
+      } else {
+        throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable");
+      }
+    }
+
+    // only commit if validations passed previously
+    transactions.forEach(Transaction::commitTransaction);

Review Comment:
   @singhpk234 the intention here was to have a very simple approach that does only validate the requirements. Hence we don't have a way of providing true atomicity here, which is left to the implementation details of a server. 
   I've also added a comment to make this clear.
   The main idea of this PR was to only include the REST-specific changes from https://github.com/apache/iceberg/pull/6948



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.catalog;
+
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {

Review Comment:
   indeed this would allow passing two random `TableMetadata` objects, which isn't ideal. I've replaced `updated` with the actual metadata changes.
   The `base` metadata here is only needed to build the actual `requirements` for the 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] nastra commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,30 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:

Review Comment:
   `CommitTableRequest` does already exist in this file unfortunately, hence naming it `CommitTransactionTableRequest`



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,30 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:
+      type: object
+      required:
+        - identifier
+        - requirements
+        - updates
+      properties:
+        identifier:
+          $ref: '#/components/schemas/TableIdentifier'
+        requirements:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableRequirement'
+        updates:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableUpdate'
+
+    CommitTransactionRequest:
+      type: array

Review Comment:
   this will need to be updated once the JSON representation has been finalized



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -56,6 +66,10 @@ public List<MetadataUpdate> updates() {
     return updates != null ? updates : ImmutableList.of();
   }
 
+  public TableIdentifier identifier() {

Review Comment:
   I think the parser and tests for this class need to be updated for the new field. We should not drop the identifier when it is used in certain cases.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:

Review Comment:
   I like this route. Maybe we should base the method name on it? `catalog.commitTransaction`?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequest.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTRequest;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface CommitTransactionRequest extends RESTRequest {
+  List<CommitTableRequest> tableChanges();
+
+  @Override
+  default void validate() {
+    check();
+  }
+
+  @Value.Check
+  default void check() {
+    Preconditions.checkArgument(!tableChanges().isEmpty(), "Invalid table changes: empty");
+  }
+
+  @Value.Immutable
+  interface CommitTableRequest {

Review Comment:
   It seems confusing to have a `CommitTableRequest` and an `UpdateTableRequest` that are basically the same thing but with or without the identifier. Then there is also the [naming problem in the spec](https://github.com/apache/iceberg/pull/7569/files#r1189516857). Those issues are a bit of a red flag that we need to address duplication.
   
   I think there are two options. First, we could reuse the `UpdateTableRequest` directly and pull the identifier to a higher level, like this:
   
   ```yaml
       CommitTransactionRequest:
         type: array
         items:
           $ref: '#/components/schemas/TransactionCommit'
   
       TransactionCommit:
         type: object
         required:
           - identifier
           - commit
         properties:
           identifier:
             $ref: '#/components/schemas/TableIdentifier'
           request:
             $ref: '#/components/schemas/CommitTableRequest'
   
       CommitTableRequest:
         type: object
         required:
           - requirements
           - updates
         properties:
           requirements:
             type: array
             items:
               $ref: '#/components/schemas/TableRequirement'
           updates:
             type: array
             items:
               $ref: '#/components/schemas/TableUpdate'
   ```
   
   That's okay, but still fairly awkward. The next option is to reuse the existing schema directly and just add an optional identifier field:
   
   ```yaml
       CommitTransactionRequest:
         type: array
         items:
           description: Each table commit request must provide an `identifier`
           $ref: '#/components/schemas/CommitTableRequest'
   
       CommitTableRequest:
         type: object
         required:
           - requirements
           - updates
         properties:
           identifier:
             description: Table identifier to update; must be present for CommitTransactionRequest
             $ref: '#/components/schemas/TableIdentifier'
           requirements:
             type: array
             items:
               $ref: '#/components/schemas/TableRequirement'
           updates:
             type: array
             items:
               $ref: '#/components/schemas/TableUpdate'
   ```
   
   I prefer the second option. It's not a problem to add an optional field and ensure it is set when the object is used in a transaction. We'd also want to check that the identifier is either not set or is set to the same table in a normal table update.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitTransactionRequest'
+        required: true
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:

Review Comment:
   I'm undecided. On one hand, that would already work and we wouldn't need to find a response code for "concurrent delete" that avoids the retry. On the other, the route doesn't represent that table.
   
   I think I'm inclined to go with what you have here right now. It makes sense and we can always deprecate its use later.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,29 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:
+      type: object
+      required:
+        - identifier
+        - requirements
+        - updates
+      properties:
+        identifier:
+          $ref: '#/components/schemas/TableIdentifier'
+        requirements:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableRequirement'
+        updates:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableUpdate'
+
+    CommitTransactionRequest:
+      type: array

Review Comment:
   I don't think this should be an array. The problem with using an array is that you can't make top-level additions to the request without changing its type. Using an object allows you to add optional keys later. For example, what happens if we want to create and track a transaction ID? That would need to be added here.
   
   Instead of an array, I think we should use a structure like this:
   
   ```yaml
       CommitTransactionRequest:
         type: object
         required:
           - tables
         properties:
           tables:
             type: array
             items:
               description: Table commit request; must provide an `identifier`
               $ref: '#/components/schemas/CommitTableRequest'
   ```



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.UpdateRequirement;
+import org.apache.iceberg.UpdateRequirements;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  List<UpdateRequirement> requirements();
+
+  List<MetadataUpdate> updates();
+
+  static TableCommit create(TableIdentifier identifier, TableMetadata base, TableMetadata updated) {

Review Comment:
   added in https://github.com/apache/iceberg/pull/7867



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequest.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTRequest;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface CommitTransactionRequest extends RESTRequest {
+  List<CommitTableRequest> tableChanges();
+
+  @Override
+  default void validate() {
+    check();
+  }
+
+  @Value.Check
+  default void check() {
+    Preconditions.checkArgument(!tableChanges().isEmpty(), "Invalid table changes: empty");
+  }
+
+  @Value.Immutable
+  interface CommitTableRequest {

Review Comment:
   It seems confusing to have a `CommitTableRequest` and an `UpdateTableRequest` that are basically the same thing but with or without the identifier. Then there is also the [naming problem in the spec](https://github.com/apache/iceberg/pull/7569/files#r1189516857). Those issues are a bit of a red flag that we need to address duplication.
   
   I think there are two options. First, we could reuse the `UpdateTableRequest` directly and pull the identifier to a higher level, like this:
   
   ```json
       CommitTransactionRequest:
         type: array
         items:
           $ref: '#/components/schemas/TransactionCommit'
   
       TransactionCommit:
         type: object
         required:
           - identifier
           - commit
         properties:
           identifier:
             $ref: '#/components/schemas/TableIdentifier'
           request:
             $ref: '#/components/schemas/CommitTableRequest'
   
       CommitTableRequest:
         type: object
         required:
           - requirements
           - updates
         properties:
           requirements:
             type: array
             items:
               $ref: '#/components/schemas/TableRequirement'
           updates:
             type: array
             items:
               $ref: '#/components/schemas/TableUpdate'
   ```
   
   That's okay, but still fairly awkward. The next option is to reuse the existing schema directly and just add an optional identifier field:
   
   ```json
       CommitTransactionRequest:
         type: array
         items:
           description: Each table commit request must provide an `identifier`
           $ref: '#/components/schemas/CommitTableRequest'
   
       CommitTableRequest:
         type: object
         required:
           - requirements
           - updates
         properties:
           identifier:
             description: Table identifier to update; must be present for CommitTransactionRequest
             $ref: '#/components/schemas/TableIdentifier'
           requirements:
             type: array
             items:
               $ref: '#/components/schemas/TableRequirement'
           updates:
             type: array
             items:
               $ref: '#/components/schemas/TableUpdate'
   ```
   
   I prefer the second option. It's not a problem to add an optional field and ensure it is set when the object is used in a transaction. We'd also want to check that the identifier is either not set or is set to the same table in a normal table update.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -62,6 +62,11 @@ public <T> T withContext(SessionContext context, Function<Catalog, T> task) {
     return task.apply(asCatalog(context));
   }
 
+  public void commitTransaction(SessionContext context, List<TableCommit> commits) {

Review Comment:
   Why add this here? I would expect to either add this to the `SessionCatalog` interface and to `Catalog`, or to create optional interfaces to add this behavior. Adding this here requires casting to this base class (or `AsCatalog` rather than extending the API.
   
   I think maybe not extending the API was the point, but in that case I'd prefer to add capabilities to just the `RESTSessionCatalog` and `RESTCatalog` so that we can move them to interfaces later.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1756,6 +1871,29 @@ components:
           items:
             $ref: '#/components/schemas/TableUpdate'
 
+    CommitTransactionTableRequest:
+      type: object
+      required:
+        - identifier
+        - requirements
+        - updates
+      properties:
+        identifier:
+          $ref: '#/components/schemas/TableIdentifier'
+        requirements:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableRequirement'
+        updates:
+          type: array
+          items:
+            $ref: '#/components/schemas/TableUpdate'
+
+    CommitTransactionRequest:
+      type: array

Review Comment:
   I don't think this should be an array. The problem with using an array is that you can't make top-level additions to the request without changing its type. Using an object allows you to add optional keys later. For example, what happens if we want to create and track a transaction ID? That would need to be added here.
   
   Instead of an array, I think we should use a structure like this:
   
   ```yaml
       CommitTransactionRequest:
         type: object
         required:
           - table-changes
         properties:
           table-changes:
             type: array
             items:
               description: Table commit request; must provide an `identifier`
               $ref: '#/components/schemas/CommitTableRequest'
   ```



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.catalog;
+
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {

Review Comment:
   Do we want this to be based on `TableMetadata` or do we want it based on changes?
   
   I think we want it based on changes that come from `TableMetadata`. That way we don't need to diff `TableMetadata` objects. We can do that in `TableOperations` because that controls the metadata base. But here, we could be passed two objects that are completely unrelated. So it makes more sense for the caller to manage `base` and `updated` and to get the change set from updated.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  TableMetadata base();
+
+  List<MetadataUpdate> changes();

Review Comment:
   sure, changed this to `updates`



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/BaseTransaction.java:
##########
@@ -107,7 +107,11 @@ public String tableName() {
   }
 
   public TableMetadata startMetadata() {
-    return current;
+    return this.base;

Review Comment:
   `startMetadata()` wasn't used before and I was expecting it to point to `this.base`



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -84,6 +98,7 @@ public static class Builder {
     private final List<MetadataUpdate> updates = Lists.newArrayList();
     private final Set<String> changedRefs = Sets.newHashSet();
     private final boolean isReplace;
+    private TableIdentifier identifier;

Review Comment:
   I prefer to specifically default to null so it is clear when you read the class.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -35,21 +35,22 @@
 public class UpdateTableRequest implements RESTRequest {
 
   private TableIdentifier identifier;
-  private List<UpdateRequirement> requirements;
+  private List<org.apache.iceberg.UpdateRequirement> requirements;
   private List<MetadataUpdate> updates;
 
   public UpdateTableRequest() {
     // needed for Jackson deserialization
   }
 
-  public UpdateTableRequest(List<UpdateRequirement> requirements, List<MetadataUpdate> updates) {
+  public UpdateTableRequest(
+      List<org.apache.iceberg.UpdateRequirement> requirements, List<MetadataUpdate> updates) {
     this.requirements = requirements;
     this.updates = updates;
   }
 
-  UpdateTableRequest(
+  public UpdateTableRequest(

Review Comment:
   this is required to be public so that it can be used in `RESTSessionCatalog`. 
   It might be also worth adding a Builder to `UpdateTableRequest` so that we don't have to make it public here.
   In this case we have two options:
   * write a manual builder
   * generate a builder from the constructor definition (I've opened https://github.com/apache/iceberg/pull/7838 to indicate how this would look like)
   



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java:
##########
@@ -35,21 +35,22 @@
 public class UpdateTableRequest implements RESTRequest {
 
   private TableIdentifier identifier;
-  private List<UpdateRequirement> requirements;
+  private List<org.apache.iceberg.UpdateRequirement> requirements;
   private List<MetadataUpdate> updates;
 
   public UpdateTableRequest() {
     // needed for Jackson deserialization
   }
 
-  public UpdateTableRequest(List<UpdateRequirement> requirements, List<MetadataUpdate> updates) {
+  public UpdateTableRequest(
+      List<org.apache.iceberg.UpdateRequirement> requirements, List<MetadataUpdate> updates) {
     this.requirements = requirements;
     this.updates = updates;
   }
 
-  UpdateTableRequest(
+  public UpdateTableRequest(

Review Comment:
   sure that would also work. I'll make the respective changes in the next PR



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

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

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


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


[GitHub] [iceberg] rdblue merged pull request #7569: Core: Add REST API for committing changes against multiple tables

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


-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1742,4 +1753,129 @@ public void testCatalogTokenRefreshDisabledWithCredential() {
             eq(catalogHeaders),
             any());
   }
+
+  @Test
+  public void multipleDiffsAgainstSingleTable() {
+    Namespace namespace = Namespace.of("namespace");
+    TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable");
+
+    Transaction transaction = catalog().buildTable(identifier, SCHEMA).create().newTransaction();
+
+    UpdateSchema updateSchema =
+        transaction.updateSchema().addColumn("new_col", Types.LongType.get());
+    Schema expectedSchema = updateSchema.apply();
+    updateSchema.commit();
+
+    UpdatePartitionSpec updateSpec =
+        transaction.updateSpec().addField("shard", Expressions.bucket("id", 16));
+    PartitionSpec expectedSpec = updateSpec.apply();
+    updateSpec.commit();
+
+    TableCommit tableCommit =
+        ImmutableTableCommit.builder()
+            .identifier(identifier)
+            .base(((BaseTransaction) transaction).startMetadata())
+            .changes(((BaseTransaction) transaction).currentMetadata().changes())
+            .build();
+
+    restCatalog.commitTransaction(ImmutableList.of(tableCommit));
+
+    Table loaded = catalog().loadTable(identifier);
+    assertThat(loaded.schema().asStruct()).isEqualTo(expectedSchema.asStruct());
+    assertThat(loaded.spec().fields()).isEqualTo(expectedSpec.fields());
+  }
+
+  @Test
+  public void multipleDiffsAgainstMultipleTables() {
+    Namespace namespace = Namespace.of("multiDiffNamespace");
+    TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1");
+    TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2");
+
+    Transaction transaction1 = catalog().buildTable(identifier1, SCHEMA).create().newTransaction();
+    Transaction transaction2 = catalog().buildTable(identifier2, SCHEMA).create().newTransaction();
+
+    UpdateSchema updateSchema =
+        transaction1.updateSchema().addColumn("new_col", Types.LongType.get());
+    Schema expectedSchema = updateSchema.apply();
+    updateSchema.commit();
+
+    UpdateSchema updateSchema2 =
+        transaction2.updateSchema().addColumn("new_col2", Types.LongType.get());
+    Schema expectedSchema2 = updateSchema2.apply();
+    updateSchema2.commit();
+
+    TableCommit tableCommit1 =
+        ImmutableTableCommit.builder()
+            .identifier(identifier1)
+            .base(((BaseTransaction) transaction1).startMetadata())
+            .changes(((BaseTransaction) transaction1).currentMetadata().changes())
+            .build();
+
+    TableCommit tableCommit2 =
+        ImmutableTableCommit.builder()
+            .identifier(identifier2)
+            .base(((BaseTransaction) transaction2).startMetadata())
+            .changes(((BaseTransaction) transaction2).currentMetadata().changes())
+            .build();
+
+    restCatalog.commitTransaction(ImmutableList.of(tableCommit1, tableCommit2));

Review Comment:
   makes sense, I've added that convenience method



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitTransactionRequest'
+        required: true
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:

Review Comment:
   I think it makes more sense to have a different error. The route does exist. Plus, our existing handlers don't know how to handle a 404 where the missing object is provided in the error 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] rdblue commented on a diff in pull request #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -361,4 +364,37 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  /**
+   * This is a very simplistic approach that only validates the requirements for each table and does
+   * not do any other conflict detection. Therefore, it does not guarantee true transactional
+   * atomicity, which is left to the implementation details of a REST server.
+   */
+  public static void commitTransaction(Catalog catalog, CommitTransactionRequest request) {

Review Comment:
   I think that this method should check whether the underlying catalog implements `commitTransaction`. If it does, then it should delegate to the catalog. If not, it should throw an exception.
   
   The `CatalogHandlers` act as a reference implementation for the REST protocol, so I think it would be problematic if we included a method that took an atomic multi-table commit and implemented it as a sequence of individual commits.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  TableMetadata base();

Review Comment:
   As I noted above, I think it would be better to use `requirements` and `updates` instead of mixing the updates with a `TableMetadata`.
   
   I think there is a drawback to that approach, though. Any catalog that commits using metadata location rather than requirements and updates would be harder to update for multi-table transactions because those require the base metadata and the new metadata.
   
   Overall, I think I would still prefer the requirements and updates. There are a couple of reasons:
   1. Implementations of `TableOperations` almost always refresh immediately before attempting a commit anyway. Loading table metadata should not be a performance problem.
   2. Some operations don't refresh table metadata before attempting to commit because they do not retry. For those cases, we have to address that they are not normally retried and use generic retry logic. That's what requirements/updates already do.
   
   For a concrete example of point 2 above, consider `UpdateSchema` that does not retry. If the table changes concurrently, then the schema update fails. That's because the operation doesn't validate that it can still apply the schema changes. However, the REST protocol does provide a way to validate a schema update can still be applied, by sending an `assert-last-assigned-field-id` requirement.
   
   If we were to try to perform a schema update using a multi-table commit method, that method doesn't know what is update is happening and will always retry. To make that retry safe, we should use the requirements.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -62,6 +62,11 @@ public <T> T withContext(SessionContext context, Function<Catalog, T> task) {
     return task.apply(asCatalog(context));
   }
 
+  public void commitTransaction(SessionContext context, List<TableCommit> commits) {

Review Comment:
   Why add this here? I would expect to either add this to the `SessionCatalog` interface and to `Catalog`, or to create optional interfaces to add this behavior.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
core/src/main/java/org/apache/iceberg/catalog/TableCommit.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import org.apache.iceberg.MetadataUpdate;
+import org.apache.iceberg.TableMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface TableCommit {
+  TableIdentifier identifier();
+
+  TableMetadata base();

Review Comment:
   I completely agree with you on just using `requirements` and `updates` in `TableCommit` due to the points you mentioned. The only difficulty is that `requirements` are currently at the rest-layer and we'd have to refactor them out to make them generally available/usable. I didn't want to overcomplicate this PR with such a refactoring and decided to keep the `base` metadata in the class for now, which we use to build the `requirements` from



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation

Review Comment:
   "atomic"?



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.

Review Comment:
   Not sure you need to say this a second time.



-- 
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 #7569: Core: Add REST API for committing changes against multiple tables

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -846,6 +846,125 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/transactions/commit:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Commit updates to multiple tables in an all-or-nothing operation
+      operationId: commitTransaction
+      requestBody:
+        description:
+          Commit updates to multiple tables in an all-or-nothing operation
+
+
+          A commit for a single table consists of a table identifier with requirements and updates.
+          Requirements are assertions that will be validated before attempting to make and commit changes.
+          For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value.
+
+
+          Updates are changes to make to table metadata. For example, after asserting that the current main ref
+          is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new
+          snapshot id.
+
+
+          A server implementation needs to guarantee that updates across multiple tables are applied in an
+          all-or-nothing operation.
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitTransactionRequest'
+        required: true
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:

Review Comment:
   The only case where I thought it might make sense to throw a 404 is when the server gets changes for a table that doesn't exist (anymore). Alternatively we could handle this case via a general `CommitFailedException`, but `CommitFailedException` is retryable and I don't think we'd want to retry in this particular case



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