Skip to content

Conversation

@agnes-xinyi-lu
Copy link
Contributor

@agnes-xinyi-lu agnes-xinyi-lu commented Jan 23, 2026

When multiple processes concurrently commit to different branches of the
same table through the REST catalog, sequence number validation failures
in TableMetadata.addSnapshot() were throwing non-retryable ValidationException
instead of retryable CommitFailedException.
This fix catches the sequence number validation error in CatalogHandlers.commit()
and wraps it in ValidationFailureException(CommitFailedException) to:
- Skip server-side retry (which won't help since sequence number is in the request)
- Return CommitFailedException to the client so it can retry with refreshed metadata

Issue #15001

request.updates().forEach(update -> update.applyTo(metadataBuilder));
} catch (ValidationException e) {
// Sequence number conflicts from concurrent commits are retryable by the client,
// but server-side retry won't help since the sequence number is in the request.
Copy link
Contributor

@singhpk234 singhpk234 Jan 26, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is an interesting point ! since the snapshot obj is created in the client and sent to the server the sequence number is locked in and server can't do much fail fast seems reasonable.

I wonder if we can refactor / introduce some other mechanism rather than relying on exception message text.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review @singhpk234 !
Checking exception message is not an uncommon pattern within iceberg repo , it helps target particular scenarios that were thrown in a more generic exception type. Refactoring the exception itself will require TableMetadata change which increases risks.
I'm trying to minimize the change to get this issue fixed as per my understanding of the comment on the issue. As my original idea was to add an UpdateRequirement to the spec for this assertion.
Any thoughts?

Copy link
Contributor

@amogh-jahagirdar amogh-jahagirdar Jan 29, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with @singhpk234 here. It's a bit brittle to rely on the message in the exception. A lot of the cases where that's done in the code base is at the integration points with other libraries/systems like JDBC/hive where there isn't a better exception provided. Here it's all the Iceberg core code base, where there can be other use cases where this behavior is desirable and we have an opportunity to do it cleanly.

In that case, I think what @rdblue suggested in the issue makes a lot of sense; we could define a RetryableValidationException, throw it instead of the validation exception in addSnapshot when performing the sequence number comparison. Here in catalog handlers, then we'd then throw the ValidationFailure wrapping a CommitFailedException to break out and kick the CommitFailedException back to the client to retry.

I'm not too sure it's an additional "risk" to define a new exception and throw it in a very specific case.

Copy link
Contributor Author

@agnes-xinyi-lu agnes-xinyi-lu Jan 29, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @amogh-jahagirdar , I don't have a strong opinion on how to implement the fix. My goal is to get a solution in ASAP because it's a hard blocker for our rest catalog migration.
If we all agree adding a wrapping exception and throwing it from addSnapshot, happy to implement that as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@singhpk234 @amogh-jahagirdar I've updated the PR with the new solution, please take a look, thanks!

@amogh-jahagirdar
Copy link
Contributor

Thanks @agnes-xinyi-lu will take a look with fresh eyes tomorrow.

Comment on lines 4489 to 4498
// Verify the fix: with AssertLastSequenceNumber, there should be NO validation failures
// All concurrent conflicts should be caught as CommitFailedException (retryable)
assertThat(validationFailureCount.get())
.as(
"With the fix, sequence number conflicts should be caught by AssertLastSequenceNumber "
+ "and throw CommitFailedException (retryable), not ValidationException")
.isEqualTo(0);

// At least some should succeed (commits that don't conflict or succeed after retry)
assertThat(successCount.get()).as("At least some appends should succeed").isGreaterThan(0);
Copy link
Contributor

@amogh-jahagirdar amogh-jahagirdar Jan 29, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should aim to make this test have harder assertions. I think we could use an AtomicInteger barrier and essentially synchronize different rounds of commits and deterministically cause conflicts. At the end, I think we'd be able to have a deterministic number of failures (i'd probably organize it so the barrier causes 1 conflict per branch per round?). Checkout https://github.com/apache/iceberg/blob/main/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java#L130 for another example of this pattern

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Understand the ask, I updated the test to verify total number of commits and make sure there are failures/conflicts, for the total number of conflicts it's hard to get a deterministic number (pls let me know if you have an easy way, happy to implement that). Main problem is table.appendFiles.commit is not an atomic operation, it refreshes TableMetadata in SnapshotProducer. To create a deterministic conflict, we would need to put a barrier there or in TableOperations to make sure every thread gets exactly the same base TableMetadata.
I think the purpose of this test is to verify there could be conflicts during commit when there are multiple threads committing to non-current branch at the same time. And the fix should guarantee it's a retryable exception.

    When multiple processes concurrently commit to different branches of the
    same table through the REST catalog, sequence number validation failures
    in TableMetadata.addSnapshot() were throwing non-retryable ValidationException
    instead of retryable CommitFailedException.

    This fix catches the sequence number validation error in CatalogHandlers.commit()
    and wraps it in ValidationFailureException(CommitFailedException) to:
    - Skip server-side retry (which won't help since sequence number is in the request)
    - Return CommitFailedException to the client so it can retry with refreshed metadata
@github-actions github-actions bot removed the API label Jan 30, 2026
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants