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

[solr] 01/03: SOLR-16392: Tweak v2 ADDREPLICA to be more REST-ful

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 324731e08698c6b0a0d1ec0a94fded7e666ab1f3
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Mon Jun 5 12:44:46 2023 -0400

    SOLR-16392: Tweak v2 ADDREPLICA to be more REST-ful
    
    Following these changes, the v2 API now appears as:
    
      `POST /api/collections/cName/shards/sName/replicas {...}`
    
    Although not shown above, the request body has no 'add-replica' command
    specifier.  These changes bring v2 ADDREPLICA more into line with the
    REST-ful design we are aiming for going forward.
    
    This commit also converts the API to the new JAX-RS framework.
---
 solr/CHANGES.txt                                   |   4 +
 .../solr/handler/admin/CollectionsHandler.java     |  41 ++---
 .../solr/handler/admin/api/AddReplicaAPI.java      |  72 --------
 .../solr/handler/admin/api/CreateReplicaAPI.java   | 202 +++++++++++++++++++++
 .../org/apache/solr/cloud/TestPullReplica.java     |   8 +-
 .../org/apache/solr/cloud/TestTlogReplica.java     |  13 +-
 .../solr/handler/admin/TestApiFramework.java       |   7 +-
 .../solr/handler/admin/TestCollectionAPIs.java     |  29 ---
 .../handler/admin/api/CreateReplicaAPITest.java    | 182 +++++++++++++++++++
 .../handler/admin/api/V2ShardsAPIMappingTest.java  |  47 -----
 .../deployment-guide/pages/replica-management.adoc |  20 +-
 .../solrj/request/beans/AddReplicaPayload.java     |  56 ------
 12 files changed, 424 insertions(+), 257 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2abdb198882..dce082498f0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -129,6 +129,10 @@ Improvements
   then uses the weights to decide the optimal strategy for placing new replicas or balancing existing replicas.
   (Houston Putman, Tomás Fernández Löbbe, Jason Gerlowski, Radu Gheorghe)
 
