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 2021/03/11 18:32:03 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

RussellSpitzer opened a new pull request #2328:
URL: https://github.com/apache/iceberg/pull/2328


   #2317 - We discovered that Iceberg is currently treating all failures during commit
   as full commit failures. This can lead to an unstable/corrupt table if the
   catalog was successfully updated and it was only a network or other error
   that prevented the client from learning of this. In this state, the client
   will attempt to clean up files related to the commit while other clients and the table believe that files are successfully added to the table.
   
   To fix this we change snapshot producer to only do a cleanup when a true CommitFailureException is thrown and stop our HMSTableOperations from removing metadata.json files when an uncertain exception is thrown.


----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +241,41 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit to {}.{} exists, treating commit state as unknown: {}",
+          database, tableName, checkFailure);
+      return CommitStatus.UNKNOWN;

Review comment:
       We never want to return failure unless we are sure things are broken. That said we can retry the refresh multiple times as suggested in the longer discussion below.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);

Review comment:
       Nit: maybe a private static final constant? 




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,51 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @param config metadata to use for configuration
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) {
+    int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+
+    for (int attempt = 1; attempt <= maxAttempts; attempt++) {

Review comment:
       I think I'd also really like a "Tasks" API that is a better fit for retry, as is, Tasks really has to propagate through side effects which makes sense for async IO but less for 
   output = Try to return something and if you fail return this other thing




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       If we add the timeout then you can't do an indefinite retry, I honestly don't think anyone should be changing any of this, but if a user wants infinite retries we shouldn't stop them IMHO




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,51 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @param config metadata to use for configuration
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) {
+    int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+
+    for (int attempt = 1; attempt <= maxAttempts; attempt++) {

Review comment:
       I considered this but I didn't like how I have to set the return value inside of the lambda and can't return from there. I also can't count the number of attempts without another counter, maybe that's a good addition for the task API an "onFailure(item, Exception, Attempt)"




-- 
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +
+              "may not be needed. Manual intervention via the Remove Orphan Files Action can remove these " +
+              "files when a connection to the Catalog can be re-established if the commit was actually unsuccessful. " +

Review comment:
       I think it will be helpful to advise the user to check whether their operation succeeded before retrying. If the commit was applied and the change was retried, this will result in duplicated data.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit succeeded, attempting to reconnect and check", persistFailure);

Review comment:
       We might want to add more info about the table to the error message. So it is easier to check the table manually and see what happened based only the logs.
   
   What do you think?




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

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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   This PR has been open for a while and we had quite some eyes on it. I don't see any open points that must be addressed in this commit. We have an open thread [here](https://github.com/apache/iceberg/pull/2328#discussion_r592944422) but I consider that as a follow-up item.
   
   Please let me know if I missed anything.


-- 
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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   I think the current approach looks good. I have just a couple of nits.
   
   I checked the following places and seems like we don't have to update them:
   
   - UpdateLocation
   - UpdateSchema
   - UpdatePartitionSpec
   - ReplaceSortOrder


----------------------------------------------------------------
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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   @rdblue, when we started looking into this problem, it was not clear how the fix will look like. I tend to agree it is a behavior change so I am fine skipping it from the patch release.


-- 
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,51 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @param config metadata to use for configuration
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) {
+    int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+
+    for (int attempt = 1; attempt <= maxAttempts; attempt++) {

Review comment:
       The tasks API automatically logs retries so you don't need to.
   
   We can look at improving the tasks for retries, but I don't think we need to do that here.




-- 
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Thanks for getting this fixed, @RussellSpitzer!


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,248 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  /**
+   * Pretends we throw an error while persisting that actually fails to commit serverside
+   */
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  /**
+   * Pretends we throw an error while persisting that actually does commit serverside
+   */
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  /**
+   * Pretends we throw an exception while persisting and don't know what happened, can't check to find out,
+   * but in reality the commit failed
+   */
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Client could not determine outcome so new metadata file should also exist",
+        3, metadataFileCount(ops.current()));
+  }
+
+  /**
+   * Pretends we throw an exception while persisting and don't know what happened, can't check to find out,
+   * but in reality the commit succeeded
+   */
+  @Test
+  public void testThriftExceptionsUnknownSuccessCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    commitAndThrowException(ops, spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+

Review comment:
       Logs like
   
   ```
   [Test worker] ERROR org.apache.iceberg.hive.HiveTableOperations - Cannot tell if commit to hivedb.tbl succeeded, attempting to reconnect and check.
   org.apache.thrift.TException: Datacenter on fire
   	at org.apache.iceberg.hive.TestHiveCommits.lambda$testThriftExceptionUnknownFailedCommit$1(TestHiveCommits.java:176)
   	at org.apache.iceberg.AssertHelpers.assertThrows(AssertHelpers.java:64)
   	at org.apache.iceberg.hive.TestHiveCommits.testThriftExceptionUnknownFailedCommit(TestHiveCommits.java:174)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
   	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
   	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   [Test worker] ERROR org.apache.iceberg.hive.HiveTableOperations - Cannot check if commit to hivedb.tbl exists. Retry attempt 1 of 3.
   java.lang.RuntimeException: Still on fire
   	at org.apache.iceberg.hive.TestHiveCommits.lambda$testThriftExceptionUnknownFailedCommit$1(TestHiveCommits.java:176)
   	at org.apache.iceberg.AssertHelpers.assertThrows(AssertHelpers.java:64)
   	at org.apache.iceberg.hive.TestHiveCommits.testThriftExceptionUnknownFailedCommit(TestHiveCommits.java:174)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
   	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
   	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   [Test worker] ERROR org.apache.iceberg.hive.HiveTableOperations - Cannot check if commit to hivedb.tbl exists. Retry attempt 2 of 3.
   java.lang.RuntimeException: Still on fire
   	at org.apache.iceberg.hive.TestHiveCommits.lambda$testThriftExceptionUnknownFailedCommit$1(TestHiveCommits.java:176)
   	at org.apache.iceberg.AssertHelpers.assertThrows(AssertHelpers.java:64)
   	at org.apache.iceberg.hive.TestHiveCommits.testThriftExceptionUnknownFailedCommit(TestHiveCommits.java:174)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
   	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
   	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   [Test worker] ERROR org.apache.iceberg.hive.HiveTableOperations - Cannot check if commit to hivedb.tbl exists. Retry attempt 3 of 3.
   java.lang.RuntimeException: Still on fire
   	at org.apache.iceberg.hive.TestHiveCommits.lambda$testThriftExceptionUnknownFailedCommit$1(TestHiveCommits.java:176)
   	at org.apache.iceberg.AssertHelpers.assertThrows(AssertHelpers.java:64)
   	at org.apache.iceberg.hive.TestHiveCommits.testThriftExceptionUnknownFailedCommit(TestHiveCommits.java:174)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
   	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
   	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   [Test worker] ERROR org.apache.iceberg.hive.HiveTableOperations - Cannot determine commit state to hivedb.tbl. Failed to check 3 times. Treating commit state as unknown.
   ```




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Client could not determine outcome so new metadata file should also exist",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionsUnknownSuccessCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    commitAndThrowException(ops, spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionConcurrentCommit() throws TException, InterruptedException, UnknownHostException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);

Review comment:
       Pretends we threw an exception while persisting, the commit succeeded, the lock expired, and a second committer placed a commit on top of ours before the first committer was able to check if their commit succeeded or not
   
   1. Client 1 commits which throws an exception but suceeded
   2. Client 1's lock expires while waiting to do the recheck for commit success
   3. Client 2 acquires a lock, commits successfully on top of client 1's commit and release lock
   4. Client 1 check's to see if their commit was successful
   
   This tests to make sure a disconnected client 1 doesn't think their commit failed just because it isn't the current one during the recheck phase. See earlier discussion with @pvary here https://github.com/apache/iceberg/pull/2328#discussion_r592637015




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      if (attempt < maxAttempts) {
+        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
+            database, tableName, attempt, maxAttempts, checkFailure);
+        return innerCommitCheck(newMetadataLocation, attempt + 1, maxAttempts);

Review comment:
       I think it depends on personal style what we find easier to understand. As mentioned below, no strong feelings just an idea, so as the one writing the code it is really up to you 😃 




-- 
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I think there are a couple of issues here that we should address separately.
   
   First, I agree that it is a good idea to add a way to signal that a commit is idempotent. Some operations are already idempotent, like file rewrites because we validate that all of the rewritten files are still part of the table. If we update paths that guarantee idempotent commits to signal this and handle `UnknownCommitStateException`, then we really reduce the incidence of the problem.
   
   Second, I think we still need to agree on the default behavior. While I really don't like the idea of allowing a retry that will write duplicate data, I think that this has convinced me that silently duplicating data is a better outcome.
   
   In our environment, we use versioned buckets so we can always un-delete metadata files that are missing, but that's not always the case. If those files are actually gone, then it is much worse because you have missing data and don't know which data files were in the missing commit without a lot of work. I think this problem is worse than the duplicate data.
   
   A second compelling argument for changing the default is that deleting the metadata files leaks the problem to other writers. All concurrent processes are blocked if a table is broken, rather than blocking just a single writer.
   
   In the end, I think that the right thing is to not delete the files and to throw `UnknownCommitStateException` as proposed. That handles interactive cases and also makes it so schedulers can handle a job failure by blocking just a single workflow and not all workflows operating on a table. And idempotent jobs should not be affected.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {

Review comment:
       In our use case we saw it as a SocketException which thrift wraps in TTransportException which is a TException, we added "Throwable" here because we were getting paranoid. Theoretically I think TException & Interrupt are the only things that can be thrown here since I don't think thrift can throw runtime exceptions in the client code, only checked ones...




----------------------------------------------------------------
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] liupan664021 commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,56 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.

Review comment:
       Hello, I have a question about the past locations check here that when checkCommitStatus() is called,  we still holds the metastore lock, so is it possible for another commiter to commit on top of our commit?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I thought more about this today and I am still convinced deleting files in an attempt to fail other concurrent operations (which is not guaranteed) and requiring a manual intervention to restore the table is NOT something we should do due to the described consequences and cases that we won't handle.
   
   I also like throwing an exception rather than hanging forever as this seems to be more descriptive. I think the error message Russell throws right now is pretty elaborate and it should be propagated as a cause of `SparkException`. As a user, I'd prefer my job to fail with `UnknownCommitStateException` rather than with a job timeout exception and no signs of what went wrong.
   
   Internally, we offer job retry mechanisms but we explicitly warn our users that they may enable it only if their jobs are idempotent. We can try to reduce the chances of retries but I don't think we can solve that completely when we loose the metastore connectivity. I think it is reasonable to explore how we can skip already committed batch operations like we do in Structured Streaming but that can be done in a follow-up PR.
   
   It may be also a good call to check the commit state more than once for some reasonable time frame but it shouldn't be for too long to avoid job timeouts.




----------------------------------------------------------------
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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   It would be great to hear your thoughts, @pvary @rymurr @Parth-Brahmbhatt @danielcweeks.


----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,221 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));

Review comment:
       nit: same




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -52,6 +52,12 @@
    * Implementations must check that the base metadata is current to avoid overwriting updates.
    * Once the atomic commit operation succeeds, implementations must not perform any operations that
    * may fail because failure in this method cannot be distinguished from commit failure.
+   * <p>

Review comment:
       Looks 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -151,12 +158,13 @@ protected void doRefresh() {
     refreshFromMetadataLocation(metadataLocation);
   }
 
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
   @Override
   protected void doCommit(TableMetadata base, TableMetadata metadata) {
     String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
     boolean hiveEngineEnabled = hiveEngineEnabled(metadata, conf);
 
-    boolean threw = true;

Review comment:
       Rather than the true/false here we need the abilty to differentiate between knowing that
   1. The Commit Failed
   2. The Commit Suceeded
   3. We aren't sure




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +

Review comment:
       I am not entirely sure. Thrift client might throw exception even if the HMS server side is finished but the response did not arrive to the HMS client. We might have to check the results even in this 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.

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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   If the operation is marked idempotent, then we can suppress the UnknownCommitStateException. That's the only thing I think we need to add right away in master.


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -168,7 +168,8 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       if (tbl != null) {
         // If we try to create the table but the metadata location is already set, then we had a concurrent commit
         if (base == null && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) != null) {
-          throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+          commitFailed = true;
+          throw new CommitFailedException("Table already exists: %s.%s", database, tableName);

Review comment:
       As noted in SnapshotProducer, we must throw a commit failed exception if we expect cleanup




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +

Review comment:
       This should be a CFE too




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +

Review comment:
       I meant for this specific message, for the general case we don't know but if we get the Hive Locks error doesn't that mean we know we couldn't commit?




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", newMetadataLocation);

Review comment:
       Good catch!




----------------------------------------------------------------
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I should also mention that we handle this internally by checking in our `TableOperations` when there is a transport exception. We just try to load the table from the metastore and see if it has the new table metadata location. If so, then we return success. That's a better initial fix than updating the logic here.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       Just to contradict my previous statement (do not overcomplicate 😄) we can have a situation where another commit arrived after the previous one, so the last metadata is not the same as the one we have set.
   We might say that the exclusive lock is still there so no new commit happened in the meantime, but then we should be very sure that there were no timeout for our lock.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);

Review comment:
       Pretends we throw an exception while permitting and don't know what happened, can't check to find out, but in reality the commit failed




----------------------------------------------------------------
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] yyanyy commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", newMetadataLocation);

Review comment:
       Seems that we only pass in one argument but specified 3 placeholders?
   
   




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -151,12 +158,13 @@ protected void doRefresh() {
     refreshFromMetadataLocation(metadataLocation);
   }
 
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review comment:
       We are still at 14 (max 12) :(




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,221 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Client could not determine outcome so new metadata file should also exist",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionsUnknownSuccessCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    commitAndThrowException(ops, spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2));

Review comment:
       nit: same




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,51 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @param config metadata to use for configuration
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) {

Review comment:
       The logic here looks correct to me.




-- 
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       Depends on the lock timeouts defined on the HMS side. We do not have a heartbeater service, so hopefully we are inside but no guarantees, especially if we add more timeouts. If we do retries only once, then it should be ok.
   
   BTW adding heartbeater might not help anyway, since the heartbeater can have the same issues as the commit.




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

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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,56 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.

Review comment:
       We request an `EXCLUSIVE` lock before committing. Until we hold the lock, no other committer can get another lock on the same table, so no other committer can commit




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));

