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:23 UTC

[solr] 03/03: SOLR-16398: Tweak v2 B-S-U API to be more REST-ful (#1689)

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 a514a2d929d6a3b6ca29e81f16e487396e059771
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Thu Jun 8 14:18:29 2023 -0400

    SOLR-16398: Tweak v2 B-S-U API to be more REST-ful (#1689)
    
    This commit changes the v2 "balance-shard-unique" replica 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/collections/cName/balance-shard-unique {...}`
    
    Although not shown above, the 'balance-shard-unique' command
    specifier has also been removed from the request body.
    
    This commit also converts the API to the new JAX-RS framework.
---
 solr/CHANGES.txt                                   |   4 +
 .../solr/handler/admin/CollectionsHandler.java     |  27 +---
 .../handler/admin/api/BalanceShardUniqueAPI.java   | 151 +++++++++++++++++----
 .../solr/handler/admin/TestCollectionAPIs.java     |   7 -
 .../admin/api/BalanceShardUniqueAPITest.java       |  98 +++++++++++++
 .../admin/api/V2CollectionAPIMappingTest.java      |  21 ---
 .../pages/cluster-node-management.adoc             |   6 +-
 .../request/beans/BalanceShardUniquePayload.java   |  29 ----
 8 files changed, 229 insertions(+), 114 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7b8e9eb0450..fc9a9703ed7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -138,6 +138,10 @@ Improvements
   "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)
 
+* SOLR-16398: The v2 "balance-shard-unique" API has been tweaked to be more intuitive, by removing the top-level command
+  specifier from the request body, and changing the path. The v2 functionality can now be accessed at:
+  `POST /api/collections/cName/balance-shard-unique {...}` (Jason Gerlowski)
+
 Optimizations
 ---------------------
 
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 eef39bee14d..9413389ca92 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
@@ -25,7 +25,6 @@ import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
 import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET;
 import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE;
 import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.NUM_SLICES;
-import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
 import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.REQUESTID;
 import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
@@ -115,7 +114,6 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
@@ -135,7 +133,6 @@ import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkController.NotInClusterStateException;
 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;
@@ -1025,26 +1022,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     BALANCESHARDUNIQUE_OP(
         BALANCESHARDUNIQUE,
         (req, rsp, h) -> {
-          Map<String, Object> map =
-              copy(req.getParams().required(), null, COLLECTION_PROP, PROPERTY_PROP);
-          Boolean shardUnique = Boolean.parseBoolean(req.getParams().get(SHARD_UNIQUE));
-          String prop = req.getParams().get(PROPERTY_PROP).toLowerCase(Locale.ROOT);
-          if (!prop.startsWith(PROPERTY_PREFIX)) {
-            prop = PROPERTY_PREFIX + prop;
-          }
-
-          if (!shardUnique && !SliceMutator.SLICE_UNIQUE_BOOLEAN_PROPERTIES.contains(prop)) {
-            throw new SolrException(
-                ErrorCode.BAD_REQUEST,
-                "Balancing properties amongst replicas in a slice requires that"
-                    + " the property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'. "
-                    + " Property: "
-                    + prop
-                    + " shardUnique: "
-                    + shardUnique);
-          }
-
-          return copy(req.getParams(), map, ONLY_ACTIVE_NODES, SHARD_UNIQUE);
+          BalanceShardUniqueAPI.invokeFromV1Params(h.coreContainer, req, rsp);
+          return null;
         }),
     REBALANCELEADERS_OP(
         REBALANCELEADERS,
@@ -1385,6 +1364,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     return List.of(
         CreateReplicaAPI.class,
         AddReplicaPropertyAPI.class,
+        BalanceShardUniqueAPI.class,
         CreateAliasAPI.class,
         CreateCollectionAPI.class,
         CreateCollectionBackupAPI.class,
@@ -1418,7 +1398,6 @@ 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 BalanceShardUniqueAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new MigrateDocsAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new ModifyCollectionAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new MoveReplicaAPI(this)));
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/BalanceShardUniqueAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/BalanceShardUniqueAPI.java
index 36dbd28ce3e..64f31ac2e59 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/BalanceShardUniqueAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/BalanceShardUniqueAPI.java
@@ -16,50 +16,143 @@
  */
 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.handler.ClusterAPI.wrapParams;
+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.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
+import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_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.client.solrj.request.beans.BalanceShardUniquePayload;
+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.cloud.overseer.SliceMutator;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.annotation.JsonProperty;
+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.common.params.SolrParams;
+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;
 
 /**
  * V2 API for insuring that a particular property is distributed evenly amongst the physical nodes
  * comprising a collection.
  *
- * <p>The new API (POST /v2/collections/collectionName {'balance-shard-unique': {...}}) is analogous
- * to the v1 /admin/collections?action=BALANCESHARDUNIQUE command.
- *
- * @see BalanceShardUniquePayload
+ * <p>The new API (POST /v2/collections/collectionName/balance-shard-unique {...} ) is analogous to
+ * the v1 /admin/collections?action=BALANCESHARDUNIQUE command.
  */
-@EndPoint(
-    path = {"/c/{collection}", "/collections/{collection}"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class BalanceShardUniqueAPI {
-  private static final String V2_BALANCE_SHARD_UNIQUE_CMD = "balance-shard-unique";
+@Path("/collections/{collectionName}/balance-shard-unique")
+public class BalanceShardUniqueAPI extends AdminAPIBase {
+  @Inject
+  public BalanceShardUniqueAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  @POST
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public SubResponseAccumulatingJerseyResponse balanceShardUnique(
+      @PathParam("collectionName") String collectionName, BalanceShardUniqueRequestBody 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(PROPERTY_PROP, requestBody.property);
+    validatePropertyToBalance(requestBody.property, Boolean.TRUE.equals(requestBody.shardUnique));
+    fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collectionName, solrQueryRequest);
+
+    final ZkNodeProps remoteMessage = createRemoteMessage(collectionName, requestBody);
+    submitRemoteMessageAndHandleResponse(
+        response,
+        CollectionParams.CollectionAction.BALANCESHARDUNIQUE,
+        remoteMessage,
+        requestBody.asyncId);
+
+    return response;
+  }
+
+  public static ZkNodeProps createRemoteMessage(
+      String collectionName, BalanceShardUniqueRequestBody requestBody) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+    remoteMessage.put(
+        QUEUE_OPERATION, CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toLower());
+    remoteMessage.put(COLLECTION_PROP, collectionName);
+    remoteMessage.put(PROPERTY_PROP, requestBody.property);
+    insertIfNotNull(remoteMessage, ONLY_ACTIVE_NODES, requestBody.onlyActiveNodes);
+    insertIfNotNull(remoteMessage, SHARD_UNIQUE, requestBody.shardUnique);
+    insertIfNotNull(remoteMessage, ASYNC, requestBody.asyncId);
+
+    return new ZkNodeProps(remoteMessage);
+  }
+
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse)
+      throws Exception {
+    final var api = new BalanceShardUniqueAPI(coreContainer, solrQueryRequest, solrQueryResponse);
+    final SolrParams params = solrQueryRequest.getParams();
+    params.required().check(COLLECTION_PROP, PROPERTY_PROP);
+
+    final String collection = params.get(COLLECTION_PROP);
+    final var requestBody = new BalanceShardUniqueRequestBody();
+    requestBody.property = params.get(PROPERTY_PROP);
+    requestBody.onlyActiveNodes = params.getBool(ONLY_ACTIVE_NODES);
+    requestBody.shardUnique = params.getBool(SHARD_UNIQUE);
+    requestBody.asyncId = params.get(ASYNC);
+
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        solrQueryResponse, api.balanceShardUnique(collection, requestBody));
+  }
+
+  public static class BalanceShardUniqueRequestBody implements JacksonReflectMapWriter {
+    @JsonProperty(required = true)
+    public String property;
+
+    @JsonProperty(ONLY_ACTIVE_NODES)
+    public Boolean onlyActiveNodes;
 
-  private final CollectionsHandler collectionsHandler;
+    @JsonProperty public Boolean shardUnique;
 
-  public BalanceShardUniqueAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+    @JsonProperty(ASYNC)
+    public String asyncId;
   }
 
-  @Command(name = V2_BALANCE_SHARD_UNIQUE_CMD)
-  public void balanceShardUnique(PayloadObj<BalanceShardUniquePayload> obj) throws Exception {
-    final BalanceShardUniquePayload v2Body = obj.get();
-    final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toLower());
-    v1Params.put(COLLECTION, obj.getRequest().getPathTemplateValues().get(COLLECTION));
+  private void validatePropertyToBalance(String prop, boolean shardUnique) {
+    prop = prop.toLowerCase(Locale.ROOT);
+    if (!prop.startsWith(PROPERTY_PREFIX)) {
+      prop = PROPERTY_PREFIX + prop;
+    }
 
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+    if (!shardUnique && !SliceMutator.SLICE_UNIQUE_BOOLEAN_PROPERTIES.contains(prop)) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "Balancing properties amongst replicas in a slice requires that"
+              + " the property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'. "
+              + " Property: "
+              + prop
+              + " shardUnique: "
+              + shardUnique);
+    }
   }
 }
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 4aa4ff96eb2..a8dde0459f5 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
@@ -116,13 +116,6 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
         "{remove-role : {role : overseer, node : 'localhost_8978'} }",
         "{operation : removerole ,role : overseer, node : 'localhost_8978'}");
 
-    compareOutput(
-        apiBag,
-        "/collections/coll1",
-        POST,
-        "{balance-shard-unique : {property: preferredLeader} }",
-        "{operation : balanceshardunique ,collection : coll1, property : preferredLeader}");
-
     compareOutput(
         apiBag,
         "/collections/coll1",
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/BalanceShardUniqueAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/BalanceShardUniqueAPITest.java
new file mode 100644
index 00000000000..fea001edcb2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/BalanceShardUniqueAPITest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
+import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
+import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
+import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.junit.Test;
+
+/** Unit tests for {@link BalanceShardUniqueAPI} */
+public class BalanceShardUniqueAPITest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testReportsErrorIfRequestBodyMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new BalanceShardUniqueAPI(null, null, null);
+              api.balanceShardUnique("someCollectionName", null);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required request body", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfCollectionNameMissing() {
+    final var requestBody = new BalanceShardUniqueAPI.BalanceShardUniqueRequestBody();
+    requestBody.property = "preferredLeader";
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new BalanceShardUniqueAPI(null, null, null);
+              api.balanceShardUnique(null, requestBody);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: collection", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfPropertyToBalanceIsMissing() {
+    // Note, 'property' param on reqBody not set
+    final var requestBody = new BalanceShardUniqueAPI.BalanceShardUniqueRequestBody();
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new BalanceShardUniqueAPI(null, null, null);
+              api.balanceShardUnique("someCollName", requestBody);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: property", thrown.getMessage());
+  }
+
+  @Test
+  public void testCreateRemoteMessageAllProperties() {
+    final var requestBody = new BalanceShardUniqueAPI.BalanceShardUniqueRequestBody();
+    requestBody.property = "someProperty";
+    requestBody.shardUnique = Boolean.TRUE;
+    requestBody.onlyActiveNodes = Boolean.TRUE;
+    requestBody.asyncId = "someAsyncId";
+    final var remoteMessage =
+        BalanceShardUniqueAPI.createRemoteMessage("someCollName", requestBody).getProperties();
+
+    assertEquals(6, remoteMessage.size());
+    assertEquals("balanceshardunique", remoteMessage.get(QUEUE_OPERATION));
+    assertEquals("someCollName", remoteMessage.get(COLLECTION));
+    assertEquals("someProperty", remoteMessage.get(PROPERTY_PROP));
+    assertEquals(Boolean.TRUE, remoteMessage.get(SHARD_UNIQUE));
+    assertEquals(Boolean.TRUE, remoteMessage.get(ONLY_ACTIVE_NODES));
+    assertEquals("someAsyncId", remoteMessage.get(ASYNC));
+  }
+}
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 503be4fa6f0..98d000773c7 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
@@ -52,7 +52,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
   @Override
   public void populateApiBag() {
     final CollectionsHandler collectionsHandler = getRequestHandler();
-    apiBag.registerObject(new BalanceShardUniqueAPI(collectionsHandler));
     apiBag.registerObject(new MigrateDocsAPI(collectionsHandler));
     apiBag.registerObject(new ModifyCollectionAPI(collectionsHandler));
     apiBag.registerObject(new MoveReplicaAPI(collectionsHandler));
@@ -161,26 +160,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     assertEquals("requestTrackingId", v1Params.get(ASYNC));
   }
 
-  @Test
-  public void testBalanceShardUniqueAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName",
-            "POST",
-            "{ 'balance-shard-unique': {"
-                + "'property': 'somePropertyToBalance', "
-                + "'onlyactivenodes': false, "
-                + "'shardUnique': true"
-                + "}}");
-
-    assertEquals(
-        CollectionParams.CollectionAction.BALANCESHARDUNIQUE.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(COLLECTION));
-    assertEquals("somePropertyToBalance", v1Params.get("property"));
-    assertFalse(v1Params.getPrimitiveBool("onlyactivenodes"));
-    assertTrue(v1Params.getPrimitiveBool("shardUnique"));
-  }
-
   @Test
   public void testRebalanceLeadersAllProperties() throws Exception {
     final SolrParams v1Params =
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/cluster-node-management.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/cluster-node-management.adoc
index 577eb71eb62..dc84a3239c3 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/cluster-node-management.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/cluster-node-management.adoc
@@ -454,11 +454,9 @@ http://localhost:8983/solr/admin/collections?action=BALANCESHARDUNIQUE&collectio
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/collections/techproducts -H 'Content-Type: application/json' -d '
+curl -X POST http://localhost:8983/api/collections/techproducts/balance-shard-unique -H 'Content-Type: application/json' -d '
   {
-    "balance-shard-unique": {
-      "property": "preferredLeader"
-    }
+    "property": "preferredLeader"
   }
 '
 ----
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BalanceShardUniquePayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BalanceShardUniquePayload.java
deleted file mode 100644
index db3cfef718a..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BalanceShardUniquePayload.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 BalanceShardUniquePayload implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String property;
-
-  @JsonProperty public Boolean onlyactivenodes = null;
-
-  @JsonProperty public Boolean shardUnique;
-}