You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2017/05/10 13:36:22 UTC

[1/2] lucene-solr:master: SOLR-10647: Move the V1 <-> V2 API mapping to SolrJ

Repository: lucene-solr
Updated Branches:
  refs/heads/master fb24d451c -> 0184d6b7f


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
new file mode 100644
index 0000000..5b043e7
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
@@ -0,0 +1,318 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.common.util.StrUtils.formatString;
+import static org.apache.solr.common.util.Utils.toJSON;
+
+public class CommandOperation {
+  public final String name;
+  private Object commandData;//this is most often a map
+  private List<String> errors = new ArrayList<>();
+
+  public CommandOperation(String operationName, Object metaData) {
+    commandData = metaData;
+    this.name = operationName;
+  }
+
+  public Object getCommandData() {
+    return commandData;
+  }
+
+  public String getStr(String key, String def) {
+    if (ROOT_OBJ.equals(key)) {
+      Object obj = getRootPrimitive();
+      return obj == def ? null : String.valueOf(obj);
+    }
+    Object o = getMapVal(key);
+    return o == null ? def : String.valueOf(o);
+  }
+
+  public boolean getBoolean(String key, boolean def) {
+    String v = getStr(key,null);
+    return v == null? def:Boolean.parseBoolean(v);
+  }
+  public void setCommandData(Object o){
+    commandData = o;
+  }
+
+  public Map<String,Object> getDataMap() {
+    if (commandData instanceof Map) {
+      //noinspection unchecked
+      return (Map<String,Object>)commandData;
+    }
+    addError(StrUtils.formatString("The command ''{0}'' should have the values as a json object {key:val} format", name));
+    return Collections.emptyMap();
+  }
+
+  private Object getRootPrimitive() {
+    if (commandData instanceof Map) {
+      errors.add(StrUtils.formatString("The value has to be a string for command : ''{0}'' ", name));
+      return null;
+    }
+    return commandData;
+
+  }
+
+  public Object getVal(String key) {
+    return getMapVal(key);
+  }
+
+  private Object getMapVal(String key) {
+    if("".equals(key)){
+      if (commandData instanceof Map) {
+        addError("value of the command is an object should be primitive");
+      }
+      return commandData;
+    }
+    if (commandData instanceof Map) {
+      Map metaData = (Map) commandData;
+      return metaData.get(key);
+    } else {
+      String msg = " value has to be an object for operation :" + name;
+      if (!errors.contains(msg)) errors.add(msg);
+      return null;
+    }
+  }
+
+  public List<String> getStrs(String key) {
+    List<String> val = getStrs(key, null);
+    if (val == null) {
+      errors.add(StrUtils.formatString(REQD, key));
+    }
+    return val;
+
+  }
+
+  public void unknownOperation() {
+    addError(formatString("Unknown operation ''{0}'' ", name));
+  }
+
+  static final String REQD = "''{0}'' is a required field";
+
+
+  /**
+   * Get collection of values for a key. If only one val is present a
+   * single value collection is returned
+   */
+  public List<String> getStrs(String key, List<String> def) {
+    Object v = null;
+    if (ROOT_OBJ.equals(key)) {
+      v = getRootPrimitive();
+    } else {
+      v = getMapVal(key);
+    }
+    if (v == null) {
+      return def;
+    } else {
+      if (v instanceof List) {
+        ArrayList<String> l = new ArrayList<>();
+        for (Object o : (List) v) {
+          l.add(String.valueOf(o));
+        }
+        if (l.isEmpty()) return def;
+        return l;
+      } else {
+        return singletonList(String.valueOf(v));
+      }
+    }
+
+  }
+
+  /**
+   * Get a required field. If missing it adds to the errors
+   */
+  public String getStr(String key) {
+    if (ROOT_OBJ.equals(key)) {
+      Object obj = getRootPrimitive();
+      if (obj == null) {
+        errors.add(StrUtils.formatString(REQD, name));
+      }
+      return obj == null ? null : String.valueOf(obj);
+    }
+
+    String s = getStr(key, null);
+    if (s == null) errors.add(StrUtils.formatString(REQD, key));
+    return s;
+  }
+
+  private Map errorDetails() {
+    return Utils.makeMap(name, commandData, ERR_MSGS, errors);
+  }
+
+  public boolean hasError() {
+    return !errors.isEmpty();
+  }
+
+  public void addError(String s) {
+    if (errors.contains(s)) return;
+    errors.add(s);
+  }
+
+  /**
+   * Get all the values from the metadata for the command
+   * without the specified keys
+   */
+  public Map<String,Object> getValuesExcluding(String... keys) {
+    getMapVal(null);
+    if (hasError()) return emptyMap();//just to verify the type is Map
+    @SuppressWarnings("unchecked") 
+    LinkedHashMap<String, Object> cp = new LinkedHashMap<>((Map<String, Object>) commandData);
+    if (keys == null) return cp;
+    for (String key : keys) {
+      cp.remove(key);
+    }
+    return cp;
+  }
+
+
+  public List<String> getErrors() {
+    return errors;
+  }
+
+  public static final String ERR_MSGS = "errorMessages";
+  public static final String ROOT_OBJ = "";
+
+  public static List<Map> captureErrors(List<CommandOperation> ops) {
+    List<Map> errors = new ArrayList<>();
+    for (CommandOperation op : ops) {
+      if (op.hasError()) {
+        errors.add(op.errorDetails());
+      }
+    }
+    return errors;
+  }
+
+
+  /**
+   * Parse the command operations into command objects
+   */
+  public static List<CommandOperation> parse(Reader rdr) throws IOException {
+    JSONParser parser = new JSONParser(rdr);
+
+    ObjectBuilder ob = new ObjectBuilder(parser);
+
+    if (parser.lastEvent() != JSONParser.OBJECT_START) {
+      throw new RuntimeException("The JSON must be an Object of the form {\"command\": {...},...");
+    }
+    List<CommandOperation> operations = new ArrayList<>();
+    for (; ; ) {
+      int ev = parser.nextEvent();
+      if (ev == JSONParser.OBJECT_END) return operations;
+      Object key = ob.getKey();
+      ev = parser.nextEvent();
+      Object val = ob.getVal();
+      if (val instanceof List) {
+        List list = (List) val;
+        for (Object o : list) {
+          if (!(o instanceof Map)) {
+            operations.add(new CommandOperation(String.valueOf(key), list));
+            break;
+          } else {
+            operations.add(new CommandOperation(String.valueOf(key), o));
+          }
+        }
+      } else {
+        operations.add(new CommandOperation(String.valueOf(key), val));
+      }
+    }
+
+  }
+
+  public CommandOperation getCopy() {
+    return new CommandOperation(name, commandData);
+  }
+
+  public Map getMap(String key, Map def) {
+    Object o = getMapVal(key);
+    if (o == null) return def;
+    if (!(o instanceof Map)) {
+      addError(StrUtils.formatString("''{0}'' must be a map", key));
+      return def;
+    } else {
+      return (Map) o;
+
+    }
+  }
+
+  @Override
+  public String toString() {
+    return new String(toJSON(singletonMap(name, commandData)), StandardCharsets.UTF_8);
+  }
+
+  public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, NamedList resp)
+      throws IOException {
+    if (streams == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
+    }
+    ArrayList<CommandOperation> ops = new ArrayList<>();
+
+    for (ContentStream stream : streams)
+      ops.addAll(parse(stream.getReader()));
+    List<Map> errList = CommandOperation.captureErrors(ops);
+    if (!errList.isEmpty()) {
+      resp.add(CommandOperation.ERR_MSGS, errList);
+      return null;
+    }
+    return ops;
+  }
+
+  public static List<CommandOperation> clone(List<CommandOperation> ops) {
+    List<CommandOperation> opsCopy = new ArrayList<>(ops.size());
+    for (CommandOperation op : ops) opsCopy.add(op.getCopy());
+    return opsCopy;
+  }
+
+
+  public Integer getInt(String name, Integer def) {
+    Object o = getVal(name);
+    if (o == null) return def;
+    if (o instanceof Number) {
+      Number number = (Number) o;
+      return number.intValue();
+    } else {
+      try {
+        return Integer.parseInt(o.toString());
+      } catch (NumberFormatException e) {
+        addError(StrUtils.formatString("{0} is not a valid integer", name));
+        return null;
+      }
+    }
+  }
+
+  public Integer getInt(String name) {
+    Object o = getVal(name);
+    if(o == null) return null;
+    return getInt(name, null);
+  }
+}