Review comment:
       We still have a possibility of corruption here if the commit was successful, but more than metadata retention amount of metadata.jsons have been made in the interim ... this is probably unlikely and i'm not sure how we can guard against it unless we just treat all metadataLocation not found as "Unknown Commit State"




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

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       @pvary, could you elaborate a bit more on the idea?




-- 
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] rdsr commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Thanks @aokolnychyi . I'll have a look at this by tomorrow!




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       @Parth-Brahmbhatt one thing to remember is that in our current code we actually get both data duplication and corrupt tables, it just depends when the failure occurs. Ie if you fail to check the metastore and lose connection to the file system, you won't clean up and will still throw an error. Or if say the commit is successful but for some other reason the job fails (OOM, Interrupt, power loss, I spill my coke on the server) we still end up with a failed job which if we retry will duplicate data / work.
   
   
   So you can still get data duplication if a container loses connectivity to the network after committing.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I would go for API level flag that the change is idempotent since this is more dependent on the client / action then the actual table.
   
   Also I would go for a few connection retries and then throwing the `UnknownCommitStateException` if we are not able to determine the status of the commit. We should fail fast as soon as possible, so the user is able to mitigate the issue. If they ignore the exception it is better to have a state where we can recover so I would keep the files in case we are not able to handle the error cleanly.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       Ok so we probably have to check whether the current metadataLocation is new MetadataLocation OR if any of the past metadatalocations is new metadatalocation 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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {

Review comment:
       @rymurr This is something I want to make sure you are ok with. Basically we have to also not clean up data files, I basically changed the rules as specified in TableOperations. Now if you throw CFE we clean up, other wise the client is on their own.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I am afraid this will be dangerous. This means we have to update all places that throw a specific exception like `AlreadyExistsException` to now throw `CommitFailedException`. This is error-prone and we lose details that will be helpful. We could throw `CommitFailedException` and add the specific exception as a cause but the first point still holds.
   
   Also, an error log message will most likely be ignored by the user. This is a case where we really want to propagate as much info as possible.
   
   Would it make sense to introduce a new exception type? E.g., `UnknownCommitStateException`? That way, we can keep the existing logic for handling exceptions except cases where we really don't know what happens.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {

Review comment:
       Pretends we throw an error while persisting that actually does commit




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

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] RussellSpitzer commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   @pvary Added in a variable for controlling the number of status check attempts as well as small 1 second timeout before the attempt. 


