Add idempotency adapter and E2E coverage#14773
Conversation
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java
Outdated
Show resolved
Hide resolved
| long now = System.currentTimeMillis(); | ||
| boolean expired = | ||
| existing.status == IdempotencyEntry.Status.FINALIZED | ||
| && (now - existing.firstSeenMillis) > idempotencyLifetimeMillis; |
There was a problem hiding this comment.
can we use Instant.now().minusMillis(existing.firstSeenMillis) here as that is more readable when adding isBefore / isAfter
There was a problem hiding this comment.
you could also make this a function on the entry itself, so that you'd only call if(!entry.expired()) here
| } | ||
|
|
||
| // Test hooks/configuration for idempotency behavior | ||
| public static void simulate503OnFirstSuccessForKey(String key) { |
There was a problem hiding this comment.
I'm not really a fan of such a way to configure this. We should explore doing this similarly how remote scan planning sets the planning behavior for particular tests
There was a problem hiding this comment.
Thanks for the suggestion. I removed the static hook from CatalogHandlers and mirrored the remote planning pattern:
- Added a per-adapter IdempotencyBehavior in RESTCatalogAdapter and apply it post‑success in execute(...).
- Tests configure behavior via the adapter (e.g., adapterForRESTServer.simulate503OnFirstSuccessForKey(key)), while routes still call CatalogHandlers.withIdempotency(...).
There was a problem hiding this comment.
I don't think we should add this functionality at all to the adapter. Instead, what about just having a custom adapter that simulates this behavior for that particular test case. We already use custom adapters with custom behavior in a lot of other test methods.
My main concern is that we're trying to force "test conditions" into code that shouldn't have such test conditions, because RESTCatalogAdapter could be used in real usage scenarios and it shouldn't ideally contain test code. Additionally, this opens up a precedent for future code modifications where it might seem ok to add more and more testing conditions to the adapter or to CatalogHandlers
There was a problem hiding this comment.
I guess one alternative would be to have a custom adapter in the testFixtures, similar to
where we have a custom adapter that performs credential vendingThere was a problem hiding this comment.
Thanks — agreed. I removed the test hook from CatalogHandlers and kept RESTCatalogAdapter free of test knobs.
The transient-503 simulation is now implemented only in a test-only adapter used by TestRESTCatalog (HeaderValidatingAdapter), so production-ish code paths don’t carry test conditions.
| AuthSession httpSession = am.initSession(httpBase, conf); | ||
| RESTClient http = httpBase.withAuthSession(httpSession); | ||
|
|
||
| CreateTableRequest req = |
There was a problem hiding this comment.
seems a bit weird to create the request here. Can we not mock stuff as we do in a bunch of other tests and verify that the correct requests were sent?
There was a problem hiding this comment.
I’ve added a reusable verify helper and now assert the request shape (method/path/headers, including Idempotency‑Key) after the POSTs. I’m keeping a real CreateTableRequest in the E2E tests because it’s needed to exercise serialization and replay.
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
Outdated
Show resolved
Hide resolved
47aae84 to
a8a0a16
Compare
|
@nastra @amogh-jahagirdar @singhpk234 Could you please take a look when you have a moment? Thanks! |
| simulate503OnFirstSuccessKeys.add(key); | ||
| } | ||
|
|
||
| private static OAuthTokenResponse handleOAuthRequest(Object body) { |
There was a problem hiding this comment.
can we wrap this to plan-api endpoints too ?
There was a problem hiding this comment.
Good point. Plan endpoints aren’t wrapped with withIdempotency yet. I’ll follow up with a separate PR to add Idempotency-Key support for the plan APIs (wrap handlers + add tests) and we can extend the simulation coverage there.
| } | ||
|
|
||
| /** Test helper to simulate a transient 503 after the first successful mutation for a key. */ | ||
| public void simulate503OnFirstSuccessForKey(String key) { |
There was a problem hiding this comment.
nit : should we make it generic like simulate status code or something ?
There was a problem hiding this comment.
Good suggestion. I kept it 503-specific since the test hook is only used to exercise retry behavior for transient 503/commit-state-unknown; happy to generalize later if we add tests needing other codes.
There was a problem hiding this comment.
It would be great to not add test-specific code to the adapter/catalog handler. See my other comment on this: #14773 (comment)
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
Outdated
Show resolved
Hide resolved
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Outdated
Show resolved
Hide resolved
a8a0a16 to
4a81530
Compare
singhpk234
left a comment
There was a problem hiding this comment.
Mostly LGTM, thanks @huaxingao left minor comments
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Outdated
Show resolved
Hide resolved
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
Outdated
Show resolved
Hide resolved
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Outdated
Show resolved
Hide resolved
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
LGTM, thanks @huaxingao !
Lets give some time to @nastra @amogh-jahagirdar incase they have further feedbacks !
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Outdated
Show resolved
Hide resolved
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Outdated
Show resolved
Hide resolved
c3e5460 to
44b467c
Compare
76027b6 to
76f6da0
Compare
|
@nastra I removed the test hook from CatalogHandlers and kept RESTCatalogAdapter free of test knobs. |
amogh-jahagirdar
left a comment
There was a problem hiding this comment.
sorry for the late review, overall I think it looks reasonable, just minor comments, will wait in case @nastra wanted to do another pass. Thanks @huaxingao
| private final Set<String> simulate503OnFirstSuccessKeys = | ||
| org.apache.iceberg.relocated.com.google.common.collect.Sets.newConcurrentHashSet(); |
There was a problem hiding this comment.
Minor: Maybe this should be a mapping from the key to a generic throwable that's thrown, then we could add other failure cases and make sure idempotency returns or throws as expected.
There was a problem hiding this comment.
Good idea. I generalized the test-only hook to a key -> RuntimeException map so tests can simulate different post-success transient failures while still keeping simulate503OnFirstSuccessForKey as a convenience wrapper for the current case.
| { | ||
| if (PropertyUtil.propertyAsBoolean(vars, "purgeRequested", false)) { | ||
| CatalogHandlers.purgeTable(catalog, tableIdentFromPathVars(vars)); | ||
| CatalogHandlers.withIdempotency( |
There was a problem hiding this comment.
Minor: I wonder if withIdempotency even needs to be exposed? Feels like it could just be invoked internally in each of the existing CatalogHandlers methods?
There was a problem hiding this comment.
Good point. The idempotency behavior is driven by the HTTP request headers, so I’m keeping the wrapping at the routing/adapter layer rather than pushing HTTPRequest through every CatalogHandlers.* method.
That said, I agree it doesn’t need to be exposed publicly — I made withIdempotency(...) package-private and it’s only used internally by the REST adapter/routes.
| // replaces) the IN_PROGRESS entry for this Idempotency-Key. Only the leader executes the | ||
| // action and finalizes the entry; concurrent requests for the same key ("followers") wait on | ||
| // the latch and then replay the finalized result/error. | ||
| AtomicBoolean isLeader = new AtomicBoolean(false); |
There was a problem hiding this comment.
Minor: I'd probably just call this isFirst. Leader/follower makes it seem more than it really is.
There was a problem hiding this comment.
Changed to isFirst. Thanks
| TableIdentifier ident = env.first(); | ||
| Pair<RESTClient, Map<String, String>> httpAndHeaders = env.second(); | ||
| RESTClient http = httpAndHeaders.first(); | ||
| Map<String, String> headers = httpAndHeaders.second(); | ||
| CreateTableRequest req = createReq(ident); |
There was a problem hiding this comment.
Nit: I think this bit is repeated across tests, any way to abstract some of this behind a helper?
There was a problem hiding this comment.
I factored the repeated prepareIdempotentEnv unpacking into a small IdempotentEnv helper
|
Thanks @amogh-jahagirdar @geruh @nastra @singhpk234 for your review! Are there any more comments? If not, I plan to merge this PR in two days. |
|
I merged this PR. If there are any more comments, I will address in follow-ups. Thank you all for reviewing this PR! |
This is the 4th PR for Idempotency Key. Introduce test-only idempotency in
RESTCatalogAdapter:Add E2E tests:
First PR: #14649
SecondPR: #14700
Third PR: #14740