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/06/17 15:42:22 UTC

[solr] 02/03: SOLR-16398: Tweak 4 v2 collection APIs to be more REST-ful (#1683)

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

commit 7c71f99071deb74a48c9a19485dbcb7522c232e6
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Tue Jun 6 15:47:35 2023 -0400

    SOLR-16398: Tweak 4 v2 collection APIs to be more REST-ful (#1683)
    
    This commit tweaks several "command"-based v2 APIs to be more REST-ful, namely reload-collection, sync-shard, rename-collection, and force-shard-leader.  The v2 APIs for this functionality is now:
    
    - POST /api/collections/cName/reload {...}
    - POST /api/collections/cName/shards/sName/sync {...}
    - POST /api/collections/cName/rename {...}
    - POST /api/collections/cName/shards/sName/force-leader {...}
    
    This commit also converts these APIs to the JAX-RS framework.
---
 solr/CHANGES.txt                                   |   5 +
 ...istributedCollectionConfigSetCommandRunner.java |   7 +-
 .../solr/handler/admin/CollectionsHandler.java     | 138 +---------------
 .../solr/handler/admin/api/ForceLeaderAPI.java     | 184 +++++++++++++++++----
 .../handler/admin/api/ReloadCollectionAPI.java     | 108 ++++++++----
 .../handler/admin/api/RenameCollectionAPI.java     | 141 +++++++++-------
 .../solr/handler/admin/api/SyncShardAPI.java       | 117 +++++++++----
 .../solr/handler/admin/TestApiFramework.java       |   9 -
 .../solr/handler/admin/TestCollectionAPIs.java     |   3 -
 .../solr/handler/admin/api/ForceLeaderAPITest.java |  53 ++++++
 .../handler/admin/api/ReloadCollectionAPITest.java |  57 +++++++
 .../solr/handler/admin/api/SyncShardAPITest.java   |  53 ++++++
 .../admin/api/V2CollectionAPIMappingTest.java      |  30 ----
 .../handler/admin/api/V2ShardsAPIMappingTest.java  |  25 ---
 .../solr/util/tracing/TestDistributedTracing.java  |   6 +-
 .../pages/collection-management.adoc               |  27 +--
 .../deployment-guide/pages/shard-management.adoc   |   6 +-
 .../solrj/request/beans/ForceLeaderPayload.java    |  25 ---
 .../request/beans/ReloadCollectionPayload.java     |  24 ---
 .../request/beans/RenameCollectionPayload.java     |  29 ----
 .../solrj/request/beans/SyncShardPayload.java      |  23 ---
 21 files changed, 583 insertions(+), 487 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dce082498f0..7b8e9eb0450 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -133,6 +133,11 @@ Improvements
   changing the path.  The v2 functionality can now be accessed at: `POST /api/collections/cName/shards/sName/replicas {...}`
   (Jason Gerlowski)
 
+* SOLR-16398: Several v2 "command" APIs have been tweaked to be more intuitive. "FORCELEADER" is now available at
+  `POST /api/collections/cName/shards/sName/force-leader`, "RELOAD" is now available at `POST /api/collections/cName/reload`,
+  "SYNCSHARD" is now available at `POST /api/collections/cName/shards/sName/sync-shard`, and "RENAME" is now available at
+  `POST /api/collections/cName/rename`. (Jason Gerlowski)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DistributedCollectionConfigSetCommandRunner.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DistributedCollectionConfigSetCommandRunner.java
index 9c7697caef2..726d13812bf 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DistributedCollectionConfigSetCommandRunner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DistributedCollectionConfigSetCommandRunner.java
@@ -275,10 +275,9 @@ public class DistributedCollectionConfigSetCommandRunner {
     // Happens either in the CollectionCommandRunner below or in the catch when the runner would not
     // execute.
     if (!asyncTaskTracker.createNewAsyncJobTracker(asyncId)) {
-      NamedList<Object> resp = new NamedList<>();
-      resp.add("error", "Task with the same requestid already exists. (" + asyncId + ")");
-      resp.add(CoreAdminParams.REQUESTID, asyncId);
-      return new OverseerSolrResponse(resp);
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "Task with the same requestid already exists. (" + asyncId + ")");
     }
 
     CollectionCommandRunner commandRunner = new CollectionCommandRunner(message, action, asyncId);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index eefe2953275..eef39bee14d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -121,15 +121,11 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
 import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.JerseyResource;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrResponse;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
 import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.cloud.OverseerSolrResponse;
 import org.apache.solr.cloud.OverseerSolrResponseSerializer;
@@ -137,20 +133,17 @@ import org.apache.solr.cloud.OverseerTaskQueue;
 import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkController.NotInClusterStateException;
-import org.apache.solr.cloud.ZkShardTerms;
 import org.apache.solr.cloud.api.collections.DistributedCollectionConfigSetCommandRunner;
 import org.apache.solr.cloud.api.collections.ReindexCollectionCmd;
 import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterProperties;
-import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -567,8 +560,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     RELOAD_OP(
         RELOAD,
         (req, rsp, h) -> {
-          Map<String, Object> map = copy(req.getParams().required(), null, NAME);
-          return copy(req.getParams(), map);
+          ReloadCollectionAPI.invokeFromV1Params(h.coreContainer, req, rsp);
+          return null;
         }),
 
     RENAME_OP(
@@ -614,32 +607,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     SYNCSHARD_OP(
         SYNCSHARD,
         (req, rsp, h) -> {
-          String extCollection = req.getParams().required().get("collection");
-          String collection =
-              h.coreContainer
-                  .getZkController()
-                  .getZkStateReader()
-                  .getAliases()
-                  .resolveSimpleAlias(extCollection);
-          String shard = req.getParams().required().get("shard");
-
-          ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
-
-          DocCollection docCollection = clusterState.getCollection(collection);
-          ZkNodeProps leaderProps = docCollection.getLeader(shard);
-          ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
-
-          try (SolrClient client =
-              new Builder(nodeProps.getBaseUrl())
-                  .withConnectionTimeout(15000, TimeUnit.MILLISECONDS)
-                  .withSocketTimeout(60000, TimeUnit.MILLISECONDS)
-                  .build()) {
-            RequestSyncShard reqSyncShard = new RequestSyncShard();
-            reqSyncShard.setCollection(collection);
-            reqSyncShard.setShard(shard);
-            reqSyncShard.setCoreName(nodeProps.getCoreName());
-            client.request(reqSyncShard);
-          }
+          SyncShardAPI.invokeFromV1Params(h.coreContainer, req, rsp);
           return null;
         }),
 
@@ -755,7 +723,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     FORCELEADER_OP(
         FORCELEADER,
         (req, rsp, h) -> {
-          forceLeaderElection(req, h);
+          ForceLeaderAPI.invokeFromV1Params(h.coreContainer, req, rsp);
           return null;
         }),
     CREATESHARD_OP(
@@ -1319,96 +1287,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     }
   }
 
-  private static void forceLeaderElection(SolrQueryRequest req, CollectionsHandler handler) {
-    ZkController zkController = handler.coreContainer.getZkController();
-    ClusterState clusterState = zkController.getClusterState();
-    String extCollectionName = req.getParams().required().get(COLLECTION_PROP);
-    String collectionName =
-        zkController.zkStateReader.getAliases().resolveSimpleAlias(extCollectionName);
-    String sliceId = req.getParams().required().get(SHARD_ID_PROP);
-
-    log.info("Force leader invoked, state: {}", clusterState);
-    DocCollection collection = clusterState.getCollection(collectionName);
-    Slice slice = collection.getSlice(sliceId);
-    if (slice == null) {
-      throw new SolrException(
-          ErrorCode.BAD_REQUEST,
-          "No shard with name " + sliceId + " exists for collection " + collectionName);
-    }
-
-    try (ZkShardTerms zkShardTerms =
-        new ZkShardTerms(collectionName, slice.getName(), zkController.getZkClient())) {
-      // if an active replica is the leader, then all is fine already
-      Replica leader = slice.getLeader();
-      if (leader != null && leader.getState() == State.ACTIVE) {
-        throw new SolrException(
-            ErrorCode.SERVER_ERROR,
-            "The shard already has an active leader. Force leader is not applicable. State: "
-                + slice);
-      }
-
-      final Set<String> liveNodes = clusterState.getLiveNodes();
-      List<Replica> liveReplicas =
-          slice.getReplicas().stream()
-              .filter(rep -> liveNodes.contains(rep.getNodeName()))
-              .collect(Collectors.toList());
-      boolean shouldIncreaseReplicaTerms =
-          liveReplicas.stream()
-              .noneMatch(
-                  rep ->
-                      zkShardTerms.registered(rep.getName())
-                          && zkShardTerms.canBecomeLeader(rep.getName()));
-      // we won't increase replica's terms if exist a live replica with term equals to leader
-      if (shouldIncreaseReplicaTerms) {
-        // TODO only increase terms of replicas less out-of-sync
-        liveReplicas.stream()
-            .filter(rep -> zkShardTerms.registered(rep.getName()))
-            // TODO should this all be done at once instead of increasing each replica individually?
-            .forEach(rep -> zkShardTerms.setTermEqualsToLeader(rep.getName()));
-      }
-
-      // Wait till we have an active leader
-      boolean success = false;
-      for (int i = 0; i < 9; i++) {
-        Thread.sleep(5000);
-        clusterState = handler.coreContainer.getZkController().getClusterState();
-        collection = clusterState.getCollection(collectionName);
-        slice = collection.getSlice(sliceId);
-        if (slice.getLeader() != null && slice.getLeader().getState() == State.ACTIVE) {
-          success = true;
-          break;
-        }
-        log.warn(
-            "Force leader attempt {}. Waiting 5 secs for an active leader. State of the slice: {}",
-            (i + 1),
-            slice); // nowarn
-      }
-
-      if (success) {
-        log.info(
-            "Successfully issued FORCELEADER command for collection: {}, shard: {}",
-            collectionName,
-            sliceId);
-      } else {
-        log.info(
-            "Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}",
-            collectionName,
-            sliceId,
-            clusterState);
-      }
-    } catch (SolrException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new SolrException(
-          ErrorCode.SERVER_ERROR,
-          "Error executing FORCELEADER operation for collection: "
-              + collectionName
-              + " shard: "
-              + sliceId,
-          e);
-    }
-  }
-
   public static void waitForActiveCollection(
       String collectionName, CoreContainer cc, SolrResponse createCollResponse)
       throws KeeperException, InterruptedException {
@@ -1517,12 +1395,16 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
         DeleteReplicaAPI.class,
         DeleteReplicaPropertyAPI.class,
         DeleteShardAPI.class,
+        ForceLeaderAPI.class,
         InstallShardDataAPI.class,
         ListCollectionsAPI.class,
         ListCollectionBackupsAPI.class,
+        ReloadCollectionAPI.class,
+        RenameCollectionAPI.class,
         ReplaceNodeAPI.class,
         BalanceReplicasAPI.class,
         RestoreCollectionAPI.class,
+        SyncShardAPI.class,
         CollectionPropertyAPI.class,
         DeleteNodeAPI.class,
         ListAliasesAPI.class,
@@ -1536,16 +1418,12 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   public Collection<Api> getApis() {
     final List<Api> apis = new ArrayList<>();
     apis.addAll(AnnotatedApi.getApis(new SplitShardAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new SyncShardAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new ForceLeaderAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new BalanceShardUniqueAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new MigrateDocsAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new ModifyCollectionAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new MoveReplicaAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new RebalanceLeadersAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new ReloadCollectionAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new CollectionStatusAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new RenameCollectionAPI(this)));
     return apis;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ForceLeaderAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ForceLeaderAPI.java
index c8bf6104770..fd32b52a1f7 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/ForceLeaderAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ForceLeaderAPI.java
@@ -17,52 +17,166 @@
 
 package org.apache.solr.handler.admin.api;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
-import static org.apache.solr.common.params.CollectionParams.ACTION;
-import static org.apache.solr.common.params.CoreAdminParams.SHARD;
-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.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 
-import java.util.HashMap;
-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.client.solrj.request.beans.ForceLeaderPayload;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.handler.admin.CollectionsHandler;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+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 javax.ws.rs.core.MediaType;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkShardTerms;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * V2 API for triggering a leader election on a particular collection and shard.
  *
- * <p>This API (POST /v2/collections/collectionName/shards/shardName {'force-leader': {}}) is
- * analogous to the v1 /admin/collections?action=FORCELEADER command.
- *
- * @see ForceLeaderPayload
+ * <p>This API (POST /v2/collections/collectionName/shards/shardName/force-leader) is analogous to
+ * the v1 /admin/collections?action=FORCELEADER command.
  */
-@EndPoint(
-    path = {"/c/{collection}/shards/{shard}", "/collections/{collection}/shards/{shard}"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class ForceLeaderAPI {
-  private static final String V2_FORCE_LEADER_CMD = "force-leader";
+@Path("/collections/{collectionName}/shards/{shardName}/force-leader")
+public class ForceLeaderAPI extends AdminAPIBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Inject
+  public ForceLeaderAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  @POST
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public SolrJerseyResponse forceLeader(
+      @PathParam("collectionName") String collectionName,
+      @PathParam("shardName") String shardName) {
+    final SolrJerseyResponse response = instantiateJerseyResponse(SolrJerseyResponse.class);
+    ensureRequiredParameterProvided(COLLECTION_PROP, collectionName);
+    ensureRequiredParameterProvided(SHARD_ID_PROP, shardName);
+    fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collectionName, solrQueryRequest);
+
+    doForceLeaderElection(collectionName, shardName);
+    return response;
+  }
 
-  private final CollectionsHandler collectionsHandler;
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest request, SolrQueryResponse response) {
+    final var api = new ForceLeaderAPI(coreContainer, request, response);
+    final var params = request.getParams();
+    params.required().check(COLLECTION_PROP, SHARD_ID_PROP);
 
-  public ForceLeaderAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        response, api.forceLeader(params.get(COLLECTION_PROP), params.get(SHARD_ID_PROP)));
   }
 
-  @Command(name = V2_FORCE_LEADER_CMD)
-  public void forceLeader(PayloadObj<ForceLeaderPayload> obj) throws Exception {
-    final Map<String, Object> addedV1Params = new HashMap<>();
-    final Map<String, String> pathParams = obj.getRequest().getPathTemplateValues();
-    addedV1Params.put(ACTION, CollectionParams.CollectionAction.FORCELEADER.toLower());
-    addedV1Params.put(COLLECTION, pathParams.get(COLLECTION));
-    addedV1Params.put(SHARD, pathParams.get(SHARD));
+  private void doForceLeaderElection(String extCollectionName, String shardName) {
+    ZkController zkController = coreContainer.getZkController();
+    ClusterState clusterState = zkController.getClusterState();
+    String collectionName =
+        zkController.zkStateReader.getAliases().resolveSimpleAlias(extCollectionName);
+
+    log.info("Force leader invoked, state: {}", clusterState);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shardName);
+    if (slice == null) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "No shard with name " + shardName + " exists for collection " + collectionName);
+    }
+
+    try (ZkShardTerms zkShardTerms =
+        new ZkShardTerms(collectionName, slice.getName(), zkController.getZkClient())) {
+      // if an active replica is the leader, then all is fine already
+      Replica leader = slice.getLeader();
+      if (leader != null && leader.getState() == Replica.State.ACTIVE) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "The shard already has an active leader. Force leader is not applicable. State: "
+                + slice);
+      }
+
+      final Set<String> liveNodes = clusterState.getLiveNodes();
+      List<Replica> liveReplicas =
+          slice.getReplicas().stream()
+              .filter(rep -> liveNodes.contains(rep.getNodeName()))
+              .collect(Collectors.toList());
+      boolean shouldIncreaseReplicaTerms =
+          liveReplicas.stream()
+              .noneMatch(
+                  rep ->
+                      zkShardTerms.registered(rep.getName())
+                          && zkShardTerms.canBecomeLeader(rep.getName()));
+      // we won't increase replica's terms if exist a live replica with term equals to leader
+      if (shouldIncreaseReplicaTerms) {
+        // TODO only increase terms of replicas less out-of-sync
+        liveReplicas.stream()
+            .filter(rep -> zkShardTerms.registered(rep.getName()))
+            // TODO should this all be done at once instead of increasing each replica individually?
+            .forEach(rep -> zkShardTerms.setTermEqualsToLeader(rep.getName()));
+      }
+
+      // Wait till we have an active leader
+      boolean success = false;
+      for (int i = 0; i < 9; i++) {
+        Thread.sleep(5000);
+        clusterState = coreContainer.getZkController().getClusterState();
+        collection = clusterState.getCollection(collectionName);
+        slice = collection.getSlice(shardName);
+        if (slice.getLeader() != null && slice.getLeader().getState() == Replica.State.ACTIVE) {
+          success = true;
+          break;
+        }
+        log.warn(
+            "Force leader attempt {}. Waiting 5 secs for an active leader. State of the slice: {}",
+            (i + 1),
+            slice); // nowarn
+      }
 
-    collectionsHandler.handleRequestBody(
-        wrapParams(obj.getRequest(), addedV1Params), obj.getResponse());
+      if (success) {
+        log.info(
+            "Successfully issued FORCELEADER command for collection: {}, shard: {}",
+            collectionName,
+            shardName);
+      } else {
+        log.info(
+            "Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}",
+            collectionName,
+            shardName,
+            clusterState);
+      }
+    } catch (SolrException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error executing FORCELEADER operation for collection: "
+              + collectionName
+              + " shard: "
+              + shardName,
+          e);
+    }
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCollectionAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCollectionAPI.java
index 00146f801fd..009ee727969 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCollectionAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ReloadCollectionAPI.java
@@ -16,50 +16,102 @@
  */
 package org.apache.solr.handler.admin.api;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