----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       I like the idea of checking whether the current metadata file was ever committed by iterating through the metadata files.




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

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {

Review comment:
       Looks much cleaner 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,6 +294,8 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 

Review comment:
       Fixes deletion of data files, manifests and snapshots related to unknown state 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.

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Thoughts, @RussellSpitzer @pvary @rymurr @danielcweeks @Parth-Brahmbhatt?




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I agree it is a valid concern when clients don't handle `UnknownStateExceptions` and blindly retry. However, we can try to address it in another way. For example, if users have idempotent requirements, they either should make their retry mechanisms smarter and be aware of `UnknownCommitStateException` or use something that guarantees the same batch won't be committed twice. We already handle such cases in Structured Streaming. I hope we do the same in Flink jobs. We can generalize this approach and make Iceberg handle such cases in batch writes too.
   
   I feel the current approach is extremely dangerous. If we delete a file that was actually committed, we corrupt the table and require a manual intervention. Unfortunately, I've seen this quite frequently. Also, getting a reply from the metastore or checking the commit status may take a substantial amount of time allowing a concurrent operation to succeed (if the lock expires). That's the worst what can happen as we will silently corrupt the table and will detect it only while querying a specific portion of the table. I had to fix such a case and we were simply lucky we found this out only after a couple of days while rewriting manifests. Duplicating data is less critical than loosing it, in my view.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I agree it is a valid concern when clients don't handle `UnknownStateExceptions` and blindly retry. However, we can try to address it in another way. For example, if users have idempotent requirements, they either should make their retry mechanisms smarter and be aware of `UnknownCommitStateException` or use something that guarantees the same batch won't be committed twice. We already handle such cases in Structured Streaming. I hope we do the same in Flink jobs. We can generalize this approach and make Iceberg handle such cases in batch writes too.
   
   I feel the current approach is extremely dangerous. If we delete a file that was actually committed, we corrupt the table and require a manual intervention. Unfortunately, I've seen this quite frequently. Also, getting a reply from the metastore or checking the commit status may take a substantial amount of time allowing a concurrent operation to succeed. That's the worst what can happen as we silently corrupt the table and we will detect it only while querying the table. I had to fix such a case and we were simply lucky we detected this only after a couple of days while rewriting manifests.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       The first scenario you propose is actually the most dangerous without this change, another client could beat the deletion code and end up doing another commit on top of the files as they are being removed. This then leads to an unrecoverable situation since additional writes and operations will be based of of files that are removed.
   
   A retry can be undone, but a commit on top of files that are removed is much more difficult to unwind (i think)




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

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,221 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2));

Review comment:
       nit: `assertEquals`




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,221 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertTrue("Current metadata should not have changed", ops.current().equals(metadataV2));

Review comment:
       nit: `assertEquals`?




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit exists, treating commit state as unknown", checkFailure);

Review comment:
       Maybe some more data about the table in the log? Name/location or whatever identifier would be nice




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       Switched it around, we have to set canCleanupMetadata to false before calling persistTable which looks a little weird but I think overall it's a little cleaner




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       And this apparently violates one of our checkstyle rules for Try nesting :(




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       An API on `PendingUpdate` sounds good to me.
   
   One more reason why I'd vote for not deleting by default is due to this situation:
   > Also, getting a reply from the metastore or checking the commit status may take a substantial amount of time allowing a concurrent operation to succeed (if the lock expires). That's the worst what can happen as we will silently corrupt the table and will detect it only while querying a specific portion of the table. I had to fix such a case and we were simply lucky we found this out only after a couple of days while rewriting manifests. 




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)
+        .onFailure((location, checkException) ->
+            LOG.error("Cannot check if commit to {}.{} exists.", database, tableName, checkException))
+        .run(location -> {
+          TableMetadata metadata = refresh();
+          String currentMetadataLocation = metadata.metadataFileLocation();
+          boolean commitSuccess = currentMetadataLocation.equals(newMetadataLocation) ||
+              metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+          if (commitSuccess) {
+            LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
+            status.set(CommitStatus.SUCCESS);
+          } else {
+            LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
+            status.set(CommitStatus.FAILURE);
+          }
+        });
+
+    LOG.error("Cannot determine commit state to {}.{}. Failed during checking {} times. " +

Review comment:
       yep good call




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

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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Thanks for fixing this, @RussellSpitzer! Thanks everyone who reviewed!
   
   I merged this to master but will not include in the patch release. We still have time to iterate on this before releasing 0.12.


-- 
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       @aokolnychyi: Not sure about the target of the question, so I try to elaborate more and hope that somewhere I hit the target 😄 
   
   - In HiveServer2 we have a HMS client for every session. If we reuse this client instead of creating a new one for the Iceberg catalogs we can save resources. Also, as I have discovered lately this HMS client could filter the databases / tables based on the roles of the current user (this might not be that important since the first access of the table will go through the HMS client in the session anyway, so this will prevent unauthorized access, but still...).
   - In Hive we use [RetryingMetaStoreClient](https://github.com/apache/hive/blob/master/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java) (moved between 2.3.8 and 3.1.2 but exists in both releases under different path and configs) which retries the calls several times defined by the `THRIFT_FAILURE_RETRIES` defaults to 1 after `CLIENT_CONNECT_RETRY_DELAY` defaults to 1 second. This might be useful replacement for the retrying mechanism created here, but we have to examine if the retries would cause any issue on other use-cases of the HiveClientPool. The most important thing is that we should turn off one of the retry mechanism if we start to use the `RetryingMetaStoreClient` otherwise we will face some surprises down the road.
   




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;

Review comment:
       This one too




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -238,7 +289,8 @@ private void persistTable(Table hmsTable, boolean updateHiveTable) throws TExcep
     }
   }
 