[2/2] lucene-solr:master: SOLR-10647: Move the V1 <-> V2 API mapping to SolrJ

Posted by no...@apache.org.
SOLR-10647: Move the V1 <-> V2 API mapping to SolrJ


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/0184d6b7
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/0184d6b7
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/0184d6b7

Branch: refs/heads/master
Commit: 0184d6b7f5ebbdcdf8faa085947da455fc1dc7ab
Parents: fb24d45
Author: Noble Paul <no...@apache.org>
Authored: Wed May 10 23:06:07 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Wed May 10 23:06:07 2017 +0930

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../src/java/org/apache/solr/api/ApiBag.java    |   2 +-
 .../apache/solr/handler/SolrConfigHandler.java  |   4 +-
 .../handler/admin/BaseHandlerApiSupport.java    |  50 +--
 .../handler/admin/CollectionHandlerApi.java     | 294 +++------------
 .../handler/admin/ConfigSetsHandlerApi.java     |  63 +---
 .../solr/handler/admin/CoreAdminHandlerApi.java | 126 ++-----
 .../solr/handler/admin/SecurityConfHandler.java |   4 +-
 .../handler/admin/SecurityConfHandlerLocal.java |   2 +-
 .../handler/admin/SecurityConfHandlerZk.java    |   2 +-
 .../apache/solr/request/SolrQueryRequest.java   |   2 +-
 .../solr/request/SolrQueryRequestBase.java      |   2 +-
 .../org/apache/solr/schema/SchemaManager.java   |   2 +-
 .../security/AutorizationEditOperation.java     |   2 +-
 .../apache/solr/security/BasicAuthPlugin.java   |   2 +-
 .../solr/security/ConfigEditablePlugin.java     |   4 +-
 .../security/RuleBasedAuthorizationPlugin.java  |   2 +-
 .../security/Sha256AuthenticationProvider.java  |   2 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |   2 +-
 .../org/apache/solr/servlet/ResponseUtils.java  |   2 +-
 .../apache/solr/servlet/SolrRequestParsers.java |   2 +-
 .../org/apache/solr/util/CommandOperation.java  | 328 -----------------
 .../handler/admin/SecurityConfHandlerTest.java  |   2 +-
 .../solr/handler/admin/TestApiFramework.java    |   2 +-
 .../solr/handler/admin/TestCollectionAPIs.java  |   2 +-
 .../apache/solr/schema/TestSchemaManager.java   |   2 +-
 .../TestRuleBasedAuthorizationPlugin.java       |   4 +-
 .../TestSha256AuthenticationProvider.java       |   2 +-
 .../solrj/request/CollectionApiMapping.java     | 369 +++++++++++++++++++
 .../client/solrj/request/CoreApiMapping.java    | 117 ++++++
 .../solr/common/util/CommandOperation.java      | 318 ++++++++++++++++
 31 files changed, 936 insertions(+), 783 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index da2b2cc..7b66ed0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -136,6 +136,8 @@ Other Changes
 
 * SOLR-10572: Removed three "no longer supported in solrconfig.xml" asserts. (Christine Poerschke)
 
+* SOLR-10647: Move the V1 <-> V2 API mapping to SolrJ (noble)
+
 ==================  6.6.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/api/ApiBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index 82d6a39..14d029f 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -43,7 +43,7 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.JsonSchemaValidator;
 import org.apache.solr.util.PathTrie;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index e984f5a..46646ad 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -71,7 +71,7 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaManager;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.SolrPluginUtils;
