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 2020/08/26 08:00:38 UTC

[GitHub] [iceberg] Simon0806 opened a new pull request #1384: Remove a wrong check for commit operation

Simon0806 opened a new pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384


   


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

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] chenjunjiedada commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480069884



##########
File path: core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
##########
@@ -267,10 +267,6 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException {
         TestTables.metadataVersion("uncommitted_change"));
 
     txn.updateProperties().set("test-property", "test-value"); // not committed
-
-    AssertHelpers.assertThrows("Should reject commit when last operation has not committed",

Review comment:
       You may want to remove the entire unit test since this assertion is mainly logic for the unit test.
   
   This introduces a behaviour change which could ignore the operations that haven't been commit yet. Not sure whether the downstream client will be impacted by this or not.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r482731677



##########
File path: core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
##########
@@ -268,9 +268,13 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException {
 
     txn.updateProperties().set("test-property", "test-value"); // not committed
 
-    AssertHelpers.assertThrows("Should reject commit when last operation has not committed",
-        IllegalStateException.class, "Cannot commit transaction: last operation has not committed",
-        txn::commitTransaction);
+    txn.commitTransaction();
+
+    // nothing to commit.
+    Assert.assertNull("Starting a create transaction should not commit metadata",
+        TestTables.readMetadata("uncommitted_change"));
+    Assert.assertNull("Should have no metadata version",
+        TestTables.metadataVersion("uncommitted_change"));

Review comment:
       Hi, I have replaced the lastBase with a boolean value to check if last commit has committed. Now It will not change the behaviors of 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.

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 #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#issuecomment-686602875


   Looks great. Thanks for fixing this problem, @Simon0806! I'll merge the fix.


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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478070684



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I think if we return the original one, then the logic inside of TransactionTableOperations will put the same metadata to current, like : BaseTransaction.this.current = metadata;  this will also cause failure in commitTransaction because in commitTransaction it has to check whether lastBase != current.
   




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

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 #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#issuecomment-681185428


   Thanks for working on this, @Simon0806! I think we can improve it to still avoid an unnecessary commit without too much trouble, but I agree that this is correct and is what caused #1370.


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

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] chenjunjiedada commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480069884



##########
File path: core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
##########
@@ -267,10 +267,6 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException {
         TestTables.metadataVersion("uncommitted_change"));
 
     txn.updateProperties().set("test-property", "test-value"); // not committed
-
-    AssertHelpers.assertThrows("Should reject commit when last operation has not committed",

Review comment:
       You could keep the `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.

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478070684



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I think if we return the original one, then the logic inside of TransactionTableOperations will put the same metadata to current, like : BaseTransaction.this.current = metadata;  this will also cause failure in commitTransaction because in commitTransaction it has to check whether lastBase != current.
   IMO, I suggest just do a deepcopy if there are no updates in metadata, this will always make commitTransaction success. I have pushed the modification. 




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r477727448



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       How about also updating `internalApply` so that it does this check?
   
   This is intended to only modify the table and run a commit if the table changes. But you're right that when combined with a transaction, this will cause a failure because commit is not called at all.
   
   But we still want to avoid unnecessary commits. To do that, `internalApply` should do this check on the new `TableMetadata` it produces. If the new metadata has the same set of manifests as the original, then it should return the original. That will cause the commit to be skipped in most cases because the actual commit logic inside of `ops.commit` checks whether the metadata is the same as the base metadata and returns if it is.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478070684



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I think if we return the original one, then the logic inside of TransactionTableOperations will put the same metadata to current, like : BaseTransaction.this.current = metadata;  this will also cause failure in commitTransaction because in commitTransaction it has to check whether lastBase != current.
   IMO, I suggest to change the Preconditions.checkState in commitTransaction to a if condition block so that it will not throws IllegalStateException,  which need to modify some UTs. 




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478017113



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       Thanks for reply, I know your point, in most transaction cases in can commit, but in BaseMetastoreTableOperations, we can return a original then it will return without any commit. I will update the code and add more UTs




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478762867



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       We should definitely not copy the data. The purpose of returning the original metadata is to avoid an unnecessary commit if the metadata hasn't changed. The `TableOperations` objects use identity to determine if anything has changed.
   
   There may be a check inside of `BaseTransaction` that also needs to be updated. But the right way to handle that is to update that check to allow passing the same metadata back.




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

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 #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384


   


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

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] chenjunjiedada commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480572502



##########
File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
##########
@@ -200,9 +200,10 @@ public ExpireSnapshots expireSnapshots() {
 
   @Override
   public void commitTransaction() {
-    Preconditions.checkState(lastBase != current,
-        "Cannot commit transaction: last operation has not committed");
-
+    if (lastBase == current) {
+      LOG.info("Nothing to commit.");
+      return;
+    }

Review comment:
       I agree with this, it changes the previous commit behaviour.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478070684



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I think if we return the original one, then the logic inside of TransactionTableOperations will put the same metadata to current, like : BaseTransaction.this.current = metadata;  this will also cause failure in commitTransaction because in commitTransaction it has to check whether lastBase != current.
   IMO, I suggest to modify the Preconditions.checkState in commitTransaction, which need to modify some UTs. 




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478017113



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       Thanks for reply, in most transaction cases in can commit, but in BaseMetastoreTableOperations, we can pass an original metadata if there has the same set of manifests,  so it will return without any commits. is it right ? 




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480289379



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -138,10 +138,23 @@ public ExpireSnapshots executeDeleteWith(ExecutorService executorService) {
   private TableMetadata internalApply() {
     this.base = ops.refresh();
 
-    return base.removeSnapshotsIf(snapshot ->
+    TableMetadata updated = base.removeSnapshotsIf(snapshot ->
         idsToRemove.contains(snapshot.snapshotId()) ||
         (expireOlderThan != null && snapshot.timestampMillis() < expireOlderThan &&
             !idsToRetain.contains(snapshot.snapshotId())));
+
+    List<Snapshot> updateSnapshots = updated.snapshots();
+    List<Snapshot> baseSnapshots = base.snapshots();
+    if (updateSnapshots.size() != baseSnapshots.size()) {
+      return updated;
+    } else {
+      for (int i = 0; i < baseSnapshots.size(); i += 1) {
+        if (!updateSnapshots.get(i).equals(baseSnapshots.get(i))) {
+          return updated;
+        }

Review comment:
       I don't think this block is needed. The contract of `removeSnapshotsIf` is to only remove snapshots or return the list unmodified. We can rely on that and just check that the size hasn't changed.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478790658



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I have tried to change a preCondition check to be a if-else block, this may work but we may need to change some UTs which are assert to return a IllegalStateException.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478017113



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       Thanks for reply, in most transaction cases in can commit, but in BaseMetastoreTableOperations, we can pass an original metadata if there has the same set of manifests,  so it will return without any commits. I will update the code and add more UTs




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480292265



##########
File path: core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
##########
@@ -268,9 +268,13 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException {
 
     txn.updateProperties().set("test-property", "test-value"); // not committed
 
-    AssertHelpers.assertThrows("Should reject commit when last operation has not committed",
-        IllegalStateException.class, "Cannot commit transaction: last operation has not committed",
-        txn::commitTransaction);
+    txn.commitTransaction();
+
+    // nothing to commit.
+    Assert.assertNull("Starting a create transaction should not commit metadata",
+        TestTables.readMetadata("uncommitted_change"));
+    Assert.assertNull("Should have no metadata version",
+        TestTables.metadataVersion("uncommitted_change"));

Review comment:
       I think this test case is still valid.
   
   The problem is that this was previously assuming that the table metadata would change when each of the operations in a transaction is committed. That's not necessarily the case.
   
   Instead of getting rid of the check that each operation committed, we just need to track that it committed differently. I think that means we should replace `lastBase` in `BaseTransaction` with a boolean `hasComitted` that gets set to false each time a new operation is created inside of `checkLastOperationCommitted`.




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478790658



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I have tried to change a preCondition check to be a if-else block, this may work but we may need to change some UTs which are assert to return an IllegalStateException. There are 8 UTs need to remove the check where commitTransaction return an IllegaStateException.




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480293816



##########
File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
##########
@@ -200,9 +200,10 @@ public ExpireSnapshots expireSnapshots() {
 
   @Override
   public void commitTransaction() {
-    Preconditions.checkState(lastBase != current,
-        "Cannot commit transaction: last operation has not committed");
-
+    if (lastBase == current) {
+      LOG.info("Nothing to commit.");
+      return;
+    }

Review comment:
       This isn't correct:
   * Skipping the commit should be done in `TableOperations`, not here. This logic is incorrect because `lastBase` is not the current table state, it is the state after the previous operation in the transaction. This logic would skip the rest of the operations in a transaction if the last one didn't make any changes.
   * This still needs to check that the last operation had `commit` called on it. Otherwise, incomplete operations could be added and would be ignored instead of raising exceptions.




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

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] chenjunjiedada commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480016637



##########
File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
##########
@@ -451,7 +452,6 @@ public void commit(TableMetadata underlyingBase, TableMetadata metadata) {
       }
 
       BaseTransaction.this.current = metadata;
-

Review comment:
       nit: useless change.

##########
File path: core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
##########
@@ -267,10 +267,6 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException {
         TestTables.metadataVersion("uncommitted_change"));
 
     txn.updateProperties().set("test-property", "test-value"); // not committed
-
-    AssertHelpers.assertThrows("Should reject commit when last operation has not committed",

Review comment:
       You may want to remove the entire unit test since this assertion is mainly logic for this unit test.
   
   This introduces a behaviour change which could ignore the operations that haven't been commit yet. Not sure whether the downstream client will be impacted by this or not.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -138,10 +138,23 @@ public ExpireSnapshots executeDeleteWith(ExecutorService executorService) {
   private TableMetadata internalApply() {
     this.base = ops.refresh();
 
-    return base.removeSnapshotsIf(snapshot ->
+    TableMetadata updated = base.removeSnapshotsIf(snapshot ->
         idsToRemove.contains(snapshot.snapshotId()) ||
         (expireOlderThan != null && snapshot.timestampMillis() < expireOlderThan &&
             !idsToRetain.contains(snapshot.snapshotId())));
+
+    List<Snapshot> updateSnapshots = updated.snapshots();
+    List<Snapshot> baseSnapshots = base.snapshots();
+    if (updateSnapshots.size() != baseSnapshots.size()) {
+      return updated;
+    } else {
+      for (int i = 0; i < baseSnapshots.size(); i += 1) {
+        if (!updateSnapshots.get(i).equals(baseSnapshots.get(i))) {

Review comment:
       Can we compare the snapshot ID?




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r479873314



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       @rdblue I have change the behavior for commitTransaction, make it the same as what commit does in TableOperations. This will avoid to copy the data and also allow to pass the same metadata. 




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478070684



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I think if we return the original one, then the logic inside of TransactionTableOperations will put the same metadata to current, like : BaseTransaction.this.current = metadata;  this will also cause failure in commitTransaction because in commitTransaction it has to check whether lastBase != current.
   IMO, I suggest to change the Preconditions.checkState in commitTransaction to a if block so that it will not throws IllegalStateException,  which need to modify some UTs. 




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

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] Simon0806 commented on a change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r478790658



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -156,10 +156,7 @@ public void commit() {
         .onlyRetryOn(CommitFailedException.class)
         .run(item -> {
           TableMetadata updated = internalApply();
-          // only commit the updated metadata if at least one snapshot was removed
-          if (updated.snapshots().size() != base.snapshots().size()) {
-            ops.commit(base, updated);
-          }

Review comment:
       I have tried to change a preCondition check to be a if-else block, this may work but we may need to change some UTs which are assert to return an IllegalStateException.




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

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 change in pull request #1384: Remove a wrong check for commit operation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1384:
URL: https://github.com/apache/iceberg/pull/1384#discussion_r480293816



##########
File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
##########
@@ -200,9 +200,10 @@ public ExpireSnapshots expireSnapshots() {
 
   @Override
   public void commitTransaction() {
-    Preconditions.checkState(lastBase != current,
-        "Cannot commit transaction: last operation has not committed");
-
+    if (lastBase == current) {
+      LOG.info("Nothing to commit.");
+      return;
+    }

Review comment:
       This isn't correct:
   * Skipping the commit should be done in `TableOperations`, not here. This logic is incorrect because `lastBase` is not the current table state, it is the state after the last operation in the transaction. This logic would skip the rest of the operations in a transaction if the last one didn't make any changes.
   * This still needs to check that the last operation had `commit` called on it. Otherwise, incomplete operations could be added and would be ignored instead of raising exceptions.




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

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