-  private Table loadHmsTable() throws TException, InterruptedException {
+  // Visible for tests
+  protected Table loadHmsTable() throws TException, InterruptedException {

Review comment:
       I no longer need this to be visible for testing, note to remove it




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit exists, treating commit state as unknown", checkFailure);
+      return CommitStatus.UNKNOWN;
     }
   }
 
-  private void persistTable(Table hmsTable, boolean updateHiveTable) throws TException, InterruptedException {
+  // Visible for testing

Review comment:
       Sometimes I forget we have that annotation :)




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -318,7 +369,8 @@ private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hive
     return storageDescriptor;
   }
 
-  private long acquireLock() throws UnknownHostException, TException, InterruptedException {
+  // Visible for testing
+  protected long acquireLock() throws UnknownHostException, TException, InterruptedException {

Review comment:
       Required for injecting a lock release and second commit, we need to capture the args to this call to get the lock 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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -384,10 +436,10 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
     return lockId;
   }
 
-  private void cleanupMetadataAndUnlock(boolean errorThrown, String metadataLocation, Optional<Long> lockId) {
+  private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadataLocation, Optional<Long> lockId) {
     try {
-      if (errorThrown) {
-        // if anything went wrong, clean up the uncommitted metadata file

Review comment:
       This fixes the issue with deleting metadata.json in unknown state 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.

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       For example, we could expose a hook to provide logic for checking whether a particular operation is already committed.  Kind of what we have in Structured Streaming where Spark gives us queryId and batchId.
   
   Alternatively, blocking indefinitely may be an option. However, we keep only last 100 metadata files (by default) so the longer we wait, the less likely we will correctly check if a commit was applied.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I actually am not sure that is the case here, we are in the situation were one client, clientA has somehow lost the ability to confirm, but i'm not sure it's more likely that client B would also lose the ability to confirm. I guess it depends on if you believe a server-level failure is more likely than a client-level failure.
   
   In my experience it is much more likely that single client has connection issues than all clients have connection issues, but ymmv




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Exception for a failure to confirm either affirmatively or negatively that a commit was applied. The client
+ * cannot take any further action without possibly corrupting the table.
+ */
+public class CommitStateUnknownException extends RuntimeException {
+
+  private static final String COMMON_INFO =
+      "Cannot determine whether the commit was successful or not, the underlying data files may or " +
+      "may not be needed. Manual intervention via the Remove Orphan Files Action can remove these " +
+      "files when a connection to the Catalog can be re-established if the commit was actually unsuccessful.\n" +
+      "Please check to see whether or not your commit was successful before retying this commit. Retrying " +

Review comment:
       `retying` -> `retrying`




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I agree it is a valid concern when clients don't handle `UnknownStateExceptions` and blindly retry. However, we can try to address it in another way. For example, if users have idempotent requirements, they either should make their retry mechanisms smarter and be aware of `UnknownCommitStateException` or use something that guarantees the same batch won't be committed twice. We already handle such cases in Structured Streaming. I hope we do the same in Flink jobs. We can generalize this approach and make Iceberg handle such cases in batch writes too.
   
   I feel the current approach is extremely dangerous. If we delete a file that was actually committed, we corrupt the table and require a manual intervention. Unfortunately, I've seen this quite frequently. Also, getting a reply from the metastore or checking the commit status may take a substantial amount of time allowing a concurrent operation to succeed (if the lock expires). That's the worst what can happen as we will silently corrupt the table and will detect it only while querying a specific portion of the table. I had to fix such a case and we were simply lucky we found this out only after a couple of days while rewriting manifests. Duplicating data is less critical than loosing it, in my view.
   
   In our env, it is frequent to have multiple operations at the same time (like append + append, append + delete, append + compaction, append + rewrite 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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Switched around the implementation so now it uses a specific exception CommitStateUnknownException. We need to be care going forward with reviews of catalog code to make sure developers properly throw 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.

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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       FYI: In Hive we are considering creating a new Catalog implementation which uses the HMS client from the session. That one is a `RetryingMetaStoreClient` which have retries in itself.
   
   If we do so, we might have to change the `maxAttempts` to 0/1.




-- 
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -151,12 +158,13 @@ protected void doRefresh() {
     refreshFromMetadataLocation(metadataLocation);
   }
 
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review comment:
       Got it, resolving. 




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit succeeded, attempting to reconnect and check", persistFailure);

Review comment:
       I think that is a great idea, I was just working with a multi-tenent application and that kind of info would have been very helpful




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks

Review comment:
       I think it is common to start with `Tasks.foreach(newMetadataLocation)` on one line.

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       If we switch to using `Tasks`, I'd also add a timeout of let's say 30 mins to be safe. Checking the status may hit request timeouts, making the check time quite long. Also, using exponential backoff sounds reasonable (i.e. changing 1.0 to 2.0).
   
   Also, it is common to use `_TS` suffixes for these constants. Something like below.
   
   ```
       Tasks.foreach(newMetadataLocation)
           .retry(maxAttempts)
           .suppressFailureWhenFinished()
           .exponentialBackoff(
               COMMIT_STATUS_CHECK_WAIT_MS,
               COMMIT_STATUS_CHECK_WAIT_MS,
               COMMIT_STATUS_CHECK_TIMEOUT_MS,
               2.0)
   ```

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)
+        .onFailure((location, checkException) ->
+            LOG.error("Cannot check if commit to {}.{} exists.", database, tableName, checkException))
+        .run(location -> {
+          TableMetadata metadata = refresh();
+          String currentMetadataLocation = metadata.metadataFileLocation();
+          boolean commitSuccess = currentMetadataLocation.equals(newMetadataLocation) ||
+              metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+          if (commitSuccess) {
+            LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
+            status.set(CommitStatus.SUCCESS);
+          } else {
+            LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
+            status.set(CommitStatus.FAILURE);
+          }
+        });
+
+    LOG.error("Cannot determine commit state to {}.{}. Failed during checking {} times. " +

Review comment:
       I am afraid we will always log this error now. Shall we do this only if state is still unknown?




-- 
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] Parth-Brahmbhatt commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

Posted by GitBox <gi...@apache.org>.
Parth-Brahmbhatt commented on a change in pull request #2328:
URL: https://github.com/apache/iceberg/pull/2328#discussion_r595473531



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Any reason we can't make this configurable so the client is forced to indicate if they are idempotent or not explicitly?
   
   For non idempotent clients we should keep the current behavior with limited and configurable retries to check if a snapshot was committed before issuing final snapshot deletion. If the client indicates they are idempotent we can avoid deleting and throw the exception as proposed 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.

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I am afraid this will be dangerous. This means we have to update all places that throw a specific exception like `AlreadyExistsException` to now throw `CommitFailedException`. This is error-prone and we potentially lose helpful details. We could throw `CommitFailedException` and add the specific exception as a cause but the first point still holds.
   
   Also, an error log message will most likely be ignored by the user. This is a case where we really want to propagate as much info as possible.
   
   Would it make sense to introduce a new exception type? E.g., `UnknownCommitStateException`? That way, we can keep the existing logic for handling exceptions except cases where we really don't know what happens.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      if (attempt < maxAttempts) {
+        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
+            database, tableName, attempt, maxAttempts, checkFailure);
+        return innerCommitCheck(newMetadataLocation, attempt + 1, maxAttempts);

Review comment:
       The for loop would need early returns which i'm not a fan of, but I know not everyone has my taste. I'll switch it out if that makes it clearer




-- 
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] Parth-Brahmbhatt commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

Posted by GitBox <gi...@apache.org>.
Parth-Brahmbhatt commented on a change in pull request #2328:
URL: https://github.com/apache/iceberg/pull/2328#discussion_r595495566



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I don't think the property should be persisted at table level as a single table could be written from both idempotent and non idempotent clients (flink writer (idempotent) + background a compaction process (nonidempotent)). We can either expose the API at Table or PendingUpdate level. 
   
   In absence of more external use cases it is hard to say what is worst , corrupting a table so nothing can read or write until manual intervention or having duplicate data that can go undetected. I would pick duplicate data based on general use cases that I have seen so if we are voting, I would vote to default the behavior to assuming idempotent client.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -52,6 +52,12 @@
    * Implementations must check that the base metadata is current to avoid overwriting updates.
    * Once the atomic commit operation succeeds, implementations must not perform any operations that
    * may fail because failure in this method cannot be distinguished from commit failure.
+   * <p>

Review comment:
       Requirement for new exception detailed in api here




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Exception for a failure to confirm either affirmatively or negatively that a commit was applied. The client
+ * cannot take any further action without possibly corrupting the table.
+ */
+public class CommitStateUnknownException extends RuntimeException {
+
+  private static final String COMMON_INFO =
+      "Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Looks descriptive enough!




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {

Review comment:
       Pretends we throw an error while persisting that actually fails to commit serverside




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+/**

Review comment:
       To Reviewers: This is our new exception, unless an operations class throws this particular exception all snapshot producer behavior remains the same




----------------------------------------------------------------
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] RussellSpitzer commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Thanks @pvary + @aokolnychyi , 
   and thanks in advance to everyone else who is about to review. This is probably the best part of Open Source when we can all come together and fix issues like this as a community.


----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -270,29 +270,33 @@ private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata
     int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
         COMMIT_NUM_STATUS_CHECKS_DEFAULT);
 
-    for (int attempt = 1; attempt <= maxAttempts; attempt++) {
-      try {
-        Thread.sleep(COMMIT_STATUS_RECHECK_SLEEP);
-        TableMetadata metadata = refresh();
-        String metadataLocation = metadata.metadataFileLocation();
-        boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
-            metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
-        if (commitSuccess) {
-          LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
-          return CommitStatus.SUCCESS;
-        } else {
-          LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
-          return CommitStatus.FAILURE;
-        }
-      } catch (Throwable checkFailure) {
-        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
-            database, tableName, attempt, maxAttempts, checkFailure);
-      }
-    }
-
-    LOG.error("Cannot determine commit state to {}.{}. Failed to check {} times. Treating commit state as unknown.",
+    AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN);
+
+    Tasks
+        .foreach(newMetadataLocation)
+        .retry(maxAttempts)
+        .suppressFailureWhenFinished()
+        .exponentialBackoff(COMMIT_STATUS_RECHECK_SLEEP, COMMIT_STATUS_RECHECK_SLEEP, Long.MAX_VALUE, 1.0)

Review comment:
       Agree.




-- 
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] pvary commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   > @pvary Added in a variable for controlling the number of status check attempts as well as small 1 second timeout before the attempt.
   
   Thanks @RussellSpitzer! Checked the relevant parts. Left a few nits. Do not have strong feeligs there, so just feel free to disagree.
   
   Is there any open items remaining?
   
   Thanks, Peter 


----------------------------------------------------------------
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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   I am not sure I got it, @rdblue. Could you please elaborate a bit?


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       I cover this by refreshing the metadata and checking both the current and past metadata location for the newmetadatalocation




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       This is inline with what I was thinking
   
   I am strongly against deleting files when we are not sure of the state.
   
   I prefer the explicit exception because I also think it's probably the best way to actually get an end user information about the failure and allow them or a framework to automatically respond to it. I worry that doing a infinite blocking or even a very extended retry can lead to another user mechanism terminating the job, in that case they would have to directly inspect the logs to know that something may have failed.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -52,6 +52,12 @@
    * Implementations must check that the base metadata is current to avoid overwriting updates.
    * Once the atomic commit operation succeeds, implementations must not perform any operations that
    * may fail because failure in this method cannot be distinguished from commit failure.
+   * <p></p>
+   * Implementations must throw a CommitStateUnknownException in cases where it cannot be determined if the

Review comment:
       Should we have a ref here for `CommitStateUnknownException`?




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       @pvary I think I have this case handled now, I wrote a test which removes the lock the first commit acquires and then places another commit on top of the first before throwing an exception in the first client's action.
   
   Timeline
   
   Client 1:
     Aquire Lock
     New Metadata Committed
     Lock Released // Simulating the lock being removed or timing out
   Client 2:
     Aquire Lock
     Commit another new metadata
     Lock Released
   Client 1:
     Throws exception in persist 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.

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -52,6 +52,12 @@
    * Implementations must check that the base metadata is current to avoid overwriting updates.
    * Once the atomic commit operation succeeds, implementations must not perform any operations that
    * may fail because failure in this method cannot be distinguished from commit failure.
+   * <p></p>

Review comment:
       nit: unnecessary `</p>`




----------------------------------------------------------------
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -203,8 +218,23 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
           .orElseGet(ImmutableMap::of);
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled, summary);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit to {}.{} succeeded, attempting to reconnect and check.",
+            database, tableName, persistFailure);
+        commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+        switch (commitStatus) {
+          case SUCCESS:
+            return;

Review comment:
       It seems strange to return directly from here instead of breaking. The outer finally block uses the commit status and should be called regardless so I think this just looks more confusing than is 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.

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] rdsr commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.

Review comment:
       good call!

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit succeeded, attempting to reconnect and check", persistFailure);
+        commitStatus = checkCommitStatus(newMetadataLocation);

Review comment:
       There's retries built into the `HiveClientPool` ,  thoughts on why an additional retry helps, @aokolnychyi , @RussellSpitzer ?

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {

Review comment:
       When this exception occurred in prod, what sort of exception did you see? 
   I would have imagine any `TException` or `MetaException` implies that we do get some error response from the Metastore, a network partition e.g would have some sort of socket exception (e.g socket closed), as the HiveClientPool would try reconnecting 3 times




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I'll be okay if we want to make this configurable. How will the config look like? Will it be a table property?
   
   The question is what should be the default behavior. I'd say no matter whether clients are idempotent or not, it is safer not to delete files due to the potential table corruption. Adding the same data twice is not as critical, in my view.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       I think it may be sufficient to do this:
   
   ```
   try {
     persistTable(tbl, updateHiveTable);
   } catch (Exception commitException) {
     try {
       canCleanupMetadata = !isCommitted(newMetadataLocation);
     } catch (Exception checkException) {
       LOG.error("Cannot determine if commit was successful", checkException);
       throw new CommitStateUnknownException(commitException);
     }
   }
   ```
   
   ```
   private boolean isCommitted(String newMetadataLocation) {
     TableMetadata metadata = refresh();
     String metadataLocation = metadata.metadataFileLocation();
     return metadataLocation.equals(newMetadataLocation) ||
         metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
   }
   ```
   
   Then we don't need extra checks in catch clauses.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +
+              "may not be needed. Manual intervention via the Remove Orphan Files Action can remove these " +
+              "files when a connection to the Catalog can be re-established if the commit was actually unsuccessful. " +

Review comment:
       Ideally, that should be part of the exception 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.

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -168,7 +168,8 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       if (tbl != null) {
         // If we try to create the table but the metadata location is already set, then we had a concurrent commit
         if (base == null && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) != null) {
-          throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);

Review comment:
       Well, replacing `AlreadyExistsException` with `CommitFailedException` may lead to unexpected consequences.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Exception for a failure to confirm either affirmatively or negatively that a commit was applied. The client
+ * cannot take any further action without possibly corrupting the table.
+ */
+public class CommitStateUnknownException extends RuntimeException {
+
+  private static final String commonInfo =

Review comment:
       nit: `commonInfo` -> `COMMON_INFO`

##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Exception for a failure to confirm either affirmatively or negatively that a commit was applied. The client
+ * cannot take any further action without possibly corrupting the table.
+ */
+public class CommitStateUnknownException extends RuntimeException {
+
+  private static final String commonInfo =
+      "\nCannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       nit: shall we add "\n" directly below to keep the constant simple?
   
   ```
   cause.getMessage() + "\n" + COMMON_INFO
   ```
   
   

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        canCleanupMetadata = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
-            "transactional meta table. To fix this, use an alternative metastore", e);
+            "transactional meta table. To fix this, use an alternative metastore.\n%s", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        canCleanupMetadata = true;
+        throw metastoreException;
+      }
 
     } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
 
+      Thread.currentThread().interrupt();
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        canCleanupMetadata = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(canCleanupMetadata, newMetadataLocation, lockId);

Review comment:
       @pvary @marton-bod, we only handle `TException` and `UnknownHostException` here. Is there any chance we get another exception while we do `alter_table`?

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       I wonder whether this place is too generic. Apart from handling exceptions in alter_table (i.e. the actual commit), we also catch any exceptions during loading and locking. We don't necessarily have to do these checks in cases where we did not attempt to commit a new version.
   
   Is it a crazy idea to only add this check in `persistTable`, the one that does the actual commit?

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        canCleanupMetadata = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
-            "transactional meta table. To fix this, use an alternative metastore", e);
+            "transactional meta table. To fix this, use an alternative metastore.\n%s", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        canCleanupMetadata = true;
+        throw metastoreException;
+      }
 
     } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
 
+      Thread.currentThread().interrupt();
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        canCleanupMetadata = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(canCleanupMetadata, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      refresh();

Review comment:
       I think `refresh` already returns `TableMetadata` to us.

##########
File path: api/src/main/java/org/apache/iceberg/exceptions/CommitStateUnknownException.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Exception for a failure to confirm either affirmatively or negatively that a commit was applied. The client
+ * cannot take any further action without possibly corrupting the table.
+ */
+public class CommitStateUnknownException extends RuntimeException {
+
+  private static final String commonInfo =
+      "\nCannot determine whether the commit was successful or not, the underlying data files may or " +
+      "may not be needed. Manual intervention via the Remove Orphan Files Action can remove these " +
+      "files when a connection to the Catalog can be re-established if the commit was actually unsuccessful." +
+      "Please check to see whether or not your commit was successful when the catalog is again reachable." +

Review comment:
       nit: I'd add something `... was successful before retrying the operation when the catalog is again reachable. Retrying an operation without checking may lead to duplicated data. At this 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       The current behavior is very dangerous, it can cause changes to the table which are unrecoverable. I don't think it is a viable option and I would strongly recommend we don't make it the default even if we retry before doing it. From our own internal investigations and other reports I've heard, this behavior has actually been cropping up in a number of places but it's rather hard to track down.




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       What do you think, @RussellSpitzer @pvary @rdblue @shardulm94 @marton-bod @Parth-Brahmbhatt?




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting

Review comment:
       Thanks for the javadoc! It is going to be helpful for folks who touch this code next 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I think that's fine, I just want to make sure every catalog implements it then, if we always treat all non CME excepts as unknown commit state we know we are never deleting state we need. If we instead do it in the opposite direction we need to be extra sure that all exceptions that may have an unknown state are marked that way. In my mind this left more chance of corruption when someone forgets to mark a particular edgecase.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();

Review comment:
       FYI: The `HiveClientPool` tries to reconnect at least once.
   We might not want to overcomplicate this part of the code to write error handling of the error handling




----------------------------------------------------------------
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] RussellSpitzer commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Rebased on Master


----------------------------------------------------------------
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       It's very dangerous to change the logic here. If there is an exception where it is unknown whether or not the table reference was updated, then it is safer to drop the files and assume that the commit was not successful. That's because we still need to propagate the exception, which signals that the commit failed. Most of the time, that will cause a higher-level retry of the whole operation, like re-running a scheduled job with the same data. If that happens and both commits are actually successful, then we have a situation where incoming records were silently duplicated.
   
   I like the idea of throwing `UnknownCommitStateException`, but I think we would need some way to ensure that it is actually handled. An idempotent committer (like the Flink sink) could simply swallow the exception and retry. But without coordination between the `TableOperations` that decides whether to throw this exception and the committer I think there is still a problem. Could we add an option to `SnapshotProducer` to signal whether it would be safe and then skip cleanup if the flag is set?




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +
+              "may not be needed. Manual intervention via the Remove Orphan Files can be used to remove these files " +

Review comment:
       action ...




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -203,8 +218,23 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
           .orElseGet(ImmutableMap::of);
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled, summary);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit to {}.{} succeeded, attempting to reconnect and check.",
+            database, tableName, persistFailure);
+        commitStatus = checkCommitStatus(newMetadataLocation, metadata);
+        switch (commitStatus) {
+          case SUCCESS:
+            return;

Review comment:
       I just thought it had more symmetry with the other cases, since they are all function exits explicitly as well. I'll switch it to a break though if you like that better.




-- 
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] aokolnychyi merged pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   