+* SOLR-16392: The v2 "add-replica" API has been tweaked to be more intuitive, by removing the top-level command specifier and
+  changing the path.  The v2 functionality can now be accessed at: `POST /api/collections/cName/shards/sName/replicas {...}`
+  (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 a83895e0e36..eefe2953275 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
@@ -36,7 +36,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
@@ -46,7 +45,6 @@ import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_NAME;
 import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
 import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_VALUE;
 import static org.apache.solr.common.params.CollectionAdminParams.SHARD;
-import static org.apache.solr.common.params.CollectionAdminParams.SKIP_NODE_ASSIGNMENT;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
@@ -106,9 +104,6 @@ import static org.apache.solr.common.params.CommonParams.TIMING;
 import static org.apache.solr.common.params.CommonParams.VALUE_LONG;
 import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
 import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
-import static org.apache.solr.common.params.CoreAdminParams.DATA_DIR;
-import static org.apache.solr.common.params.CoreAdminParams.INSTANCE_DIR;
-import static org.apache.solr.common.params.CoreAdminParams.ULOG_DIR;
 import static org.apache.solr.common.params.ShardParams._ROUTE_;
 import static org.apache.solr.common.util.StrUtils.formatString;
 
@@ -175,7 +170,6 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.snapshots.CollectionSnapshotMetaData;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.handler.admin.api.AddReplicaAPI;
 import org.apache.solr.handler.admin.api.AddReplicaPropertyAPI;
 import org.apache.solr.handler.admin.api.AdminAPIBase;
 import org.apache.solr.handler.admin.api.AliasPropertyAPI;
@@ -187,6 +181,7 @@ import org.apache.solr.handler.admin.api.CreateAliasAPI;
 import org.apache.solr.handler.admin.api.CreateCollectionAPI;
 import org.apache.solr.handler.admin.api.CreateCollectionBackupAPI;
 import org.apache.solr.handler.admin.api.CreateCollectionSnapshotAPI;
+import org.apache.solr.handler.admin.api.CreateReplicaAPI;
 import org.apache.solr.handler.admin.api.CreateShardAPI;
 import org.apache.solr.handler.admin.api.DeleteAliasAPI;
 import org.apache.solr.handler.admin.api.DeleteCollectionAPI;
@@ -976,27 +971,17 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     ADDREPLICA_OP(
         ADDREPLICA,
         (req, rsp, h) -> {
-          Map<String, Object> props =
-              copy(
-                  req.getParams(),
-                  null,
-                  COLLECTION_PROP,
-                  "node",
-                  SHARD_ID_PROP,
-                  _ROUTE_,
-                  CoreAdminParams.NAME,
-                  INSTANCE_DIR,
-                  DATA_DIR,
-                  ULOG_DIR,
-                  REPLICA_TYPE,
-                  WAIT_FOR_FINAL_STATE,
-                  NRT_REPLICAS,
-                  TLOG_REPLICAS,
-                  PULL_REPLICAS,
-                  CREATE_NODE_SET,
-                  FOLLOW_ALIASES,
-                  SKIP_NODE_ASSIGNMENT);
-          return copyPropertiesWithPrefix(req.getParams(), props, PROPERTY_PREFIX);
+          final var params = req.getParams();
+          params.required().check(COLLECTION_PROP, SHARD_ID_PROP);
+
+          final var api = new CreateReplicaAPI(h.coreContainer, req, rsp);
+          final var requestBody =
+              CreateReplicaAPI.AddReplicaRequestBody.fromV1Params(req.getParams());
+          final var response =
+              api.createReplica(
+                  params.get(COLLECTION_PROP), params.get(SHARD_ID_PROP), requestBody);
+          V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, response);
+          return null;
         }),
     OVERSEERSTATUS_OP(OVERSEERSTATUS, (req, rsp, h) -> new LinkedHashMap<>()),
     DISTRIBUTEDAPIPROCESSING_OP(
@@ -1520,6 +1505,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   @Override
   public Collection<Class<? extends JerseyResource>> getJerseyResources() {
     return List.of(
+        CreateReplicaAPI.class,
         AddReplicaPropertyAPI.class,
         CreateAliasAPI.class,
         CreateCollectionAPI.class,
@@ -1550,7 +1536,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 AddReplicaAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SyncShardAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new ForceLeaderAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new BalanceShardUniqueAPI(this)));
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/AddReplicaAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/AddReplicaAPI.java
deleted file mode 100644
index caf545f029e..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/AddReplicaAPI.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.handler.admin.api;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
-import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
-import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
-import static org.apache.solr.common.params.CommonParams.ACTION;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
-import static org.apache.solr.handler.api.V2ApiUtils.flattenMapWithPrefix;
-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.AddReplicaPayload;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.handler.admin.CollectionsHandler;
-
-/**
- * V2 API for adding a new replica to an existing shard.
- *
- * <p>This API (POST /v2/collections/collectionName/shards {'add-replica': {...}}) is analogous to
- * the v1 /admin/collections?action=ADDREPLICA command.
- *
- * @see AddReplicaPayload
- */
-@EndPoint(
-    path = {"/c/{collection}/shards", "/collections/{collection}/shards"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class AddReplicaAPI {
-  private static final String V2_ADD_REPLICA_CMD = "add-replica";
-
-  private final CollectionsHandler collectionsHandler;
-
-  public AddReplicaAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
-  }
-
-  @Command(name = V2_ADD_REPLICA_CMD)
-  public void addReplica(PayloadObj<AddReplicaPayload> obj) throws Exception {
-    final AddReplicaPayload v2Body = obj.get();
-    final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.ADDREPLICA.toLower());
-    v1Params.put(COLLECTION, obj.getRequest().getPathTemplateValues().get(COLLECTION));
-
-    flattenMapWithPrefix(v2Body.coreProperties, v1Params, PROPERTY_PREFIX);
-    if (v2Body.createNodeSet != null && !v2Body.createNodeSet.isEmpty()) {
-      v1Params.replace(CREATE_NODE_SET_PARAM, String.join(",", v2Body.createNodeSet));
-    }
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateReplicaAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateReplicaAPI.java
new file mode 100644
index 00000000000..76043bff05c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateReplicaAPI.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin.api;
+
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+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.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
+import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
+import static org.apache.solr.common.params.CollectionAdminParams.SKIP_NODE_ASSIGNMENT;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
+import static org.apache.solr.common.params.CoreAdminParams.DATA_DIR;
+import static org.apache.solr.common.params.CoreAdminParams.INSTANCE_DIR;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+import static org.apache.solr.common.params.CoreAdminParams.NODE;
+import static org.apache.solr.common.params.CoreAdminParams.ULOG_DIR;
+import static org.apache.solr.common.params.ShardParams._ROUTE_;
+import static org.apache.solr.handler.admin.api.CreateCollectionAPI.copyPrefixedPropertiesWithoutPrefix;
+import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+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 org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.CollectionUtil;
+import org.apache.solr.core.CoreContainer;
+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 adding a new replica to an existing shard.
+ *
+ * <p>This API (POST /v2/collections/cName/shards/sName/replicas {...}) is analogous to the v1
+ * /admin/collections?action=ADDREPLICA command.
+ */
+@Path("/collections/{collectionName}/shards/{shardName}/replicas")
+public class CreateReplicaAPI extends AdminAPIBase {
+
+  @Inject
+  public CreateReplicaAPI(
+      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 createReplica(
+      @PathParam("collectionName") String collectionName,
+      @PathParam("shardName") String shardName,
+      AddReplicaRequestBody requestBody)
+      throws Exception {
+    final var response = instantiateJerseyResponse(SubResponseAccumulatingJerseyResponse.class);
+    if (requestBody == null) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST, "Required request-body is missing");
+    }
+    ensureRequiredParameterProvided(COLLECTION_PROP, collectionName);
+    ensureRequiredParameterProvided(SHARD_ID_PROP, shardName);
+    final String resolvedCollectionName =
+        resolveAndValidateAliasIfEnabled(
+            collectionName, Boolean.TRUE.equals(requestBody.followAliases));
+
+    final ZkNodeProps remoteMessage =
+        createRemoteMessage(resolvedCollectionName, shardName, requestBody);
+    submitRemoteMessageAndHandleResponse(
+        response, CollectionParams.CollectionAction.ADDREPLICA, remoteMessage, requestBody.asyncId);
+    return response;
+  }
+
+  public static ZkNodeProps createRemoteMessage(
+      String collectionName, String shardName, AddReplicaRequestBody requestBody) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+    remoteMessage.put(QUEUE_OPERATION, CollectionParams.CollectionAction.ADDREPLICA.toLower());
+    remoteMessage.put(COLLECTION_PROP, collectionName);
+    remoteMessage.put(SHARD_ID_PROP, shardName);
+    insertIfNotNull(remoteMessage, CoreAdminParams.NAME, requestBody.name);
+    insertIfNotNull(remoteMessage, _ROUTE_, requestBody.route);
+    insertIfNotNull(remoteMessage, NODE, requestBody.node);
+    if (CollectionUtil.isNotEmpty(requestBody.nodeSet)) {
+      remoteMessage.put(CREATE_NODE_SET_PARAM, String.join(",", requestBody.nodeSet));
+    }
+    insertIfNotNull(remoteMessage, SKIP_NODE_ASSIGNMENT, requestBody.skipNodeAssignment);
+    insertIfNotNull(remoteMessage, INSTANCE_DIR, requestBody.instanceDir);
+    insertIfNotNull(remoteMessage, DATA_DIR, requestBody.dataDir);
+    insertIfNotNull(remoteMessage, ULOG_DIR, requestBody.ulogDir);
+    insertIfNotNull(remoteMessage, REPLICA_TYPE, requestBody.type);
+    insertIfNotNull(remoteMessage, WAIT_FOR_FINAL_STATE, requestBody.waitForFinalState);
+    insertIfNotNull(remoteMessage, NRT_REPLICAS, requestBody.nrtReplicas);
+    insertIfNotNull(remoteMessage, TLOG_REPLICAS, requestBody.tlogReplicas);
+    insertIfNotNull(remoteMessage, PULL_REPLICAS, requestBody.pullReplicas);
+    insertIfNotNull(remoteMessage, FOLLOW_ALIASES, requestBody.followAliases);
+    insertIfNotNull(remoteMessage, ASYNC, requestBody.asyncId);
+
+    if (requestBody.properties != null) {
+      requestBody
+          .properties
+          .entrySet()
+          .forEach(
+              entry -> {
+                remoteMessage.put(PROPERTY_PREFIX + entry.getKey(), entry.getValue());
+              });
+    }
+
+    return new ZkNodeProps(remoteMessage);
+  }
+
+  public static class AddReplicaRequestBody implements JacksonReflectMapWriter {
+    @JsonProperty public String name;
+    @JsonProperty public String type; // TODO Make this an enum - see SOLR-15796
+    @JsonProperty public String instanceDir;
+    @JsonProperty public String dataDir;
+    @JsonProperty public String ulogDir;
+    @JsonProperty public String route;
+    @JsonProperty public Integer nrtReplicas;
+    @JsonProperty public Integer tlogReplicas;
+    @JsonProperty public Integer pullReplicas;
+    @JsonProperty public Boolean waitForFinalState;
+    @JsonProperty public Boolean followAliases;
+
+    @JsonProperty(ASYNC)
+    public String asyncId;
+
+    // TODO This cluster of properties could probably be simplified down to just "nodeSet".  See
+    // SOLR-15542
+    @JsonProperty public String node;
+
+    @JsonProperty("nodeSet")
+    public List<String> nodeSet;
+
+    @JsonProperty public Boolean skipNodeAssignment;
+
+    @JsonProperty public Map<String, String> properties;
+
+    public static AddReplicaRequestBody fromV1Params(SolrParams params) {
+      final var requestBody = new AddReplicaRequestBody();
+
+      requestBody.name = params.get(NAME);
+      requestBody.type = params.get(REPLICA_TYPE);
+      requestBody.instanceDir = params.get(INSTANCE_DIR);
+      requestBody.dataDir = params.get(DATA_DIR);
+      requestBody.ulogDir = params.get(ULOG_DIR);
+      requestBody.route = params.get(_ROUTE_);
+      requestBody.nrtReplicas = params.getInt(NRT_REPLICAS);
+      requestBody.tlogReplicas = params.getInt(TLOG_REPLICAS);
+      requestBody.pullReplicas = params.getInt(PULL_REPLICAS);
+      requestBody.waitForFinalState = params.getBool(WAIT_FOR_FINAL_STATE);
+      requestBody.followAliases = params.getBool(FOLLOW_ALIASES);
+      requestBody.asyncId = params.get(ASYNC);
+
+      requestBody.node = params.get(NODE);
+      if (params.get(CREATE_NODE_SET_PARAM) != null) {
+        requestBody.nodeSet = Arrays.asList(params.get(CREATE_NODE_SET).split(","));
+      }
+      requestBody.skipNodeAssignment = params.getBool(SKIP_NODE_ASSIGNMENT);
+
+      requestBody.properties =
+          copyPrefixedPropertiesWithoutPrefix(params, new HashMap<>(), PROPERTY_PREFIX);
+
+      return requestBody;
+    }
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 6bc17396b33..a5b0ee21781 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -867,11 +867,11 @@ public class TestPullReplica extends SolrCloudTestCase {
         url =
             String.format(
                 Locale.ROOT,
-                "%s/____v2/c/%s/shards",
+                "%s/____v2/collections/%s/shards/%s/replicas",
                 cluster.getRandomJetty(random()).getBaseUrl(),
-                collectionName);
-        String requestBody =
-            String.format(Locale.ROOT, "{add-replica:{shard:%s, type:%s}}", shardName, type);
+                collectionName,
+                shardName);
+        String requestBody = String.format(Locale.ROOT, "{\"type\": \"%s\"}", type);
         HttpPost addReplicaPost = new HttpPost(url);
         addReplicaPost.setHeader("Content-type", "application/json");
         addReplicaPost.setEntity(new StringEntity(requestBody));
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index ff391813670..937b8f1fe17 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -38,6 +38,7 @@ import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.StringEntity;
+import org.apache.http.util.EntityUtils;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -359,15 +360,19 @@ public class TestTlogReplica extends SolrCloudTestCase {
         url =
             String.format(
                 Locale.ROOT,
-                "%s/____v2/c/%s/shards",
+                "%s/____v2/collections/%s/shards/%s/replicas",
                 cluster.getRandomJetty(random()).getBaseUrl(),
-                collectionName);
-        String requestBody =
-            String.format(Locale.ROOT, "{add-replica:{shard:%s, type:%s}}", shardName, type);
+                collectionName,
+                shardName);
+        String requestBody = String.format(Locale.ROOT, "{\"type\": \"%s\"}", type);
         HttpPost addReplicaPost = new HttpPost(url);
         addReplicaPost.setHeader("Content-type", "application/json");
         addReplicaPost.setEntity(new StringEntity(requestBody));
         httpResponse = getHttpClient().execute(addReplicaPost);
+        if (httpResponse.getStatusLine().getStatusCode() == 400) {
+          final String entity = EntityUtils.toString(httpResponse.getEntity());
+          System.out.println(entity);
+        }
         assertEquals(200, httpResponse.getStatusLine().getStatusCode());
         break;
     }
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 07601cc624d..e416d35bf5b 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
@@ -100,12 +100,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     parts = new HashMap<>();
     api =
         V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards", "POST", null, parts);
-    assertConditions(
-        api.getSpec(),
-        Map.of(
-            "/methods[0]", "POST",
-            "/commands/split", NOT_NULL,
-            "/commands/add-replica", NOT_NULL));
+    assertConditions(api.getSpec(), Map.of("/methods[0]", "POST", "/commands/split", NOT_NULL));
 
     parts = new HashMap<>();
     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 1411452d66a..fa11d4aa4cb 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
@@ -98,13 +98,6 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
         "{split:{shard:shard1, ranges: '0-1f4,1f5-3e8,3e9-5dc', coreProperties : {prop1:prop1Val, prop2:prop2Val} }}",
         "{collection: collName , shard : shard1, ranges :'0-1f4,1f5-3e8,3e9-5dc', operation : splitshard, property.prop1:prop1Val, property.prop2: prop2Val}");
 
-    compareOutput(
-        apiBag,
-        "/collections/collName/shards",
-        POST,
-        "{add-replica:{shard: shard1, node: 'localhost_8978' , coreProperties : {prop1:prop1Val, prop2:prop2Val} }}",
-        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, property.prop1:prop1Val, property.prop2: prop2Val}");
-
     compareOutput(
         apiBag,
         "/collections/collName/shards",
@@ -112,28 +105,6 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
         "{split:{ splitKey:id12345, coreProperties : {prop1:prop1Val, prop2:prop2Val} }}",
         "{collection: collName , split.key : id12345 , operation : splitshard, property.prop1:prop1Val, property.prop2: prop2Val}");
 
-    compareOutput(
-        apiBag,
-        "/collections/collName/shards",
-        POST,
-        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'TLOG' }}",
-        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, type: TLOG}");
-
-    compareOutput(
-        apiBag,
-        "/collections/collName/shards",
-        POST,
-        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'PULL' }}",
-        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, type: PULL}");
-
-    // TODO annotation-based v2 APIs still miss enum support to validate the 'type' parameter as
-    // this test requires.
-    // Uncomment this test after fixing SOLR-15796
-    //    assertErrorContains(apiBag, "/collections/collName/shards", POST,
-    //        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'foo' }}", null,
-    //        "Value of enum must be one of"
-    //    );
-
     compareOutput(
         apiBag,
         "/cluster",
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/CreateReplicaAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/CreateReplicaAPITest.java
new file mode 100644
index 00000000000..d4e9772e866
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/CreateReplicaAPITest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.NRT_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
+import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
+import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
+import static org.apache.solr.common.params.CollectionAdminParams.SKIP_NODE_ASSIGNMENT;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
+import static org.apache.solr.common.params.CoreAdminParams.DATA_DIR;
+import static org.apache.solr.common.params.CoreAdminParams.INSTANCE_DIR;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+import static org.apache.solr.common.params.CoreAdminParams.NODE;
+import static org.apache.solr.common.params.CoreAdminParams.ULOG_DIR;
+import static org.apache.solr.common.params.ShardParams._ROUTE_;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.Test;
+
+/** Unit tests for {@link CreateReplicaAPI} */
+public class CreateReplicaAPITest extends SolrTestCaseJ4 {
+  @Test
+  public void testReportsErrorIfRequestBodyMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new CreateReplicaAPI(null, null, null);
+              api.createReplica("someCollName", "someShardName", null);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Required request-body is missing", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfCollectionNameMissing() {
+    final var requestBody = new CreateReplicaAPI.AddReplicaRequestBody();
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new CreateReplicaAPI(null, null, null);
+              api.createReplica(null, "shardName", requestBody);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: collection", thrown.getMessage());
+  }
+
+  @Test
+  public void testReportsErrorIfShardNameMissing() {
+    final var requestBody = new CreateReplicaAPI.AddReplicaRequestBody();
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new CreateReplicaAPI(null, null, null);
+              api.createReplica("someCollectionName", null, requestBody);
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: shard", thrown.getMessage());
+  }
+
+  @Test
+  public void testCreateRemoteMessageAllProperties() {
+    final var requestBody = new CreateReplicaAPI.AddReplicaRequestBody();
+    requestBody.name = "someName";
+    requestBody.type = "NRT";
+    requestBody.instanceDir = "/some/dir1";
+    requestBody.dataDir = "/some/dir2";
+    requestBody.ulogDir = "/some/dir3";
+    requestBody.route = "someRoute";
+    requestBody.nrtReplicas = 123;
+    requestBody.tlogReplicas = 456;
+    requestBody.pullReplicas = 789;
+    requestBody.nodeSet = List.of("node1", "node2");
+    requestBody.node = "node3";
+    requestBody.skipNodeAssignment = Boolean.TRUE;
+    requestBody.waitForFinalState = true;
+    requestBody.followAliases = true;
+    requestBody.asyncId = "someAsyncId";
+    requestBody.properties = Map.of("propName1", "propVal1", "propName2", "propVal2");
+
+    final var remoteMessage =
+        CreateReplicaAPI.createRemoteMessage("someCollectionName", "someShardName", requestBody)
+            .getProperties();
+
+    assertEquals(20, remoteMessage.size());
+    assertEquals("addreplica", remoteMessage.get(QUEUE_OPERATION));
+    assertEquals("someCollectionName", remoteMessage.get(COLLECTION));
+    assertEquals("someShardName", remoteMessage.get(SHARD_ID_PROP));
+    assertEquals("someName", remoteMessage.get(NAME));
+    assertEquals("NRT", remoteMessage.get(REPLICA_TYPE));
+    assertEquals("/some/dir1", remoteMessage.get(INSTANCE_DIR));
+    assertEquals("/some/dir2", remoteMessage.get(DATA_DIR));
+    assertEquals("/some/dir3", remoteMessage.get(ULOG_DIR));
+    assertEquals("someRoute", remoteMessage.get(_ROUTE_));
+    assertEquals(123, remoteMessage.get(NRT_REPLICAS));
+    assertEquals(456, remoteMessage.get(TLOG_REPLICAS));
+    assertEquals(789, remoteMessage.get(PULL_REPLICAS));
+    assertEquals("node1,node2", remoteMessage.get(CREATE_NODE_SET_PARAM));
+    assertEquals("node3", remoteMessage.get(NODE));
+    assertEquals(Boolean.TRUE, remoteMessage.get(SKIP_NODE_ASSIGNMENT));
+    assertEquals(true, remoteMessage.get(WAIT_FOR_FINAL_STATE));
+    assertEquals(true, remoteMessage.get(FOLLOW_ALIASES));
+    assertEquals("someAsyncId", remoteMessage.get(ASYNC));
+    assertEquals("propVal1", remoteMessage.get("property.propName1"));
+    assertEquals("propVal2", remoteMessage.get("property.propName2"));
+  }
+
+  @Test
+  public void testCanConvertV1ParamsToV2RequestBody() {
+    final var v1Params = new ModifiableSolrParams();
+    v1Params.add(COLLECTION, "someCollectionName");
+    v1Params.add(SHARD_ID_PROP, "someShardName");
+    v1Params.add(NAME, "someName");
+    v1Params.add(REPLICA_TYPE, "NRT");
+    v1Params.add(INSTANCE_DIR, "/some/dir1");
+    v1Params.add(DATA_DIR, "/some/dir2");
+    v1Params.add(ULOG_DIR, "/some/dir3");
+    v1Params.add(_ROUTE_, "someRoute");
+    v1Params.set(NRT_REPLICAS, 123);
+    v1Params.set(TLOG_REPLICAS, 456);
+    v1Params.set(PULL_REPLICAS, 789);
+    v1Params.add(CREATE_NODE_SET_PARAM, "node1,node2");
+    v1Params.add(NODE, "node3");
+    v1Params.set(SKIP_NODE_ASSIGNMENT, true);
+    v1Params.set(WAIT_FOR_FINAL_STATE, true);
+    v1Params.set(FOLLOW_ALIASES, true);
+    v1Params.add(ASYNC, "someAsyncId");
+    v1Params.add("property.propName1", "propVal1");
+    v1Params.add("property.propName2", "propVal2");
+
+    final var requestBody = CreateReplicaAPI.AddReplicaRequestBody.fromV1Params(v1Params);
+
+    assertEquals("someName", requestBody.name);
+    assertEquals("NRT", requestBody.type);
+    assertEquals("/some/dir1", requestBody.instanceDir);
+    assertEquals("/some/dir2", requestBody.dataDir);
+    assertEquals("/some/dir3", requestBody.ulogDir);
+    assertEquals("someRoute", requestBody.route);
+    assertEquals(Integer.valueOf(123), requestBody.nrtReplicas);
+    assertEquals(Integer.valueOf(456), requestBody.tlogReplicas);
+    assertEquals(Integer.valueOf(789), requestBody.pullReplicas);
+    assertEquals(List.of("node1", "node2"), requestBody.nodeSet);
+    assertEquals("node3", requestBody.node);
+    assertEquals(Boolean.TRUE, requestBody.skipNodeAssignment);
+    assertEquals(Boolean.TRUE, requestBody.waitForFinalState);
+    assertEquals(Boolean.TRUE, requestBody.followAliases);
+    assertEquals("someAsyncId", requestBody.asyncId);
+    assertEquals("propVal1", requestBody.properties.get("propName1"));
+    assertEquals("propVal2", requestBody.properties.get("propName2"));
+  }
+}
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 3d8d1448329..0d584973a7a 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
@@ -19,9 +19,7 @@ package org.apache.solr.handler.admin.api;
 
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
-import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
 import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
