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

[GitHub] [iceberg] gaborkaszab opened a new pull request, #6074: API,Core: SnapshotManager to be created through Transaction

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

   Currently, SnapshotManager encapsulates its own BaseTransaction object and there is no way to expose it opposed to for example ExpireSnapshots that can be created through the Transaction API. As a result the operations in SnapshotManager can't run in the same transaction with the operations in e.g. BaseTransaction.
   
   One real world use-case would be from Apache Impala where each transactional operation through the Iceberg API has to set some Catalog properties (CATALOG_SERVICE_ID, CATALOG_VERSION) in the same transaction. For explanation see this ticket:
   https://issues.apache.org/jira/browse/IMPALA-11508


-- 
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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   > This looks good to me other than a couple of small things to fix:
   > 
   > * revapi suppressions are reordered, which is going to introduce churn
   > * It isn't clear why this wouldn't work with a create transaction
   
   Thanks for the reviews so far, @rdblue!
   - revapi part should be ok now
   - I removed that Precondition that didn't allow to use SnapshotManager with a createTransaction. However, I think that keeping that Precondition would make sense as with a createTransction we wouldn't have snapshot IDs nor branch names so we wouldn't be able to do much on that SnapshotManager anyway.


-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,13 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");

Review Comment:
   ```suggestion
       Preconditions.checkArgument(transaction != null, "Invalid input transaction: null");
   ```



##########
core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java:
##########
@@ -111,6 +111,12 @@ public ExpireSnapshots expireSnapshots() {
     return wrapped.expireSnapshots();
   }
 