-- 
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", database, tableName, newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", database, tableName, newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      if (attempt < maxAttempts) {
+        LOG.error("Cannot check if commit to {}.{} exists. Retry attempt {} of {}.",
+            database, tableName, attempt, maxAttempts, checkFailure);
+        return innerCommitCheck(newMetadataLocation, attempt + 1, maxAttempts);

Review comment:
       Nit: wouldn't it be easier to read with a loop instead of recursion?




----------------------------------------------------------------
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   @RussellSpitzer, @aokolnychyi, I think the approach here is good. I like that existing behavior is kept but the new exception provides a way to change it. I don't think that we need the ability to mark a commit idempotent yet since this is an alternative way to avoid deleting the files. Once the comments above are addressed, I think this can be committed to master.
   
   I know the intent is to get this into a patch release, but I don't think that we should do that with a new behavior change for Hive catalogs by default. The problem used to cause tables to be unwritable (in most cases) until rolled back. But this changes the behavior and makes the table still writable. That's the right direction, but I don't think the change should be in a patch release.
   
   We could, however, add this with a flag to still delete the data files or to fall back to not throwing the unknown commit state exception. I'd be comfortable with that going in a patch release.


-- 
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] aokolnychyi commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Could you do another pass too, @rdsr @pvary @rymurr @Parth-Brahmbhatt @yyanyy?


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();
+      String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP);
+      return metadataLocation.equals(newMetadataLocation);

