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 2022/11/30 14:57:13 UTC

[solr] branch branch_9x updated: SOLR-16392: Refactor and update v2 DELETEREPLICAPROP API (#1053)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new b8ed5504f2e SOLR-16392: Refactor and update v2 DELETEREPLICAPROP API  (#1053)
b8ed5504f2e is described below

commit b8ed5504f2ea5838cc6aee2074e508ec86cd4fde
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Mon Nov 28 08:39:49 2022 -0500

    SOLR-16392: Refactor and update v2 DELETEREPLICAPROP API  (#1053)
    
    This commit modifies the existing DeleteReplicaPropertyAPI definition to
    be more REST-ful, and use the new JAX-RS framework. Specifically, this
    commit changes the API from:
    
    POST /collections/collName
        {"delete-replica-property" : {...}}
    
    to
    
    DELETE /collections/<coll>/shards/<shard>/replicas/<replica>/properties/<prop>
    
    This is a breaking change for users of the v2 API, but one that is
    allowed because of v2's "experimental" designation.
    
    In support of this, this commit also refactors DeleteReplicaPropertyAPI
    to use the new JAX-RS framework.
---
 solr/CHANGES.txt                                   |   1 +
 .../solr/handler/admin/CollectionsHandler.java     |  19 +--
 .../admin/api/DeleteReplicaPropertyAPI.java        | 123 ++++++++++++-----
 .../solr/handler/admin/TestApiFramework.java       |   4 +-
 .../solr/handler/admin/TestCollectionAPIs.java     |   7 -
 .../admin/api/DeleteReplicaPropertyAPITest.java    | 150 +++++++++++++++++++++
 .../admin/api/V2CollectionAPIMappingTest.java      |  21 ---
 .../deployment-guide/pages/replica-management.adoc |  10 +-
 .../beans/DeleteReplicaPropertyPayload.java        |  31 -----
 9 files changed, 253 insertions(+), 113 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f230b2bfd01..2bcfc93e3d0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -26,6 +26,7 @@ Improvements
 
 * SOLR-16392: A number of v2 "replica" endpoints have been updated to be more REST-ful, and hopefully, intuitive.
   ADDREPLICAPROP is now `PUT /api/collections/$coll/shards/$shard/replicas/$rep/properties/$prop {"value": $val}`
+  DELETEREPLICAPROP is now `DELETE /api/collections/$coll/shards/$shard/replicas/$rep/properties/$prop`
 
 * SOLR-11657: Deprecate ContentStream in favour of RequestWriter.ContentWriter.   (Joshua Ouma via Eric Pugh)
 
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 4a64fc893e9..449558f78bf 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
@@ -1299,19 +1299,14 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, addReplicaPropResponse);
           return null;
         }),
