You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2023/11/07 19:27:15 UTC

(solr) branch main updated: SOLR-16397: Tweak v2 'swapcore' API to be more REST-ful

This is an automated email from the ASF dual-hosted git repository.

gerlowskija pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new 88dbe12873d SOLR-16397: Tweak v2 'swapcore' API to be more REST-ful
88dbe12873d is described below

commit 88dbe12873d0474f48e087194a33e8fc411849a9
Author: Sanjay Dutt <sa...@gmail.com>
AuthorDate: Wed Nov 8 00:57:09 2023 +0530

    SOLR-16397: Tweak v2 'swapcore' API to be more REST-ful
    
    This commit changes the v2 "swapcore" API to be more in line
    with the REST-ful design we're targeting for Solr's v2 APIs.
    
    Following these changes, the v2 API now appears as:
      `POST /api/cores/coreName/swap {...}`
    
    Although not shown above, the 'swap' command specifier
    has been removed from the request body.
    
    This commit also converts the API to the new JAX-RS framework.
    
    ---------
    
    Co-authored-by: iamsanjay <sa...@yahoo.com>
    Co-authored-by: Jason Gerlowski <ge...@apache.org>
---
 solr/CHANGES.txt                                   |  3 +
 .../solr/client/api/endpoint/SwapCoresApi.java     | 43 ++++++++++++
 .../client/api/model/SwapCoresRequestBody.java     | 30 +++++++++
 .../solr/handler/admin/CoreAdminHandler.java       |  6 +-
 .../solr/handler/admin/CoreAdminOperation.java     | 13 +++-
 .../apache/solr/handler/admin/api/SwapCores.java   | 59 ++++++++++++++++
 .../solr/handler/admin/api/SwapCoresAPI.java       | 78 ----------------------
 .../solr/handler/admin/TestCoreAdminApis.java      | 14 ----
 .../handler/admin/api/V2CoreAPIMappingTest.java    | 12 ----
 .../configuration-guide/pages/coreadmin-api.adoc   | 24 +++++++
 10 files changed, 173 insertions(+), 109 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ea34d79b657..15f70ad21c7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -89,6 +89,9 @@ Improvements
 
 * SOLR-17046: SchemaCodecFactory is now the implicit default codec factory.  (hossman)
 
+* SOLR-16397: Swap core v2 endpoints have been updated to be more REST-ful.
+  SWAP is now available at `POST /api/cores/coreName/swap` (Sanjay Dutt via Jason Gerlowski)
+
 Optimizations
 ---------------------
 (No changes)