Review comment:
       I Was trying to think about that, shouldn't we still be within our original lock at this point? We don't remove it till we hit the finally block




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

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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       Then we could handle all exception types there and don't worry about getting only `TException`.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       The first scenario you propose is actually the most dangerous without this change, another client could beat the deletion code and end up doing another commit on top of the files as they are being removed. This then leads to an unrecoverable situation since additional writes and operations will be based of of files that are removed.




----------------------------------------------------------------
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] RussellSpitzer commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   Thanks everyone! 


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       ugh and it fails because we need to rethrow the exception if canCleanupMetadata is false




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;
+      }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException("Interrupted during commit", e);
-
+      RuntimeException interruptException = new RuntimeException("Interrupted during commit", e);
+      if (checkCommitSuccessful(newMetadataLocation, interruptException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw interruptException;
+      }
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitFailed, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if the current metadata location matches our new commit path. This as used as
+   * a last resort when we are dealing with exceptions which may indicate that our commit has failed but we are not
+   * certain if that is the case.
+   * @param newMetadataLocation the path of the new commit file
+   * @param originalFailure the exception which leads us to believe the commit has failed
+   * @return true if the commit was successful, false if not, and rethrows the original exception if we cannot
+   * determine
+   */
+  private boolean checkCommitSuccessful(String newMetadataLocation, RuntimeException originalFailure) {
+    try {
+      Table tbl = loadHmsTable();

Review comment:
       We could try this even more times, would be glad to hear other folks opinion 




----------------------------------------------------------------
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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       You're right that there is a race condition there. But it is far more unlikely that there is an unknown state problem followed by an immediate race commit. It's more likely that deleting the metadata file immediately will cause other commits to fail.
   
   I think that it's reasonable to do this if the committer is idempotent, but otherwise I would not recommend doing it by default. We'd be trading the right behavior almost all of the time for the right behavior in an extreme 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I think that's fine, I just want to make sure every catalog implements it then, if we always treat all non CME excepts as unknown commit state we know we are never deleting state we need. If we instead do it in the opposite direction we need to be extra sure that all exceptions that may have a unknown state are marked that way.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        canCleanupMetadata = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
-            "transactional meta table. To fix this, use an alternative metastore", e);
+            "transactional meta table. To fix this, use an alternative metastore.\n%s", e);

Review comment:
       clean this up




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I agree it is a valid concern when clients don't handle `UnknownStateExceptions` and blindly retry. However, we can try to address it in another way. For example, if users have idempotent requirements, they either should make their retry mechanisms smarter and be aware of `UnknownCommitStateException` or use something that guarantees the same batch won't be committed twice. We already handle such cases in Structured Streaming. I hope we do the same in Flink jobs. We can generalize this approach and make Iceberg handle such cases in batch writes too.
   
   I feel the current approach is extremely dangerous. If we delete a file that was actually committed, we corrupt the table and require a manual intervention. Unfortunately, I've seen this quite frequently. Also, getting a reply from the metastore or checking the commit status may take a substantial amount of time allowing a concurrent operation to succeed (if the lock expires). That's the worst what can happen as we will silently corrupt the table and will detect it only while querying the table. I had to fix such a case and we were simply lucky we found this out only after a couple of days while rewriting manifests. Duplicating data is less critical than loosing it, in my view.




----------------------------------------------------------------
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] rymurr commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       I agree wtih @aokolnychyi here, I think changing the contract for the table ops is dangerous, especially as the change reduces the information available to the user.
   
   I think Anton is right, rather than saying "`CommitFailedException` is the only valid known failure mode" we should probably say "`NewExceptionTypeX` means the commit failed in an unknown way and the user may be required to do some immediate intervention"

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -168,7 +168,8 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       if (tbl != null) {
         // If we try to create the table but the metadata location is already set, then we had a concurrent commit
         if (base == null && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) != null) {
-          throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);