-    // XXX should this command support followAliases?
     DELETEREPLICAPROP_OP(
         DELETEREPLICAPROP,
         (req, rsp, h) -> {
-          Map<String, Object> map =
-              copy(
-                  req.getParams().required(),
-                  null,
-                  COLLECTION_PROP,
-                  PROPERTY_PROP,
-                  SHARD_ID_PROP,
-                  REPLICA_PROP);
-          return copy(req.getParams(), map, PROPERTY_PROP);
+          final var api = new DeleteReplicaPropertyAPI(h.coreContainer, req, rsp);
+          final var deleteReplicaPropResponse =
+              DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, req.getParams());
+          V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, deleteReplicaPropResponse);
+          return null;
         }),
     // XXX should this command support followAliases?
     BALANCESHARDUNIQUE_OP(
@@ -2079,7 +2074,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
   @Override
   public Collection<Class<? extends JerseyResource>> getJerseyResources() {
-    return List.of(AddReplicaPropertyAPI.class, ReplaceNodeAPI.class);
+    return List.of(
+        AddReplicaPropertyAPI.class, DeleteReplicaPropertyAPI.class, ReplaceNodeAPI.class);
   }
 
   @Override
@@ -2094,7 +2090,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     apis.addAll(AnnotatedApi.getApis(new DeleteReplicaAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new BalanceShardUniqueAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new DeleteCollectionAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new DeleteReplicaPropertyAPI(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/DeleteReplicaPropertyAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPI.java
index 2bc092513ff..933d31e640d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPI.java
@@ -17,49 +17,112 @@
 
 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.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
+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.cloud.ZkStateReader.PROPERTY_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
+import static org.apache.solr.handler.admin.CollectionsHandler.DEFAULT_COLLECTION_OP_TIMEOUT;
 
-import java.util.HashMap;
+import io.swagger.v3.oas.annotations.Parameter;
 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.DeleteReplicaPropertyPayload;
+import javax.inject.Inject;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.RequiredSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
 
 /**
  * V2 API for removing a property from a collection replica
  *
- * <p>This API (POST /v2/collections/collectionName {'delete-replica-property': {...}}) is analogous
- * to the v1 /admin/collections?action=DELETEREPLICAPROP command.
- *
- * @see DeleteReplicaPropertyPayload
+ * <p>This API is analogous to the v1 /admin/collections?action=DELETEREPLICAPROP command.
  */
-@EndPoint(
-    path = {"/c/{collection}", "/collections/{collection}"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class DeleteReplicaPropertyAPI {
-  private static final String V2_DELETE_REPLICA_PROPERTY_CMD = "delete-replica-property";
+@Path("/collections/{collName}/shards/{shardName}/replicas/{replicaName}/properties/{propName}")
+public class DeleteReplicaPropertyAPI extends AdminAPIBase {
+
+  @Inject
+  public DeleteReplicaPropertyAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
 
-  private final CollectionsHandler collectionsHandler;
+  public SolrJerseyResponse deleteReplicaProperty(
+      @Parameter(
+              description = "The name of the collection the replica belongs to.",
+              required = true)
+          @PathParam("collName")
+          String collName,
+      @Parameter(description = "The name of the shard the replica belongs to.", required = true)
+          @PathParam("shardName")
+          String shardName,
+      @Parameter(description = "The replica, e.g., `core_node1`.", required = true)
+          @PathParam("replicaName")
+          String replicaName,
+      @Parameter(description = "The name of the property to delete.", required = true)
+          @PathParam("propName")
+          String propertyName)
+      throws Exception {
+    final SolrJerseyResponse response = instantiateJerseyResponse(SolrJerseyResponse.class);
+    final CoreContainer coreContainer = fetchAndValidateZooKeeperAwareCoreContainer();
+    recordCollectionForLogAndTracing(collName, solrQueryRequest);
 
-  public DeleteReplicaPropertyAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
+    final ZkNodeProps remoteMessage =
+        createRemoteMessage(collName, shardName, replicaName, propertyName);
+    final SolrResponse remoteResponse =
+        CollectionsHandler.submitCollectionApiCommand(
+            coreContainer,
+            coreContainer.getDistributedCollectionCommandRunner(),
+            remoteMessage,
+            CollectionParams.CollectionAction.DELETEREPLICAPROP,
+            DEFAULT_COLLECTION_OP_TIMEOUT);
+    if (remoteResponse.getException() != null) {
+      throw remoteResponse.getException();
+    }
+
+    return response;
   }
 
-  @Command(name = V2_DELETE_REPLICA_PROPERTY_CMD)
-  public void deleteReplicaProperty(PayloadObj<DeleteReplicaPropertyPayload> obj) throws Exception {
-    final DeleteReplicaPropertyPayload v2Body = obj.get();
-    final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower());
-    v1Params.put(COLLECTION, obj.getRequest().getPathTemplateValues().get(COLLECTION));
+  public static SolrJerseyResponse invokeUsingV1Inputs(
+      DeleteReplicaPropertyAPI apiInstance, SolrParams solrParams) throws Exception {
+    final RequiredSolrParams requiredParams = solrParams.required();
+    final String propNameToDelete = requiredParams.get(PROPERTY_PROP);
+    final String trimmedPropNameToDelete =
+        propNameToDelete.startsWith(PROPERTY_PREFIX)
+            ? propNameToDelete.substring(PROPERTY_PREFIX.length())
+            : propNameToDelete;
+    return apiInstance.deleteReplicaProperty(
+        requiredParams.get(COLLECTION_PROP),
+        requiredParams.get(SHARD_ID_PROP),
+        requiredParams.get(REPLICA_PROP),
+        trimmedPropNameToDelete);
+  }
 
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+  // XXX should this command support followAliases?
+  public static ZkNodeProps createRemoteMessage(
+      String collName, String shardName, String replicaName, String propName) {
+    final Map<String, Object> messageProperties =
+        Map.of(
+            QUEUE_OPERATION,
+            CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+            COLLECTION_PROP,
+            collName,
+            SHARD_ID_PROP,
+            shardName,
+            REPLICA_PROP,
+            replicaName,
+            PROPERTY_PROP,
+            propName);
+    return new ZkNodeProps(messageProperties);
   }
 }
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 999e9142f73..c5352514356 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
@@ -121,9 +121,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
     parts = new HashMap<>();
     api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello", "POST", null, parts);
-    assertConditions(
-        api.getSpec(),
-        Map.of("/methods[0]", "POST", "/commands/delete-replica-property", NOT_NULL));
+    assertConditions(api.getSpec(), Map.of("/methods[0]", "POST", "/commands/modify", NOT_NULL));
     assertEquals("hello", parts.get("collection"));
 
     api =
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 0db79432f51..e074f0ba40d 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
@@ -199,13 +199,6 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
     //        "Value of enum must be one of"
     //    );
 
-    compareOutput(
-        apiBag,
-        "/collections/collName",
-        POST,
-        "{delete-replica-property : {property: propA , shard: shard1, replica:replica1} }",
-        "{collection: collName, shard: shard1, replica : replica1 , property : propA , operation : deletereplicaprop}");
-
     compareOutput(
         apiBag,
         "/cluster",
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPITest.java
new file mode 100644
index 00000000000..d03193072e0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteReplicaPropertyAPITest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import java.util.Map;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link DeleteReplicaPropertyAPI}
+ *
+ * <p>End-to-end functionality is tested implicitly through v1 integration tests, so the unit tests
+ * here focus primarily on how the v1 code invokes the v2 API and how the v2 API crafts its
+ * overseer/Distributed State Processing RPC message.
+ */
+public class DeleteReplicaPropertyAPITest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void ensureWorkingMockito() {
+    assumeWorkingMockito();
+  }
+
+  @Test
+  public void testV1InvocationThrowsErrorsIfRequiredParametersMissing() {
+    final var api = mock(DeleteReplicaPropertyAPI.class);
+    final var allParams = new ModifiableSolrParams();
+    allParams.add(COLLECTION_PROP, "someColl");
+    allParams.add(SHARD_ID_PROP, "someShard");
+    allParams.add(REPLICA_PROP, "someReplica");
+    allParams.add(PROPERTY_PROP, "somePropName");
+
+    {
+      final var noCollectionParams = new ModifiableSolrParams(allParams);
+      noCollectionParams.remove(COLLECTION_PROP);
+      final SolrException e =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, noCollectionParams);
+              });
+      assertEquals("Missing required parameter: " + COLLECTION_PROP, e.getMessage());
+    }
+
+    {
+      final var noShardParams = new ModifiableSolrParams(allParams);
+      noShardParams.remove(SHARD_ID_PROP);
+      final SolrException e =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, noShardParams);
+              });
+      assertEquals("Missing required parameter: " + SHARD_ID_PROP, e.getMessage());
+    }
+
+    {
+      final var noReplicaParams = new ModifiableSolrParams(allParams);
+      noReplicaParams.remove(REPLICA_PROP);
+      final SolrException e =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, noReplicaParams);
+              });
+      assertEquals("Missing required parameter: " + REPLICA_PROP, e.getMessage());
+    }
+
+    {
+      final var noPropertyParams = new ModifiableSolrParams(allParams);
+      noPropertyParams.remove(PROPERTY_PROP);
+      final SolrException e =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, noPropertyParams);
+              });
+      assertEquals("Missing required parameter: " + PROPERTY_PROP, e.getMessage());
+    }
+  }
+
+  @Test
+  public void testV1InvocationPassesAllProvidedParameters() throws Exception {
+    final var api = mock(DeleteReplicaPropertyAPI.class);
+    final var allParams = new ModifiableSolrParams();
+    allParams.add(COLLECTION_PROP, "someColl");
+    allParams.add(SHARD_ID_PROP, "someShard");
+    allParams.add(REPLICA_PROP, "someReplica");
+    allParams.add(PROPERTY_PROP, "somePropName");
+
+    DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, allParams);
+
+    verify(api).deleteReplicaProperty("someColl", "someShard", "someReplica", "somePropName");
+  }
+
+  @Test
+  public void testV1InvocationTrimsPropertyNamePrefixIfProvided() throws Exception {
+    final var api = mock(DeleteReplicaPropertyAPI.class);
+    final var allParams = new ModifiableSolrParams();
+    allParams.add(COLLECTION_PROP, "someColl");
+    allParams.add(SHARD_ID_PROP, "someShard");
+    allParams.add(REPLICA_PROP, "someReplica");
+    allParams.add(PROPERTY_PROP, "property.somePropName"); // NOTE THE "property." prefix!
+
+    DeleteReplicaPropertyAPI.invokeUsingV1Inputs(api, allParams);
+
+    verify(api).deleteReplicaProperty("someColl", "someShard", "someReplica", "somePropName");
+  }
+
+  @Test
+  public void testRPCMessageCreation() {
+    final ZkNodeProps message =
+        DeleteReplicaPropertyAPI.createRemoteMessage(
+            "someColl", "someShard", "someReplica", "somePropName");
+    final Map<String, Object> props = message.getProperties();
+
+    assertEquals(5, props.size());
+    assertEquals("deletereplicaprop", props.get(QUEUE_OPERATION));
+    assertEquals("someColl", props.get(COLLECTION_PROP));
+    assertEquals("someShard", props.get(SHARD_ID_PROP));
+    assertEquals("someReplica", props.get(REPLICA_PROP));
+    assertEquals("somePropName", props.get(PROPERTY_PROP));
+  }
+}
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 cebbaee2227..65feab737db 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
@@ -56,7 +56,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     final CollectionsHandler collectionsHandler = getRequestHandler();
     apiBag.registerObject(new BalanceShardUniqueAPI(collectionsHandler));
     apiBag.registerObject(new DeleteCollectionAPI(collectionsHandler));
