You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ge...@apache.org on 2021/02/08 15:12:14 UTC

[lucene-solr] branch master updated: SOLR-15118: Convert /v2/collections APIs to annotations (#2281)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e89fba6  SOLR-15118: Convert /v2/collections APIs to annotations (#2281)
e89fba6 is described below

commit e89fba6fe7739cb213560b94b6a4a76e40c2e4a5
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Mon Feb 8 10:11:58 2021 -0500

    SOLR-15118: Convert /v2/collections APIs to annotations (#2281)
    
    Solr supports two different ways to write v2 APIs: a JSON spec based
    approach, and one based on annotated POJOs.  The POJO method is now
    preferred.
    
    This commit switches the /v2/collections APIs over to the
    annotation-based approach.  Since V2RequestSupport only works with
    jsonspec-based APIs, this commit also changes CollectionAdminRequest
    to no longer implement that interface.
---
 .../java/org/apache/solr/core/CoreContainer.java   |   4 +-
 .../java/org/apache/solr/handler/ClusterAPI.java   |  14 +-
 .../org/apache/solr/handler/CollectionsAPI.java    | 177 +++++++++++-
 .../org/apache/solr/servlet/QueryRateLimiter.java  |   8 +-
 .../cloud/api/collections/TestCollectionAPI.java   |   7 +-
 .../solr/handler/admin/TestCollectionAPIs.java     |   4 +-
 .../handler/admin/V2CollectionsAPIMappingTest.java | 293 ++++++++++++++++++++
 .../solrj/request/CollectionAdminRequest.java      |  51 ++--
 .../client/solrj/request/CollectionApiMapping.java |  74 +----
 ...onfigInfo.java => BackupCollectionPayload.java} |  40 ++-
 ...lusterPropInfo.java => ClusterPropPayload.java} |  10 +-
 .../solrj/request/beans/CreateAliasPayload.java    |  79 ++++++
 ...ateConfigInfo.java => CreateConfigPayload.java} |   2 +-
 .../{ClusterPropInfo.java => CreatePayload.java}   |  54 ++--
 ...eateConfigInfo.java => DeleteAliasPayload.java} |  14 +-
 ...ateLimiterMeta.java => RateLimiterPayload.java} |  10 +-
 ...nfigInfo.java => RestoreCollectionPayload.java} |  39 ++-
 ...onfigInfo.java => SetAliasPropertyPayload.java} |  19 +-
 .../client/solrj/request/beans/V2ApiConstants.java |  55 ++++
 .../solr/common/params/CollectionAdminParams.java  |   5 +
 .../resources/apispec/collections.Commands.json    | 298 ---------------------
 .../client/solrj/request/TestV1toV2ApiMapper.java  |  26 +-
 .../apache/solr/common/util/JsonValidatorTest.java |  52 ++--
 23 files changed, 793 insertions(+), 542 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 26bb7d3..89d9df4 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -743,7 +743,9 @@ public class CoreContainer {
     createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
     createHandler(ZK_STATUS_PATH, ZookeeperStatusHandler.class.getName(), ZookeeperStatusHandler.class);
     collectionsHandler = createHandler(COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
-    containerHandlers.getApiBag().registerObject(new CollectionsAPI(collectionsHandler));
+    final CollectionsAPI collectionsAPI = new CollectionsAPI(collectionsHandler);
+    containerHandlers.getApiBag().registerObject(collectionsAPI);
+    containerHandlers.getApiBag().registerObject(collectionsAPI.collectionsCommands);
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
     ClusterAPI clusterAPI = new ClusterAPI(collectionsHandler, configSetsHandler);
     containerHandlers.getApiBag().registerObject(clusterAPI);
diff --git a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
index ee77e3d..aee2571 100644
--- a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
@@ -23,9 +23,9 @@ 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.ClusterPropInfo;
-import org.apache.solr.client.solrj.request.beans.CreateConfigInfo;
-import org.apache.solr.client.solrj.request.beans.RateLimiterMeta;
+import org.apache.solr.client.solrj.request.beans.ClusterPropPayload;
+import org.apache.solr.client.solrj.request.beans.CreateConfigPayload;
+import org.apache.solr.client.solrj.request.beans.RateLimiterPayload;
 import org.apache.solr.cloud.OverseerConfigSetMessageHandler;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
@@ -120,7 +120,7 @@ public class ClusterAPI {
 
     @Command(name = "create")
     @SuppressWarnings("unchecked")
-    public void create(PayloadObj<CreateConfigInfo> obj) throws Exception {
+    public void create(PayloadObj<CreateConfigPayload> obj) throws Exception {
       Map<String, Object> mapVals = obj.get().toMap(new HashMap<>());
       Map<String,Object> customProps = (Map<String, Object>) mapVals.remove("properties");
       if(customProps!= null) {
@@ -223,7 +223,7 @@ public class ClusterAPI {
 
     @Command(name = "set-obj-property")
     @SuppressWarnings({"rawtypes", "unchecked"})
-    public void setObjProperty(PayloadObj<ClusterPropInfo> obj) {
+    public void setObjProperty(PayloadObj<ClusterPropPayload> obj) {
       //Not using the object directly here because the API differentiate between {name:null} and {}
       Map m = obj.getDataMap();
       ClusterProperties clusterProperties = new ClusterProperties(getCoreContainer().getZkController().getZkClient());
@@ -242,8 +242,8 @@ public class ClusterAPI {
     }
 
     @Command(name = "set-ratelimiter")
-    public void setRateLimiters(PayloadObj<RateLimiterMeta> payLoad) {
-      RateLimiterMeta rateLimiterConfig = payLoad.get();
+    public void setRateLimiters(PayloadObj<RateLimiterPayload> payLoad) {
+      RateLimiterPayload rateLimiterConfig = payLoad.get();
       ClusterProperties clusterProperties = new ClusterProperties(getCoreContainer().getZkController().getZkClient());
 
       try {
diff --git a/solr/core/src/java/org/apache/solr/handler/CollectionsAPI.java b/solr/core/src/java/org/apache/solr/handler/CollectionsAPI.java
index e02e5f2..a5b6b12 100644
--- a/solr/core/src/java/org/apache/solr/handler/CollectionsAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/CollectionsAPI.java
@@ -17,16 +17,38 @@
 
 package org.apache.solr.handler;
 
+import org.apache.commons.collections4.CollectionUtils;
+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.BackupCollectionPayload;
+import org.apache.solr.client.solrj.request.beans.CreateAliasPayload;
+import org.apache.solr.client.solrj.request.beans.CreatePayload;
+import org.apache.solr.client.solrj.request.beans.DeleteAliasPayload;
+import org.apache.solr.client.solrj.request.beans.RestoreCollectionPayload;
+import org.apache.solr.client.solrj.request.beans.SetAliasPropertyPayload;
+import org.apache.solr.client.solrj.request.beans.V2ApiConstants;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
+import static org.apache.solr.client.solrj.request.beans.V2ApiConstants.ROUTER_KEY;
+import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
+import static org.apache.solr.common.params.CollectionAdminParams.ROUTER_PREFIX;
+import static org.apache.solr.common.params.CommonParams.ACTION;
 import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.handler.ClusterAPI.wrapParams;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_READ_PERM;
 
@@ -36,7 +58,16 @@ import static org.apache.solr.security.PermissionNameProvider.Name.COLL_READ_PER
  */
 public class CollectionsAPI {
 
-  private final CollectionsHandler collectionsHandler;
+    public static final String V2_CREATE_COLLECTION_CMD = "create";
+    public static final String V2_BACKUP_CMD = "backup-collection";
+    public static final String V2_RESTORE_CMD = "restore-collection";
+    public static final String V2_CREATE_ALIAS_CMD = "create-alias";
+    public static final String V2_SET_ALIAS_PROP_CMD = "set-alias-property";
+    public static final String V2_DELETE_ALIAS_CMD = "delete-alias";
+
+    private final CollectionsHandler collectionsHandler;
+
+  public  final CollectionsCommands collectionsCommands = new CollectionsCommands();
 
   public CollectionsAPI(CollectionsHandler collectionsHandler) {
     this.collectionsHandler = collectionsHandler;
@@ -50,11 +81,149 @@ public class CollectionsAPI {
     CollectionsHandler.CollectionOperation.LIST_OP.execute(req, rsp, collectionsHandler);
   }
 
+    @EndPoint(
+            path = {"/c", "/collections"},
+            method = POST,
+            permission = COLL_EDIT_PERM)
+    public class CollectionsCommands {
+
+        @Command(name = V2_BACKUP_CMD)
+        @SuppressWarnings("unchecked")
+        public void backupCollection(PayloadObj<BackupCollectionPayload> obj) throws Exception {
+            final Map<String, Object> v1Params = obj.get().toMap(new HashMap<>());
+            v1Params.put(ACTION, CollectionAction.BACKUP.toLower());
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @Command(name = V2_RESTORE_CMD)
+        @SuppressWarnings("unchecked")
+        public void restoreBackup(PayloadObj<RestoreCollectionPayload> obj) throws Exception {
+            final RestoreCollectionPayload v2Body = obj.get();
+            final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
+
+            v1Params.put(ACTION, CollectionAction.RESTORE.toLower());
+            if (v2Body.createCollectionParams != null && !v2Body.createCollectionParams.isEmpty()) {
+                final Map<String, Object> createCollParams = (Map<String, Object>) v1Params.remove(V2ApiConstants.CREATE_COLLECTION_KEY);
+                convertV2CreateCollectionMapToV1ParamMap(createCollParams);
+                v1Params.putAll(createCollParams);
+            }
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @Command(name = V2_CREATE_ALIAS_CMD)
+        @SuppressWarnings("unchecked")
+        public void createAlias(PayloadObj<CreateAliasPayload> obj) throws Exception {
+            final CreateAliasPayload v2Body = obj.get();
+            final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
+
+            v1Params.put(ACTION, CollectionAction.CREATEALIAS.toLower());
+            if (! CollectionUtils.isEmpty(v2Body.collections)) {
+                final String collectionsStr = String.join(",", v2Body.collections);
+                v1Params.remove(V2ApiConstants.COLLECTIONS);
+                v1Params.put(V2ApiConstants.COLLECTIONS, collectionsStr);
+            }
+            if (v2Body.router != null) {
+                Map<String, Object> routerProperties = (Map<String, Object>) v1Params.remove(V2ApiConstants.ROUTER_KEY);
+                flattenMapWithPrefix(routerProperties, v1Params, ROUTER_PREFIX);
+            }
+            if (v2Body.createCollectionParams != null && !v2Body.createCollectionParams.isEmpty()) {
+                final Map<String, Object> createCollectionMap = (Map<String, Object>) v1Params.remove(V2ApiConstants.CREATE_COLLECTION_KEY);
+                convertV2CreateCollectionMapToV1ParamMap(createCollectionMap);
+                flattenMapWithPrefix(createCollectionMap, v1Params, CREATE_COLLECTION_PREFIX);
+            }
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @Command(name= V2_SET_ALIAS_PROP_CMD)
+        @SuppressWarnings("unchecked")
+        public void setAliasProperty(PayloadObj<SetAliasPropertyPayload> obj) throws Exception {
+            final SetAliasPropertyPayload v2Body = obj.get();
+            final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
+
+            v1Params.put(ACTION, CollectionAction.ALIASPROP.toLower());
+            // Flatten "properties" map into individual prefixed params
+            final Map<String, Object> propertiesMap = (Map<String, Object>) v1Params.remove(V2ApiConstants.PROPERTIES_KEY);
+            flattenMapWithPrefix(propertiesMap, v1Params, PROPERTY_PREFIX);
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @Command(name= V2_DELETE_ALIAS_CMD)
+        @SuppressWarnings("unchecked")
+        public void deleteAlias(PayloadObj<DeleteAliasPayload> obj) throws Exception {
+            final DeleteAliasPayload v2Body = obj.get();
+            final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
+            v1Params.put(ACTION, CollectionAction.DELETEALIAS.toLower());
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @Command(name = V2_CREATE_COLLECTION_CMD)
+        @SuppressWarnings("unchecked")
+        public void create(PayloadObj<CreatePayload> obj) throws Exception {
+            final CreatePayload v2Body = obj.get();
+            final Map<String, Object> v1Params = v2Body.toMap(new HashMap<>());
+
+            v1Params.put(ACTION, CollectionAction.CREATE.toLower());
+            convertV2CreateCollectionMapToV1ParamMap(v1Params);
+
+            collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
+        }
+
+        @SuppressWarnings("unchecked")
+        private void convertV2CreateCollectionMapToV1ParamMap(Map<String, Object> v2MapVals) {
+            // Keys are copied so that map can be modified as keys are looped through.
+            final Set<String> v2Keys = v2MapVals.keySet().stream().collect(Collectors.toSet());
+            for (String key : v2Keys) {
+                switch (key) {
+                    case V2ApiConstants.PROPERTIES_KEY:
+                        final Map<String, Object> propertiesMap = (Map<String, Object>) v2MapVals.remove(V2ApiConstants.PROPERTIES_KEY);
+                        flattenMapWithPrefix(propertiesMap, v2MapVals, PROPERTY_PREFIX);
+                        break;
+                    case ROUTER_KEY:
+                        final Map<String, Object> routerProperties = (Map<String, Object>) v2MapVals.remove(V2ApiConstants.ROUTER_KEY);
+                        flattenMapWithPrefix(routerProperties, v2MapVals, CollectionAdminParams.ROUTER_PREFIX);
+                        break;
+                    case V2ApiConstants.CONFIG:
+                        v2MapVals.put(CollectionAdminParams.COLL_CONF, v2MapVals.remove(V2ApiConstants.CONFIG));
+                        break;
+                    case V2ApiConstants.SHUFFLE_NODES:
+                        v2MapVals.put(CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM, v2MapVals.remove(V2ApiConstants.SHUFFLE_NODES));
+                        break;
+                    case V2ApiConstants.NODE_SET:
+                        final Object nodeSetValUncast = v2MapVals.remove(V2ApiConstants.NODE_SET);
+                        if (nodeSetValUncast instanceof String) {
+                            v2MapVals.put(CollectionAdminParams.CREATE_NODE_SET_PARAM, nodeSetValUncast);
+                        } else {
+                            final List<String> nodeSetList = (List<String>) nodeSetValUncast;
+                            final String nodeSetStr = String.join(",", nodeSetList);
+                            v2MapVals.put(CollectionAdminParams.CREATE_NODE_SET_PARAM, nodeSetStr);
+                        }
+                        break;
+                    default:
+                        break;
+                }
+            }
+        }
+
+        private void flattenMapWithPrefix(Map<String, Object> toFlatten, Map<String, Object> destination,
+                                          String additionalPrefix) {
+            if (toFlatten == null || toFlatten.isEmpty() || destination == null) {
+                return;
+            }
+
+            toFlatten.forEach((k, v) -> destination.put(additionalPrefix + k, v));
+        }
+  }
+
   @EndPoint(path = {"/c/{collection}", "/collections/{collection}"},
       method = DELETE,
       permission = COLL_EDIT_PERM)
   public void deleteCollection(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    req = ClusterAPI.wrapParams(req, "action",
+    req = wrapParams(req, ACTION,
         CollectionAction.DELETE.toString(),
         NAME, req.getPathTemplateValues().get(ZkStateReader.COLLECTION_PROP));
     collectionsHandler.handleRequestBody(req, rsp);
diff --git a/solr/core/src/java/org/apache/solr/servlet/QueryRateLimiter.java b/solr/core/src/java/org/apache/solr/servlet/QueryRateLimiter.java
index f746aca..36e5d3b 100644
--- a/solr/core/src/java/org/apache/solr/servlet/QueryRateLimiter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/QueryRateLimiter.java
@@ -22,7 +22,7 @@ import java.util.Map;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.request.beans.RateLimiterMeta;
+import org.apache.solr.client.solrj.request.beans.RateLimiterPayload;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
@@ -51,7 +51,7 @@ public class QueryRateLimiter extends RequestRateLimiter {
       return;
     }
 
-    RateLimiterMeta rateLimiterMeta = mapper.readValue(configInput, RateLimiterMeta.class);
+    RateLimiterPayload rateLimiterMeta = mapper.readValue(configInput, RateLimiterPayload.class);
 
     constructQueryRateLimiterConfigInternal(rateLimiterMeta, rateLimiterConfig);
   }
@@ -74,7 +74,7 @@ public class QueryRateLimiter extends RequestRateLimiter {
         return rateLimiterConfig;
       }
 
-      RateLimiterMeta rateLimiterMeta = mapper.readValue(configInput, RateLimiterMeta.class);
+      RateLimiterPayload rateLimiterMeta = mapper.readValue(configInput, RateLimiterPayload.class);
 
       constructQueryRateLimiterConfigInternal(rateLimiterMeta, rateLimiterConfig);
 
@@ -88,7 +88,7 @@ public class QueryRateLimiter extends RequestRateLimiter {
     }
   }
 
-  private static void constructQueryRateLimiterConfigInternal(RateLimiterMeta rateLimiterMeta, RateLimiterConfig rateLimiterConfig) {
+  private static void constructQueryRateLimiterConfigInternal(RateLimiterPayload rateLimiterMeta, RateLimiterConfig rateLimiterConfig) {
 
     if (rateLimiterMeta == null) {
       // No Rate limiter configuration defined in clusterprops.json
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
index ef718e0..17f34e3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
@@ -69,9 +69,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
       } else {
         req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2, 1, 0, 1);
       }
-      setV2(req);
       client.request(req);
-      assertV2CallsCount();
       createCollection(null, COLLECTION_NAME1, 1, 1, client, null, "conf1");
     }
 
@@ -414,8 +412,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
   private void clusterStatusZNodeVersion() throws Exception {
     String cname = "clusterStatusZNodeVersion";
     try (CloudSolrClient client = createCloudClient(null)) {
-      setV2(CollectionAdminRequest.createCollection(cname, "conf1", 1, 1)).process(client);
-      assertV2CallsCount();
+      CollectionAdminRequest.createCollection(cname, "conf1", 1, 1).process(client);
       waitForRecoveriesToFinish(cname, true);
 
       ModifiableSolrParams params = new ModifiableSolrParams();
@@ -438,9 +435,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
       assertNotNull(znodeVersion);
 
       CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(cname, "shard1");
-      setV2(addReplica);
       addReplica.process(client);
-      assertV2CallsCount();
       waitForRecoveriesToFinish(cname, true);
 
       rsp = client.request(request);
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 f423f0f..cb3562c 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
@@ -84,7 +84,9 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
     ApiBag apiBag;
     try (MockCollectionsHandler collectionsHandler = new MockCollectionsHandler()) {
       apiBag = new ApiBag(false);
+      final CollectionsAPI collectionsAPI = new CollectionsAPI(collectionsHandler);
       apiBag.registerObject(new CollectionsAPI(collectionsHandler));
+      apiBag.registerObject(collectionsAPI.collectionsCommands);
       Collection<Api> apis = collectionsHandler.getApis();
       for (Api api : apis) apiBag.register(api, Collections.emptyMap());
 
@@ -112,7 +114,7 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
 
 
     compareOutput(apiBag, "/collections", POST,
-        "{create-alias:{name: aliasName , collections:[c1,c2] }}", null, "{operation : createalias, name: aliasName, collections:[c1,c2] }");
+        "{create-alias:{name: aliasName , collections:[c1,c2] }}", null, "{operation : createalias, name: aliasName, collections:\"c1,c2\" }");
 
     compareOutput(apiBag, "/collections", POST,
         "{delete-alias:{ name: aliasName}}", null, "{operation : deletealias, name: aliasName}");
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionsAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionsAPIMappingTest.java
new file mode 100644
index 0000000..689073c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionsAPIMappingTest.java
@@ -0,0 +1,293 @@
+/*
+ * 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;
+
+import com.google.common.collect.Maps;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.api.collections.CategoryRoutedAlias;
+import org.apache.solr.cloud.api.collections.RoutedAlias;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.core.backup.BackupManager;
+import org.apache.solr.handler.CollectionsAPI;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.solr.common.params.CommonParams.ACTION;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Unit tests for the API mappings found in {@link org.apache.solr.handler.CollectionsAPI}.
+ *
+ * This test bears many similarities to {@link TestCollectionAPIs} which appears to test the mappings indirectly by
+ * checking message sent to the ZK overseer (which is similar, but not identical to the v1 param list).  If there's no
+ * particular benefit to testing the mappings in this way (there very well may be), then we should combine these two
+ * test classes at some point in the future using the simpler approach here.
+ *
+ * Note that the V2 requests made by these tests are not necessarily semantically valid.  They shouldn't be taken as
+ * examples. In several instances, mutually exclusive JSON parameters are provided.  This is done to exercise conversion
+ * of all parameters, even if particular combinations are never expected in the same request.
+ */
+public class V2CollectionsAPIMappingTest extends SolrTestCaseJ4 {
+
+    private ApiBag apiBag;
+
+    private ArgumentCaptor<SolrQueryRequest> queryRequestCaptor;
+    private CollectionsHandler mockCollectionsHandler;
+
+    @BeforeClass
+    public static void ensureWorkingMockito() {
+        assumeWorkingMockito();
+    }
+
+    @Before
+    public void setupApiBag() throws Exception {
+        mockCollectionsHandler = mock(CollectionsHandler.class);
+        queryRequestCaptor = ArgumentCaptor.forClass(SolrQueryRequest.class);
+
+        apiBag = new ApiBag(false);
+        final CollectionsAPI collectionsAPI = new CollectionsAPI(mockCollectionsHandler);
+        apiBag.registerObject(collectionsAPI);
+        apiBag.registerObject(collectionsAPI.collectionsCommands);
+    }
+
+    @Test
+    public void testCreateCollectionAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'create': {" +
+                        "'name': 'techproducts', " +
+                        "'config':'_default', " +
+                        "'router': {'name': 'composite', 'field': 'routeField', 'foo': 'bar'}, " +
+                        "'shards': 'customShardName,anotherCustomShardName', " +
+                        "'replicationFactor': 3," +
+                        "'nrtReplicas': 1, " +
+                        "'tlogReplicas': 1, " +
+                        "'pullReplicas': 1, " +
+                        "'nodeSet': ['localhost:8983_solr', 'localhost:7574_solr']," +
+                        "'shuffleNodes': true," +
+                        "'properties': {'foo': 'bar', 'foo2': 'bar2'}, " +
+                        "'async': 'requestTrackingId', " +
+                        "'waitForFinalState': false, " +
+                        "'perReplicaState': false," +
+                        "'numShards': 1}}");
+
+        assertEquals(CollectionParams.CollectionAction.CREATE.lowerName, v1Params.get(ACTION));
+        assertEquals("techproducts", v1Params.get(CommonParams.NAME));
+        assertEquals("_default", v1Params.get(CollectionAdminParams.COLL_CONF));
+        assertEquals("composite", v1Params.get("router.name"));
+        assertEquals("routeField", v1Params.get("router.field"));
+        assertEquals("bar", v1Params.get("router.foo"));
+        assertEquals("customShardName,anotherCustomShardName", v1Params.get(ShardParams.SHARDS));
+        assertEquals(3, v1Params.getPrimitiveInt(ZkStateReader.REPLICATION_FACTOR));
+        assertEquals(1, v1Params.getPrimitiveInt(ZkStateReader.NRT_REPLICAS));
+        assertEquals(1, v1Params.getPrimitiveInt(ZkStateReader.TLOG_REPLICAS));
+        assertEquals(1, v1Params.getPrimitiveInt(ZkStateReader.PULL_REPLICAS));
+        assertEquals("localhost:8983_solr,localhost:7574_solr", v1Params.get(CollectionAdminParams.CREATE_NODE_SET_PARAM));
+        assertEquals(true, v1Params.getPrimitiveBool(CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM));
+        assertEquals("bar", v1Params.get("property.foo"));
+        assertEquals("bar2", v1Params.get("property.foo2"));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+        assertEquals(false, v1Params.getPrimitiveBool(CommonAdminParams.WAIT_FOR_FINAL_STATE));
+        assertEquals(false, v1Params.getPrimitiveBool(DocCollection.PER_REPLICA_STATE));
+        assertEquals(1, v1Params.getPrimitiveInt(CollectionAdminParams.NUM_SHARDS));
+    }
+
+    @Test
+    public void testCreateAliasAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'create-alias': {" +
+                        "'name': 'aliasName', " +
+                        "'collections': ['techproducts1', 'techproducts2'], " +
+                        "'tz': 'someTimeZone', " +
+                        "'async': 'requestTrackingId', " +
+                        "'router': {" +
+                        "    'name': 'time', " +
+                        "    'field': 'date_dt', " +
+                        "    'interval': '+1HOUR', " +
+                        "     'maxFutureMs': 3600, " +
+                        "     'preemptiveCreateMath': 'somePreemptiveCreateMathString', " +
+                        "     'autoDeleteAge': 'someAutoDeleteAgeExpression', " +
+                        "     'maxCardinality': 36, " +
+                        "     'mustMatch': 'someRegex', " +
+                        "}, " +
+                        "'create-collection': {" +
+                        "     'numShards': 1, " +
+                        "     'properties': {'foo': 'bar', 'foo2': 'bar2'}, " +
+                        "     'replicationFactor': 3 " +
+                        "}" +
+                        "}}");
+
+        assertEquals(CollectionParams.CollectionAction.CREATEALIAS.lowerName, v1Params.get(ACTION));
+        assertEquals("aliasName", v1Params.get(CommonParams.NAME));
+        assertEquals("techproducts1,techproducts2", v1Params.get("collections"));
+        assertEquals("someTimeZone", v1Params.get(CommonParams.TZ.toLowerCase(Locale.ROOT)));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+        assertEquals("time", v1Params.get(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_TYPE_NAME));
+        assertEquals("date_dt", v1Params.get(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_FIELD));
+        assertEquals("+1HOUR", v1Params.get(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_INTERVAL));
+        assertEquals(3600, v1Params.getPrimitiveInt(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_MAX_FUTURE));
+        assertEquals("somePreemptiveCreateMathString", v1Params.get(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_PREEMPTIVE_CREATE_WINDOW));
+        assertEquals("someAutoDeleteAgeExpression", v1Params.get(CollectionAdminRequest.CreateTimeRoutedAlias.ROUTER_AUTO_DELETE_AGE));
+        assertEquals(36, v1Params.getPrimitiveInt(CategoryRoutedAlias.ROUTER_MAX_CARDINALITY));
+        assertEquals("someRegex", v1Params.get(CategoryRoutedAlias.ROUTER_MUST_MATCH));
+        assertEquals(1, v1Params.getPrimitiveInt(RoutedAlias.CREATE_COLLECTION_PREFIX + CollectionAdminParams.NUM_SHARDS));
+        assertEquals("bar", v1Params.get(RoutedAlias.CREATE_COLLECTION_PREFIX + "property.foo"));
+        assertEquals("bar2", v1Params.get(RoutedAlias.CREATE_COLLECTION_PREFIX + "property.foo2"));
+        assertEquals(3, v1Params.getPrimitiveInt(RoutedAlias.CREATE_COLLECTION_PREFIX + ZkStateReader.REPLICATION_FACTOR));
+    }
+
+    @Test
+    public void testDeleteAliasAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'delete-alias': {" +
+                        "'name': 'aliasName', " +
+                        "'async': 'requestTrackingId'" +
+                        "}}");
+
+        assertEquals(CollectionParams.CollectionAction.DELETEALIAS.lowerName, v1Params.get(ACTION));
+        assertEquals("aliasName", v1Params.get(CommonParams.NAME));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+    }
+
+    @Test
+    public void testSetAliasAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'set-alias-property': {" +
+                        "'name': 'aliasName', " +
+                        "'async': 'requestTrackingId', " +
+                        "'properties': {'foo':'bar', 'foo2':'bar2'}" +
+                        "}}");
+
+        assertEquals(CollectionParams.CollectionAction.ALIASPROP.lowerName, v1Params.get(ACTION));
+        assertEquals("aliasName", v1Params.get(CommonParams.NAME));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+        assertEquals("bar", v1Params.get("property.foo"));
+        assertEquals("bar2", v1Params.get("property.foo2"));
+    }
+
+    @Test
+    public void testBackupAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'backup-collection': {" +
+                        "'name': 'backupName', " +
+                        "'collection': 'collectionName', " +
+                        "'location': '/some/location/uri', " +
+                        "'repository': 'someRepository', " +
+                        "'followAliases': true, " +
+                        "'indexBackup': 'copy-files', " +
+                        "'commitName': 'someSnapshotName', " +
+                        "'incremental': true, " +
+                        "'async': 'requestTrackingId' " +
+                        "}}");
+
+        assertEquals(CollectionParams.CollectionAction.BACKUP.lowerName, v1Params.get(ACTION));
+        assertEquals("backupName", v1Params.get(CommonParams.NAME));
+        assertEquals("collectionName", v1Params.get(BackupManager.COLLECTION_NAME_PROP));
+        assertEquals("/some/location/uri", v1Params.get(CoreAdminParams.BACKUP_LOCATION));
+        assertEquals("someRepository", v1Params.get(CoreAdminParams.BACKUP_REPOSITORY));
+        assertEquals(true, v1Params.getPrimitiveBool(CollectionAdminParams.FOLLOW_ALIASES));
+        assertEquals("copy-files", v1Params.get(CollectionAdminParams.INDEX_BACKUP_STRATEGY));
+        assertEquals("someSnapshotName", v1Params.get(CoreAdminParams.COMMIT_NAME));
+        assertEquals(true, v1Params.getPrimitiveBool(CoreAdminParams.BACKUP_INCREMENTAL));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+    }
+
+    @Test
+    public void testRestoreAllProperties() throws Exception {
+        final SolrParams v1Params = captureConvertedV1Params("/collections", "POST",
+                "{'restore-collection': {" +
+                        "'name': 'backupName', " +
+                        "'collection': 'collectionName', " +
+                        "'location': '/some/location/uri', " +
+                        "'repository': 'someRepository', " +
+                        "'backupId': 123, " +
+                        "'async': 'requestTrackingId', " +
+                        "'create-collection': {" +
+                        "     'numShards': 1, " +
+                        "     'properties': {'foo': 'bar', 'foo2': 'bar2'}, " +
+                        "     'replicationFactor': 3 " +
+                        "}" +
+                        "}}");
+
+        assertEquals(CollectionParams.CollectionAction.RESTORE.lowerName, v1Params.get(ACTION));
+        assertEquals("backupName", v1Params.get(CommonParams.NAME));
+        assertEquals("collectionName", v1Params.get(BackupManager.COLLECTION_NAME_PROP));
+        assertEquals("/some/location/uri", v1Params.get(CoreAdminParams.BACKUP_LOCATION));
+        assertEquals("someRepository", v1Params.get(CoreAdminParams.BACKUP_REPOSITORY));
+        assertEquals(123, v1Params.getPrimitiveInt(CoreAdminParams.BACKUP_ID));
+        assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
+        // NOTE: Unlike other v2 APIs that have a nested object for collection-creation params, restore's v1 equivalent
+        // for these properties doesn't have a "create-collection." prefix.
+        assertEquals(1, v1Params.getPrimitiveInt(CollectionAdminParams.NUM_SHARDS));
+        assertEquals("bar", v1Params.get("property.foo"));
+        assertEquals("bar2", v1Params.get("property.foo2"));
+        assertEquals(3, v1Params.getPrimitiveInt(ZkStateReader.REPLICATION_FACTOR));
+    }
+
+    private SolrParams captureConvertedV1Params(String path, String method, String v2RequestBody) throws Exception {
+        final HashMap<String, String> parts = new HashMap<>();
+        final Api api = apiBag.lookup(path, method, parts);
+        final SolrQueryResponse rsp = new SolrQueryResponse();
+        final LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, Maps.newHashMap()) {
+            @Override
+            public List<CommandOperation> getCommands(boolean validateInput) {
+                if (v2RequestBody == null) return Collections.emptyList();
+                return ApiBag.getCommandOperations(new ContentStreamBase.StringStream(v2RequestBody), api.getCommandSchema(), true);
+            }
+
+            @Override
+            public Map<String, String> getPathTemplateValues() {
+                return parts;
+            }
+
+            @Override
+            public String getHttpMethod() {
+                return method;
+            }
+        };
+
+
+        api.call(req, rsp);
+        verify(mockCollectionsHandler).handleRequestBody(queryRequestCaptor.capture(), any());
+        return queryRequestCaptor.getValue().getParams();
+    }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index e09ebb3..4037609 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -16,25 +16,11 @@
  */
 package org.apache.solr.client.solrj.request;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.solr.client.solrj.RoutedAliasTypes;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.V2RequestSupport;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
@@ -54,26 +40,29 @@ import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
 import static org.apache.solr.common.cloud.DocCollection.PER_REPLICA_STATE;
-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.READ_ONLY;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
-import static org.apache.solr.common.params.CollectionAdminParams.ALIAS;
-import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
-import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
-import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
-import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
-import static org.apache.solr.common.params.CollectionAdminParams.ROUTER_PREFIX;
-import static org.apache.solr.common.params.CollectionAdminParams.SKIP_NODE_ASSIGNMENT;
+import static org.apache.solr.common.cloud.ZkStateReader.*;
+import static org.apache.solr.common.params.CollectionAdminParams.*;
 
 /**
  * This class is experimental and subject to change.
  *
  * @since solr 4.5
  */
-public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> implements V2RequestSupport, MapWriter {
+public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> implements MapWriter {
 
   /**
    * The set of modifiable collection properties
@@ -98,14 +87,6 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   }
 
   @Override
-  @SuppressWarnings({"rawtypes"})
-  public SolrRequest getV2Request() {
-    return usev2 ?
-        V1toV2ApiMapper.convert(this).useBinary(useBinaryV2).build() :
-        this;
-  }
-
-  @Override
   public SolrParams getParams() {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CoreAdminParams.ACTION, action.toString());
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index 9b106ab..a88a656 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -18,6 +18,13 @@
 package org.apache.solr.client.solrj.request;
 
 
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.beans.V2ApiConstants;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.Utils;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -25,26 +32,11 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.common.util.CommandOperation;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.Utils;
-
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.CLUSTER_ALIASES;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.COLLECTIONS_COMMANDS;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.COLLECTION_STATE;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.PER_COLLECTION;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.PER_COLLECTION_PER_SHARD_DELETE;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE;
-import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.PER_COLLECTION_SHARDS_COMMANDS;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
+import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint.*;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -57,17 +49,6 @@ public class CollectionApiMapping {
   public enum Meta implements CommandMeta {
     GET_A_COLLECTION(COLLECTION_STATE, GET, CLUSTERSTATUS),
     LIST_ALIASES(CLUSTER_ALIASES, GET, LISTALIASES),
-    CREATE_COLLECTION(COLLECTIONS_COMMANDS,
-        POST,
-        CREATE,
-        CREATE.toLower(),
-        Utils.makeMap(
-            "collection.configName", "config",
-            "createNodeSet.shuffle", "shuffleNodes",
-            "createNodeSet", "nodeSet"
-        ),
-        Utils.makeMap("property.", "properties.")),
-
     RELOAD_COLL(PER_COLLECTION,
         POST,
         RELOAD,
@@ -91,34 +72,11 @@ public class CollectionApiMapping {
         POST,
         REBALANCELEADERS,
         "rebalance-leaders", null),
-    CREATE_ALIAS(COLLECTIONS_COMMANDS,
-        POST,
-        CREATEALIAS,
-        "create-alias",
-        CREATE_COLLECTION.paramsToAttrs.entrySet().stream().collect(Collectors.toMap(
-            entry -> "create-collection." + entry.getKey(),
-            entry -> "create-collection." + entry.getValue()
-        )),
-        CREATE_COLLECTION.prefixParamsToAttrs.entrySet().stream().collect(Collectors.toMap(
-            entry -> "create-collection." + entry.getKey(),
-            entry -> "create-collection." + entry.getValue()
-        ))),
-    DELETE_ALIAS(COLLECTIONS_COMMANDS,
-        POST,
-        DELETEALIAS,
-        "delete-alias",
-        null),
-    ALIAS_PROP(COLLECTIONS_COMMANDS,
-        POST,
-        ALIASPROP,
-        "set-alias-property",
-        null,
-        Utils.makeMap("property.", "properties.")),
     CREATE_SHARD(PER_COLLECTION_SHARDS_COMMANDS,
         POST,
         CREATESHARD,
         "create",
-        Utils.makeMap("createNodeSet", "nodeSet"),
+        Utils.makeMap("createNodeSet", V2ApiConstants.NODE_SET),
         Utils.makeMap("property.", "coreProperties.")) {
       @Override
       public String getParamSubstitute(String param) {
@@ -169,17 +127,6 @@ public class CollectionApiMapping {
             NAME, "collection",
             "propertyName", "name",
             "propertyValue", "value")),
-    BACKUP_COLLECTION(COLLECTIONS_COMMANDS,
-        POST,
-        BACKUP,
-        "backup-collection", null
-    ),
-    RESTORE_COLLECTION(COLLECTIONS_COMMANDS,
-        POST,
-        RESTORE,
-        "restore-collection",
-        null
-    ),
     FORCE_LEADER(PER_COLLECTION_PER_SHARD_COMMANDS, POST, CollectionAction.FORCELEADER, "force-leader", null),
     BALANCE_SHARD_UNIQUE(PER_COLLECTION, POST, BALANCESHARDUNIQUE,"balance-shard-unique" , null)
     ;
@@ -304,7 +251,6 @@ public class CollectionApiMapping {
 
   public enum EndPoint implements V2EndPoint {
     CLUSTER_ALIASES("cluster.aliases"),
-    COLLECTIONS_COMMANDS("collections.Commands"),
     COLLECTION_STATE("collections.collection"),
     PER_COLLECTION("collections.collection.Commands"),
     PER_COLLECTION_SHARDS_COMMANDS("collections.collection.shards.Commands"),
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BackupCollectionPayload.java
similarity index 58%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BackupCollectionPayload.java
index 5cd10b6..5d5e7ce 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/BackupCollectionPayload.java
@@ -16,17 +16,39 @@
  */
 package org.apache.solr.client.solrj.request.beans;
 
-import java.util.Map;
-
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class CreateConfigInfo implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String name;
-  @JsonProperty
-  public String baseConfigSet;
-  @JsonProperty
-  public Map<String,Object> properties;
+/**
+ * V2 API POJO for the /v2/collections 'backup-collection' command.
+ *
+ * Analogous to the request parameters for v1 /admin/collections?action=BACKUP API.
+ */
+public class BackupCollectionPayload implements ReflectMapWriter {
+    @JsonProperty(required = true)
+    public String collection;
+
+    @JsonProperty(required = true)
+    public String name;
+
+    @JsonProperty
+    public String location;
+
+    @JsonProperty
+    public String repository;
+
+    @JsonProperty
+    public Boolean followAliases;
+
+    @JsonProperty
+    public String indexBackup;
+
+    @JsonProperty
+    public String commitName;
+
+    @JsonProperty
+    public Boolean incremental;
 
+    @JsonProperty
+    public String async;
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropPayload.java
similarity index 86%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropPayload.java
index b8de08d..64b233a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropPayload.java
@@ -20,7 +20,7 @@ package org.apache.solr.client.solrj.request.beans;
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class ClusterPropInfo implements ReflectMapWriter {
+public class ClusterPropPayload implements ReflectMapWriter {
 
   @JsonProperty
   public String urlScheme;
@@ -31,7 +31,7 @@ public class ClusterPropInfo implements ReflectMapWriter {
   public String location;
 
   @JsonProperty
-  public DefaultsInfo defaults;
+  public Defaults defaults;
 
   @JsonProperty
   public CollectionDefaults collectionDefaults;
@@ -48,17 +48,17 @@ public class ClusterPropInfo implements ReflectMapWriter {
 
   }
 
-  public static class DefaultsInfo implements ReflectMapWriter {
+  public static class Defaults implements ReflectMapWriter {
 
     @JsonProperty
     public CollectionDefaults collection;
 
     @JsonProperty
-    public ClusterInfo cluster;
+    public Cluster cluster;
 
   }
 
-  public static class ClusterInfo implements ReflectMapWriter {
+  public static class Cluster implements ReflectMapWriter {
     @JsonProperty
     public Boolean useLegacyReplicaAssignment;
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateAliasPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateAliasPayload.java
new file mode 100644
index 0000000..b84b49c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateAliasPayload.java
@@ -0,0 +1,79 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.solr.client.solrj.request.beans.V2ApiConstants.CREATE_COLLECTION_KEY;
+
+public class CreateAliasPayload implements ReflectMapWriter {
+    @JsonProperty(required = true)
+    public String name;
+
+    @JsonProperty
+    public List<String> collections;
+
+    @JsonProperty
+    public AliasRouter router;
+
+    @JsonProperty
+    public String tz;
+
+    @JsonProperty(CREATE_COLLECTION_KEY)
+    public Map<String, Object> createCollectionParams;
+
+    @JsonProperty
+    public String async;
+
+    public static class AliasRouter implements ReflectMapWriter {
+        @JsonProperty(required = true)
+        public String name;
+
+        @JsonProperty
+        public String field;
+
+        @JsonProperty
+        public String start;
+
+        @JsonProperty
+        public String interval;
+
+        @JsonProperty
+        public Integer maxFutureMs;
+
+        @JsonProperty
+        public String preemptiveCreateMath;
+
+        @JsonProperty
+        public String autoDeleteAge;
+
+        @JsonProperty
+        public Integer maxCardinality;
+
+        @JsonProperty
+        public String mustMatch;
+
+        @JsonProperty
+        public List<Map<String, Object>> routerList;
+    }
+}
+
+
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigPayload.java
similarity index 94%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigPayload.java
index 5cd10b6..98b22b4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigPayload.java
@@ -21,7 +21,7 @@ import java.util.Map;
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class CreateConfigInfo implements ReflectMapWriter {
+public class CreateConfigPayload implements ReflectMapWriter {
   @JsonProperty(required = true)
   public String name;
   @JsonProperty
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreatePayload.java
similarity index 64%
rename from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java
rename to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreatePayload.java
index b8de08d..403af4d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ClusterPropInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreatePayload.java
@@ -14,59 +14,57 @@
  * 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 ClusterPropInfo implements ReflectMapWriter {
-
-  @JsonProperty
-  public String urlScheme;
+import java.util.List;
+import java.util.Map;
 
-  @JsonProperty
-  public Integer maxCoresPerNode;
-  @JsonProperty
-  public String location;
+public class CreatePayload implements ReflectMapWriter {
+    @JsonProperty(required = true)
+    public String name;
 
-  @JsonProperty
-  public DefaultsInfo defaults;
+    @JsonProperty
+    public String config;
 
-  @JsonProperty
-  public CollectionDefaults collectionDefaults;
+    @JsonProperty
+    public Map<String, Object> router;
 
-  public static class CollectionDefaults implements ReflectMapWriter {
     @JsonProperty
     public Integer numShards;
+
     @JsonProperty
-    public Integer tlogReplicas;
+    public String shards;
+
     @JsonProperty
-    public Integer pullReplicas;
+    public Integer replicationFactor;
+
     @JsonProperty
     public Integer nrtReplicas;
 
-  }
-
-  public static class DefaultsInfo implements ReflectMapWriter {
-
     @JsonProperty
-    public CollectionDefaults collection;
+    public Integer tlogReplicas;
 
     @JsonProperty
-    public ClusterInfo cluster;
-
-  }
+    public Integer pullReplicas;
 
-  public static class ClusterInfo implements ReflectMapWriter {
     @JsonProperty
-    public Boolean useLegacyReplicaAssignment;
+    public List<String> nodeSet;
 
+    @JsonProperty
+    public Boolean shuffleNodes;
 
     @JsonProperty
-    public CollectionDefaults collection;
+    public Map<String, Object> properties;
 
-  }
+    @JsonProperty
+    public String async;
 
+    @JsonProperty
+    public Boolean waitForFinalState;
 
+    @JsonProperty
+    public Boolean perReplicaState;
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteAliasPayload.java
similarity index 80%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteAliasPayload.java
index 5cd10b6..7565c3d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteAliasPayload.java
@@ -16,17 +16,13 @@
  */
 package org.apache.solr.client.solrj.request.beans;
 
-import java.util.Map;
-
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class CreateConfigInfo implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String name;
-  @JsonProperty
-  public String baseConfigSet;
-  @JsonProperty
-  public Map<String,Object> properties;
+public class DeleteAliasPayload implements ReflectMapWriter {
+    @JsonProperty(required = true)
+    public String name;
 
+    @JsonProperty
+    public String async;
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterMeta.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterPayload.java
similarity index 89%
rename from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterMeta.java
rename to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterPayload.java
index 7cf70fd..42058bc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterMeta.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RateLimiterPayload.java
@@ -25,7 +25,7 @@ import org.apache.solr.common.util.ReflectMapWriter;
 /**
  * POJO for Rate Limiter Metadata Configuration
  */
-public class RateLimiterMeta implements ReflectMapWriter {
+public class RateLimiterPayload implements ReflectMapWriter {
   @JsonProperty
   public Boolean enabled;
 
@@ -41,8 +41,8 @@ public class RateLimiterMeta implements ReflectMapWriter {
   @JsonProperty
   public Integer slotAcquisitionTimeoutInMS;
 
-  public RateLimiterMeta copy() {
-    RateLimiterMeta result = new RateLimiterMeta();
+  public RateLimiterPayload copy() {
+    RateLimiterPayload result = new RateLimiterPayload();
 
     result.enabled = enabled;
     result.guaranteedSlots = guaranteedSlots;
@@ -55,8 +55,8 @@ public class RateLimiterMeta implements ReflectMapWriter {
 
   @Override
   public boolean equals(Object obj) {
-    if (obj instanceof RateLimiterMeta) {
-      RateLimiterMeta that = (RateLimiterMeta) obj;
+    if (obj instanceof RateLimiterPayload) {
+      RateLimiterPayload that = (RateLimiterPayload) obj;
       return Objects.equals(this.enabled, that.enabled) &&
           Objects.equals(this.guaranteedSlots, that.guaranteedSlots) &&
           Objects.equals(this.allowedRequests, that.allowedRequests) &&
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RestoreCollectionPayload.java
similarity index 57%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RestoreCollectionPayload.java
index 5cd10b6..2634802 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/RestoreCollectionPayload.java
@@ -16,17 +16,38 @@
  */
 package org.apache.solr.client.solrj.request.beans;
 
-import java.util.Map;
-
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class CreateConfigInfo implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String name;
-  @JsonProperty
-  public String baseConfigSet;
-  @JsonProperty
-  public Map<String,Object> properties;
+import java.util.Map;
+
+import static org.apache.solr.client.solrj.request.beans.V2ApiConstants.CREATE_COLLECTION_KEY;
+
+/**
+ * V2 API POJO for the /v2/collections 'restore-collection' command.
+ *
+ * Analogous to the request parameters for v1 /admin/collections?action=RESTORE API.
+ */
+public class RestoreCollectionPayload implements ReflectMapWriter {
+
+    @JsonProperty(required = true)
+    public String collection;
+
+    @JsonProperty(required = true)
+    public String name;
+
+    @JsonProperty
+    public String location;
+
+    @JsonProperty
+    public String repository;
+
+    @JsonProperty
+    public Integer backupId;
+
+    @JsonProperty(CREATE_COLLECTION_KEY)
+    public Map<String, Object> createCollectionParams;
 
+    @JsonProperty
+    public String async;
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SetAliasPropertyPayload.java
similarity index 81%
rename from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
rename to solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SetAliasPropertyPayload.java
index 5cd10b6..c3c8585 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/CreateConfigInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/SetAliasPropertyPayload.java
@@ -16,17 +16,18 @@
  */
 package org.apache.solr.client.solrj.request.beans;
 
-import java.util.Map;
-
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-public class CreateConfigInfo implements ReflectMapWriter {
-  @JsonProperty(required = true)
-  public String name;
-  @JsonProperty
-  public String baseConfigSet;
-  @JsonProperty
-  public Map<String,Object> properties;
+import java.util.Map;
+
+public class SetAliasPropertyPayload implements ReflectMapWriter {
+    @JsonProperty(required = true)
+    public String name;
+
+    @JsonProperty
+    public String async;
 
+    @JsonProperty
+    public Map<String, Object> properties;
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/V2ApiConstants.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/V2ApiConstants.java
new file mode 100644
index 0000000..174b8bf
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/V2ApiConstants.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+public class V2ApiConstants {
+    private V2ApiConstants() { /* Private ctor prevents instantiation */ }
+
+    /**
+     * Parent key for collection or alias properties to set.
+     */
+    public static final String PROPERTIES_KEY = "properties";
+    /**
+     * Parent key for v2 params used to create a collection.
+     */
+    public static final String CREATE_COLLECTION_KEY = "create-collection";
+
+    /**
+     * Parent key holding alias-router parameters.
+     */
+    public static final String ROUTER_KEY = "router";
+
+    /**
+     * Parameter name for the configset used by a collection
+     */
+    public static final String CONFIG = "config";
+
+    /**
+     * Property controlling whether 'nodeSet' should be shuffled before use.
+     */
+    public static final String SHUFFLE_NODES = "shuffleNodes";
+
+    /**
+     * The set of nodes to consider as potential locations for a new collection or its constituent components.
+     */
+    public static final String NODE_SET = "nodeSet";
+
+    /**
+     * The collections to be included in an alias.
+     */
+    public static final String COLLECTIONS = "collections";
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
index c38f397..d259f35 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
@@ -39,6 +39,11 @@ public interface CollectionAdminParams {
   String CREATE_NODE_SET_PARAM = "createNodeSet";
 
   /**
+   * The number of shards to create a particular collection with.
+   */
+  String NUM_SHARDS = "numShards";
+
+  /**
    * A parameter which specifies if the provided list of Solr nodes (via {@linkplain #CREATE_NODE_SET_PARAM})
    * should be shuffled before being used.
    */
diff --git a/solr/solrj/src/resources/apispec/collections.Commands.json b/solr/solrj/src/resources/apispec/collections.Commands.json
deleted file mode 100644
index 4d4a178..0000000
--- a/solr/solrj/src/resources/apispec/collections.Commands.json
+++ /dev/null
@@ -1,298 +0,0 @@
-{
-  "documentation": "https://lucene.apache.org/solr/guide/collection-management.html#create",
-  "description": "Create collections and collection aliases, backup or restore collections, and delete collections and aliases.",
-  "methods": [
-    "POST"
-  ],
-  "url": {
-    "paths": [
-      "/collections",
-      "/c"
-    ]
-  },
-  "commands": {
-    "create": {
-      "type": "object",
-      "documentation": "https://lucene.apache.org/solr/guide/collection-management.html#create",
-      "description": "Create a collection.",
-      "properties": {
-        "name": {
-          "type": "string",
-          "description": "The name of the collection to be created."
-        },
-        "config": {
-          "type": "string",
-          "description": "The name of the configuration set (which must already be stored in ZooKeeper) to use for this collection. If not provided, Solr will default to the collection name as the configuration set name."
-        },
-        "router": {
-          "type": "object",
-          "documentation": "https://lucene.apache.org/solr/guide/shards-and-indexing-data-in-solrcloud.html",
-          "description": "These properties define how to distribute documents across a collection's shards.",
-          "properties": {
-            "name": {
-              "type": "string",
-              "enum":["implicit","compositeId"],
-              "description": "The router implementation to use for this collection. There are two options: compositeId or implicit. The compositeId option has Solr decide how to distribute documents (with some possibilities for customization). The implicit option requires you define your own routing strategy, and puts the balancing of documents in shards entirely in your hands.",
-              "default": "compositeId"
-            },
-            "field": {
-              "type": "string",
-              "description": "A field to be used by Solr to identify the shard a document should be routed to. By default, the field defined as the unique ID for each document is used, but an alternative field can be defined with this parameter."
-            }
-          }
-        },
-        "numShards": {
-          "type": "integer",
-          "description": "The number of shards to be created as part of the collection. Shards are logical partitions of a single collection. Each shard has at least one replica, but more replicas for each shard can be defined with the replicationFactor property. This is a required parameter when using the 'compositeId' router."
-        },
-        "shards": {
-          "type": "string",
-          "description": "A comma-separated list of shard names, e.g., shard-x,shard-y,shard-z. This is a required parameter when using the 'implicit' router."
-        },
-        "replicationFactor": {
-          "type": "integer",
-          "description": "The number of NRT replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard."
-        },
-        "nrtReplicas": {
-          "type": "integer",
-          "description": "The number of NRT replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard. Replicas of type NRT will be updated with each document that is added to the cluster, and can use \"softCommits\" to get a new view of the index in Near Real Time. This parameter works in the same way as 'replicationFactor'"
-        },
-        "tlogReplicas": {
-          "type": "integer",
-          "description": "The number of TLOG replicas to be created for each shard. TLOG replicas update their transaction log for every update to the cluster, but only the shard leader updates the local index, other TLOG replicas will use segment replication and copy the latest index files from the leader."
-        },
-        "pullReplicas": {
-          "type": "integer",
-          "description": "The number of PULL replicas to be created for each shard. PULL replicas don't receive copies of the documents on update requests, they just replicate the latest segments periodically from the shard leader. PULL replicas can't become shard leaders, and need at least one active TLOG(recommended) or NRT replicas in the shard to replicate from."
-        },
-        "nodeSet": {
-          "type": "array",
-          "items": {
-            "type": "string"
-          },
-          "description": "Defines nodes to spread the new collection across. If not provided, the collection will be spread across all live Solr nodes. The names to use are the 'node_name', which can be found by a request to the cluster/nodes endpoint. A special value of EMPTY will create no shards or replicas for the new collection. In this case, shards and replicas can be added later with the add-replica command available on the /collections/{collection}/shards endpoint."
-        },
-        "shuffleNodes": {
-          "type": "boolean",
-          "description": "Controls whether or not the shard-replicas created for this collection will be assigned to the nodes specified by the nodeSet property in a sequential manner, or if the list of nodes should be shuffled prior to creating individual replicas. A 'false' value makes the results of a collection creation predictable and gives more exact control over the location of the individual shard-replicas, but 'true' can be a better choice for ensuring replicas are distributed e [...]
-        },
-        "properties": {
-          "type": "object",
-          "documentation": "https://lucene.apache.org/solr/guide/defining-core-properties.html",
-          "description": "Allows adding core.properties for the collection. Some examples of core properties you may want to modify include the config set, the node name, the data directory, among others.",
-          "additionalProperties": true
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        },
-        "waitForFinalState": {
-          "type": "boolean",
-          "description": "If true then request will complete only when all affected replicas become active.",
-          "default": false
-        },
-        "perReplicaState": {
-          "type": "boolean",
-          "description": "Use Per replica states",
-          "default": false
-        }
-      },
-      "required": [
-        "name"
-      ]
-    },
-    "create-alias": {
-      "documentation": "https://lucene.apache.org/solr/guide/collection-aliasing.html#createalias",
-      "description": "Allows one or more collections to be known by another name (to include time partitioned collections). If this command is used on an existing alias, the existing alias will be replaced with the new collection details.",
-      "type": "object",
-      "properties": {
-        "name": {
-          "type": "string",
-          "description": "The alias name to be created."
-        },
-        "collections": {
-          "type": "array",
-          "description": "The list of collections to be known as this alias. Incompatible with any of the routing parameters. Either this parameter or a complete set of routing parameters is required.",
-          "items": {
-            "type": "string"
-          }
-        },
-        "router" : {
-          "type":"object",
-          "documentation": "https://lucene.apache.org/solr/guide/collection-aliasing.html#createalias",
-          "description":"Routing specific properties to define a time routed alias.  Do not specify 'collections' when creating a time routed alias.",
-          "properties" : {
-            "name" : {
-              "type" : "string",
-              "description": "The type of routing to perform. Currently only 'time' is supported, and it's required."
-            },
-            "field" : {
-              "type": "string",
-              "description": "The date field name in incoming documents that is consulted to decide which collection the document should be routed to."
-            },
-            "start": {
-              "type": "string",
-              "description": "The earliest date/time in a document that may be indexed into this alias. Documents with values less than this will return an error. For time based routing this may be a date math expression."
-            },
-            "interval" : {
-              "type": "string",
-              "description": "A specification of the width of the interval for each partition collection. For time based routing this should be a date math expression fragment starting with the + character."
-            },
-            "maxFutureMs": {
-              "type": "integer",
-              "description":"How many milliseconds into the future to accept document. Documents with a value in router.field that is greater than now() + maxFutureMs will be rejected to avoid provisioning too much resources."
-            },
-            "preemptiveCreateMath":{
-              "type": "string",
-              "description": "If a document arrives with a timestamp that is after the end time of the most recent collection minus this interval, then the next collection will be created asynchronously. Without this setting, collections are created synchronously when required by the document time stamp and thus block the flow of documents until the collection is created (possibly several seconds). Preemptive creation reduces these hiccups. If set to enough time (perhaps an hour or more) [...]
-            },
-            "autoDeleteAge": {
-              "type": "string",
-              "description": "A date math expressions yielding a time in the past. Collections covering a period of time entirely before this age will be automatically deleted."
-            },
-            "maxCardinality": {
-              "type": "integer",
-              "description": "The maximum number of categories allowed for this alias."
-            },
-            "mustMatch": {
-              "type": "string",
-              "description": "A regular expression that the value of the field specified by `router.field` must match before a corresponding collection will be created."
-            },
-            "routerList": {
-              "type": "array",
-              "description": "A list of router property sets to be used with router type Dimensional[foo,bar] where foo and bar are valid router type names (i.e. time or category). The order must correspond to the type specification in [] in the Dimensional type, so Dimensional[category,time] would require the first set of router properties to be valid for a category routed alias, and the second set to be valid for a time routed alias. In these sets of properties, router.name will be ign [...]
-              "items": {
-                "type": "object",
-                "additionalProperties": true
-              }
-            }
-          }
-        },
-        "TZ": {
-          "type": "string",
-          "description": "Optional timezone for use with any date math that may exist in other parameters.  Defaults to UTC."
-        },
-        "create-collection": {
-          "type": "object",
-          "documentation": "https://lucene.apache.org/solr/guide/collection-management.html#create",
-          "description": "The settings to use to create a collection for each new time partition. Most options from the collection create command are available, except for 'name', 'async' and 'waitForFinalState'.",
-          "additionalProperties": true
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        }
-      },
-      "required": [
-        "name"
-      ]
-    },
-    "delete-alias": {
-      "documentation": "https://lucene.apache.org/solr/guide/collection-aliasing.html#deletealias",
-      "description": "Deletes a collection alias",
-      "type": "object",
-      "properties": {
-        "name": {
-          "type": "string",
-          "description": "The name of the alias to delete."
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        }
-      },
-      "required": [
-        "name"
-      ]
-    },
-    "set-alias-property": {
-      "documentation": "https://lucene.apache.org/solr/guide/collection-aliasing.html#modifyalias",
-      "description": "Allows changing the properties on an alias. If a key is set with an empty string then it will be removed",
-      "type": "object",
-      "properties": {
-        "name": {
-          "type": "string",
-          "description": "The alias name on which to set properties."
-        },
-        "properties" : {
-          "type": "object",
-          "description": "A map of key/value pairs that will be associated with the alias as alias properties (metadata). An empty value will delete any existing value for a given key.",
-          "additionalProperties": true
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        }
-      },
-      "required": [
-        "name"
-      ]
-    },
-    "backup-collection": {
-      "documentation": "https://lucene.apache.org/solr/guide/collection-management.html#backup",
-      "description": "Backup Solr indexes and configurations for a specific collection. One copy of the indexes will be taken from each shard, and the config set for the collection will also be copied.",
-      "type": "object",
-      "properties": {
-        "collection": {
-          "type": "string",
-          "description": "The name of the collection to back up."
-        },
-        "name": {
-          "type": "string",
-          "description": "The name of the backup."
-        },
-        "location": {
-          "type": "string",
-          "description": "A location on a shared drive for the backup-collection command to write to. Alternately, it can be set as a cluster property with the cluster endpoint, which also supports setting a location."
-        },
-        "followAliases": {
-          "type": "boolean",
-          "description": "Controls whether aliases are resolved when trying to back up the specified collection, or whether Solr should only backup the provided collection name if it matches a concrete collection."
-        },
-        "incremental": {
-          "type": "boolean",
-          "description": "An internal property that controls whether the backup should use the standard 'incremental' file format or the deprecated 'full-snapshot' based format."
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        }
-      },
-      "required": [
-        "collection",
-        "name"
-      ]
-    },
-    "restore-collection": {
-      "documentation": "https://lucene.apache.org/solr/guide/collection-management.html#restore",
-      "description": "Restore Solr indexes and configurations from a backup. You cannot restore into the same collection you took the backup from. The target collection must not exist before calling this command, as it will be created by the restore action. The new collection will have the same number of shards and replicas as the original collection, and all routing strategies will be retained.",
-      "type": "object",
-      "properties": {
-        "collection": {
-          "type": "string",
-          "description": "The name of the collection the backup will be restored to. This collection must not exist prior to this "
-        },
-        "name": {
-          "type": "string",
-          "description": "The name of the backup file."
-        },
-        "location": {
-          "type": "string",
-          "description": "The location on the shared drive for the restore-collection command to read from. Alternately, it can be set as a cluster property with the cluster endpoint, which also supports setting a location."
-        },
-        "backupId": {
-          "type": "integer",
-          "description": "The ID of the backup to restore, when the provided location and backup name hold multiple backups for the provided collection.  Defaults to the most recent backup if not specified."
-        },
-        "async": {
-          "type": "string",
-          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
-        }
-      },
-      "required": [
-        "collection",
-        "name"
-      ]
-    }
-  }
-}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
index b144e40..a57d859 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
@@ -17,36 +17,16 @@
 
 package org.apache.solr.client.solrj.request;
 
-import java.io.IOException;
-import java.util.Map;
-
-import com.google.common.collect.ImmutableMap;
 import org.apache.solr.SolrTestCase;
 import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.Utils;
 import org.junit.Test;
 
-public class TestV1toV2ApiMapper extends SolrTestCase {
+import java.io.IOException;
+import java.util.Map;
 
-  @Test
-  // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Sep-2018
-  public void testCreate() throws IOException {
-    Create cmd = CollectionAdminRequest
-        .createCollection("mycoll", "conf1", 3, 2)
-        .setProperties(ImmutableMap.<String,String>builder()
-            .put("p1","v1")
-            .put("p2","v2")
-            .build());
-    V2Request v2r = V1toV2ApiMapper.convert(cmd).build();
-    Map<?,?> m = (Map<?,?>) Utils.fromJSON(ContentStreamBase.create(new BinaryRequestWriter(), v2r).getStream());
-    assertEquals("/c", v2r.getPath());
-    assertEquals("v1", Utils.getObjectByPath(m,true,"/create/properties/p1"));
-    assertEquals("v2", Utils.getObjectByPath(m,true,"/create/properties/p2"));
-    assertEquals("3", Utils.getObjectByPath(m,true,"/create/numShards"));
-    assertEquals("2", Utils.getObjectByPath(m,true,"/create/nrtReplicas"));
-  }
+public class TestV1toV2ApiMapper extends SolrTestCase {
 
   @Test
   // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Sep-2018
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java b/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
index d539088..66aa39f 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
@@ -28,7 +28,6 @@ import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
 public class JsonValidatorTest extends SolrTestCaseJ4  {
 
   public void testSchema() {
-    checkSchema("collections.Commands");
     checkSchema("collections.collection.Commands");
     checkSchema("collections.collection.shards.Commands");
     checkSchema("collections.collection.shards.shard.Commands");
@@ -43,38 +42,42 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
 
 
   public void testSchemaValidation() {
-    ValidatingJsonMap spec = Utils.getSpec("collections.Commands").getSpec();
-    @SuppressWarnings({"rawtypes"})
-    Map createSchema = spec.getMap("commands", NOT_NULL).getMap("create-alias", NOT_NULL);
-    JsonSchemaValidator validator = new JsonSchemaValidator(createSchema);
-    List<String> errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ c1 , c2]}"));
+    // merge-indexes chosen to exercise string and array/list props.
+    ValidatingJsonMap spec = Utils.getSpec("cores.core.Commands").getSpec();
+    final Map<String, Object> mergeIndexesSchema = spec.getMap("commands", NOT_NULL).getMap("merge-indexes", NOT_NULL);
+    final JsonSchemaValidator mergeIndexesSchemaValidator = new JsonSchemaValidator(mergeIndexesSchema);
+
+    List<String> errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{async : x, indexDir: [ c1 , c2]}"));
     assertNull(toJSONString(errs), errs);
-    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [c1] }"));
+    errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{async : x, indexDir: [c1] }"));
     assertNull(toJSONString(errs), errs);
-    errs = validator.validateJson(Utils.fromJSONString("{name : x, x:y, collections: [ c1 , c2]}"));
+    errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{async : x, x:y, indexDir: [ c1 , c2]}"));
     assertNotNull(toJSONString(errs), errs);
     assertTrue(toJSONString(errs), errs.get(0).contains("Unknown"));
-    errs = validator.validateJson(Utils.fromJSONString("{name : 123, collections: c1 }"));
+    errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{async : 123, indexDir: c1 }"));
     assertNotNull(toJSONString(errs), errs);
     assertTrue(toJSONString(errs), errs.get(0).contains("expected"));
-    errs = validator.validateJson(Utils.fromJSONString("{x:y, collections: [ c1 , c2]}"));
+    errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{x:y, indexDir: [ c1 , c2]}"));
     assertTrue(toJSONString(errs), StrUtils.join(errs, '|').contains("Unknown"));
-    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ 1 , 2]}"));
+    errs = mergeIndexesSchemaValidator.validateJson(Utils.fromJSONString("{async : x, indexDir: [ 1 , 2]}"));
     assertFalse(toJSONString(errs), errs.isEmpty());
     assertTrue(toJSONString(errs), errs.get(0).contains("expected"));
-    validator = new JsonSchemaValidator("{" +
+
+
+    final JsonSchemaValidator personSchemaValidator = new JsonSchemaValidator("{" +
         "  type:object," +
         "  properties: {" +
         "   age : {type: number}," +
         "   adult : {type: boolean}," +
         "   name: {type: string}}}");
-    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:21, adult:true}"));
+    errs = personSchemaValidator.validateJson(Utils.fromJSONString("{name:x, age:21, adult:true}"));
     assertNull(errs);
-    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'21', adult:'true'}"));
+    errs = personSchemaValidator.validateJson(Utils.fromJSONString("{name:x, age:'21', adult:'true'}"));
     assertNotNull(errs);
-
-    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'x21', adult:'true'}"));
+    errs = personSchemaValidator.validateJson(Utils.fromJSONString("{name:x, age:'x21', adult:'true'}"));
     assertEquals(1, errs.size());
+
+
     Exception e = expectThrows(Exception.class, () -> {
       new JsonSchemaValidator("{" +
           "  type:object," +
@@ -106,16 +109,16 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
     });
     assertTrue(e.getMessage().contains("Unknown key : propertes"));
 
-    validator = new JsonSchemaValidator("{" +
+    final JsonSchemaValidator personWithEnumValidator = new JsonSchemaValidator("{" +
         "  type:object," +
         "  properties: {" +
         "   age : {type: number}," +
         "   sex: {type: string, enum:[M, F]}," +
         "   adult : {type: boolean}," +
         "   name: {type: string}}}");
-    errs = validator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:M}"));
+    errs = personWithEnumValidator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:M}"));
     assertNull("errs are " + errs, errs);
-    errs = validator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:m}"));
+    errs = personWithEnumValidator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:m}"));
     assertEquals(1, errs.size());
     assertTrue(errs.get(0).contains("Value of enum"));
 
@@ -139,8 +142,8 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
         "\n" +
         "  }\n" +
         "}";
-    validator = new JsonSchemaValidator(schema);
-    validator.validateJson(Utils.fromJSONString("{\n" +
+    final JsonSchemaValidator nestedObjectValidator = new JsonSchemaValidator(schema);
+    nestedObjectValidator.validateJson(Utils.fromJSONString("{\n" +
         "  'links': [\n" +
         "    {\n" +
         "        'rel': 'x',\n" +
@@ -161,11 +164,12 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
         "'type' : 'object',\n" +
         "'oneOf' : ['a', 'b']\n" +
         "}";
-    validator = new JsonSchemaValidator(schema);
-    errs = validator.validateJson(Utils.fromJSONString("" +
+
+    final JsonSchemaValidator mutuallyExclusivePropertiesValidator = new JsonSchemaValidator(schema);
+    errs = mutuallyExclusivePropertiesValidator.validateJson(Utils.fromJSONString("" +
         "{'c':'val'}"));
     assertNotNull(errs);
-    errs = validator.validateJson(Utils.fromJSONString("" +
+    errs = mutuallyExclusivePropertiesValidator.validateJson(Utils.fromJSONString("" +
         "{'a':'val'}"));
     assertNull(errs);