Review comment:
       Agreed, I think `AlreadyExistsExpection` is the correct exception for the problem and we should raise as much info as possible 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {
+        LOG.error("Cannot tell if commit succeeded, attempting to reconnect and check", persistFailure);
+        commitStatus = checkCommitStatus(newMetadataLocation);

Review comment:
       This was from the discussion from the issue with @marton-bod and @pvary, the thought was we should give it one more chance just incase a full new connection attempt would succeed because the cost of failure is that the user needs to do manual investigation, this design also lets us slot in other behaviors later if we decide that makes sense.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit exists, treating commit state as unknown", checkFailure);
+      return CommitStatus.UNKNOWN;
     }
   }
 
-  private void persistTable(Table hmsTable, boolean updateHiveTable) throws TException, InterruptedException {
+  // Visible for testing

Review comment:
       nit: Could we use
   `org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting`?

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -318,7 +369,8 @@ private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hive
     return storageDescriptor;
   }
 
-  private long acquireLock() throws UnknownHostException, TException, InterruptedException {
+  // Visible for testing

Review comment:
       nit: Could we use
   `org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting`?




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);

Review comment:
       Pretends we throw an exception while persisting and don't know what happened, can't check to find out, but in reality the commit failed




----------------------------------------------------------------
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] RussellSpitzer commented on pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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


   @rdblue You mean just fail with the root exception? Because we still want the job to be marked as failed even if it's idempotent. 


-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       The first scenario you propose is actually the most dangerous without this change, another client could beat the deletion code and end up doing another commit on top of the files as they are being removed. This then leads to an unrecoverable situation since additional writes and operations will be based off of files that are removed.
   
   A retry can be undone, but a commit on top of files that are removed is much more difficult to unwind (i think)




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       Also you'll note we do have the check here in table operations as well, we are checking the metastore (or at least attempting to) immediately after failure to try to avoid going down this path.




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +239,40 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit exists, treating commit state as unknown", checkFailure);
+      return CommitStatus.UNKNOWN;
     }
   }
 