-    apiBag.registerObject(new DeleteReplicaPropertyAPI(collectionsHandler));
     apiBag.registerObject(new MigrateDocsAPI(collectionsHandler));
     apiBag.registerObject(new ModifyCollectionAPI(collectionsHandler));
     apiBag.registerObject(new MoveReplicaAPI(collectionsHandler));
@@ -229,26 +228,6 @@ public class V2CollectionAPIMappingTest extends V2ApiMappingTest<CollectionsHand
     assertEquals(456, v1Params.getPrimitiveInt("maxWaitSeconds"));
   }
 
-  @Test
-  public void testDeleteReplicaPropertyAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName",
-            "POST",
-            "{ 'delete-replica-property': {"
-                + "'shard': 'someShardName', "
-                + "'replica': 'someReplicaName', "
-                + "'property': 'somePropertyName' "
-                + "}}");
-
-    assertEquals(
-        CollectionParams.CollectionAction.DELETEREPLICAPROP.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(COLLECTION));
-    assertEquals("someShardName", v1Params.get("shard"));
-    assertEquals("someReplicaName", v1Params.get("replica"));
-    assertEquals("somePropertyName", v1Params.get("property"));
-  }
-
   @Test
   public void testSetCollectionPropertyAllProperties() throws Exception {
     final SolrParams v1Params =
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/replica-management.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/replica-management.adoc
index 57381c362d6..0123679fc91 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/replica-management.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/replica-management.adoc
@@ -747,15 +747,7 @@ http://localhost:8983/solr/admin/collections?action=DELETEREPLICAPROP&collection
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/collections/techproducts -H 'Content-Type: application/json' -d '
-  {
-    "delete-replica-property":{
-      "shard":"shard1",
-      "replica":"core_node2",
-      "name":"preferredLeader"
-    }
-  }
-'
+curl -X DELETE http://localhost:8983/api/collections/techproducts/shards/shard1/replicas/core_node2/properties/preferredLeader
 ----
 ====
 --
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteReplicaPropertyPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteReplicaPropertyPayload.java
deleted file mode 100644
index fd8d95fd896..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteReplicaPropertyPayload.java
+++ /dev/null
@@ -1,31 +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 DeleteReplicaPropertyPayload implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String shard;
-
-  @JsonProperty(required = true)
-  public String replica;
-
-  @JsonProperty(required = true)
-  public String property;
-}