@@ -330,7 +330,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
 
     private void handlePOST() throws IOException {
-      List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), resp);
+      List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), resp.getValues());
       if (ops == null) return;
       try {
         for (; ; ) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
index 0e58ccc..6e2ec36 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
@@ -27,16 +27,18 @@ import java.util.Map;
 import java.util.Objects;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionApiMapping;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.V2EndPoint;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.CommandOperation;
-import org.apache.solr.api.Api;
-import org.apache.solr.api.ApiBag;
-import org.apache.solr.api.ApiSupport;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
@@ -53,10 +55,10 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
   protected BaseHandlerApiSupport() {
     commandsMapping = new HashMap<>();
     for (ApiCommand cmd : getCommands()) {
-      Map<V2EndPoint, List<ApiCommand>> m = commandsMapping.get(cmd.getHttpMethod());
-      if (m == null) commandsMapping.put(cmd.getHttpMethod(), m = new HashMap<>());
-      List<ApiCommand> list = m.get(cmd.getEndPoint());
-      if (list == null) m.put(cmd.getEndPoint(), list = new ArrayList<>());
+      Map<V2EndPoint, List<ApiCommand>> m = commandsMapping.get(cmd.meta().getHttpMethod());
+      if (m == null) commandsMapping.put(cmd.meta().getHttpMethod(), m = new HashMap<>());
+      List<ApiCommand> list = m.get(cmd.meta().getEndPoint());
+      if (list == null) m.put(cmd.meta().getEndPoint(), list = new ArrayList<>());
       list.add(cmd);
     }
   }
@@ -86,7 +88,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
             ApiCommand command = null;
             String commandName = c == null ? null : c.name;
             for (ApiCommand cmd : commands) {
-              if (Objects.equals(cmd.getName(), commandName)) {
+              if (Objects.equals(cmd.meta().getName(), commandName)) {
                 command = cmd;
                 break;
               }
@@ -105,7 +107,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
             }
             if (commands.size() > 1) {
               for (ApiCommand command : commands) {
-                if (command.getName().equals(req.getPath())) {
+                if (command.meta().getName().equals(req.getPath())) {
                   commands = Collections.singletonList(command);
                   break;
                 }
@@ -147,7 +149,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
           }
 
           private Object getParams0(String param) {
-            param = cmd.getParamSubstitute(param);
+            param = cmd.meta().getParamSubstitute(param);
             Object o = param.indexOf('.') > 0 ?
                 Utils.getObjectByPath(map, true, splitSmart(param, '.')) :
                 map.get(param);
@@ -171,7 +173,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
 
           @Override
           public Iterator<String> getParameterNamesIterator() {
-            return cmd.getParamNames(co).iterator();
+            return cmd.meta().getParamNames(co).iterator();
 
           }
 
@@ -207,30 +209,10 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
   protected abstract List<V2EndPoint> getEndPoints();
 
 
-  public interface ApiCommand {
-    String getName();
-
-    /**
-     * the http method supported by this command
-     */
-    SolrRequest.METHOD getHttpMethod();
-
-    V2EndPoint getEndPoint();
-
-    default Collection<String> getParamNames(CommandOperation op) {
-      return BaseHandlerApiSupport.getParamNames(op, this);
-    }
-
-
-    default String getParamSubstitute(String name) {
-      return name;
-    }
+  public interface ApiCommand  {
+    CollectionApiMapping.CommandMeta meta();
 
     void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception;
   }
 
-  public interface V2EndPoint {
-
-    String getSpecName();
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
index 3cb21ab..8513e090 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -17,29 +17,18 @@
 
 package org.apache.solr.handler.admin;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
-import com.google.common.collect.ImmutableMap;
-import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionApiMapping;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.Meta;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.V2EndPoint;
 import org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.CommandOperation;
 
-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.cloud.OverseerCollectionMessageHandler.COLL_CONF;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET;
-import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation.*;
 
-
 public class CollectionHandlerApi extends BaseHandlerApiSupport {
   final CollectionsHandler handler;
 
@@ -54,268 +43,69 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
 
   @Override
   protected List<V2EndPoint> getEndPoints() {
-    return Arrays.asList(EndPoint.values());
+    return Arrays.asList(CollectionApiMapping.EndPoint.values());
   }
 
 
   enum Cmd implements ApiCommand {
-    GET_COLLECTIONS(EndPoint.COLLECTIONS, GET, LIST_OP),
-    GET_CLUSTER(EndPoint.CLUSTER, GET, LIST_OP, "/cluster", null),
-    GET_CLUSTER_OVERSEER(EndPoint.CLUSTER, GET, OVERSEERSTATUS_OP, "/cluster/overseer", null),
-    GET_CLUSTER_STATUS_CMD(EndPoint.CLUSTER_CMD_STATUS, GET, REQUESTSTATUS_OP),
-    DELETE_CLUSTER_STATUS(EndPoint.CLUSTER_CMD_STATUS_DELETE, DELETE, DELETESTATUS_OP),
-    GET_A_COLLECTION(EndPoint.COLLECTION_STATE, GET, CLUSTERSTATUS_OP),
-    LIST_ALIASES(EndPoint.CLUSTER_ALIASES, GET, LISTALIASES_OP),
-    CREATE_COLLECTION(EndPoint.COLLECTIONS_COMMANDS,
-        POST,
-        CREATE_OP,
-        CREATE_OP.action.toLower(),
-        ImmutableMap.of(
-            COLL_CONF, "config",
-            "createNodeSet.shuffle", "shuffleNodes",
-            "createNodeSet", "nodeSet"
-        ),
-        ImmutableMap.of("properties.", "property.")),
-
-    DELETE_COLL(EndPoint.PER_COLLECTION_DELETE,
-        DELETE,
-        DELETE_OP,
-        DELETE_OP.action.toLower(),
-        ImmutableMap.of(NAME, "collection")),
-
-    RELOAD_COLL(EndPoint.PER_COLLECTION,
-        POST,
-        RELOAD_OP,
-        RELOAD_OP.action.toLower(),
-        ImmutableMap.of(NAME, "collection")),
-    MODIFYCOLLECTION(EndPoint.PER_COLLECTION,
-        POST,
-        MODIFYCOLLECTION_OP,
-        "modify",null),
-    MIGRATE_DOCS(EndPoint.PER_COLLECTION,
-        POST,
-        MIGRATE_OP,
-        "migrate-docs",
-        ImmutableMap.of("split.key", "splitKey",
-            "target.collection", "target",
-            "forward.timeout", "forwardTimeout"
-        )),
-    REBALANCELEADERS(EndPoint.PER_COLLECTION,
-        POST,
-        REBALANCELEADERS_OP,
-        "rebalance-leaders", null),
-    CREATE_ALIAS(EndPoint.COLLECTIONS_COMMANDS,
-        POST,
-        CREATEALIAS_OP,
-        "create-alias",
-        null),
-
-    DELETE_ALIAS(EndPoint.COLLECTIONS_COMMANDS,
-        POST,
-        DELETEALIAS_OP,
-        "delete-alias",
-        null),
-    CREATE_SHARD(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
-        POST,
-        CREATESHARD_OP,
-        "create",
-        ImmutableMap.of(CREATE_NODE_SET, "nodeSet"),
-        ImmutableMap.of("coreProperties.", "property.")) {
-      @Override
-      public String getParamSubstitute(String param) {
-        return super.getParamSubstitute(param);
-      }
-    },
-
-    SPLIT_SHARD(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
-        POST,
-        SPLITSHARD_OP,
-        "split",
-        ImmutableMap.of(
-            "split.key", "splitKey"),
-        ImmutableMap.of("coreProperties.", "property.")),
-    DELETE_SHARD(EndPoint.PER_COLLECTION_PER_SHARD_DELETE,
-        DELETE,
-        DELETESHARD_OP),
-
-    CREATE_REPLICA(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
-        POST,
-        ADDREPLICA_OP,
-        "add-replica",
-        null,
-        ImmutableMap.of("coreProperties.", "property.")),
-
-    DELETE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE,
-        DELETE,
-        DELETEREPLICA_OP),
-
-    SYNC_SHARD(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,
-        POST,
-        SYNCSHARD_OP,
-        "synch-shard",
-        null),
-    ADDREPLICAPROP(EndPoint.PER_COLLECTION,
-        POST,
-        ADDREPLICAPROP_OP,
-        "add-replica-property",
-        ImmutableMap.of("property", "name", "property.value", "value")),
-    DELETEREPLICAPROP(EndPoint.PER_COLLECTION,
-        POST,
-        DELETEREPLICAPROP_OP,
-        "delete-replica-property",
-        null),
-    ADDROLE(EndPoint.CLUSTER_CMD,
-        POST,
-        ADDROLE_OP,
-        "add-role",null),
-    REMOVEROLE(EndPoint.CLUSTER_CMD,
-        POST,
-        REMOVEROLE_OP,
-        "remove-role",null),
-
-    CLUSTERPROP(EndPoint.CLUSTER_CMD,
-        POST,
-        CLUSTERPROP_OP,
-        "set-property",null),
-
-    BACKUP(EndPoint.COLLECTIONS_COMMANDS,
-        POST,
-        BACKUP_OP,
-        "backup-collection", null
-        ),
-    RESTORE(EndPoint.COLLECTIONS_COMMANDS,
-        POST,
-        RESTORE_OP,
-        "restore-collection",
-        null
-    ),
-    GET_NODES(EndPoint.CLUSTER_NODES, GET, null) {
+    GET_COLLECTIONS(Meta.GET_COLLECTIONS,LIST_OP),
+    GET_CLUSTER(Meta.GET_CLUSTER,LIST_OP),
+    GET_CLUSTER_OVERSEER(Meta.GET_CLUSTER_OVERSEER,OVERSEERSTATUS_OP),
+    GET_CLUSTER_STATUS_CMD(Meta.GET_CLUSTER_STATUS_CMD,REQUESTSTATUS_OP),
+    DELETE_CLUSTER_STATUS(Meta.DELETE_CLUSTER_STATUS,DELETESTATUS_OP),
+    GET_A_COLLECTION(Meta.GET_A_COLLECTION,CLUSTERSTATUS_OP),
+    LIST_ALIASES(Meta.LIST_ALIASES,LISTALIASES_OP),
+    CREATE_COLLECTION(Meta.CREATE_COLLECTION, CREATE_OP),
+    DELETE_COLL(Meta.DELETE_COLL, DELETE_OP),
+    RELOAD_COLL(Meta.RELOAD_COLL, RELOAD_OP),
+    MODIFYCOLLECTION(Meta.MODIFYCOLLECTION, MODIFYCOLLECTION_OP),
+    MIGRATE_DOCS(Meta.MIGRATE_DOCS,MIGRATE_OP),
+    REBALANCELEADERS(Meta.REBALANCELEADERS, REBALANCELEADERS_OP),
+    CREATE_ALIAS(Meta.CREATE_ALIAS, CREATEALIAS_OP),
+    DELETE_ALIAS(Meta.DELETE_ALIAS, DELETEALIAS_OP),
+    CREATE_SHARD(Meta.CREATE_SHARD,CREATESHARD_OP),
+    SPLIT_SHARD(Meta.SPLIT_SHARD, SPLITSHARD_OP),
+    DELETE_SHARD(Meta.DELETE_SHARD,DELETESHARD_OP),
+    CREATE_REPLICA(Meta.CREATE_REPLICA,ADDREPLICA_OP),
+    DELETE_REPLICA(Meta.DELETE_REPLICA,DELETEREPLICA_OP),
+    SYNC_SHARD(Meta.SYNC_SHARD, SYNCSHARD_OP),
+    ADDREPLICAPROP(Meta.ADDREPLICAPROP, ADDREPLICAPROP_OP),
+    DELETEREPLICAPROP(Meta.DELETEREPLICAPROP, DELETEREPLICAPROP_OP),
+    ADDROLE(Meta.ADDROLE, ADDROLE_OP),
+    REMOVEROLE(Meta.REMOVEROLE, REMOVEROLE_OP),
+    CLUSTERPROP(Meta.CLUSTERPROP,CLUSTERPROP_OP),
+    BACKUP(Meta.BACKUP, BACKUP_OP),
+    RESTORE(Meta.RESTORE, RESTORE_OP),
+    GET_NODES(Meta.GET_NODES, null) {
       @Override
       public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
         rsp.add("nodes", ((CollectionHandlerApi) apiHandler).handler.coreContainer.getZkController().getClusterState().getLiveNodes());
       }
     },
-    FORCELEADER(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,POST, FORCELEADER_OP,"force-leader",null),
-    SYNCSHARD(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,POST, SYNCSHARD_OP, "sync-shard",null),
-    BALANCESHARDUNIQUE(EndPoint.PER_COLLECTION, POST, BALANCESHARDUNIQUE_OP, "balance-shard-unique",null)
+    FORCELEADER(Meta.FORCELEADER,FORCELEADER_OP),
+    SYNCSHARD(Meta.SYNCSHARD,SYNCSHARD_OP),
+    BALANCESHARDUNIQUE(Meta.BALANCESHARDUNIQUE,BALANCESHARDUNIQUE_OP)
 
     ;
-    public final String commandName;
-    public final EndPoint endPoint;
-    public final SolrRequest.METHOD method;
-    public final CollectionOperation target;
-    //mapping of http param name to json attribute
-    public final Map<String, String> paramstoAttr;
-    //mapping of old prefix to new for instance properties.a=val can be substituted with property:{a:val}
-    public final Map<String, String> prefixSubstitutes;
-
-    public SolrRequest.METHOD getMethod() {
-      return method;
-    }
-
-
-    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CollectionOperation target) {
-      this(endPoint, method, target, null, null);
-    }
 
-    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CollectionOperation target,
-        String commandName, Map<String, String> paramstoAttr) {
-      this(endPoint, method, target, commandName, paramstoAttr, Collections.EMPTY_MAP);
+    public final CollectionApiMapping.CommandMeta meta;
 
-    }
+    public final CollectionOperation target;
 
-    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CollectionOperation target,
-        String commandName, Map<String, String> paramstoAttr, Map<String, String> prefixSubstitutes) {
-      this.commandName = commandName;
-      this.endPoint = endPoint;
-      this.method = method;
+    Cmd(CollectionApiMapping.CommandMeta meta, CollectionOperation target) {
+      this.meta = meta;
       this.target = target;
-      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : paramstoAttr;
-      this.prefixSubstitutes = prefixSubstitutes;
-
     }
 
     @Override
-    public String getName() {
-      return commandName;
-    }
-
-    @Override
-    public SolrRequest.METHOD getHttpMethod() {
-      return method;
-    }
-
-    @Override
-    public V2EndPoint getEndPoint() {
-      return endPoint;
-    }
-
-
-    @Override
-    public Collection<String> getParamNames(CommandOperation op) {
-      Collection<String> paramNames = BaseHandlerApiSupport.getParamNames(op, this);
-      if (!prefixSubstitutes.isEmpty()) {
-        Collection<String> result = new ArrayList<>(paramNames.size());
-        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
-          for (String paramName : paramNames) {
-            if (paramName.startsWith(e.getKey())) {
-              result.add(paramName.replace(e.getKey(), e.getValue()));
-            } else {
-              result.add(paramName);
-            }
-          }
-          paramNames = result;
-        }
-      }
-
-      return paramNames;
-    }
-
-    @Override
-    public String getParamSubstitute(String param) {
-      String s = paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
-      if (prefixSubstitutes != null) {
-        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
-          if (s.startsWith(e.getValue())) return s.replace(e.getValue(), e.getKey());
-        }
-      }
-      return s;
+    public CollectionApiMapping.CommandMeta meta() {
+      return meta;
     }
 
     public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler)
         throws Exception {
       ((CollectionHandlerApi) apiHandler).handler.invokeAction(req, rsp, ((CollectionHandlerApi) apiHandler).handler.coreContainer, target.action, target);
     }
-
-  }
-
-  enum EndPoint implements V2EndPoint {
-    CLUSTER("cluster"),
-    CLUSTER_ALIASES("cluster.aliases"),
-    CLUSTER_CMD("cluster.Commands"),
-    CLUSTER_NODES("cluster.nodes"),
-    CLUSTER_CMD_STATUS("cluster.commandstatus"),
-    CLUSTER_CMD_STATUS_DELETE("cluster.commandstatus.delete"),
-    COLLECTIONS_COMMANDS("collections.Commands"),
-    COLLECTIONS("collections"),
-    COLLECTION_STATE("collections.collection"),
-    PER_COLLECTION("collections.collection.Commands"),
-    PER_COLLECTION_DELETE("collections.collection.delete"),
-    PER_COLLECTION_SHARDS_COMMANDS("collections.collection.shards.Commands"),
-    PER_COLLECTION_PER_SHARD_COMMANDS("collections.collection.shards.shard.Commands"),
-    PER_COLLECTION_PER_SHARD_DELETE("collections.collection.shards.shard.delete"),
-    PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE("collections.collection.shards.shard.replica.delete");
-    final String specName;
-
-
-    EndPoint(String specName) {
-      this.specName = specName;
-    }
-
-    @Override
-    public String getSpecName() {
-      return specName;
-    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
index 6037bcd..db396d3 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
@@ -20,14 +20,12 @@ package org.apache.solr.handler.admin;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionApiMapping;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.ConfigSetMeta;
 import org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation;
 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 static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.CREATE_OP;
 import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.DELETE_OP;
 import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.LIST_OP;
@@ -47,44 +45,27 @@ public class ConfigSetsHandlerApi extends BaseHandlerApiSupport {
   }
 
   @Override
-  protected List<V2EndPoint> getEndPoints() {
-    return Arrays.asList(EndPoint.values());
+  protected List<CollectionApiMapping.V2EndPoint> getEndPoints() {
+    return Arrays.asList(CollectionApiMapping.ConfigSetEndPoint.values());
   }
 
   enum Cmd implements ApiCommand {
-    LIST(EndPoint.LIST_CONFIG, LIST_OP, GET),
-    CREATE(EndPoint.CONFIG_COMMANDS, CREATE_OP, POST, "create"),
-    DEL(EndPoint.CONFIG_DEL, DELETE_OP, DELETE)
-    ;
-    private final EndPoint endPoint;
-    private final ConfigSetOperation op;
-    private final SolrRequest.METHOD method;
-    private final String cmdName;
-
-    Cmd(EndPoint endPoint, ConfigSetOperation op, SolrRequest.METHOD method) {
-      this(endPoint, op, method, null);
-    }
+    LIST(ConfigSetMeta.LIST, LIST_OP),
+    CREATE(ConfigSetMeta.CREATE, CREATE_OP),
+    DEL(ConfigSetMeta.DEL,DELETE_OP);
 
-    Cmd(EndPoint endPoint, ConfigSetOperation op, SolrRequest.METHOD method, String cmdName) {
-      this.cmdName = cmdName;
-      this.endPoint = endPoint;
-      this.op = op;
-      this.method = method;
-    }
+    public ConfigSetMeta meta;
 
-    @Override
-    public String getName() {
-      return cmdName;
-    }
+    private final ConfigSetOperation op;
 
-    @Override
-    public SolrRequest.METHOD getHttpMethod() {
-      return method;
+    Cmd(ConfigSetMeta meta, ConfigSetOperation op) {
+      this.meta = meta;
+      this.op = op;
     }
 
     @Override
-    public V2EndPoint getEndPoint() {
-      return endPoint;
+    public CollectionApiMapping.CommandMeta meta() {
+      return meta;
     }
 
     @Override
@@ -93,20 +74,4 @@ public class ConfigSetsHandlerApi extends BaseHandlerApiSupport {
     }
 
   }
-  enum EndPoint implements V2EndPoint {
-    LIST_CONFIG("cluster.configs"),
-    CONFIG_COMMANDS("cluster.configs.Commands"),
-    CONFIG_DEL("cluster.configs.delete");
-
-    public final String spec;
-
-    EndPoint(String spec) {
-      this.spec = spec;
-    }
-
-    @Override
-    public String getSpecName() {
-      return spec;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
index 9d256e6..e9d8474 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@ -18,23 +18,15 @@
 package org.apache.solr.handler.admin;
 
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
-import java.util.Locale;
-import java.util.Map;
 
-import com.google.common.collect.ImmutableMap;
-import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionApiMapping;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.V2EndPoint;
+import org.apache.solr.client.solrj.request.CoreApiMapping;
+import org.apache.solr.client.solrj.request.CoreApiMapping.Meta;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
-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.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_COMMANDS;
-import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_STATUS;
-import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEAPIS;
-import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEINVOKE;
-import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.PER_CORE_COMMANDS;
 import static org.apache.solr.handler.admin.CoreAdminOperation.CREATE_OP;
 import static org.apache.solr.handler.admin.CoreAdminOperation.FORCEPREPAREFORLEADERSHIP_OP;
 import static org.apache.solr.handler.admin.CoreAdminOperation.INVOKE_OP;
@@ -62,69 +54,38 @@ public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
   }
 
   enum Cmd implements ApiCommand {
-    CREATE(CORES_COMMANDS, POST, CREATE_OP, null, ImmutableMap.of("config", "configSet")),
-    UNLOAD(PER_CORE_COMMANDS, POST, UNLOAD_OP, null, null),
-    RELOAD(PER_CORE_COMMANDS, POST, RELOAD_OP, null, null),
-    STATUS(CORES_STATUS, GET, STATUS_OP),
-    SWAP(PER_CORE_COMMANDS, POST, SWAP_OP, null, ImmutableMap.of("other", "with")),
-    RENAME(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
-    MERGEINDEXES(PER_CORE_COMMANDS, POST, MERGEINDEXES_OP, "merge-indexes", null),
-    SPLIT(PER_CORE_COMMANDS, POST, SPLIT_OP, null, ImmutableMap.of("split.key", "splitKey")),
-    PREPRECOVERY(PER_CORE_COMMANDS, POST, PREPRECOVERY_OP, "prep-recovery", null),
-    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
-    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, REQUESTSYNCSHARD_OP, "request-sync-shard", null),
-    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, REQUESTBUFFERUPDATES_OP, "request-buffer-updates", null),
-    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, REQUESTAPPLYUPDATES_OP, "request-apply-updates", null),
-    REQUESTSTATUS(PER_CORE_COMMANDS, POST, REQUESTSTATUS_OP, null, null),
-    OVERSEEROP(NODEAPIS, POST, OVERSEEROP_OP, "overseer-op", null),
-    REJOINLEADERELECTION(NODEAPIS, POST, REJOINLEADERELECTION_OP, "rejoin-leader-election", null),
-    INVOKE(NODEINVOKE, GET, INVOKE_OP, null, null),
-    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, FORCEPREPAREFORLEADERSHIP_OP, "force-prepare-for-leadership", null);
-
-    public final String commandName;
-    public final BaseHandlerApiSupport.V2EndPoint endPoint;
-    public final SolrRequest.METHOD method;
-    public final Map<String, String> paramstoAttr;
-    final CoreAdminOperation target;
-
-
-    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target) {
-      this.endPoint = endPoint;
-      this.method = method;
+    CREATE(Meta.CREATE, CREATE_OP),
+    UNLOAD(Meta.UNLOAD, UNLOAD_OP),
+    RELOAD(Meta.RELOAD, RELOAD_OP),
+    STATUS(Meta.STATUS, STATUS_OP),
+    SWAP(Meta.SWAP, SWAP_OP),
+    RENAME(Meta.RENAME, RENAME_OP),
+    MERGEINDEXES(Meta.MERGEINDEXES, MERGEINDEXES_OP),
+    SPLIT(Meta.SPLIT, SPLIT_OP),
+    PREPRECOVERY(Meta.PREPRECOVERY, PREPRECOVERY_OP),
+    REQUESTRECOVERY(Meta.REQUESTRECOVERY, REQUESTRECOVERY_OP),
+    REQUESTSYNCSHARD(Meta.REQUESTSYNCSHARD, REQUESTSYNCSHARD_OP),
+    REQUESTBUFFERUPDATES(Meta.REQUESTBUFFERUPDATES, REQUESTBUFFERUPDATES_OP),
+    REQUESTAPPLYUPDATES(Meta.REQUESTAPPLYUPDATES, REQUESTAPPLYUPDATES_OP),
+    REQUESTSTATUS(Meta.REQUESTSTATUS, REQUESTSTATUS_OP),
+    OVERSEEROP(Meta.OVERSEEROP, OVERSEEROP_OP),
+    REJOINLEADERELECTION(Meta.REJOINLEADERELECTION, REJOINLEADERELECTION_OP),
+    INVOKE(Meta.INVOKE, INVOKE_OP),
+    FORCEPREPAREFORLEADERSHIP(Meta.FORCEPREPAREFORLEADERSHIP, FORCEPREPAREFORLEADERSHIP_OP);
+
+    public final Meta meta;
+    public final CoreAdminOperation target;
+
+
+    Cmd(Meta meta, CoreAdminOperation target) {
+      this.meta = meta;
       this.target = target;
-      commandName = null;
-      paramstoAttr = Collections.EMPTY_MAP;
-
-    }
-
-
-    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target, String commandName,
-        Map<String, String> paramstoAttr) {
-      this.commandName = commandName == null ? target.action.toString().toLowerCase(Locale.ROOT) : commandName;
-      this.endPoint = endPoint;
-      this.method = method;
-      this.target = target;
-      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : paramstoAttr;
-    }
-
-    @Override
-    public String getName() {
-      return commandName;
-    }
-
-    @Override
-    public SolrRequest.METHOD getHttpMethod() {
-      return method;
     }
 
-    @Override
-    public V2EndPoint getEndPoint() {
-      return endPoint;
-    }
 
     @Override
-    public String getParamSubstitute(String param) {
-      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+    public CollectionApiMapping.CommandMeta meta() {
+      return meta;
     }
 
     @Override
@@ -135,29 +96,6 @@ public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
           target));
 
     }
-
-  }
-
-
-
-  enum EndPoint implements BaseHandlerApiSupport.V2EndPoint {
-    CORES_STATUS("cores.Status"),
-    CORES_COMMANDS("cores.Commands"),
-    PER_CORE_COMMANDS("cores.core.Commands"),
-    NODEINVOKE("node.invoke"),
-    NODEAPIS("node.Commands")
-    ;
-
-    final String specName;
-
-    EndPoint(String specName) {
-      this.specName = specName;
-    }
-
-    @Override
-    public String getSpecName() {
-      return specName;
-    }
   }
 
 
@@ -168,7 +106,7 @@ public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
 
   @Override
   protected List<V2EndPoint> getEndPoints() {
-    return Arrays.asList(EndPoint.values());
+    return Arrays.asList(CoreApiMapping.EndPoint.values());
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
index eceb4b7..36a3b57 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
@@ -42,7 +42,7 @@ import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.AuthorizationPlugin;
 import org.apache.solr.security.ConfigEditablePlugin;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag.ReqHandlerToApi;
 import org.apache.solr.api.SpecProvider;
@@ -102,7 +102,7 @@ public abstract class SecurityConfHandler extends RequestHandlerBase implements
     if (req.getContentStreams() == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No contentStream");
     }
-    List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), rsp);
+    List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), rsp.getValues());
     if (ops == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
index 852d501..69ae3ed 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
@@ -30,7 +30,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
index 8323b8a..9e77fe3 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
@@ -26,7 +26,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
index a494ec4..5bad237 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
@@ -22,7 +22,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.HttpSolrCall;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.RTimerTree;
 
 import java.security.Principal;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
index 19350f0..8275991 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
@@ -21,7 +21,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.RefCounted;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index 1658e67..d75fccd 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -35,7 +35,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.rest.BaseSolrResource;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
index 6a5230c..fc8b877 100644
--- a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
+++ b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
@@ -24,7 +24,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 
 import static org.apache.solr.common.util.Utils.getDeepCopy;
 import static org.apache.solr.handler.admin.SecurityConfHandler.getListValue;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index f1665c7..296a2a6 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -38,7 +38,7 @@ import org.apache.http.auth.BasicUserPrincipal;
 import org.apache.http.message.BasicHeader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ValidatingJsonMap;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.api.SpecProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java b/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
index a8bef9c..5c95f80 100644
--- a/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
@@ -19,7 +19,7 @@ package org.apache.solr.security;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 
 /**An interface to be implemented by a Plugin whose Configuration is runtime editable
  *
@@ -33,6 +33,6 @@ public interface ConfigEditablePlugin {
    * of the implementation to ensure that the returned config is valid . The framework
    * does no validation of the data
    */
-  public Map<String,Object> edit(Map<String,Object> latestConf, List<CommandOperation> commands);
+  Map<String,Object> edit(Map<String, Object> latestConf, List<CommandOperation> commands);
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java b/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
index a8a97ed..c54453d 100644
--- a/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
@@ -30,7 +30,7 @@ import java.util.function.Function;
 import org.apache.solr.api.ApiBag;
 import org.apache.solr.api.SpecProvider;
 import org.apache.solr.common.util.ValidatingJsonMap;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
index 91bbe74..10ed71a 100644
--- a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
@@ -30,9 +30,9 @@ import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.ValidatingJsonMap;
 
-import org.apache.solr.util.CommandOperation;
 import org.apache.solr.api.ApiBag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 4318621..f529bd4 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -101,7 +101,7 @@ import org.apache.solr.servlet.SolrDispatchFilter.Action;
 import org.apache.solr.servlet.cache.HttpCacheHeaderUtil;
 import org.apache.solr.servlet.cache.Method;
 import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.TimeOut;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java b/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
index 00733f5..90ca968 100644
--- a/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
+++ b/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
@@ -18,7 +18,7 @@ package org.apache.solr.servlet;
 import org.apache.solr.api.ApiBag;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.slf4j.Logger;
 
 import java.io.PrintWriter;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 93baace..9aae8e9 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -61,7 +61,7 @@ import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.SolrFileCleaningTracker;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/java/org/apache/solr/util/CommandOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/CommandOperation.java b/solr/core/src/java/org/apache/solr/util/CommandOperation.java
deleted file mode 100644
index 88dfbb9..0000000
--- a/solr/core/src/java/org/apache/solr/util/CommandOperation.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.util;
-
-import java.io.IOException;
-import java.io.Reader;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.util.IOUtils;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.response.SolrQueryResponse;
-import org.noggit.JSONParser;
-import org.noggit.ObjectBuilder;
-
-import static java.util.Collections.emptyMap;
-import static java.util.Collections.singletonList;
-import static java.util.Collections.singletonMap;
-import static org.apache.solr.common.util.StrUtils.formatString;
-import static org.apache.solr.common.util.Utils.toJSON;
-
-public class CommandOperation {
-  public final String name;
-  private Object commandData;//this is most often a map
-  private List<String> errors = new ArrayList<>();
-
-  public CommandOperation(String operationName, Object metaData) {
-    commandData = metaData;
-    this.name = operationName;
-  }
-
-  public Object getCommandData() {
-    return commandData;
-  }
-
-  public String getStr(String key, String def) {
-    if (ROOT_OBJ.equals(key)) {
-      Object obj = getRootPrimitive();
-      return obj == def ? null : String.valueOf(obj);
-    }
-    Object o = getMapVal(key);
-    return o == null ? def : String.valueOf(o);
-  }
-
-  public boolean getBoolean(String key, boolean def) {
-    String v = getStr(key,null);
-    return v == null? def:Boolean.parseBoolean(v);
-  }
-  public void setCommandData(Object o){
-    commandData = o;
-  }
-
-  public Map<String,Object> getDataMap() {
-    if (commandData instanceof Map) {
-      //noinspection unchecked
-      return (Map<String,Object>)commandData;
-    }
-    addError(StrUtils.formatString("The command ''{0}'' should have the values as a json object {key:val} format", name));
-    return Collections.emptyMap();
-  }
-
-  private Object getRootPrimitive() {
-    if (commandData instanceof Map) {
-      errors.add(StrUtils.formatString("The value has to be a string for command : ''{0}'' ", name));
-      return null;
-    }
-    return commandData;
-
-  }
-
-  public Object getVal(String key) {
-    return getMapVal(key);
-  }
-
-  private Object getMapVal(String key) {
-    if("".equals(key)){
-      if (commandData instanceof Map) {
-        addError("value of the command is an object should be primitive");
-      }
-      return commandData;
-    }
-    if (commandData instanceof Map) {
-      Map metaData = (Map) commandData;
-      return metaData.get(key);
-    } else {
-      String msg = " value has to be an object for operation :" + name;
-      if (!errors.contains(msg)) errors.add(msg);
-      return null;
-    }
-  }
-
-  public List<String> getStrs(String key) {
-    List<String> val = getStrs(key, null);
-    if (val == null) {
-      errors.add(StrUtils.formatString(REQD, key));
-    }
-    return val;
-
-  }
-
-  public void unknownOperation() {
-    addError(formatString("Unknown operation ''{0}'' ", name));
-  }
-
-  static final String REQD = "''{0}'' is a required field";
-
-
-  /**
-   * Get collection of values for a key. If only one val is present a
-   * single value collection is returned
-   */
-  public List<String> getStrs(String key, List<String> def) {
-    Object v = null;
-    if (ROOT_OBJ.equals(key)) {
-      v = getRootPrimitive();
-    } else {
-      v = getMapVal(key);
-    }
-    if (v == null) {
-      return def;
-    } else {
-      if (v instanceof List) {
-        ArrayList<String> l = new ArrayList<>();
-        for (Object o : (List) v) {
-          l.add(String.valueOf(o));
-        }
-        if (l.isEmpty()) return def;
-        return l;
-      } else {
-        return singletonList(String.valueOf(v));
-      }
-    }
-
-  }
-
-  /**
-   * Get a required field. If missing it adds to the errors
-   */
-  public String getStr(String key) {
-    if (ROOT_OBJ.equals(key)) {
-      Object obj = getRootPrimitive();
-      if (obj == null) {
-        errors.add(StrUtils.formatString(REQD, name));
-      }
-      return obj == null ? null : String.valueOf(obj);
-    }
-
-    String s = getStr(key, null);
-    if (s == null) errors.add(StrUtils.formatString(REQD, key));
-    return s;
-  }
-
-  private Map errorDetails() {
-    return Utils.makeMap(name, commandData, ERR_MSGS, errors);
-  }
-
-  public boolean hasError() {
-    return !errors.isEmpty();
-  }
-
-  public void addError(String s) {
-    if (errors.contains(s)) return;
-    errors.add(s);
-  }
-
-  /**
-   * Get all the values from the metadata for the command
-   * without the specified keys
-   */
-  public Map<String,Object> getValuesExcluding(String... keys) {
-    getMapVal(null);
-    if (hasError()) return emptyMap();//just to verify the type is Map
-    @SuppressWarnings("unchecked") 
-    LinkedHashMap<String, Object> cp = new LinkedHashMap<>((Map<String, Object>) commandData);
-    if (keys == null) return cp;
-    for (String key : keys) {
-      cp.remove(key);
-    }
-    return cp;
-  }
-
-
-  public List<String> getErrors() {
-    return errors;
-  }
-
-  public static final String ERR_MSGS = "errorMessages";
-  public static final String ROOT_OBJ = "";
-
-  public static List<Map> captureErrors(List<CommandOperation> ops) {
-    List<Map> errors = new ArrayList<>();
-    for (CommandOperation op : ops) {
-      if (op.hasError()) {
-        errors.add(op.errorDetails());
-      }
-    }
-    return errors;
-  }
-
-
-  /**
-   * Parse the command operations into command objects
-   */
-  public static List<CommandOperation> parse(Reader rdr) throws IOException {
-    JSONParser parser = new JSONParser(rdr);
-
-    ObjectBuilder ob = new ObjectBuilder(parser);
-
-    if (parser.lastEvent() != JSONParser.OBJECT_START) {
-      throw new RuntimeException("The JSON must be an Object of the form {\"command\": {...},...");
-    }
-    List<CommandOperation> operations = new ArrayList<>();
-    for (; ; ) {
-      int ev = parser.nextEvent();
-      if (ev == JSONParser.OBJECT_END) return operations;
-      Object key = ob.getKey();
-      ev = parser.nextEvent();
-      Object val = ob.getVal();
-      if (val instanceof List) {
-        List list = (List) val;
-        for (Object o : list) {
-          if (!(o instanceof Map)) {
-            operations.add(new CommandOperation(String.valueOf(key), list));
-            break;
-          } else {
-            operations.add(new CommandOperation(String.valueOf(key), o));
-          }
-        }
-      } else {
-        operations.add(new CommandOperation(String.valueOf(key), val));
-      }
-    }
-
-  }
-
-  public CommandOperation getCopy() {
-    return new CommandOperation(name, commandData);
-  }
-
-  public Map getMap(String key, Map def) {
-    Object o = getMapVal(key);
-    if (o == null) return def;
-    if (!(o instanceof Map)) {
-      addError(StrUtils.formatString("''{0}'' must be a map", key));
-      return def;
-    } else {
-      return (Map) o;
-
-    }
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return new String(toJSON(singletonMap(name, commandData)), IOUtils.UTF_8);
-    } catch (UnsupportedEncodingException e) {
-      //should not happen
-      return "";
-    }
-  }
-
-  public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, SolrQueryResponse resp)
-      throws IOException {
-    if (streams == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
-    }
-    ArrayList<CommandOperation> ops = new ArrayList<>();
-
-    for (ContentStream stream : streams)
-      ops.addAll(parse(stream.getReader()));
-    List<Map> errList = CommandOperation.captureErrors(ops);
-    if (!errList.isEmpty()) {
-      resp.add(CommandOperation.ERR_MSGS, errList);
-      return null;
-    }
-    return ops;
-  }
-
-  public static List<CommandOperation> clone(List<CommandOperation> ops) {
-    List<CommandOperation> opsCopy = new ArrayList<>(ops.size());
-    for (CommandOperation op : ops) opsCopy.add(op.getCopy());
-    return opsCopy;
-  }
-
-
-  public Integer getInt(String name, Integer def) {
-    Object o = getVal(name);
-    if (o == null) return def;
-    if (o instanceof Number) {
-      Number number = (Number) o;
-      return number.intValue();
-    } else {
-      try {
-        return Integer.parseInt(o.toString());
-      } catch (NumberFormatException e) {
-        addError(StrUtils.formatString("{0} is not a valid integer", name));
-        return null;
-      }
-    }
-  }
-
-  public Integer getInt(String name) {
-    Object o = getVal(name);
-    if(o == null) return null;
-    return getInt(name, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
index 54c8587..e6dbddb 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
@@ -30,7 +30,7 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.BasicAuthPlugin;
 import org.apache.solr.security.RuleBasedAuthorizationPlugin;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 
 import static org.apache.solr.common.util.Utils.makeMap;
 import static org.apache.solr.handler.admin.SecurityConfHandler.SecurityConfig;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index b784f87..2f4fcda 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -41,7 +41,7 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.V2HttpCall;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.util.PathTrie;
 
 import static org.apache.solr.api.ApiBag.EMPTY_SPEC;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
----------------------------------------------------------------------
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 7f072ec..dec1e04 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
@@ -39,7 +39,7 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.servlet.SolrRequestParsers;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/schema/TestSchemaManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSchemaManager.java b/solr/core/src/test/org/apache/solr/schema/TestSchemaManager.java
index b93db26..a6f7fc4 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestSchemaManager.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestSchemaManager.java
@@ -17,7 +17,7 @@
 package org.apache.solr.schema;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 import org.junit.BeforeClass;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
index 4cdc555..f4509a8 100644
--- a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
@@ -41,13 +41,13 @@ import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.component.SearchHandler;
 import org.apache.solr.security.AuthorizationContext.CollectionRequest;
 import org.apache.solr.security.AuthorizationContext.RequestType;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 
 import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.getObjectByPath;
 import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.util.CommandOperation.captureErrors;
+import static org.apache.solr.common.util.CommandOperation.captureErrors;
 
 public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
   String permissions = "{" +

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/core/src/test/org/apache/solr/security/TestSha256AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestSha256AuthenticationProvider.java b/solr/core/src/test/org/apache/solr/security/TestSha256AuthenticationProvider.java
index adcccda..9139bdf 100644
--- a/solr/core/src/test/org/apache/solr/security/TestSha256AuthenticationProvider.java
+++ b/solr/core/src/test/org/apache/solr/security/TestSha256AuthenticationProvider.java
@@ -21,7 +21,7 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.CommandOperation;
+import org.apache.solr.common.util.CommandOperation;
 
 import static java.util.Collections.singletonMap;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..9d7ec8a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -0,0 +1,369 @@
+/*
+ * 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;
+
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+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.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.common.params.CommonParams.NAME;
+
+/** stores the mapping of v1 API parameters to v2 API parameters
+ * for collection API and configset API
+ *
+ */
+public class CollectionApiMapping {
+
+  public enum Meta implements CommandMeta {
+    GET_COLLECTIONS(EndPoint.COLLECTIONS, GET),
+    GET_CLUSTER(EndPoint.CLUSTER, GET,  "/cluster", null),
+    GET_CLUSTER_OVERSEER(EndPoint.CLUSTER, GET, "/cluster/overseer", null),
+    GET_CLUSTER_STATUS_CMD(EndPoint.CLUSTER_CMD_STATUS, GET ),
+    DELETE_CLUSTER_STATUS(EndPoint.CLUSTER_CMD_STATUS_DELETE, DELETE),
+    GET_A_COLLECTION(EndPoint.COLLECTION_STATE, GET),
+    LIST_ALIASES(EndPoint.CLUSTER_ALIASES, GET),
+    CREATE_COLLECTION(EndPoint.COLLECTIONS_COMMANDS,
+        POST,
+        CollectionAction.CREATE.toLower(),
+        Utils.makeMap(
+            "collection.configName", "config",
+            "createNodeSet.shuffle", "shuffleNodes",
+            "createNodeSet", "nodeSet"
+        ),
+        Utils.makeMap("properties.", "property.")),
+
+    DELETE_COLL(EndPoint.PER_COLLECTION_DELETE,
+        DELETE,
+        CollectionAction.DELETE.toLower(),
+        Utils.makeMap(NAME, "collection")),
+
+    RELOAD_COLL(EndPoint.PER_COLLECTION,
+        POST,
+        CollectionAction.RELOAD.toLower(),
+        Utils.makeMap(NAME, "collection")),
+    MODIFYCOLLECTION(EndPoint.PER_COLLECTION,
+        POST,
+        "modify",null),
+    MIGRATE_DOCS(EndPoint.PER_COLLECTION,
+        POST,
+        "migrate-docs",
+        Utils.makeMap("split.key", "splitKey",
+            "target.collection", "target",
+            "forward.timeout", "forwardTimeout"
+        )),
+    REBALANCELEADERS(EndPoint.PER_COLLECTION,
+        POST,
+        "rebalance-leaders", null),
+    CREATE_ALIAS(EndPoint.COLLECTIONS_COMMANDS,
+        POST,
+        "create-alias",
+        null),
+
+    DELETE_ALIAS(EndPoint.COLLECTIONS_COMMANDS,
+        POST,
+        "delete-alias",
+        null),
+    CREATE_SHARD(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
+        POST,
+        "create",
+        Utils.makeMap("createNodeSet", "nodeSet"),
+        Utils.makeMap("coreProperties.", "property.")) {
+      @Override
+      public String getParamSubstitute(String param) {
+        return super.getParamSubstitute(param);
+      }
+    },
+
+    SPLIT_SHARD(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
+        POST,
+        "split",
+        Utils.makeMap(
+            "split.key", "splitKey"),
+        Utils.makeMap("coreProperties.", "property.")),
+    DELETE_SHARD(EndPoint.PER_COLLECTION_PER_SHARD_DELETE,
+        DELETE),
+
+    CREATE_REPLICA(EndPoint.PER_COLLECTION_SHARDS_COMMANDS,
+        POST,
+        "add-replica",
+        null,
+        Utils.makeMap("coreProperties.", "property.")),
+
+    DELETE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE,
+        DELETE),
+
+    SYNC_SHARD(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,
+        POST,
+        "synch-shard",
+        null),
+    ADDREPLICAPROP(EndPoint.PER_COLLECTION,
+        POST,
+        "add-replica-property",
+        Utils.makeMap("property", "name", "property.value", "value")),
+    DELETEREPLICAPROP(EndPoint.PER_COLLECTION,
+        POST,
+        "delete-replica-property",
+        null),
+    ADDROLE(EndPoint.CLUSTER_CMD,
+        POST,
+        "add-role",null),
+    REMOVEROLE(EndPoint.CLUSTER_CMD,
+        POST,
+        "remove-role",null),
+
+    CLUSTERPROP(EndPoint.CLUSTER_CMD,
+        POST,
+        "set-property",null),
+
+    BACKUP(EndPoint.COLLECTIONS_COMMANDS,
+        POST,
+        "backup-collection", null
+    ),
+    RESTORE(EndPoint.COLLECTIONS_COMMANDS,
+        POST,
+        "restore-collection",
+        null
+    ),
+    GET_NODES(EndPoint.CLUSTER_NODES, null),
+    FORCELEADER(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,POST, "force-leader",null),
+    SYNCSHARD(EndPoint.PER_COLLECTION_PER_SHARD_COMMANDS,POST, "sync-shard",null),
+    BALANCESHARDUNIQUE(EndPoint.PER_COLLECTION, POST, "balance-shard-unique",null)
+    ;
+
+    public final String commandName;
+    public final EndPoint endPoint;
+    public final SolrRequest.METHOD method;
+    //mapping of http param name to json attribute
+    public final Map<String, String> paramstoAttr;
+    //mapping of old prefix to new for instance properties.a=val can be substituted with property:{a:val}
+    public final Map<String, String> prefixSubstitutes;
+
+    public SolrRequest.METHOD getMethod() {
+      return method;
+    }
+
+
+    Meta(EndPoint endPoint, SolrRequest.METHOD method) {
+      this(endPoint, method,  null, null);
+    }
+
+    Meta(EndPoint endPoint, SolrRequest.METHOD method,
+         String commandName, Map paramstoAttr) {
+      this(endPoint, method,  commandName, paramstoAttr, Collections.EMPTY_MAP);
+
+    }
+
+    Meta(EndPoint endPoint, SolrRequest.METHOD method,
+         String commandName, Map paramstoAttr, Map prefixSubstitutes) {
+      this.commandName = commandName;
+      this.endPoint = endPoint;
+      this.method = method;
+      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : Collections.unmodifiableMap(paramstoAttr);
+      this.prefixSubstitutes = Collections.unmodifiableMap(prefixSubstitutes);
+
+    }
+
+    @Override
+    public String getName() {
+      return commandName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+
+    @Override
+    public Collection<String> getParamNames(CommandOperation op) {
+      Collection<String> paramNames = getParamNames_(op, this);
+      if (!prefixSubstitutes.isEmpty()) {
+        Collection<String> result = new ArrayList<>(paramNames.size());
+        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
+          for (String paramName : paramNames) {
+            if (paramName.startsWith(e.getKey())) {
+              result.add(paramName.replace(e.getKey(), e.getValue()));
+            } else {
+              result.add(paramName);
+            }
+          }
+          paramNames = result;
+        }
+      }
+
+      return paramNames;
+    }
+
+    @Override
+    public String getParamSubstitute(String param) {
+      String s = paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+      if (prefixSubstitutes != null) {
+        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
+          if (s.startsWith(e.getValue())) return s.replace(e.getValue(), e.getKey());
+        }
+      }
+      return s;
+    }
+
+  }
+
+  public enum EndPoint implements V2EndPoint {
+    CLUSTER("cluster"),
+    CLUSTER_ALIASES("cluster.aliases"),
+    CLUSTER_CMD("cluster.Commands"),
+    CLUSTER_NODES("cluster.nodes"),
+    CLUSTER_CMD_STATUS("cluster.commandstatus"),
+    CLUSTER_CMD_STATUS_DELETE("cluster.commandstatus.delete"),
+    COLLECTIONS_COMMANDS("collections.Commands"),
+    COLLECTIONS("collections"),
+    COLLECTION_STATE("collections.collection"),
+    PER_COLLECTION("collections.collection.Commands"),
+    PER_COLLECTION_DELETE("collections.collection.delete"),
+    PER_COLLECTION_SHARDS_COMMANDS("collections.collection.shards.Commands"),
+    PER_COLLECTION_PER_SHARD_COMMANDS("collections.collection.shards.shard.Commands"),
+    PER_COLLECTION_PER_SHARD_DELETE("collections.collection.shards.shard.delete"),
+    PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE("collections.collection.shards.shard.replica.delete");
+    final String specName;
+
+
+    EndPoint(String specName) {
+      this.specName = specName;
+    }
+
+    @Override
+    public String getSpecName() {
+      return specName;
+    }
+  }
+
+  public interface V2EndPoint {
+
+    String getSpecName();
+  }
+
+  public enum ConfigSetMeta implements CommandMeta {
+    LIST(ConfigSetEndPoint.LIST_CONFIG, GET),
+    CREATE(ConfigSetEndPoint.CONFIG_COMMANDS, POST, "create"),
+    DEL(ConfigSetEndPoint.CONFIG_DEL,  DELETE)
+    ;
+    private final ConfigSetEndPoint endPoint;
+    private final SolrRequest.METHOD method;
+    private final String cmdName;
+
+    ConfigSetMeta(ConfigSetEndPoint endPoint, SolrRequest.METHOD method) {
+      this(endPoint, method, null);
+    }
+
+    ConfigSetMeta(ConfigSetEndPoint endPoint, SolrRequest.METHOD method, String cmdName) {
+      this.cmdName = cmdName;
+      this.endPoint = endPoint;
+      this.method = method;
+    }
+
+    @Override
+    public String getName() {
+      return cmdName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+
+  }
+  public enum ConfigSetEndPoint implements V2EndPoint {
+    LIST_CONFIG("cluster.configs"),
+    CONFIG_COMMANDS("cluster.configs.Commands"),
+    CONFIG_DEL("cluster.configs.delete");
+
+    public final String spec;
+
+    ConfigSetEndPoint(String spec) {
+      this.spec = spec;
+    }
+
+    @Override
+    public String getSpecName() {
+      return spec;
+    }
+  }
+
+
+
+  private static Collection<String> getParamNames_(CommandOperation op, CommandMeta command) {
+    List<String> result = new ArrayList<>();
+    Object o = op.getCommandData();
+    if (o instanceof Map) {
+      Map map = (Map) o;
+      collectKeyNames(map, result, "");
+    }
+    return result;
+
+  }
+
+  public static void collectKeyNames(Map<String, Object> map, List<String> result, String prefix) {
+    for (Map.Entry<String, Object> e : map.entrySet()) {
+      if (e.getValue() instanceof Map) {
+        collectKeyNames((Map) e.getValue(), result, prefix + e.getKey() + ".");
+      } else {
+        result.add(prefix + e.getKey());
+      }
+    }
+  }
+  public interface CommandMeta {
+    String getName();
+
+    /**
+     * the http method supported by this command
+     */
+    SolrRequest.METHOD getHttpMethod();
+
+    V2EndPoint getEndPoint();
+
+    default Collection<String> getParamNames(CommandOperation op) {
+      return getParamNames_(op, CommandMeta.this);
+    }
+
+
+    default String getParamSubstitute(String name) {
+      return name;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0184d6b7/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreApiMapping.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreApiMapping.java
new file mode 100644
index 0000000..c7e20f9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreApiMapping.java
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionApiMapping.CommandMeta;
+import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
+import org.apache.solr.common.util.Utils;
+
+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.CoreApiMapping.EndPoint.CORES_COMMANDS;
+import static org.apache.solr.client.solrj.request.CoreApiMapping.EndPoint.CORES_STATUS;
+import static org.apache.solr.client.solrj.request.CoreApiMapping.EndPoint.NODEAPIS;
+import static org.apache.solr.client.solrj.request.CoreApiMapping.EndPoint.NODEINVOKE;
+import static org.apache.solr.client.solrj.request.CoreApiMapping.EndPoint.PER_CORE_COMMANDS;
+
+/** stores the mapping of v1 API parameters to v2 API parameters
+ * for core admin API
+ *
+ */
+public class CoreApiMapping {
+  public enum Meta implements CommandMeta {
+    CREATE(CORES_COMMANDS, POST, CoreAdminAction.CREATE, Utils.makeMap("config", "configSet")),
+    UNLOAD(PER_CORE_COMMANDS, POST, CoreAdminAction.UNLOAD, null),
+    RELOAD(PER_CORE_COMMANDS, POST, CoreAdminAction.RELOAD, null),
+    STATUS(CORES_STATUS, GET, CoreAdminAction.STATUS, null),
+    SWAP(PER_CORE_COMMANDS, POST, CoreAdminAction.SWAP, Utils.makeMap("other", "with")),
+    RENAME(PER_CORE_COMMANDS, POST, CoreAdminAction.RENAME, null),
+    MERGEINDEXES(PER_CORE_COMMANDS, POST, "merge-indexes", null),
+    SPLIT(PER_CORE_COMMANDS, POST, CoreAdminAction.SPLIT, Utils.makeMap("split.key", "splitKey")),
+    PREPRECOVERY(PER_CORE_COMMANDS, POST, "prep-recovery", null),
+    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, CoreAdminAction.REQUESTRECOVERY, null),
+    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, "request-sync-shard", null),
+    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, "request-buffer-updates", null),
+    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, "request-apply-updates", null),
+    REQUESTSTATUS(PER_CORE_COMMANDS, POST, CoreAdminAction.REQUESTSTATUS, null),
+    OVERSEEROP(NODEAPIS, POST, "overseer-op", null),
+    REJOINLEADERELECTION(NODEAPIS, POST, "rejoin-leader-election", null),
+    INVOKE(NODEINVOKE, GET, CoreAdminAction.INVOKE, null),
+    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, "force-prepare-for-leadership", null);
+
+    public final String commandName;
+    public final EndPoint endPoint;
+    public final SolrRequest.METHOD method;
+    public final Map<String, String> paramstoAttr;
+
+    Meta(EndPoint endPoint, SolrRequest.METHOD method, Object commandName,
+         Map paramstoAttr) {
+      this.commandName = commandName.toString().toLowerCase(Locale.ROOT);
+      this.endPoint = endPoint;
+      this.method = method;
+      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : Collections.unmodifiableMap(paramstoAttr);
+    }
+
+    @Override
+    public String getName() {
+      return commandName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public CollectionApiMapping.V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+    @Override
+    public String getParamSubstitute(String param) {
+      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+    }
+
+
+  }
+
+  public enum EndPoint implements CollectionApiMapping.V2EndPoint {
+    CORES_STATUS("cores.Status"),
+    CORES_COMMANDS("cores.Commands"),
+    PER_CORE_COMMANDS("cores.core.Commands"),
+    NODEINVOKE("node.invoke"),
+    NODEAPIS("node.Commands");
+
+    final String specName;
+
+    EndPoint(String specName) {
+      this.specName = specName;
+    }
+
+    @Override
+    public String getSpecName() {
+      return specName;
+    }
+  }
+}