diff --git a/solr/api/src/java/org/apache/solr/client/api/endpoint/SwapCoresApi.java b/solr/api/src/java/org/apache/solr/client/api/endpoint/SwapCoresApi.java
new file mode 100644
index 00000000000..23e8b7f8f14
--- /dev/null
+++ b/solr/api/src/java/org/apache/solr/client/api/endpoint/SwapCoresApi.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.api.endpoint;
+
+import io.swagger.v3.oas.annotations.Operation;
+import io.swagger.v3.oas.annotations.parameters.RequestBody;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import org.apache.solr.client.api.model.SolrJerseyResponse;
+import org.apache.solr.client.api.model.SwapCoresRequestBody;
+
+/**
+ * V2 API for swapping two existing Solr cores.
+ *
+ * <p>Not intended for use in SolrCloud mode.
+ */
+@Path("/cores/{coreName}/swap")
+public interface SwapCoresApi {
+  @POST
+  @Operation(
+      summary = "SWAP atomically swaps the names used to access two existing Solr cores.",
+      tags = {"cores"})
+  SolrJerseyResponse swapCores(
+      @PathParam("coreName") String coreName,
+      @RequestBody(description = "Additional properties related to core swapping.")
+          SwapCoresRequestBody requestBody)
+      throws Exception;
+}
diff --git a/solr/api/src/java/org/apache/solr/client/api/model/SwapCoresRequestBody.java b/solr/api/src/java/org/apache/solr/client/api/model/SwapCoresRequestBody.java
new file mode 100644
index 00000000000..7ab42eb20ce
--- /dev/null
+++ b/solr/api/src/java/org/apache/solr/client/api/model/SwapCoresRequestBody.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.api.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.media.Schema;
+
+public class SwapCoresRequestBody {
+  @Schema(description = "The name of the other core to be swapped.")
+  @JsonProperty
+  public String with;
+
+  @Schema(description = "Request ID to track this action which will be processed asynchronously.")
+  @JsonProperty
+  public String async;
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index c0759543da9..60c83d10d15 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -73,7 +73,7 @@ import org.apache.solr.handler.admin.api.RequestSyncShardAPI;
 import org.apache.solr.handler.admin.api.RestoreCore;
 import org.apache.solr.handler.admin.api.SingleCoreStatusAPI;
 import org.apache.solr.handler.admin.api.SplitCoreAPI;
-import org.apache.solr.handler.admin.api.SwapCoresAPI;
+import org.apache.solr.handler.admin.api.SwapCores;
 import org.apache.solr.handler.admin.api.UnloadCore;
 import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrMetricManager;
@@ -383,7 +383,6 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     apis.addAll(AnnotatedApi.getApis(new CreateCoreAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new RejoinLeaderElectionAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new OverseerOperationAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new SwapCoresAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new RenameCoreAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new MergeIndexesAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SplitCoreAPI(this)));
@@ -406,7 +405,8 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
         BackupCoreAPI.class,
         RestoreCore.class,
         ReloadCore.class,
-        UnloadCore.class);
+        UnloadCore.class,
+        SwapCores.class);
   }
 
   public interface CoreAdminOp {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index c1580cdc5bb..c8947803400 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -58,9 +58,11 @@ import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.util.Locale;
 import java.util.Map;
+import org.apache.solr.client.api.endpoint.SwapCoresApi;
 import org.apache.solr.client.api.model.ListCoreSnapshotsResponse;
 import org.apache.solr.client.api.model.ReloadCoreRequestBody;
 import org.apache.solr.client.api.model.SolrJerseyResponse;
+import org.apache.solr.client.api.model.SwapCoresRequestBody;
 import org.apache.solr.client.api.model.UnloadCoreRequestBody;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -77,6 +79,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.admin.CoreAdminHandler.CoreAdminOp;
 import org.apache.solr.handler.admin.api.CoreSnapshot;
 import org.apache.solr.handler.admin.api.ReloadCore;
+import org.apache.solr.handler.admin.api.SwapCores;
 import org.apache.solr.handler.admin.api.UnloadCore;
 import org.apache.solr.handler.api.V2ApiUtils;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -155,8 +158,14 @@ public enum CoreAdminOperation implements CoreAdminOp {
       it -> {
         final SolrParams params = it.req.getParams();
         final String cname = params.required().get(CoreAdminParams.CORE);
-        String other = params.required().get(CoreAdminParams.OTHER);
-        it.handler.coreContainer.swap(cname, other);
+        final var swapCoresRequestBody = new SwapCoresRequestBody();
+        swapCoresRequestBody.with = params.required().get(CoreAdminParams.OTHER);
+        ;
+        SwapCoresApi swapCoresApi =
+            new SwapCores(
+                it.handler.coreContainer, it.handler.getCoreAdminAsyncTracker(), it.req, it.rsp);
+        SolrJerseyResponse response = swapCoresApi.swapCores(cname, swapCoresRequestBody);
+        V2ApiUtils.squashIntoSolrResponseWithoutHeader(it.rsp, response);
       }),
 
   RENAME_OP(
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCores.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCores.java
new file mode 100644
index 00000000000..eff87bc3f86
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCores.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.admin.api;
+
+import static org.apache.solr.security.PermissionNameProvider.Name.CORE_EDIT_PERM;
+
+import javax.inject.Inject;
+import org.apache.solr.client.api.endpoint.SwapCoresApi;
+import org.apache.solr.client.api.model.SolrJerseyResponse;
+import org.apache.solr.client.api.model.SwapCoresRequestBody;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+public class SwapCores extends CoreAdminAPIBase implements SwapCoresApi {
+  @Inject
+  public SwapCores(
+      CoreContainer coreContainer,
+      CoreAdminHandler.CoreAdminAsyncTracker coreAdminAsyncTracker,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, coreAdminAsyncTracker, solrQueryRequest, solrQueryResponse);
+  }
+
+  @PermissionName(CORE_EDIT_PERM)
+  @Override
+  public SolrJerseyResponse swapCores(String coreName, final SwapCoresRequestBody requestBody)
+      throws Exception {
+    ensureRequiredParameterProvided("coreName", coreName);
+    ensureRequiredRequestBodyProvided(requestBody);
+    ensureRequiredParameterProvided("with", requestBody.with);
+    SolrJerseyResponse solrJerseyResponse = instantiateJerseyResponse(SolrJerseyResponse.class);
+    return handlePotentiallyAsynchronousTask(
+        solrJerseyResponse,
+        coreName,
+        requestBody.async,
+        "swap",
+        () -> {
+          coreContainer.swap(coreName, requestBody.with);
+          return solrJerseyResponse;
+        });
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCoresAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCoresAPI.java
deleted file mode 100644
index e130b6e5540..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SwapCoresAPI.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.handler.admin.api;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
-import static org.apache.solr.security.PermissionNameProvider.Name.CORE_EDIT_PERM;
-
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-import org.apache.solr.api.Command;
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.api.PayloadObj;
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.util.ReflectMapWriter;
-import org.apache.solr.handler.admin.CoreAdminHandler;
-
-/**
- * V2 API for swapping two existing Solr cores.
- *
- * <p>Not intended for use in SolrCloud mode.
- *
- * <p>The new API (POST /v2/cores/coreName {'swap': {...}}) is equivalent to the v1
- * /admin/cores?action=swap command.
- */
-@EndPoint(
-    path = {"/cores/{core}"},
-    method = POST,
-    permission = CORE_EDIT_PERM)
-public class SwapCoresAPI {
-  private static final String V2_SWAP_CORES_CMD = "swap";
-
-  private final CoreAdminHandler coreHandler;
-
-  public SwapCoresAPI(CoreAdminHandler coreHandler) {
-    this.coreHandler = coreHandler;
-  }
-
-  @Command(name = V2_SWAP_CORES_CMD)
-  public void swapCores(PayloadObj<SwapCoresPayload> obj) throws Exception {
-    final SwapCoresPayload v2Body = obj.get();
-    final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
-    v1Params.put(
-        CoreAdminParams.ACTION,
-        CoreAdminParams.CoreAdminAction.SWAP.name().toLowerCase(Locale.ROOT));
-    v1Params.put(
-        CoreAdminParams.CORE, obj.getRequest().getPathTemplateValues().get(CoreAdminParams.CORE));
-
-    // V1 API uses 'other' instead of 'with' to represent the second/replacement core.
-    v1Params.put(CoreAdminParams.OTHER, v1Params.remove("with"));
-
-    coreHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
-  }
-
-  public static class SwapCoresPayload implements ReflectMapWriter {
-    @JsonProperty(required = true)
-    public String with;
-
-    @JsonProperty public String async;
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
index 3cf5b06261d..825cb996a5e 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
@@ -58,12 +58,6 @@ public class TestCoreAdminApis extends SolrTestCaseJ4 {
     assertEquals("hello", params[0]);
     assertEquals(fromJSONString("{schema : schema.xml}"), params[2]);
 
-    TestCollectionAPIs.makeCall(
-        apiBag, "/cores/core1", SolrRequest.METHOD.POST, "{swap:{with: core2}}");
-    params = calls.get("swap");
-    assertEquals("core1", params[0]);
-    assertEquals("core2", params[1]);
-
     TestCollectionAPIs.makeCall(
         apiBag, "/cores/core1", SolrRequest.METHOD.POST, "{rename:{to: core2}}");
     params = calls.get("rename");
@@ -84,14 +78,6 @@ public class TestCoreAdminApis extends SolrTestCaseJ4 {
               return null;
             });
 
-    doAnswer(
-            invocationOnMock -> {
-              in.put("swap", invocationOnMock.getArguments());
-              return null;
-            })
-        .when(mockCC)
-        .swap(any(String.class), any(String.class));
-
     doAnswer(
             invocationOnMock -> {
               in.put("rename", invocationOnMock.getArguments());
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/V2CoreAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/V2CoreAPIMappingTest.java
index ed2ea3079ee..83befc7d523 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/api/V2CoreAPIMappingTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/V2CoreAPIMappingTest.java
@@ -64,7 +64,6 @@ public class V2CoreAPIMappingTest extends V2ApiMappingTest<CoreAdminHandler> {
   @Override
   public void populateApiBag() {
     final CoreAdminHandler handler = getRequestHandler();
-    apiBag.registerObject(new SwapCoresAPI(handler));
     apiBag.registerObject(new RenameCoreAPI(handler));
     apiBag.registerObject(new MergeIndexesAPI(handler));
     apiBag.registerObject(new SplitCoreAPI(handler));
@@ -76,17 +75,6 @@ public class V2CoreAPIMappingTest extends V2ApiMappingTest<CoreAdminHandler> {
     apiBag.registerObject(new RequestCoreCommandStatusAPI(handler));
   }
 
-  @Test
-  public void testSwapCoresAllParams() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/cores/coreName", "POST", "{\"swap\": {\"with\": \"otherCore\"}}");
-
-    assertEquals("swap", v1Params.get(ACTION));
-    assertEquals("coreName", v1Params.get(CORE));
-    assertEquals("otherCore", v1Params.get(OTHER));
-  }
-
   @Test
   public void testRenameCoreAllParams() throws Exception {
     final SolrParams v1Params =
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
index aa4cc2f76a8..f2cf158ce95 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
@@ -390,7 +390,31 @@ This can be used to swap new content into production.
 The prior core remains available and can be swapped back, if necessary.
 Each core will be known by the name of the other, after the swap.
 
+[.dynamic-tabs]
+--
+[example.tab-pane#v1coreadmin-swap]
+====
+[.tab-label]*V1 API*
+
+[source,bash]
+----
 `admin/cores?action=SWAP&core=_core-name_&other=_other-core-name_`
+----
+====
+
+[example.tab-pane#v2coreadmin-reload]
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+`curl -X POST http://localhost:8983/api/cores/_core-name_/swap -H 'Content-Type: application/json' -d '
+  {
+    "with": "_other-core-name_"
+  }
+'`
+----
+====
 
 [IMPORTANT]
 ====