-import static org.apache.solr.common.params.CollectionParams.NAME;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
 import static org.apache.solr.common.params.CommonAdminParams.SPLIT_BY_PREFIX;
@@ -55,7 +53,6 @@ public class V2ShardsAPIMappingTest extends V2ApiMappingTest<CollectionsHandler>
   public void populateApiBag() {
     final CollectionsHandler collectionsHandler = getRequestHandler();
     apiBag.registerObject(new SplitShardAPI(collectionsHandler));
-    apiBag.registerObject(new AddReplicaAPI(collectionsHandler));
     apiBag.registerObject(new SyncShardAPI(collectionsHandler));
     apiBag.registerObject(new ForceLeaderAPI(collectionsHandler));
   }
@@ -131,48 +128,4 @@ public class V2ShardsAPIMappingTest extends V2ApiMappingTest<CollectionsHandler>
     assertEquals("foo1", v1Params.get("property.foo"));
     assertEquals("bar1", v1Params.get("property.bar"));
   }
-
-  @Test
-  public void testAddReplicaAllProperties() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/collName/shards",
-            "POST",
-            "{ 'add-replica': {"
-                + "'shard': 'shard1', "
-                + "'_route_': 'someRouteValue', "
-                + "'node': 'someNodeValue', "
-                + "'name': 'someName', "
-                + "'instanceDir': 'dir1', "
-                + "'dataDir': 'dir2', "
-                + "'ulogDir': 'dir3', "
-                + "'createNodeSet': ['foo', 'bar', 'baz'], "
-                + "'followAliases': true, "
-                + "'async': 'some_async_id', "
-                + "'waitForFinalState': true, "
-                + "'skipNodeAssignment': true, "
-                + "'type': 'tlog', "
-                + "'coreProperties': {"
-                + "    'foo': 'foo1', "
-                + "    'bar': 'bar1', "
-                + "}}}");
-
-    assertEquals(CollectionParams.CollectionAction.ADDREPLICA.lowerName, v1Params.get(ACTION));
-    assertEquals("collName", v1Params.get(COLLECTION));
-    assertEquals("shard1", v1Params.get(SHARD_ID_PROP));
-    assertEquals("someRouteValue", v1Params.get("_route_"));
-    assertEquals("someNodeValue", v1Params.get("node"));
-    assertEquals("foo,bar,baz", v1Params.get(CREATE_NODE_SET_PARAM));
-    assertEquals("someName", v1Params.get(NAME));
-    assertEquals("dir1", v1Params.get("instanceDir"));
-    assertEquals("dir2", v1Params.get("dataDir"));
-    assertEquals("dir3", v1Params.get("ulogDir"));
-    assertTrue(v1Params.getPrimitiveBool(FOLLOW_ALIASES));
-    assertEquals("some_async_id", v1Params.get(ASYNC));
-    assertTrue(v1Params.getPrimitiveBool(WAIT_FOR_FINAL_STATE));
-    assertTrue(v1Params.getPrimitiveBool("skipNodeAssignment"));
-    assertEquals("tlog", v1Params.get("type"));
-    assertEquals("foo1", v1Params.get("property.foo"));
-    assertEquals("bar1", v1Params.get("property.bar"));
-  }
 }
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 39667652845..3a0e7c700d2 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
@@ -73,12 +73,9 @@ http://localhost:8983/solr/admin/collections?action=ADDREPLICA&collection=techpr
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/api/collections/techproducts/shards -H 'Content-Type: application/json' -d '
+curl -X POST http://localhost:8983/api/collections/techproducts/shards/shard1/replicas -H 'Content-Type: application/json' -d '
   {
-    "add-replica":{
-      "shard":"shard1",
-      "node":"localhost:8983_solr"
-    }
+    "node":"localhost:8983_solr"
   }
 '
 ----
