singhpk234 commented on code in PR #14773:
URL: https://github.com/apache/iceberg/pull/14773#discussion_r2636621534


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -122,6 +124,11 @@ public RESTCatalogAdapter(Catalog catalog) {
     this.asViewCatalog = catalog instanceof ViewCatalog ? (ViewCatalog) 
catalog : null;
   }
 
+  /** Test helper to simulate a transient 503 after the first successful 
mutation for a key. */
+  public void simulate503OnFirstSuccessForKey(String key) {
+    simulate503OnFirstSuccessKeys.add(key);
+  }
+
   private static OAuthTokenResponse handleOAuthRequest(Object body) {

Review Comment:
   can we wrap this to plan-api endpoints too ? 



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -204,8 +211,11 @@ public <T extends RESTResponse> T handleRequest(
       case CREATE_NAMESPACE:
         if (asNamespaceCatalog != null) {
           CreateNamespaceRequest request = 
castRequest(CreateNamespaceRequest.class, body);
-          return castResponse(
-              responseType, 
CatalogHandlers.createNamespace(asNamespaceCatalog, request));
+          return CatalogHandlers.withIdempotency(

Review Comment:
   can we add this for mutable remote scan plan endpoint ?



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -122,6 +124,11 @@ public RESTCatalogAdapter(Catalog catalog) {
     this.asViewCatalog = catalog instanceof ViewCatalog ? (ViewCatalog) 
catalog : null;
   }
 
+  /** Test helper to simulate a transient 503 after the first successful 
mutation for a key. */
+  public void simulate503OnFirstSuccessForKey(String key) {

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



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -3312,6 +3319,160 @@ public void 
testClientDoesNotSendIdempotencyWhenServerNotAdvertising() {
     local.dropTable(ident);
   }
 
+  @Test
+  public void testIdempotentDuplicateCreateReturnsCached() {
+    String key = "dup-create-key";
+    Namespace ns = Namespace.of("ns_dup");
+    TableIdentifier ident = TableIdentifier.of(ns, "t_dup");
+    restCatalog.createNamespace(ns, ImmutableMap.of());
+    Pair<RESTClient, Map<String, String>> httpAndHeaders = httpAndHeaders(key);
+    RESTClient http = httpAndHeaders.first();
+    Map<String, String> headers = httpAndHeaders.second();
+    CreateTableRequest req = createReq(ident);
+
+    // First create succeeds
+    LoadTableResponse first =
+        http.post(
+            ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+            req,
+            LoadTableResponse.class,
+            headers,
+            ErrorHandlers.tableErrorHandler());
+    assertThat(first).isNotNull();
+
+    // Verify request shape (method, path, headers including Idempotency-Key)
+    verifyCreatePost(ns, headers);
+
+    // Duplicate with same key returns cached 200 OK
+    LoadTableResponse second =
+        http.post(
+            ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+            req,
+            LoadTableResponse.class,
+            headers,
+            ErrorHandlers.tableErrorHandler());
+    assertThat(second).isNotNull();
+
+    // Clean up
+    restCatalog.dropTable(ident);
+  }
+
+  @Test
+  public void testIdempotencyKeyLifetimeExpiredTreatsAsNew() {
+    // Set TTL to 0 so cached success expires immediately
+    CatalogHandlers.setIdempotencyLifetimeFromIso("PT0S");
+    try {
+      String key = "expired-create-key";
+      Namespace ns = Namespace.of("ns_exp");
+      TableIdentifier ident = TableIdentifier.of(ns, "t_exp");
+      restCatalog.createNamespace(ns, ImmutableMap.of());
+      Pair<RESTClient, Map<String, String>> httpAndHeaders = 
httpAndHeaders(key);
+      RESTClient http = httpAndHeaders.first();
+      Map<String, String> headers = httpAndHeaders.second();
+      CreateTableRequest req = createReq(ident);
+
+      // First create succeeds
+      LoadTableResponse created =
+          http.post(
+              ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+              req,
+              LoadTableResponse.class,
+              headers,
+              ErrorHandlers.tableErrorHandler());
+      assertThat(created).isNotNull();
+
+      // Verify request shape (method, path, headers including Idempotency-Key)
+      verifyCreatePost(ns, headers);
+
+      // TTL expired -> duplicate with same key should be treated as new and 
fail with AlreadyExists
+      assertThatThrownBy(
+              () ->
+                  http.post(
+                      
ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+                      req,
+                      LoadTableResponse.class,
+                      headers,
+                      ErrorHandlers.tableErrorHandler()))
+          .isInstanceOf(AlreadyExistsException.class)
+          .hasMessageContaining(ident.toString());
+
+      // Clean up
+      restCatalog.dropTable(ident);
+    } finally {
+      // Restore default TTL for other tests
+      CatalogHandlers.setIdempotencyLifetimeFromIso("PT30M");
+    }
+  }
+
+  @Test
+  public void testIdempotentCreateReplayAfterSimulated503() {
+    // Use a fixed key and simulate 503 after first success for that key
+    String key = "idemp-create-503";
+    adapterForRESTServer.simulate503OnFirstSuccessForKey(key);
+    Namespace ns = Namespace.of("ns_idemp");
+    TableIdentifier ident = TableIdentifier.of(ns, "t_idemp");
+    restCatalog.createNamespace(ns, ImmutableMap.of());
+    Pair<RESTClient, Map<String, String>> httpAndHeaders = httpAndHeaders(key);
+    RESTClient http = httpAndHeaders.first();
+    Map<String, String> headers = httpAndHeaders.second();
+    CreateTableRequest req = createReq(ident);
+
+    // First attempt: server finalizes success but responds 503
+    assertThatThrownBy(
+            () ->
+                http.post(
+                    
ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+                    req,
+                    LoadTableResponse.class,
+                    headers,
+                    ErrorHandlers.tableErrorHandler()))
+        .isInstanceOf(RuntimeException.class)
+        .hasMessageContaining("simulated transient 503");
+
+    // Verify request shape (method, path, headers including Idempotency-Key)
+    verifyCreatePost(ns, headers);
+
+    // Retry with same key: server should replay 200 OK
+    LoadTableResponse replay =
+        http.post(
+            ResourcePaths.forCatalogProperties(ImmutableMap.of()).tables(ns),
+            req,
+            LoadTableResponse.class,
+            headers,
+            ErrorHandlers.tableErrorHandler());
+    assertThat(replay).isNotNull();
+
+    // Clean up
+    restCatalog.dropTable(ident);
+  }
+
+  @Test
+  public void testIdempotentDropDuplicateNoop() {
+    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();

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



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to