Skip to content

Conversation

@huaxingao
Copy link
Contributor

This is the 4th PR for Idempotency Key. Introduce test-only idempotency in RESTCatalogAdapter:

  • in-memory store keyed by Idempotency-Key,
  • cached-success replay
  • TTL expiry
  • optional “503 after success” simulation.

Add E2E tests:

  • testIdempotentDuplicateCreateReturnsCached
  • testIdempotencyKeyLifetimeExpiredTreatsAsNew
  • testIdempotentCreateReplayAfterSimulated503

First PR: #14649
SecondPR: #14700
Third PR: #14740

@github-actions github-actions bot added the core label Dec 5, 2025
long now = System.currentTimeMillis();
boolean expired =
existing.status == IdempotencyEntry.Status.FINALIZED
&& (now - existing.firstSeenMillis) > idempotencyLifetimeMillis;
Copy link
Contributor

Choose a reason for hiding this comment

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

can we use Instant.now().minusMillis(existing.firstSeenMillis) here as that is more readable when adding isBefore / isAfter

Copy link
Contributor

Choose a reason for hiding this comment

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

you could also make this a function on the entry itself, so that you'd only call if(!entry.expired()) here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed. Thanks!

}

// Test hooks/configuration for idempotency behavior
public static void simulate503OnFirstSuccessForKey(String key) {
Copy link
Contributor

Choose a reason for hiding this comment

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

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

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

AuthSession httpSession = am.initSession(httpBase, conf);
RESTClient http = httpBase.withAuthSession(httpSession);

CreateTableRequest req =
Copy link
Contributor

Choose a reason for hiding this comment

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

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

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.

return new IdempotentCreateEnv(ns, ident, headers, http, req);
}

private static class IdempotentCreateEnv {
Copy link
Contributor

Choose a reason for hiding this comment

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

do we really need this wrapper class?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I removed this wrapped class

@huaxingao huaxingao force-pushed the idempotency_4 branch 2 times, most recently from 47aae84 to a8a0a16 Compare December 15, 2025 18:34
@huaxingao
Copy link
Contributor Author

@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) {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we wrap this to plan-api endpoints too ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

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) {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit : should we make it generic like simulate status code or something ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

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.

Comment on lines 3451 to 3461
String key = "idemp-drop-void";
Namespace ns = Namespace.of("ns_void");
TableIdentifier ident = TableIdentifier.of(ns, "t_void");
restCatalog.createNamespace(ns, ImmutableMap.of());
Pair<RESTClient, Map<String, String>> httpAndHeaders = httpAndHeaders(key);
RESTClient http = httpAndHeaders.first();
Map<String, String> headers = httpAndHeaders.second();
Copy link
Contributor

Choose a reason for hiding this comment

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

i wonder if all this can be abstracted in a utils which takes key, namespace and table_name ? wdyt ?

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 comment! I have added a smaller helper method for the common code.

String key = keyHeader.get().value();

// check existing entry and TTL
IdempotencyEntry existing = IDEMPOTENCY_STORE.get(key);
Copy link
Contributor

Choose a reason for hiding this comment

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

This is rare in this use case but maybe we should use computeIfAbsent here to avoid any race conditions get and put issues

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good call — I updated this to avoid the get/remove/put race by using a single atomic map operation. I now use IDEMPOTENCY_STORE.compute(...) to either reuse the current entry or replace it if missing/expired, and use a per-call isLeader flag so only the thread that created the new entry executes the action while others wait and replay the finalized result. I used compute (instead of computeIfAbsent) because we also need to replace expired entries.

Copy link
Contributor

@singhpk234 singhpk234 left a comment

Choose a reason for hiding this comment

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

Mostly LGTM, thanks @huaxingao left minor comments


String key = keyHeader.get().value();

AtomicBoolean isLeader = new AtomicBoolean(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

whats a Leader ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

“Leader” here means the request thread that won the IDEMPOTENCY_STORE.compute(...) and created (or replaced) the IN_PROGRESS entry for this Idempotency-Key. That thread executes the action and finalizes the entry; all other concurrent requests for the same key are “followers” that wait on the latch and then replay the finalized result/error.

.hasMessageContaining(ident.toString());

// Clean up
restCatalog.dropTable(ident);
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need explicit drop ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch. This is not needed. Removed.

try {
idempotencyLifetimeMillis = Duration.parse(isoDuration).toMillis();
} catch (Exception e) {
// ignore parse errors; keep default
Copy link
Contributor

Choose a reason for hiding this comment

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

sorry i might have missed this in first pass, this is public imho we should throw illegalArgs ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch. I made this test-only (@VisibleForTesting + package-private) and changed it to throw IllegalArgumentException on invalid ISO durations rather than silently ignoring parse errors.


private AuthSession authSession = AuthSession.EMPTY;
private PlanningBehavior planningBehavior;
private final java.util.Set<String> simulate503OnFirstSuccessKeys = Sets.newConcurrentHashSet();
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: can we import this instead of inline ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed. Thanks

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.

4 participants