-import static org.apache.solr.common.params.CommonParams.ACTION;
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 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.client.solrj.request.beans.ReloadCollectionPayload;
+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 javax.ws.rs.core.MediaType;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SubResponseAccumulatingJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * V2 API for reloading collections.
  *
- * <p>The new API (POST /v2/collections/collectionName {'reload': {...}}) is analogous to the v1
+ * <p>The new API (POST /v2/collections/collectionName/reload {...}) is analogous to the v1
  * /admin/collections?action=RELOAD command.
- *
- * @see ReloadCollectionPayload
  */
-@EndPoint(
-    path = {"/c/{collection}", "/collections/{collection}"},
-    method = POST,
-    permission = COLL_EDIT_PERM) // TODO Does this permission make sense for reload?
-public class ReloadCollectionAPI {
-  private static final String V2_RELOAD_COLLECTION_CMD = "reload";
+@Path("/collections/{collectionName}/reload")
+public class ReloadCollectionAPI extends AdminAPIBase {
 
-  private final CollectionsHandler collectionsHandler;
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public ReloadCollectionAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+  @Inject
+  public ReloadCollectionAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
   }
 
-  @Command(name = V2_RELOAD_COLLECTION_CMD)
-  public void reloadCollection(PayloadObj<ReloadCollectionPayload> obj) throws Exception {
-    final ReloadCollectionPayload v2Body = obj.get();
-    final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.RELOAD.toLower());
-    v1Params.put(NAME, obj.getRequest().getPathTemplateValues().get(COLLECTION));
+  @POST
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public SubResponseAccumulatingJerseyResponse reloadCollection(
+      @PathParam("collectionName") String collectionName, ReloadCollectionRequestBody requestBody)
+      throws Exception {
+    final var response = instantiateJerseyResponse(SubResponseAccumulatingJerseyResponse.class);
+    ensureRequiredParameterProvided(COLLECTION_PROP, collectionName);
+    fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collectionName, solrQueryRequest);
+
+    final ZkNodeProps remoteMessage = createRemoteMessage(collectionName, requestBody);
+    submitRemoteMessageAndHandleResponse(
+        response,
+        CollectionParams.CollectionAction.RELOAD,
+        remoteMessage,
+        requestBody != null ? requestBody.asyncId : null);
+    return response;
+  }
+
+  public static ZkNodeProps createRemoteMessage(
+      String collectionName, ReloadCollectionRequestBody requestBody) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+    remoteMessage.put(QUEUE_OPERATION, CollectionParams.CollectionAction.RELOAD.toLower());
+    remoteMessage.put(NAME, collectionName);
+    if (requestBody != null) {
+      insertIfNotNull(remoteMessage, ASYNC, requestBody.asyncId);
+    }
+
+    return new ZkNodeProps(remoteMessage);
+  }
+
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest request, SolrQueryResponse response)
+      throws Exception {
+    final var api = new ReloadCollectionAPI(coreContainer, request, response);
+    final var params = request.getParams();
+    params.required().check(NAME);
+    final var requestBody = new ReloadCollectionRequestBody();
+    requestBody.asyncId = params.get(ASYNC); // Note, 'async' may or may not have been provided.
+
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        response, api.reloadCollection(params.get(NAME), requestBody));
+  }
 
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+  // TODO Is it worth having this in a request body, or should we just make it a query param?
+  public static class ReloadCollectionRequestBody implements JacksonReflectMapWriter {
+    @JsonProperty(ASYNC)
+    public String asyncId;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/RenameCollectionAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/RenameCollectionAPI.java
index de6e0106701..c63c518426b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/RenameCollectionAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/RenameCollectionAPI.java
@@ -16,87 +16,110 @@
  */
 package org.apache.solr.handler.admin.api;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
 import static org.apache.solr.common.params.CollectionAdminParams.TARGET;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
-import static org.apache.solr.common.params.CommonParams.ACTION;
 import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.MapperFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import java.io.IOException;
-import java.util.Locale;
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.client.solrj.request.beans.RenameCollectionPayload;
-import org.apache.solr.common.params.CollectionAdminParams;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.HashMap;
+import java.util.Map;
+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 javax.ws.rs.core.MediaType;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SubResponseAccumulatingJerseyResponse;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.SolrJacksonAnnotationInspector;
 
 /**
  * V2 API for "renaming" an existing collection
  *
  * <p>This API is analogous to the v1 /admin/collections?action=RENAME command.
  */
-public class RenameCollectionAPI {
+@Path("/collections/{collectionName}/rename")
+public class RenameCollectionAPI extends AdminAPIBase {
 
-  private final CollectionsHandler collectionsHandler;
-  private static final ObjectMapper REQUEST_BODY_PARSER =
-      SolrJacksonAnnotationInspector.createObjectMapper()
-          .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false)
-          .disable(MapperFeature.AUTO_DETECT_FIELDS);
+  @Inject
+  public RenameCollectionAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  @POST
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public SubResponseAccumulatingJerseyResponse renameCollection(
+      @PathParam("collectionName") String collectionName, RenameCollectionRequestBody requestBody)
+      throws Exception {
+    final var response = instantiateJerseyResponse(SubResponseAccumulatingJerseyResponse.class);
+    if (requestBody == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing required request body");
+    }
+    ensureRequiredParameterProvided(COLLECTION_PROP, collectionName);
+    ensureRequiredParameterProvided("to", requestBody.to);
+    fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collectionName, solrQueryRequest);
 
-  public RenameCollectionAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+    final ZkNodeProps remoteMessage = createRemoteMessage(collectionName, requestBody);
+    submitRemoteMessageAndHandleResponse(
+        response,
+        CollectionParams.CollectionAction.RENAME,
+        remoteMessage,
+        requestBody != null ? requestBody.asyncId : null);
+    return response;
   }
 
-  @EndPoint(
-      path = {"/collections/{collection}/rename"},
-      method = POST,
-      permission = COLL_EDIT_PERM)
-  public void renameCollection(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    final RenameCollectionPayload v2Body = parseRenameParamsFromRequestBody(req);
+  public static ZkNodeProps createRemoteMessage(
+      String collectionName, RenameCollectionRequestBody requestBody) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+    remoteMessage.put(QUEUE_OPERATION, CollectionParams.CollectionAction.RENAME.toLower());
+    remoteMessage.put(NAME, collectionName);
+    remoteMessage.put(TARGET, requestBody.to);
+    insertIfNotNull(remoteMessage, FOLLOW_ALIASES, requestBody.followAliases);
+    insertIfNotNull(remoteMessage, ASYNC, requestBody.asyncId);
 
-    req =
-        wrapParams(
-            req,
-            ACTION,
-            CollectionParams.CollectionAction.RENAME.name().toLowerCase(Locale.ROOT),
-            NAME,
-            req.getPathTemplateValues().get(CollectionAdminParams.COLLECTION),
-            TARGET,
-            v2Body.to,
-            ASYNC,
-            v2Body.async,
-            FOLLOW_ALIASES,
-            v2Body.followAliases);
-    collectionsHandler.handleRequestBody(req, rsp);
+    return new ZkNodeProps(remoteMessage);
   }
 
-  // TODO This is a bit hacky, but it's not worth investing in the request-body parsing code much
-  // here, as it's
-  //  something that's already somewhat built-in when this eventually moves to JAX-RS
-  private RenameCollectionPayload parseRenameParamsFromRequestBody(
-      SolrQueryRequest solrQueryRequest) throws IOException {
-    Iterable<ContentStream> contentStreams = solrQueryRequest.getContentStreams();
-    ContentStream cs = null;
-    if (contentStreams != null) {
-      cs = contentStreams.iterator().next();
-    }
-    if (cs == null) {
-      // An empty request-body is invalid (the 'to' field is required at a minimum), but we'll lean
-      // on the input-validation in CollectionsHandler to
-      // catch this, rather than duplicating the check for that here
-      return new RenameCollectionPayload();
-    }
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest request, SolrQueryResponse response)
+      throws Exception {
+    final var api = new RenameCollectionAPI(coreContainer, request, response);
+    final var params = request.getParams();
+    params.required().check(COLLECTION_PROP, TARGET);
+    final var requestBody = new RenameCollectionRequestBody();
+    requestBody.to = params.get(TARGET);
+    // Optional parameters
+    requestBody.asyncId = params.get(ASYNC);
+    requestBody.followAliases = params.getBool(FOLLOW_ALIASES);
+
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        response, api.renameCollection(params.get(COLLECTION_PROP), requestBody));
+  }
+
+  public static class RenameCollectionRequestBody implements JacksonReflectMapWriter {
+    @JsonProperty(required = true)
+    public String to;
+
+    @JsonProperty(ASYNC)
+    public String asyncId;
 
-    return REQUEST_BODY_PARSER.readValue(cs.getStream(), RenameCollectionPayload.class);
+    @JsonProperty public Boolean followAliases;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SyncShardAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SyncShardAPI.java
index 0819b09ad51..07488e3c5ed 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SyncShardAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/SyncShardAPI.java
@@ -17,52 +17,99 @@
 
 package org.apache.solr.handler.admin.api;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
-import static org.apache.solr.common.params.CollectionParams.ACTION;
-import static org.apache.solr.common.params.CoreAdminParams.SHARD;
-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.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 
-import java.util.HashMap;
-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.client.solrj.request.beans.SyncShardPayload;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.handler.admin.CollectionsHandler;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+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 javax.ws.rs.core.MediaType;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
 
 /**
  * V2 API for triggering a shard-sync operation within a particular collection and shard.
  *
- * <p>This API (POST /v2/collections/collectionName/shards/shardName {'sync-shard': {}}) is
- * analogous to the v1 /admin/collections?action=SYNCSHARD command.
- *
- * @see SyncShardPayload
+ * <p>This API (POST /v2/collections/cName/shards/sName/sync {...}) is analogous to the v1
+ * /admin/collections?action=SYNCSHARD command.
  */
-@EndPoint(
-    path = {"/c/{collection}/shards/{shard}", "/collections/{collection}/shards/{shard}"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class SyncShardAPI {
-  private static final String V2_SYNC_SHARD_CMD = "sync-shard";
+@Path("/collections/{collectionName}/shards/{shardName}/sync")
+public class SyncShardAPI extends AdminAPIBase {
+
+  @Inject
+  public SyncShardAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  @POST
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public SolrJerseyResponse syncShard(
+      @PathParam("collectionName") String collectionName, @PathParam("shardName") String shardName)
+      throws Exception {
+    final SolrJerseyResponse response = instantiateJerseyResponse(SolrJerseyResponse.class);
+    ensureRequiredParameterProvided(COLLECTION_PROP, collectionName);
+    ensureRequiredParameterProvided(SHARD_ID_PROP, shardName);
+    fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collectionName, solrQueryRequest);
+
+    doSyncShard(collectionName, shardName);
+
+    return response;
+  }
+
+  private void doSyncShard(String extCollectionName, String shardName)
+      throws IOException, SolrServerException {
+    String collection = coreContainer.getAliases().resolveSimpleAlias(extCollectionName);
+
+    ClusterState clusterState = coreContainer.getZkController().getClusterState();
 
-  private final CollectionsHandler collectionsHandler;
+    DocCollection docCollection = clusterState.getCollection(collection);
+    ZkNodeProps leaderProps = docCollection.getLeader(shardName);
+    ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
 
-  public SyncShardAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+    try (SolrClient client =
+        new HttpSolrClient.Builder(nodeProps.getBaseUrl())
+            .withConnectionTimeout(15000, TimeUnit.MILLISECONDS)
+            .withSocketTimeout(60000, TimeUnit.MILLISECONDS)
+            .build()) {
+      CoreAdminRequest.RequestSyncShard reqSyncShard = new CoreAdminRequest.RequestSyncShard();
+      reqSyncShard.setCollection(collection);
+      reqSyncShard.setShard(shardName);
+      reqSyncShard.setCoreName(nodeProps.getCoreName());
+      client.request(reqSyncShard);
+    }
   }
 
-  @Command(name = V2_SYNC_SHARD_CMD)
-  public void syncShard(PayloadObj<SyncShardPayload> obj) throws Exception {
-    final Map<String, Object> addedV1Params = new HashMap<>();
-    final Map<String, String> pathParams = obj.getRequest().getPathTemplateValues();
-    addedV1Params.put(ACTION, CollectionParams.CollectionAction.SYNCSHARD.toLower());
-    addedV1Params.put(COLLECTION, pathParams.get(COLLECTION));
-    addedV1Params.put(SHARD, pathParams.get(SHARD));
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest request, SolrQueryResponse response)
+      throws Exception {
+    final var api = new SyncShardAPI(coreContainer, request, response);
+    final var params = request.getParams();
+    params.required().check(COLLECTION_PROP, SHARD_ID_PROP);
 
-    collectionsHandler.handleRequestBody(
-        wrapParams(obj.getRequest(), addedV1Params), obj.getResponse());
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        response, api.syncShard(params.get(COLLECTION_PROP), params.get(SHARD_ID_PROP)));
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index e416d35bf5b..deefc0e4df3 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -102,15 +102,6 @@ public class TestApiFramework extends SolrTestCaseJ4 {
         V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards", "POST", null, parts);
     assertConditions(api.getSpec(), Map.of("/methods[0]", "POST", "/commands/split", NOT_NULL));
 
-    parts = new HashMap<>();
-    api =
-        V2HttpCall.getApiInfo(
-            containerHandlers, "/collections/hello/shards/shard1", "POST", null, parts);
-    assertConditions(
-        api.getSpec(), Map.of("/methods[0]", "POST", "/commands/force-leader", NOT_NULL));
-    assertEquals("hello", parts.get("collection"));
-    assertEquals("shard1", parts.get("shard"));
-
     parts = new HashMap<>();
     api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello", "POST", null, parts);
     assertConditions(api.getSpec(), Map.of("/methods[0]", "POST", "/commands/modify", NOT_NULL));
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
index fa11d4aa4cb..4aa4ff96eb2 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -88,9 +88,6 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
       apiBag.registerObject(clusterAPI.commands);
     }
 
-    compareOutput(
-        apiBag, "/collections/collName", POST, "{reload:{}}", "{name:collName, operation :reload}");
-
     compareOutput(
         apiBag,
         "/collections/collName/shards",
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/ForceLeaderAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/ForceLeaderAPITest.java
new file mode 100644
index 00000000000..09f97a4757f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/ForceLeaderAPITest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.common.SolrException;
+import org.junit.Test;
+
+/** Unit tests for {@link ForceLeaderAPI} */
+public class ForceLeaderAPITest extends SolrTestCaseJ4 {
+  @Test
+  public void testReportsErrorIfCollectionNameMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new ForceLeaderAPI(null, null, null);
+              api.forceLeader(null, "someShard");
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: collection", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfShardNameMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new ForceLeaderAPI(null, null, null);
+              api.forceLeader("someCollection", null);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: shard", thrown.getMessage());
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCollectionAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCollectionAPITest.java
new file mode 100644
index 00000000000..e7534dc0c1a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/ReloadCollectionAPITest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.junit.Test;
+
+/** Unit tests for {@link ReloadCollectionAPI} */
+public class ReloadCollectionAPITest extends SolrTestCaseJ4 {
+  @Test
+  public void testReportsErrorIfCollectionNameMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new ReloadCollectionAPI(null, null, null);
+              api.reloadCollection(null, new ReloadCollectionAPI.ReloadCollectionRequestBody());
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: collection", thrown.getMessage());
+  }
+
+  // TODO message creation
+  @Test
+  public void testCreateRemoteMessageAllProperties() {
+    final var requestBody = new ReloadCollectionAPI.ReloadCollectionRequestBody();
+    requestBody.asyncId = "someAsyncId";
+    final var remoteMessage =
+        ReloadCollectionAPI.createRemoteMessage("someCollName", requestBody).getProperties();
+
+    assertEquals(3, remoteMessage.size());
+    assertEquals("reload", remoteMessage.get(QUEUE_OPERATION));
+    assertEquals("someCollName", remoteMessage.get(NAME));
+    assertEquals("someAsyncId", remoteMessage.get(ASYNC));
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/SyncShardAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/SyncShardAPITest.java
new file mode 100644
index 00000000000..25ca5bb9cd1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/SyncShardAPITest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.common.SolrException;
+import org.junit.Test;
+
+/** Unit tests for {@link SyncShardAPI} */
+public class SyncShardAPITest extends SolrTestCaseJ4 {
+  @Test
+  public void testReportsErrorIfCollectionNameMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new SyncShardAPI(null, null, null);
+              api.syncShard(null, "someShard");
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: collection", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfShardNameMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new SyncShardAPI(null, null, null);
+              api.syncShard("someCollection", null);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: shard", thrown.getMessage());
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/V2CollectionAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/V2CollectionAPIMappingTest.java
index fa943c80740..503be4fa6f0 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/api/V2CollectionAPIMappingTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/V2CollectionAPIMappingTest.java
@@ -19,10 +19,8 @@ package org.apache.solr.handler.admin.api;
 
 import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
-import static org.apache.solr.common.params.CollectionAdminParams.TARGET;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonParams.ACTION;
-import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CoreAdminParams.SHARD;
 
 import java.util.Map;
@@ -59,9 +57,7 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     apiBag.registerObject(new ModifyCollectionAPI(collectionsHandler));
     apiBag.registerObject(new MoveReplicaAPI(collectionsHandler));
     apiBag.registerObject(new RebalanceLeadersAPI(collectionsHandler));
-    apiBag.registerObject(new ReloadCollectionAPI(collectionsHandler));
     apiBag.registerObject(new CollectionStatusAPI(collectionsHandler));
-    apiBag.registerObject(new RenameCollectionAPI(collectionsHandler));
   }
 
   @Override
@@ -85,21 +81,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     assertEquals("shard2", v1Params.get(SHARD));
   }
 
-  @Test
-  public void testRenameCollectionAllParams() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName/rename",
-            "POST",
-            "{\"to\": \"targetColl\", \"async\": \"requestTrackingId\", \"followAliases\": true}");
-
-    assertEquals("rename", v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(NAME));
-    assertEquals("targetColl", v1Params.get(TARGET));
-    assertEquals("requestTrackingId", v1Params.get(ASYNC));
-    assertEquals(true, v1Params.getPrimitiveBool("followAliases"));
-  }
-
   @Test
   public void testModifyCollectionAllProperties() throws Exception {
     final SolrParams v1Params =
@@ -128,17 +109,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     assertEquals(456, v1Params.getPrimitiveInt("property.baz"));
   }
 
-  @Test
-  public void testReloadCollectionAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName", "POST", "{ 'reload': {'async': 'requestTrackingId'}}");
-
-    assertEquals(CollectionParams.CollectionAction.RELOAD.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(NAME));
-    assertEquals("requestTrackingId", v1Params.get(ASYNC));
-  }
-
   @Test
   public void testMoveReplicaAllProperties() throws Exception {
     final SolrParams v1Params =
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/V2ShardsAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/V2ShardsAPIMappingTest.java
index 0d584973a7a..9731922de46 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/api/V2ShardsAPIMappingTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/V2ShardsAPIMappingTest.java
@@ -29,7 +29,6 @@ import static org.apache.solr.common.params.CommonAdminParams.SPLIT_METHOD;
 import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
 import static org.apache.solr.common.params.CommonParams.ACTION;
 import static org.apache.solr.common.params.CommonParams.TIMING;
-import static org.apache.solr.common.params.CoreAdminParams.SHARD;
 
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
@@ -53,8 +52,6 @@ public class V2ShardsAPIMappingTest extends V2ApiMappingTest<CollectionsHandler>
   public void populateApiBag() {
     final CollectionsHandler collectionsHandler = getRequestHandler();
     apiBag.registerObject(new SplitShardAPI(collectionsHandler));
-    apiBag.registerObject(new SyncShardAPI(collectionsHandler));
-    apiBag.registerObject(new ForceLeaderAPI(collectionsHandler));
   }
 
   @Override
@@ -67,28 +64,6 @@ public class V2ShardsAPIMappingTest extends V2ApiMappingTest<CollectionsHandler>
     return false;
   }
 
-  @Test
-  public void testForceLeaderAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName/shards/shardName", "POST", "{ 'force-leader': {}}");
-
-    assertEquals(CollectionParams.CollectionAction.FORCELEADER.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(COLLECTION));
-    assertEquals("shardName", v1Params.get(SHARD));
-  }
-
-  @Test
-  public void testSyncShardAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName/shards/shardName", "POST", "{ 'sync-shard': {}}");
-
-    assertEquals(CollectionParams.CollectionAction.SYNCSHARD.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(COLLECTION));
-    assertEquals("shardName", v1Params.get(SHARD));
-  }
-
   @Test
   public void testSplitShardAllProperties() throws Exception {
     final SolrParams v1Params =
diff --git a/solr/core/src/test/org/apache/solr/util/tracing/TestDistributedTracing.java b/solr/core/src/test/org/apache/solr/util/tracing/TestDistributedTracing.java
index cbfd4a64df9..b28f15e18bd 100644
--- a/solr/core/src/test/org/apache/solr/util/tracing/TestDistributedTracing.java
+++ b/solr/core/src/test/org/apache/solr/util/tracing/TestDistributedTracing.java
@@ -134,13 +134,13 @@ public class TestDistributedTracing extends SolrCloudTestCase {
     CloudSolrClient cloudClient = cluster.getSolrClient();
     List<MockSpan> finishedSpans;
 
-    new V2Request.Builder("/c/" + COLLECTION)
+    new V2Request.Builder("/collections/" + COLLECTION + "/reload")
         .withMethod(SolrRequest.METHOD.POST)
-        .withPayload("{\n" + " \"reload\" : {}\n" + "}")
+        .withPayload("{}")
         .build()
         .process(cloudClient);
     finishedSpans = getAndClearSpans();
-    assertEquals("reload:/c/{collection}", finishedSpans.get(0).operationName());
+    assertEquals("post:/collections/{collection}/reload", finishedSpans.get(0).operationName());
     assertDbInstanceColl(finishedSpans.get(0));
 
     new V2Request.Builder("/c/" + COLLECTION + "/update/json")
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
index 2c8de19cbc4..61e987509c6 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
@@ -321,28 +321,14 @@ http://localhost:8983/solr/admin/collections?action=RELOAD&name=techproducts_v2
 ====
 [.tab-label]*V2 API*
 
-With the v2 API, the `reload` command is provided as part of the JSON data that contains the required parameters:
+With the v2 API, the `reload` command is provided as a part of URL path. The request body is optional if the optional `async` parameter is omitted:
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/collections/techproducts_v2 -H 'Content-Type: application/json' -d '
-  {
-    "reload": {}
-  }
-'
-----
-
-Additional parameters can be passed in via the `reload` key:
-
-[source,bash]
-----
-curl -X POST http://localhost:8983/api/collections/techproducts_v2 -H 'Content-Type: application/json' -d '
+curl -X POST http://localhost:8983/api/collections/techproducts_v2/reload -H 'Content-Type: application/json' -d '
   {
-    "reload": {
-      "async": "reload1"
-    }
+    "async": "someAsyncId"
   }
-'
 ----
 ====
 --
@@ -357,7 +343,8 @@ curl -X POST http://localhost:8983/api/collections/techproducts_v2 -H 'Content-T
 |===
 +
 The name of the collection to reload.
-This parameter is required by the V1 API.
+This parameter is required.
+It appears as a query-parameter on v1 requests, and in the URL path of v2 requests.
 
 `async`::
 +
@@ -542,7 +529,7 @@ Aliases that refer to more than 1 collection are not supported.
 
 [source,bash]
 ----
-http://localhost:8983/solr/admin/collections?action=RENAME&name=techproducts_v2&target=renamedCollection
+http://localhost:8983/solr/admin/collections?action=RENAME&name=techproducts_v2&target=newName
 ----
 ====
 
@@ -554,7 +541,7 @@ http://localhost:8983/solr/admin/collections?action=RENAME&name=techproducts_v2&
 ----
 curl -X POST http://localhost:8983/api/collections/techproducts/rename -H 'Content-Type: application/json' -d '
   {
-    "to": "new_name"
+    "to": "newName"
   }
 '
 ----
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/shard-management.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/shard-management.adoc
index c8548b573b9..672ffa199f6 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/shard-management.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/shard-management.adoc
@@ -591,11 +591,7 @@ http://localhost:8983/solr/admin/collections?action=FORCELEADER&collection=techp
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/collections/techproducts/shards/shard1 -H 'Content-Type: application/json' -d '
-  {
-    "force-leader":{}
-  }
-'
+curl -X POST http://localhost:8983/api/collections/techproducts/shards/shard1/force-leader
 ----
 *Output*
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ForceLeaderPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ForceLeaderPayload.java
deleted file mode 100644
index fb23dd95c50..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ForceLeaderPayload.java
+++ /dev/null
@@ -1,25 +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.client.solrj.request.beans;
-
-import org.apache.solr.common.util.ReflectMapWriter;
-
-/**
- * Empty payload for the POST /v2/collections/collName/shards/shardName {"force-leader": {}} API.
- */
-public class ForceLeaderPayload implements ReflectMapWriter {}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ReloadCollectionPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ReloadCollectionPayload.java
deleted file mode 100644
index 91331d44008..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ReloadCollectionPayload.java
+++ /dev/null
@@ -1,24 +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.client.solrj.request.beans;
-
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
-
-public class ReloadCollectionPayload implements ReflectMapWriter {
-  @JsonProperty public String async;
-}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RenameCollectionPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RenameCollectionPayload.java
deleted file mode 100644
index c85fd42daee..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RenameCollectionPayload.java
+++ /dev/null
@@ -1,29 +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.client.solrj.request.beans;
-
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
-
-public class RenameCollectionPayload implements ReflectMapWriter {
-  @JsonProperty public String async;
-
-  @JsonProperty public Boolean followAliases;
-
-  @JsonProperty(required = true)
-  public String to;
-}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SyncShardPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SyncShardPayload.java
deleted file mode 100644
index b6636e65b6e..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SyncShardPayload.java
+++ /dev/null
@@ -1,23 +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.client.solrj.request.beans;
-
-import org.apache.solr.common.util.ReflectMapWriter;
-
-/** Empty payload for the /v2/collections/collName/shards/shardName {"sync-shard": {}} API. */
-public class SyncShardPayload implements ReflectMapWriter {}