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/09/22 19:19:22 UTC

[solr] branch branch_9x updated: SOLR-16397: Convert reload-core API to JAX-RS (#1903)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 5bf79fd1864 SOLR-16397: Convert reload-core API to JAX-RS (#1903)
5bf79fd1864 is described below

commit 5bf79fd18648a10d612008f74e4d00d6b1b5301f
Author: Sanjay Dutt <sa...@gmail.com>
AuthorDate: Wed Sep 20 00:53:27 2023 +0530

    SOLR-16397: Convert reload-core API to JAX-RS (#1903)
    
    The API is now available at `POST /api/cores/coreName/reload`
    
    ----
    Co-authored-by: iamsanjay <sa...@yahoo.com>
    Co-authored-by: Jason Gerlowski <ge...@apache.org>
---
 solr/CHANGES.txt                                   |  3 +
 .../solr/handler/admin/CoreAdminHandler.java       |  7 +-
 .../solr/handler/admin/CoreAdminOperation.java     | 10 ++-
 .../solr/handler/admin/api/ReloadCoreAPI.java      | 87 +++++++++++++---------
 .../solr/handler/admin/api/ReloadCoreAPITest.java  | 81 ++++++++++++++++++++
 .../handler/admin/api/V2CoreAPIMappingTest.java    | 10 ---
 .../configuration-guide/pages/coreadmin-api.adoc   |  6 +-
 7 files changed, 151 insertions(+), 53 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 934a130521d..9978cea69ff 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -354,6 +354,9 @@ Improvements
 
 * SOLR-16878: Use Log4J JUL manager when starting Java. This is necessary for Lucene logs to be included with Solr logs. (Houston Putman, Uwe Schindler)
 
+* SOLR-16397: Reload core v2 endpoints have been updated to be more REST-ful.
+  RELOAD is now available at `POST /api/cores/coreName/reload` (Sanjay Dutt via Jason Gerlowski)
+
 Optimizations
 ---------------------
 
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 8ee1bf3254a..d928282df5c 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
@@ -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 ReloadCoreAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SwapCoresAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new RenameCoreAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new UnloadCoreAPI(this)));
@@ -403,7 +402,11 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   @Override
   public Collection<Class<? extends JerseyResource>> getJerseyResources() {
     return List.of(
-        CoreSnapshotAPI.class, InstallCoreDataAPI.class, BackupCoreAPI.class, RestoreCoreAPI.class);
+        CoreSnapshotAPI.class,
+        InstallCoreDataAPI.class,
+        BackupCoreAPI.class,
+        RestoreCoreAPI.class,
+        ReloadCoreAPI.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 31251759c76..987c79d376b 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,6 +58,7 @@ import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.util.Locale;
 import java.util.Map;
+import org.apache.solr.client.api.model.SolrJerseyResponse;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -72,6 +73,7 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.admin.CoreAdminHandler.CoreAdminOp;
 import org.apache.solr.handler.admin.api.CoreSnapshotAPI;
+import org.apache.solr.handler.admin.api.ReloadCoreAPI;
 import org.apache.solr.handler.api.V2ApiUtils;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.UpdateLog;
@@ -133,7 +135,13 @@ public enum CoreAdminOperation implements CoreAdminOp {
         SolrParams params = it.req.getParams();
         String cname = params.required().get(CoreAdminParams.CORE);
 
-        it.handler.coreContainer.reload(cname);
+        ReloadCoreAPI reloadCoreAPI =
+            new ReloadCoreAPI(
+                it.req, it.rsp, it.handler.coreContainer, it.handler.getCoreAdminAsyncTracker());
+        ReloadCoreAPI.ReloadCoreRequestBody reloadCoreRequestBody =
+            new ReloadCoreAPI.ReloadCoreRequestBody();
+        SolrJerseyResponse response = reloadCoreAPI.reloadCore(cname, reloadCoreRequestBody);
+        V2ApiUtils.squashIntoSolrResponseWithoutHeader(it.rsp, response);
       }),
   STATUS_OP(STATUS, new StatusOp()),
   SWAP_OP(
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCoreAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCoreAPI.java
index e330d713340..b0252b83fa9 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCoreAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCoreAPI.java
@@ -17,52 +17,69 @@
 
 package org.apache.solr.handler.admin.api;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CommonParams.ACTION;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
 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.params.CoreAdminParams;
-import org.apache.solr.common.util.ReflectMapWriter;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.Parameter;
+import io.swagger.v3.oas.annotations.media.Schema;
+import io.swagger.v3.oas.annotations.parameters.RequestBody;
+import javax.inject.Inject;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import org.apache.solr.client.api.model.SolrJerseyResponse;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
 
 /**
  * V2 API for reloading an individual core.
  *
- * <p>The new API (POST /v2/cores/coreName {'reload': {...}}) is equivalent to the v1
- * /admin/cores?action=reload command.
- *
- * @see ReloadCorePayload
+ * <p>The new API (POST /v2/cores/coreName/reload is analogous to the v1 /admin/cores?action=RELOAD
+ * command.
  */
-@EndPoint(
-    path = {"/cores/{core}"},
-    method = POST,
-    permission = CORE_EDIT_PERM)
-public class ReloadCoreAPI {
-  private static final String V2_RELOAD_CORE_CMD = "reload";
-
-  private final CoreAdminHandler coreHandler;
+@Path("/cores/{coreName}/reload")
+public class ReloadCoreAPI extends CoreAdminAPIBase {
 
-  public ReloadCoreAPI(CoreAdminHandler coreHandler) {
-    this.coreHandler = coreHandler;
+  @Inject
+  public ReloadCoreAPI(
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse,
+      CoreContainer coreContainer,
+      CoreAdminHandler.CoreAdminAsyncTracker coreAdminAsyncTracker) {
+    super(coreContainer, coreAdminAsyncTracker, solrQueryRequest, solrQueryResponse);
   }
 
-  @Command(name = V2_RELOAD_CORE_CMD)
-  public void reloadCore(PayloadObj<ReloadCorePayload> obj) throws Exception {
-    final String coreName = obj.getRequest().getPathTemplateValues().get(CoreAdminParams.CORE);
-
-    final Map<String, Object> v1Params = new HashMap<>();
-    v1Params.put(ACTION, CoreAdminParams.CoreAdminAction.RELOAD.name().toLowerCase(Locale.ROOT));
-    v1Params.put(CoreAdminParams.CORE, coreName);
-
-    coreHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+  @POST
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(CORE_EDIT_PERM)
+  public SolrJerseyResponse reloadCore(
+      @Parameter(description = "The name of the core to reload.", required = true)
+          @PathParam("coreName")
+          String coreName,
+      @Schema(description = "Additional parameters for reloading the core") @RequestBody
+          ReloadCoreAPI.ReloadCoreRequestBody reloadCoreRequestBody)
+      throws Exception {
+    SolrJerseyResponse solrJerseyResponse = instantiateJerseyResponse(SolrJerseyResponse.class);
+    return handlePotentiallyAsynchronousTask(
+        solrJerseyResponse,
+        coreName,
+        (reloadCoreRequestBody == null) ? null : reloadCoreRequestBody.async,
+        "reload",
+        () -> {
+          coreContainer.reload(coreName);
+          return solrJerseyResponse;
+        });
   }
 
-  public static class ReloadCorePayload implements ReflectMapWriter {}
+  public static class ReloadCoreRequestBody implements JacksonReflectMapWriter {
+    @Schema(description = "Request ID to track this action which will be processed asynchronously.")
+    @JsonProperty("async")
+    public String async;
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCoreAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCoreAPITest.java
new file mode 100644
index 00000000000..cd0fdc32324
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCoreAPITest.java
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.api.model.SolrJerseyResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class ReloadCoreAPITest extends SolrTestCaseJ4 {
+
+  private ReloadCoreAPI reloadCoreAPI;
+  private static final String NON_EXISTENT_CORE = "non_existent_core";
+
+  @BeforeClass
+  public static void initializeCoreAndRequestFactory() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+    lrf = h.getRequestFactory("/api", 0, 10);
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    SolrQueryRequest solrQueryRequest = req();
+    SolrQueryResponse solrQueryResponse = new SolrQueryResponse();
+    CoreContainer coreContainer = h.getCoreContainer();
+    CoreAdminHandler.CoreAdminAsyncTracker coreAdminAsyncTracker =
+        new CoreAdminHandler.CoreAdminAsyncTracker();
+    reloadCoreAPI =
+        new ReloadCoreAPI(
+            solrQueryRequest, solrQueryResponse, coreContainer, coreAdminAsyncTracker);
+  }
+
+  @Test
+  public void testValidReloadCoreAPIResponse() throws Exception {
+    SolrJerseyResponse response =
+        reloadCoreAPI.reloadCore(coreName, new ReloadCoreAPI.ReloadCoreRequestBody());
+    assertEquals(0, response.responseHeader.status);
+    assertNotNull(response.responseHeader.qTime);
+  }
+
+  @Test
+  public void testNonExistentCoreExceptionResponse() {
+    final SolrException solrException =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              reloadCoreAPI.reloadCore(
+                  NON_EXISTENT_CORE, new ReloadCoreAPI.ReloadCoreRequestBody());
+            });
+    assertEquals(400, solrException.code());
+    assertTrue(solrException.getMessage().contains("No such core: " + NON_EXISTENT_CORE));
+  }
+
+  @AfterClass // unique core per test
+  public static void coreDestroy() {
+    deleteCore();
+  }
+}
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 e6ecf9fd903..a2a36af2c4f 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
@@ -67,7 +67,6 @@ public class V2CoreAPIMappingTest extends V2ApiMappingTest<CoreAdminHandler> {
   @Override
   public void populateApiBag() {
     final CoreAdminHandler handler = getRequestHandler();
-    apiBag.registerObject(new ReloadCoreAPI(handler));
     apiBag.registerObject(new SwapCoresAPI(handler));
     apiBag.registerObject(new RenameCoreAPI(handler));
     apiBag.registerObject(new UnloadCoreAPI(handler));
@@ -81,15 +80,6 @@ public class V2CoreAPIMappingTest extends V2ApiMappingTest<CoreAdminHandler> {
     apiBag.registerObject(new RequestCoreCommandStatusAPI(handler));
   }
 
-  @Test
-  public void testReloadCoreAllParams() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params("/cores/coreName", "POST", "{\"reload\": {}}");
-
-    assertEquals("reload", v1Params.get(ACTION));
-    assertEquals("coreName", v1Params.get(CORE));
-  }
-
   @Test
   public void testSwapCoresAllParams() 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 cd33ea975e6..a62e4288199 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
@@ -318,11 +318,7 @@ http://localhost:8983/solr/admin/cores?action=RELOAD&core=techproducts
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/cores/techproducts -H 'Content-Type: application/json' -d '
-  {
-    "reload": {}
-  }
-'
+curl -X POST http://localhost:8983/api/cores/techproducts/reload
 ----
 ====
 --