@@ -123,7 +120,7 @@ The name of the shard to which replica is to be added.
 +
 If `shard` is not specified, then `\_route_` must be.
 
-`\_route_`::
+`\_route_` (v1), `route` (v2)::
 +
 [%autowidth,frame=none]
 |===
@@ -143,18 +140,19 @@ Ignored if the `shard` parameter is also specified.
 +
 The name of the node where the replica should be created.
 
-`createNodeSet`::
+`createNodeSet` (v1), `nodeSet` (v2)::
 +
 [%autowidth,frame=none]
 |===
 |Optional |Default: none
 |===
 +
-A comma-separated list of nodes among which the best ones will be chosen to place the replicas.
+Placement candidates for the newly created replica(s).
 +
-The format is a comma-separated list of node_names, such as `localhost:8983_solr,localhost:8984_solr,localhost:8985_solr`.
+Provided as a comma-separated list of node names in v1 requests, such as `localhost:8983_solr,localhost:8984_solr,localhost:8985_solr`.
+In v2 requests, `nodeSet` expects the values as a true list, such as `["localhost:8983_solr", "localhost:8984_solr", "localhost:8985_solr"]`.
 
-NOTE: If neither `node` nor `createNodeSet` are specified then the best node(s) from among all the live nodes in the cluster are chosen.
+NOTE: If neither `node` nor `createNodeSet`/`nodeSet` are specified then the best node(s) from among all the live nodes in the cluster are chosen.
 
 `instanceDir`::
 +