+  @Override
+  public ManageSnapshots manageSnapshots() {
+    throw new UnsupportedOperationException(

Review Comment:
   alternatively to throwing, does anything speak against just returning `wrapped.manageSnapshots()` here?



##########
api/src/main/java/org/apache/iceberg/Transaction.java:
##########
@@ -155,6 +155,13 @@ default UpdateStatistics updateStatistics() {
    */
   ExpireSnapshots expireSnapshots();
 
+  /**
+   * Create a new {@link ManageSnapshots manage snapshot API} to manage snapshots in this table.
+   *
+   * @return a new {@link ManageSnapshots}
+   */
+  ManageSnapshots manageSnapshots();

Review Comment:
   rather than introducing an API-breaking change, we should probably make this a `default` method that throws an UOE (similar to `updateStatistics()`). Then we wouldn't need the revapi 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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isTransactionExternal = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(
+        transaction.underlyingOps().current(),
+        "Cannot manage snapshots: table %s does not exist",
+        transaction.tableName());

Review Comment:
   sure, 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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   This fixes #5882


-- 
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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   @rdblue I saw you have touched this area a few times. Could you please take a look?
   The motivation here was that in Impala we'd like to use SnapshotManager.rollback() in the same transaction with UpdateProperties (the reason is described in https://issues.apache.org/jira/browse/IMPALA-11508) but currently SnapshotManager hides the BaseTransaction it uses.
   I also thought that once SnapshotManager could be created through an external Transaction then the support for the internal one should be dropped but I guess that would break backward compatibility. Should we mark then that behavior deprecated and drop 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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   Hey @rdblue,
   I added this to the 1.2.0 milestone if I'm not mistaken there is only an open question wrt a Precondition check, and a related Impala commit has been waiting for this since Oct.


-- 
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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(

Review Comment:
   Ohh, I totally misunderstood your comment then, and writing such a long answer wasn't necessary at all :)
   
   This precondition is let's say partially intentional as I wanted to follow the existing constructor that has a [similar check](https://github.com/apache/iceberg/blob/c07f2aabc0a1d02f068ecf1514d2479c0fbdd3b0/core/src/main/java/org/apache/iceberg/SnapshotManager.java#L29). 
   After giving this a thought this check might make sense as everything in SnapshotManager requires snapshot IDs or branch names that we won't have with a createTransaction().



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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -22,6 +22,7 @@
 
 public class SnapshotManager implements ManageSnapshots {
 
+  private final boolean isTransactionExternal;

Review Comment:
   Nit naming: could we call it "isExternalTransaction" or even just "externalTransaction"



##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isTransactionExternal = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(
+        transaction.underlyingOps().current(),
+        "Cannot manage snapshots: table %s does not exist",
+        transaction.tableName());

Review Comment:
   Could we add tests for these 2 unhappy 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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java:
##########
@@ -111,6 +111,12 @@ public ExpireSnapshots expireSnapshots() {
     return wrapped.expireSnapshots();
   }
 
+  @Override
+  public ManageSnapshots manageSnapshots() {
+    throw new UnsupportedOperationException(

Review Comment:
   Good point. Done



##########
api/src/main/java/org/apache/iceberg/Transaction.java:
##########
@@ -155,6 +155,13 @@ default UpdateStatistics updateStatistics() {
    */
   ExpireSnapshots expireSnapshots();
 
+  /**
+   * Create a new {@link ManageSnapshots manage snapshot API} to manage snapshots in this table.
+   *
+   * @return a new {@link ManageSnapshots}
+   */
+  ManageSnapshots manageSnapshots();

Review Comment:
   Done. Just for my understanding, this is only to trick revapi but there is not other point, right?



##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,13 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");

Review Comment:
   Sure, 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 pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   Thanks, @gaborkaszab! I merged 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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(

Review Comment:
   In what case does this matter? Shouldn't you be able to start a create transaction, add a couple of commits, and then use this 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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
.palantir/revapi.yml:
##########
@@ -273,6 +273,15 @@ acceptedBreaks:
     - code: "java.method.addedToInterface"
       new: "method java.util.List<org.apache.iceberg.StatisticsFile> org.apache.iceberg.Table::statisticsFiles()"
       justification: "new API method"
+    - code: "java.method.addedToInterface"

Review Comment:
   Yeah, I know. I missed to amend the changes to my patch after re-doing the revapi. But anyway thanks for checking! :)



-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


-- 
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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   Thanks @amogh-jahagirdar, @rdblue and @nastra for reviewing!


-- 
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] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   > This looks good to me other than a couple of small things to fix:
   > 
   > * revapi suppressions are reordered, which is going to introduce churn
   > * It isn't clear why this wouldn't work with a create transaction
   
   Thanks for taking a look, @rdblue!
   
   1: The revapi thing got solved after a rebase and re-doing the revapi changes.
   
   2: I'm not sure I entirely get your comment with the 'create transaction' approach. Do you mean that I should create a transaction via Table.newTransaction() use it for do some stuff on the table and then in a next step do a Table.manageSnapshots(). I figured this would happen in 2 separate transaction then, one for the stuff going through the Transaction from Table.newTransaction() and then the stuff I do in the other Transaction from Table.manageSnapshot(). Do I get this wrong?
   
   What in fact I need is to do tasks from ManageSnapshots in the same transaction with other stuff through the Transaction interface (like updateProperties() in the same transaction with rollbackTo() in particular).
   
   **The motivation** here is kind of Impala vs HMS specific. When we are executing various `ALTER TABLE` statements on an Iceberg table we do it in a Transaction (things like expireSnapshots(), updateSchema() and others) but we observed that in the same Transaction we have to do an updateProperties() as well to set a new Catalog version. Doing this in the same transaction will generate a single Catalog event containing both changes, but in separate transactions this would generate multiple Catalog events where the catalog version update is in a separate event. Well, without going into even more Impala specific details this would cause us some headaches ([link](https://issues.apache.org/jira/browse/IMPALA-11508) for one such issue).


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

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

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


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


[GitHub] [iceberg] gaborkaszab commented on pull request #6074: API,Core: SnapshotManager to be created through Transaction

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

   Hey @rdblue ,
   May I ask for another round of review? I believe I've addresses everything that came up so far.


-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
.palantir/revapi.yml:
##########
@@ -43,9 +49,6 @@ acceptedBreaks:
     - code: "java.method.removed"
       old: "method org.apache.iceberg.RowDelta org.apache.iceberg.RowDelta::validateNoConflictingAppends(org.apache.iceberg.expressions.Expression)"
       justification: "Deprecations for 1.0 release"
-    - code: "java.method.addedToInterface"
-      new: "method org.apache.iceberg.TableScan org.apache.iceberg.TableScan::useRef(java.lang.String)"
-      justification: "Adding table scan APIs to support scanning from refs"

Review Comment:
   @nastra, do you know why revapi suppressions get reordered? Is this a Java version issue?



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

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

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


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
.palantir/revapi.yml:
##########
@@ -43,9 +49,6 @@ acceptedBreaks:
     - code: "java.method.removed"
       old: "method org.apache.iceberg.RowDelta org.apache.iceberg.RowDelta::validateNoConflictingAppends(org.apache.iceberg.expressions.Expression)"
       justification: "Deprecations for 1.0 release"
-    - code: "java.method.addedToInterface"
-      new: "method org.apache.iceberg.TableScan org.apache.iceberg.TableScan::useRef(java.lang.String)"
-      justification: "Adding table scan APIs to support scanning from refs"

Review Comment:
   I've got the same diff by running `./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "ManageSnapshots creation through TX"` so it must be RevAPI re-ordering stuff, because previously some things were manually added there in the wrong order. 
   Btw, rebasing the PR fixes this, because in some recent commit we fixed the ordering in the revapi file. 
   So after rebasing you get the right diff:
   ```
   diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml
   index f21466470..1dbfe00e9 100644
   --- a/.palantir/revapi.yml
   +++ b/.palantir/revapi.yml
   @@ -66,6 +66,10 @@ acceptedBreaks:
          old: "method void org.apache.iceberg.io.DataWriter<T>::add(T)"
          justification: "Removing deprecated method"
      "1.1.0":
   +    org.apache.iceberg:iceberg-api:
   +    - code: "java.method.addedToInterface"
   +      new: "method org.apache.iceberg.ManageSnapshots org.apache.iceberg.Transaction::manageSnapshots()"
   +      justification: "ManageSnapshots creation through TX"
        org.apache.iceberg:iceberg-core:
        - code: "java.method.exception.checkedAdded"
   ```



-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
api/src/main/java/org/apache/iceberg/Transaction.java:
##########
@@ -155,6 +155,13 @@ default UpdateStatistics updateStatistics() {
    */
   ExpireSnapshots expireSnapshots();
 
+  /**
+   * Create a new {@link ManageSnapshots manage snapshot API} to manage snapshots in this table.
+   *
+   * @return a new {@link ManageSnapshots}
+   */
+  ManageSnapshots manageSnapshots();

Review Comment:
   RevAPI is checking API and ABI compatibility, so it's not just tricking RevAPI, it's also to make sure that users of this particular `Transaction` API don't run into API/ABI breakages when they upgrade the Iceberg version.



-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/test/java/org/apache/iceberg/TestSnapshotManager.java:
##########
@@ -632,4 +632,82 @@ public void testAttemptToRollbackToCurrentSnapshot() {
     long currentSnapshotId = table.currentSnapshot().snapshotId();
     table.manageSnapshots().rollbackTo(currentSnapshotId).commit();
   }
+
+  @Test
+  public void testSnapshotManagerThroughTransaction() {
+    table.newAppend().appendFile(FILE_A).commit();
+    Snapshot snapshotAfterFirstAppend = readMetadata().currentSnapshot();
+    validateSnapshot(null, snapshotAfterFirstAppend, FILE_A);
+
+    table.newAppend().appendFile(FILE_B).commit();
+    validateSnapshot(snapshotAfterFirstAppend, readMetadata().currentSnapshot(), FILE_B);
+    Assert.assertEquals("Table should be on version 2 after appending twice", 2, (int) version());

Review Comment:
   Why is this assertion here? The table version should not matter to this test.



-- 
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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(

Review Comment:
   Thanks for the clarification, @rdblue! I totally misunderstood your comment then, and writing such a long answer wasn't necessary at all :)
   
   This precondition is let's say partially intentional as I wanted to follow the existing constructor that has a [similar check](https://github.com/apache/iceberg/blob/c07f2aabc0a1d02f068ecf1514d2479c0fbdd3b0/core/src/main/java/org/apache/iceberg/SnapshotManager.java#L29). 
   After giving this a thought this check might make sense as everything in SnapshotManager requires snapshot IDs or branch names that we won't have with a createTransaction().



-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -30,6 +31,17 @@ public class SnapshotManager implements ManageSnapshots {
         ops.current() != null, "Cannot manage snapshots: table %s does not exist", tableName);
     this.transaction =
         new BaseTransaction(tableName, ops, BaseTransaction.TransactionType.SIMPLE, ops.refresh());
+    this.isExternalTransaction = false;
+  }
+
+  SnapshotManager(BaseTransaction transaction) {
+    Preconditions.checkNotNull(transaction, "Input transaction cannot be null");
+    Preconditions.checkNotNull(

Review Comment:
   @gaborkaszab, this is what I'm talking about with the create transaction comment. Why is this precondition needed?



-- 
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 #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
.palantir/revapi.yml:
##########
@@ -273,6 +273,15 @@ acceptedBreaks:
     - code: "java.method.addedToInterface"
       new: "method java.util.List<org.apache.iceberg.StatisticsFile> org.apache.iceberg.Table::statisticsFiles()"
       justification: "new API method"
+    - code: "java.method.addedToInterface"

Review Comment:
   @gaborkaszab I think you need to just re-do the changes on this file. Only `new: "method org.apache.iceberg.ManageSnapshots org.apache.iceberg.Transaction::manageSnapshots()"` should be added to `1.1.0`



-- 
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] gaborkaszab commented on a diff in pull request #6074: API,Core: SnapshotManager to be created through Transaction

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


##########
core/src/main/java/org/apache/iceberg/SnapshotManager.java:
##########
@@ -22,6 +22,7 @@
 
 public class SnapshotManager implements ManageSnapshots {
 
+  private final boolean isTransactionExternal;

Review Comment:
   Renamed this to isExternalTransaction



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