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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]