@@ -228,7 +226,7 @@ Defaults to `1` if `type` is `pull` otherwise `0`.
 |Optional |Default: none
 |===
 +
-Set core property _name_ to _value_.
+Name/value pairs to use as additional properties in the created core.
 See xref:configuration-guide:core-discovery.adoc[] for details about supported properties and values.
 
 [WARNING]
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/AddReplicaPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/AddReplicaPayload.java
deleted file mode 100644
index ca061cf7823..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/AddReplicaPayload.java
+++ /dev/null
@@ -1,56 +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 java.util.List;
-import java.util.Map;
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
-
-public class AddReplicaPayload implements ReflectMapWriter {
-  @JsonProperty public String shard;
-
-  @JsonProperty public String _route_;
-
-  // TODO Remove in favor of a createNodeSet/nodeSet param with size=1 (see SOLR-15542)
-  @JsonProperty public String node;
-
-  // TODO Rename to 'nodeSet' to match the name used by create-shard and other APIs (see SOLR-15542)
-  @JsonProperty public List<String> createNodeSet;
-
-  @JsonProperty public String name;
-
-  @JsonProperty public String instanceDir;
-
-  @JsonProperty public String dataDir;
-
-  @JsonProperty public String ulogDir;
-
-  @JsonProperty public Map<String, Object> coreProperties;
-
-  @JsonProperty public String async;
-
-  @JsonProperty public Boolean waitForFinalState;
-
-  @JsonProperty public Boolean followAliases;
-
-  @JsonProperty public Boolean skipNodeAssignment;
-
-  // TODO Make this an enum - see SOLR-15796
-  @JsonProperty public String type;
-}