-  private void persistTable(Table hmsTable, boolean updateHiveTable) throws TException, InterruptedException {
+  // Visible for testing
+  protected void persistTable(Table hmsTable, boolean updateHiveTable) throws TException, InterruptedException {

Review comment:
       Required so I can mock this method in the test cases and inject errors




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -151,12 +158,13 @@ protected void doRefresh() {
     refreshFromMetadataLocation(metadataLocation);
   }
 
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review comment:
       Do we still need 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -198,8 +206,22 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
-      persistTable(tbl, updateHiveTable);
-      threw = false;
+      try {
+        persistTable(tbl, updateHiveTable);
+        commitStatus = CommitStatus.SUCCESS;
+      } catch (Throwable persistFailure) {

Review comment:
       ```Caused by: java.lang.RuntimeException: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out```
   
   The runtime wrapper is from how this code wraps 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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +

Review comment:
       I am not entirely sure. Thrift client might throw exception even if the HMS server side is finished but the response did not arrive to the HMS client.




----------------------------------------------------------------
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] pvary commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {

Review comment:
       nit: These onliner descriptions could be very useful as a comment in the code too so next generations could understand better the goal of these 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.

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +250,55 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata metadata) {
+    int maxAttempts = PropertyUtil.propertyAsInt(metadata.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+    return innerCommitCheck(newMetadataLocation, 1, maxAttempts);
+  }
+
+  private CommitStatus innerCommitCheck(String newMetadataLocation, int attempt, int maxAttempts) {
+    try {
+      Thread.sleep(1000);

Review comment:
       SGTM




-- 
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] wg1026688210 commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -217,11 +241,41 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation) {
+    try {
+      TableMetadata metadata = refresh();
+      String metadataLocation = metadata.metadataFileLocation();
+      boolean commitSuccess = metadataLocation.equals(newMetadataLocation) ||
+          metadata.previousFiles().stream().anyMatch(log -> log.file().equals(newMetadataLocation));
+      if (commitSuccess) {
+        LOG.info("Commit status check: Commit to {}.{} of {} succeeded", newMetadataLocation);
+        return CommitStatus.SUCCESS;
+      } else {
+        LOG.info("Commit status check: Commit to {}.{} of {} failed", newMetadataLocation);
+        return CommitStatus.FAILURE;
+      }
+    } catch (Throwable checkFailure) {
+      LOG.error("Cannot check if commit to {}.{} exists, treating commit state as unknown: {}",
+          database, tableName, checkFailure);
+      return CommitStatus.UNKNOWN;

Review comment:
       the check action may produce exception which can be retried such as  io exception when visiting HMS  ,shall we return `CommitStatus.FAILURE` for retrying




----------------------------------------------------------------
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java
##########
@@ -66,4 +75,220 @@ public void testSuppressUnlockExceptions() throws TException, InterruptedExcepti
     // the commit must succeed
     Assert.assertEquals(1, ops.current().schema().columns().size());
   }
+
+  @Test
+  public void testThriftExceptionFailureOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+
+    AssertHelpers.assertThrows("We should rethrow generic runtime errors if the " +
+        "commit actually doesn't succeed", RuntimeException.class, "Metastore operation failed",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2));
+    Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    // Simulate a communication error after a successful commit
+    commitAndThrowException(ops, spyOps);
+
+    // Shouldn't throw because the commit actually succeeds even though persistTable throws an exception
+    spyOps.commit(metadataV2, metadataV1);
+
+    ops.refresh();
+    Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Commit should have been successful and new metadata file should be made",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+    HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations();
+
+    TableMetadata metadataV1 = ops.current();
+
+    table.updateSchema()
+        .addColumn("n", Types.IntegerType.get())
+        .commit();
+
+    ops.refresh();
+
+    TableMetadata metadataV2 = ops.current();
+
+    Assert.assertEquals(2, ops.current().schema().columns().size());
+
+    HiveTableOperations spyOps = spy(ops);
+
+    failCommitAndThrowException(spyOps);
+    breakFallbackCatalogCommitCheck(spyOps);
+
+    AssertHelpers.assertThrows("Should throw CommitStateUnknownException since the catalog check was blocked",
+        CommitStateUnknownException.class, "Datacenter on fire",
+        () -> spyOps.commit(metadataV2, metadataV1));
+
+    ops.refresh();
+
+    Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current());
+    Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current()));
+    Assert.assertEquals("Client could not determine outcome so new metadata file should also exist",
+        3, metadataFileCount(ops.current()));
+  }
+
+  @Test
+  public void testThriftExceptionsUnknownSuccessCommit() throws TException, InterruptedException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);

Review comment:
       Pretends we throw an exception while persisting and don't know what happened, can't check to find out, but in reality the commit succeeded




----------------------------------------------------------------
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] aokolnychyi commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,8 +293,15 @@ public void commit() {
             taskOps.commit(base, updated.withUUID());
           });
 
+    } catch (CommitFailedException commitFailedException) {
+      // We have an acknowledged failure from the Catalog. We are confident that the commit has not been applied
+      Exceptions.suppressAndThrow(commitFailedException, this::cleanAll);
     } catch (RuntimeException e) {
-      Exceptions.suppressAndThrow(e, this::cleanAll);
+      LOG.error("Cannot determine whether the commit was successful or not, the underlying data files may or " +

Review comment:
       cc @omalley @rdblue @shardulm94 @rdsr too




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

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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,29 +203,68 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      canCleanupMetadata = true;
       throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {

Review comment:
       I think this is still not very clear, but If other folks can read it let's keep it.




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

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 #2328: (#2317) Stop removal of files when catalog state is uncertain - HiveCatalog

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -222,11 +252,51 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       throw new RuntimeException("Interrupted during commit", e);
 
     } finally {
-      cleanupMetadataAndUnlock(threw, newMetadataLocation, lockId);
+      cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId);
+    }
+  }
+
+  /**
+   * Attempt to load the table and see if any current or past metadata location matches the one we were attempting
+   * to set. This is used as a last resort when we are dealing with exceptions that may indicate the commit has
+   * failed but are not proof that this is the case. Past locations must also be searched on the chance that a second
+   * committer was able to successfully commit on top of our commit.
+   *
+   * @param newMetadataLocation the path of the new commit file
+   * @param config metadata to use for configuration
+   * @return Commit Status of Success, Failure or Unknown
+   */
+  private CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) {
+    int maxAttempts = PropertyUtil.propertyAsInt(config.properties(), COMMIT_NUM_STATUS_CHECKS,
+        COMMIT_NUM_STATUS_CHECKS_DEFAULT);
+
+    for (int attempt = 1; attempt <= maxAttempts; attempt++) {

Review comment:
       I think this should use `Tasks` instead of a custom loop for retry logic. You can customize the retries and that would keep all of the retry logic in one place.




-- 
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] RussellSpitzer commented on a change in pull request #2328: (#2317) Stop removal of files when catalog state is uncertain

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -199,25 +201,63 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       setHmsTableParameters(newMetadataLocation, tbl, metadata.properties(), removedProps, hiveEngineEnabled);
 
       persistTable(tbl, updateHiveTable);
-      threw = false;
     } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
-      throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName);
+      commitFailed = true;
+      throw new CommitFailedException("Table already exists: %s.%s", database, tableName);
 
     } catch (TException | UnknownHostException e) {
       if (e.getMessage() != null && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) {
+        commitFailed = true;
         throw new RuntimeException("Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " +
             "exist, this probably happened when using embedded metastore or doesn't create a " +
             "transactional meta table. To fix this, use an alternative metastore", e);
       }
 
-      throw new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
+      RuntimeException metastoreException =
+              new RuntimeException(String.format("Metastore operation failed for %s.%s", database, tableName), e);
 
+      if (checkCommitSuccessful(newMetadataLocation, metastoreException)) {
+        return; // We are able to verify the commit succeed
+      } else {
+        // We were able to check and the commit did not succeed
+        commitFailed = true;
+        throw metastoreException;

Review comment:
       This should be converted to a commit failed exception




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