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/01/31 05:57:51 UTC

[1/5] lucene-solr:master: SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new

Repository: lucene-solr
Updated Branches:
  refs/heads/master c8edbe866 -> 71abe1306


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java b/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
index b9e1e4a..a34febc 100644
--- a/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
@@ -55,6 +55,7 @@ import org.apache.solr.servlet.SolrRequestParsers.MultipartRequestParser;
 import org.apache.solr.servlet.SolrRequestParsers.FormDataRequestParser;
 import org.apache.solr.servlet.SolrRequestParsers.RawRequestParser;
 import org.apache.solr.servlet.SolrRequestParsers.StandardRequestParser;
+import org.easymock.EasyMock;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -481,6 +482,7 @@ public class SolrRequestParserTest extends SolrTestCaseJ4 {
     // we dont pass a content-length to let the security mechanism limit it:
     expect(request.getQueryString()).andReturn("foo=1&bar=2").anyTimes();
     expect(request.getInputStream()).andReturn(new ByteServletInputStream(body.getBytes(StandardCharsets.US_ASCII)));
+    expect(request.getAttribute(EasyMock.anyObject(String.class))).andReturn(null).anyTimes();
     replay(request);
 
     SolrRequestParsers parsers = new SolrRequestParsers(h.getCore().getSolrConfig());
@@ -518,7 +520,7 @@ public class SolrRequestParserTest extends SolrTestCaseJ4 {
     expect(request.getRequestURI()).andReturn(uri).anyTimes();
     expect(request.getContentType()).andReturn(contentType).anyTimes();
     expect(request.getContentLength()).andReturn(contentLength).anyTimes();
-    expect(request.getAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE)).andReturn(null).anyTimes();
+    expect(request.getAttribute(EasyMock.anyObject(String.class))).andReturn(null).anyTimes();
     return request;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java b/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
new file mode 100644
index 0000000..4492586
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+
+import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
+import static org.apache.solr.common.util.Utils.toJSONString;
+
+public class JsonValidatorTest extends SolrTestCaseJ4 {
+
+  public void testSchema() {
+    checkSchema("collections.Commands");
+    checkSchema("collections.collection.Commands");
+    checkSchema("collections.collection.shards.Commands");
+    checkSchema("collections.collection.shards.shard.Commands");
+    checkSchema("cores.Commands");
+    checkSchema("cores.core.Commands");
+    checkSchema("node.Commands");
+    checkSchema("cluster.security.BasicAuth.Commands");
+    checkSchema("cluster.security.RuleBasedAuthorization");
+    checkSchema("core.config.Commands");
+    checkSchema("core.SchemaEdit");
+    checkSchema("cluster.configs.Commands");
+  }
+
+
+  public void testSchemaValidation() {
+    ValidatingJsonMap spec = ApiBag.getSpec("collections.Commands").getSpec();
+    Map createSchema = spec.getMap("commands", NOT_NULL).getMap("create-alias", NOT_NULL);
+    JsonSchemaValidator validator = new JsonSchemaValidator(createSchema);
+    List<String> errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ c1 , c2]}"));
+    assertNull(toJSONString(errs), errs);
+    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: c1 }"));
+    assertNull(toJSONString(errs), errs);
+    errs = validator.validateJson(Utils.fromJSONString("{name : x, x:y, collections: [ c1 , c2]}"));
+    assertNotNull(toJSONString(errs), errs);
+    assertTrue(toJSONString(errs), errs.get(0).contains("Unknown"));
+    errs = validator.validateJson(Utils.fromJSONString("{name : 123, collections: c1 }"));
+    assertNotNull(toJSONString(errs), errs);
+    assertTrue(toJSONString(errs), errs.get(0).contains("Expected type"));
+    errs = validator.validateJson(Utils.fromJSONString("{x:y, collections: [ c1 , c2]}"));
+    assertEquals(toJSONString(errs), 2, errs.size());
+    assertTrue(toJSONString(errs), StrUtils.join(errs, '|').contains("Missing field"));
+    assertTrue(toJSONString(errs), StrUtils.join(errs, '|').contains("Unknown"));
+    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ 1 , 2]}"));
+    assertFalse(toJSONString(errs), errs.isEmpty());
+    assertTrue(toJSONString(errs), errs.get(0).contains("Expected elements of type"));
+    validator = new JsonSchemaValidator("{" +
+        "  type:object," +
+        "  properties: {" +
+        "   age : {type: number}," +
+        "   adult : {type: boolean}," +
+        "   name: {type: string}}}");
+    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:21, adult:true}"));
+    assertNull(errs);
+    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'21', adult:'true'}"));
+    assertNull(errs);
+
+    errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'x21', adult:'true'}"));
+    assertEquals(1, errs.size());
+    try {
+      validator = new JsonSchemaValidator("{" +
+          "  type:object," +
+          "  properties: {" +
+          "   age : {type: int}," +
+          "   adult : {type: Boolean}," +
+          "   name: {type: string}}}");
+      fail("should have failed");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Unknown type"));
+    }
+
+    try {
+      new JsonSchemaValidator("{" +
+          "  type:object," +
+          "   x : y," +
+          "  properties: {" +
+          "   age : {type: number}," +
+          "   adult : {type: boolean}," +
+          "   name: {type: string}}}");
+      fail("should have failed");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Unknown key"));
+    }
+    try {
+      new JsonSchemaValidator("{" +
+          "  type:object," +
+          "  propertes: {" +
+          "   age : {type: number}," +
+          "   adult : {type: boolean}," +
+          "   name: {type: string}}}");
+      fail("should have failed");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Unknown key : propertes"));
+    }
+
+    validator = new JsonSchemaValidator("{" +
+        "  type:object," +
+        "  properties: {" +
+        "   age : {type: number}," +
+        "   sex: {type: string, enum:[M, F]}," +
+        "   adult : {type: boolean}," +
+        "   name: {type: string}}}");
+    errs = validator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:M}"));
+    assertNull("errs are " + errs, errs);
+    errs = validator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:m}"));
+    assertEquals(1, errs.size());
+    assertTrue(errs.get(0).contains("value of enum"));
+    
+    String schema = "{\n" +
+        "  'type': 'object',\n" +
+        "  'properties': {\n" +
+        "    'links': {\n" +
+        "      'type': 'array',\n" +
+        "      'items':{" +
+        "          'type': 'object',\n" +
+        "          'properties': {\n" +
+        "            'rel': {\n" +
+        "              'type': 'string'\n" +
+        "            },\n" +
+        "            'href': {\n" +
+        "              'type': 'string'\n" +
+        "            }\n" +
+        "          }\n" +
+        "        }\n" +
+        "    }\n" +
+        "\n" +
+        "  }\n" +
+        "}";
+    validator = new JsonSchemaValidator(schema);
+    validator.validateJson(Utils.fromJSONString("{\n" +
+        "  'links': [\n" +
+        "    {\n" +
+        "        'rel': 'x',\n" +
+        "        'href': 'x'\n" +
+        "    },\n" +
+        "    {\n" +
+        "        'rel': 'x',\n" +
+        "        'href': 'x'\n" +
+        "    },\n" +
+        "    {\n" +
+        "        'rel': 'x',\n" +
+        "        'href': 'x'\n" +
+        "    }\n" +
+        "  ]\n" +
+        "}"));
+    
+
+
+
+  }
+
+  private void checkSchema(String name) {
+    ValidatingJsonMap spec = ApiBag.getSpec(name).getSpec();
+    Map commands = (Map) spec.get("commands");
+    for (Object o : commands.entrySet()) {
+      Map.Entry cmd = (Map.Entry) o;
+      try {
+        JsonSchemaValidator validator = new JsonSchemaValidator((Map) cmd.getValue());
+      } catch (Exception e) {
+        throw new RuntimeException("Error in command  " + cmd.getKey() + " in schema " + name, e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index 69a1519..a9ddb25 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -850,7 +850,7 @@
     </requestHandler>
 
   <!-- A request handler that returns indented JSON by default -->
-  <requestHandler name="/query" class="solr.SearchHandler">
+  <requestHandler name="/query" class="solr.SearchHandler" registerPath="/,/v2">
      <lst name="defaults">
        <str name="echoParams">explicit</str>
        <str name="wt">json</str>
@@ -958,7 +958,7 @@
   </requestHandler>
 
 
-  <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell,/browse">
+  <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell,/browse,update">
     <lst name="defaults">
       <str name="df">text</str>
     </lst>
@@ -966,12 +966,12 @@
 
   <!-- The following are implicitly added
   <requestHandler name="/update/json" class="solr.UpdateRequestHandler">
-        <lst name="defaults">
+        <lst name="invariants">
          <str name="stream.contentType">application/json</str>
        </lst>
   </requestHandler>
   <requestHandler name="/update/csv" class="solr.UpdateRequestHandler">
-        <lst name="defaults">
+        <lst name="invariants">
          <str name="stream.contentType">application/csv</str>
        </lst>
   </requestHandler>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
index 3e31edf..4dbba5b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
@@ -21,10 +21,14 @@ import org.apache.solr.common.util.ContentStream;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import static java.util.Collections.unmodifiableSet;
+
 /**
  * 
  *
@@ -35,9 +39,16 @@ public abstract class SolrRequest<T extends SolrResponse> implements Serializabl
   public enum METHOD {
     GET,
     POST,
-    PUT
+    PUT,
+    DELETE
   };
 
+  public static final Set<String> SUPPORTED_METHODS = unmodifiableSet(new HashSet<>(Arrays.<String>asList(
+      METHOD.GET.toString(),
+      METHOD.POST.toString(),
+      METHOD.PUT.toString(),
+      METHOD.DELETE.toString())));
+
   private METHOD method = METHOD.GET;
   private String path = null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 3b69484..d0263c8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -23,6 +23,7 @@ import java.net.ConnectException;
 import java.net.SocketException;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -83,7 +84,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
 
-import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
+import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
+import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
 
 /**
  * SolrJ client class to communicate with SolrCloud.
@@ -1036,6 +1041,15 @@ public class CloudSolrClient extends SolrClient {
       collection = (reqParams != null) ? reqParams.get("collection", getDefaultCollection()) : getDefaultCollection();
     return requestWithRetryOnStaleState(request, 0, collection);
   }
+  private static final Set<String> ADMIN_PATHS = new HashSet<>(Arrays.asList(
+      CORES_HANDLER_PATH,
+      COLLECTIONS_HANDLER_PATH,
+      CONFIGSETS_HANDLER_PATH,
+      AUTHC_PATH,
+      AUTHZ_PATH,
+      "/v2/cluster/security/authentication",
+      "/v2/cluster/security/authorization"
+      ));
 
   /**
    * As this class doesn't watch external collections on the client side,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index b47cf00..4968cf2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -108,6 +108,9 @@ public class Replica extends ZkNodeProps {
   public String getCoreUrl() {
     return ZkCoreNodeProps.getCoreUrl(getStr(BASE_URL_PROP), getStr(CORE_NAME_PROP));
   }
+  public String getBaseUrl(){
+    return getStr(ZkStateReader.BASE_URL_PROP);
+  }
 
   public String getCoreName() {
     return getStr(CORE_NAME_PROP);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
index 995e142..7313597 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
@@ -33,13 +33,18 @@ public class StrUtils {
   public static final char[] HEX_DIGITS = { '0', '1', '2', '3', '4', '5', '6',
       '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f' };
 
+  public static List<String> splitSmart(String s, char separator) {
+    ArrayList<String> lst = new ArrayList<>(4);
+    splitSmart(s, separator, lst);
+    return lst;
+
+  }
   /**
    * Split a string based on a separator, but don't split if it's inside
    * a string.  Assume '\' escapes the next char both inside and
    * outside strings.
    */
-  public static List<String> splitSmart(String s, char separator) {
-    ArrayList<String> lst = new ArrayList<>(4);
+  public static void splitSmart(String s, char separator, List<String> lst) {
     int pos=0, start=0, end=s.length();
     char inString=0;
     char ch=0;
@@ -72,7 +77,6 @@ public class StrUtils {
     }
     ***/
 
-    return lst;
   }
 
   /** Splits a backslash escaped string on the separator.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index a8de0ac..4cb6b8e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -146,7 +146,9 @@ public class Utils {
   }
 
   public static Object getObjectByPath(Map root, boolean onlyPrimitive, String hierarchy) {
-    return getObjectByPath(root, onlyPrimitive, StrUtils.splitSmart(hierarchy, '/'));
+    List<String> parts = StrUtils.splitSmart(hierarchy, '/');
+    if (parts.get(0).isEmpty()) parts.remove(0);
+    return getObjectByPath(root, onlyPrimitive, parts);
   }
 
   public static Object getObjectByPath(Map root, boolean onlyPrimitive, List<String> hierarchy) {
@@ -172,6 +174,7 @@ public class Utils {
         obj = (Map) o;
       } else {
         Object val = obj.get(s);
+        if (val == null) return null;
         if (idx > -1) {
           List l = (List) val;
           val = idx < l.size() ? l.get(idx) : null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
new file mode 100644
index 0000000..eef3823
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
@@ -0,0 +1,349 @@
+/*
+ * 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.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.common.SolrException;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableSet;
+
+public class ValidatingJsonMap implements Map<String, Object> {
+
+  private static final String INCLUDE = "#include";
+  private static final String RESOURCE_EXTENSION = ".json";
+  public static final PredicateWithErrMsg<Object> NOT_NULL = o -> {
+    if (o == null) return " Must not be NULL";
+    return null;
+  };
+  public static final PredicateWithErrMsg<Pair> ENUM_OF = pair -> {
+    if (pair.second() instanceof Set) {
+      Set set = (Set) pair.second();
+      if (pair.first() instanceof Collection) {
+        for (Object o : (Collection) pair.first()) {
+          if (!set.contains(o)) {
+            return " Must be one of " + pair.second();
+          }
+        }
+      } else {
+        if (!set.contains(pair.first())) return " Must be one of " + pair.second() + ", got " + pair.first();
+      }
+      return null;
+    } else {
+      return " Unknown type";
+    }
+
+  };
+  private final Map<String, Object> delegate;
+
+  public ValidatingJsonMap(Map<String, Object> delegate) {
+    this.delegate = delegate;
+  }
+
+  public ValidatingJsonMap(int i) {
+    delegate = new LinkedHashMap<>(i);
+  }
+
+  public ValidatingJsonMap() {
+    delegate = new LinkedHashMap<>();
+  }
+
+  @Override
+  public int size() {
+    return delegate.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return delegate.isEmpty();
+  }
+
+  @Override
+  public boolean containsKey(Object key) {
+    return delegate.containsKey(key);
+  }
+
+  @Override
+  public boolean containsValue(Object value) {
+    return delegate.containsValue(value);
+  }
+
+  @Override
+  public Object get(Object key) {
+    return delegate.get(key);
+  }
+
+  @Override
+  public Object put(String key, Object value) {
+    return delegate.put(key, value);
+  }
+
+  @Override
+  public Object remove(Object key) {
+    return delegate.remove(key);
+  }
+
+  @Override
+  public void putAll(Map<? extends String, ?> m) {
+    delegate.putAll(m);
+  }
+
+  @Override
+  public void clear() {
+    delegate.clear();
+
+  }
+
+  @Override
+  public Set<String> keySet() {
+    return delegate.keySet();
+  }
+
+  @Override
+  public Collection<Object> values() {
+    return delegate.values();
+  }
+
+  @Override
+  public Set<Entry<String, Object>> entrySet() {
+    return delegate.entrySet();
+  }
+
+  public Object get(String key, PredicateWithErrMsg predicate) {
+    Object v = get(key);
+    if (predicate != null) {
+      String msg = predicate.test(v);
+      if (msg != null) {
+        throw new RuntimeException("" + key + msg);
+      }
+    }
+    return v;
+  }
+
+  public Boolean getBool(String key, Boolean def) {
+    Object v = get(key);
+    if (v == null) return def;
+    if (v instanceof Boolean) return (Boolean) v;
+    try {
+      return Boolean.parseBoolean(v.toString());
+    } catch (NumberFormatException e) {
+      throw new RuntimeException("value of " + key + "must be an boolean");
+    }
+  }
+
+  public Integer getInt(String key, Integer def) {
+    Object v = get(key);
+    if (v == null) return def;
+    if (v instanceof Integer) return (Integer) v;
+    try {
+      return Integer.parseInt(v.toString());
+    } catch (NumberFormatException e) {
+      throw new RuntimeException("value of " + key + "must be an integer");
+    }
+  }
+
+  public ValidatingJsonMap getMap(String key) {
+    return getMap(key, null, null);
+  }
+
+  public ValidatingJsonMap getMap(String key, PredicateWithErrMsg predicate) {
+    return getMap(key, predicate, null);
+
+  }
+
+  public ValidatingJsonMap getMap(String key, PredicateWithErrMsg predicate, String message) {
+    Object v = get(key);
+    if (v != null && !(v instanceof Map)) {
+      throw new RuntimeException("" + key + " should be of type map");
+    }
+
+    if (predicate != null) {
+      String msg = predicate.test(v);
+      if (msg != null) {
+        msg = message != null ? message : key + msg;
+        throw new RuntimeException(msg);
+      }
+    }
+    return wrap((Map) v);
+  }
+
+  public List getList(String key, PredicateWithErrMsg predicate) {
+    return getList(key, predicate, null);
+  }
+
+  public List getList(String key, PredicateWithErrMsg predicate, Object test) {
+    Object v = get(key);
+    if (v != null && !(v instanceof List)) {
+      throw new RuntimeException("" + key + " should be of type List");
+    }
+
+    if (predicate != null) {
+      String msg = predicate.test(test == null ? v : new Pair(v, test));
+      if (msg != null) {
+        throw new RuntimeException("" + key + msg);
+      }
+    }
+
+    return (List) v;
+  }
+
+  public Object get(String key, PredicateWithErrMsg<Pair> predicate, Object arg) {
+    Object v = get(key);
+    String test = predicate.test(new Pair(v, arg));
+    if (test != null) {
+      throw new RuntimeException("" + key + test);
+    }
+    return v;
+  }
+
+  public Object get(String k, Object def) {
+    Object v = get(k);
+    if (v == null) return def;
+    return v;
+  }
+
+  static ValidatingJsonMap wrap(Map<String, Object> map) {
+    if (map == null) return null;
+    if (map instanceof ValidatingJsonMap) {
+      return (ValidatingJsonMap) map;
+    } else {
+      return new ValidatingJsonMap(map);
+    }
+
+  }
+
+  public static ValidatingJsonMap fromJSON(InputStream is, String includeLocation) {
+    return fromJSON(new InputStreamReader(is, UTF_8), includeLocation);
+  }
+
+  public static ValidatingJsonMap fromJSON(Reader s, String includeLocation) {
+    try {
+      ValidatingJsonMap map = (ValidatingJsonMap) getObjectBuilder(new JSONParser(s)).getObject();
+      handleIncludes(map, includeLocation, 4);
+      return map;
+    } catch (IOException e) {
+      throw new RuntimeException();
+    }
+  }
+
+  /**
+   * In the given map, recursively replace "#include":"resource-name" with the key/value pairs
+   * parsed from the resource at {location}/{resource-name}.json
+   */
+  private static void handleIncludes(ValidatingJsonMap map, String location, int maxDepth) {
+    final String loc = location == null ? "" // trim trailing slash
+        : (location.endsWith("/") ? location.substring(0, location.length() - 1) : location);
+    String resourceToInclude = (String) map.get(INCLUDE);
+    if (resourceToInclude != null) {
+      ValidatingJsonMap includedMap = parse(loc + "/" + resourceToInclude + RESOURCE_EXTENSION, loc);
+      map.remove(INCLUDE);
+      map.putAll(includedMap);
+    }
+    if (maxDepth > 0) {
+      map.entrySet().stream()
+          .filter(e -> e.getValue() instanceof Map)
+          .map(Map.Entry::getValue)
+          .forEach(m -> handleIncludes((ValidatingJsonMap) m, loc, maxDepth - 1));
+    }
+  }
+
+  public static ValidatingJsonMap getDeepCopy(Map map, int maxDepth, boolean mutable) {
+    if (map == null) return null;
+    if (maxDepth < 1) return ValidatingJsonMap.wrap(map);
+    ValidatingJsonMap copy = mutable ? new ValidatingJsonMap(map.size()) : new ValidatingJsonMap();
+    for (Object o : map.entrySet()) {
+      Map.Entry<String, Object> e = (Entry<String, Object>) o;
+      Object v = e.getValue();
+      if (v instanceof Map) v = getDeepCopy((Map) v, maxDepth - 1, mutable);
+      else if (v instanceof Collection) v = getDeepCopy((Collection) v, maxDepth - 1, mutable);
+      copy.put(e.getKey(), v);
+    }
+    return mutable ? copy : new ValidatingJsonMap(Collections.unmodifiableMap(copy));
+  }
+
+  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
+    if (c == null || maxDepth < 1) return c;
+    Collection result = c instanceof Set ? new HashSet() : new ArrayList();
+    for (Object o : c) {
+      if (o instanceof Map) {
+        o = getDeepCopy((Map) o, maxDepth - 1, mutable);
+      }
+      result.add(o);
+    }
+    return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
+  }
+
+  private static ObjectBuilder getObjectBuilder(final JSONParser jp) throws IOException {
+    return new ObjectBuilder(jp) {
+      @Override
+      public Object newObject() throws IOException {
+        return new ValidatingJsonMap();
+      }
+    };
+  }
+
+  public static ValidatingJsonMap parse(String resourceName, String includeLocation) {
+    InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(resourceName);
+    if (is == null)
+      throw new RuntimeException("invalid API spec: " + resourceName);
+    ValidatingJsonMap map = null;
+    try {
+      map = fromJSON(is, includeLocation);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in JSON : " + resourceName, e);
+    }
+    if (map == null) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Empty value for " + resourceName);
+
+    return getDeepCopy(map, 5, false);
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    return that instanceof Map && this.delegate.equals(that);
+  }
+
+  public static final ValidatingJsonMap EMPTY = new ValidatingJsonMap(Collections.EMPTY_MAP);
+
+  public interface PredicateWithErrMsg<T> {
+
+    /**
+     * Test the object and return null if the predicate is true
+     * or return a string with a message;
+     *
+     * @param t test value
+     * @return null if test succeeds or an error description if test fails
+     */
+    String test(T t);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
index 2cf8e04..b7ac7de 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
@@ -73,7 +73,7 @@ public class SolrExampleJettyTest extends SolrExampleTests {
     // two docs, one with uniqueKey, another without it
     String json = "{\"id\":\"abc1\", \"name\": \"name1\"} {\"name\" : \"name2\"}";
     HttpClient httpClient = client.getHttpClient();
-    HttpPost post = new HttpPost(client.getBaseURL() + "/update/json/docs");
+    HttpPost post = new HttpPost(getUri(client));
     post.setHeader("Content-Type", "application/json");
     post.setEntity(new InputStreamEntity(new ByteArrayInputStream(json.getBytes("UTF-8")), -1));
     HttpResponse response = httpClient.execute(post, HttpClientUtil.createNewHttpClientRequestContext());
@@ -94,4 +94,11 @@ public class SolrExampleJettyTest extends SolrExampleTests {
     assertEquals("name2",m.get("name"));
 
   }
+
+  private String getUri(HttpSolrClient client) {
+    String baseURL = client.getBaseURL();
+    return random().nextBoolean() ?
+        baseURL.replace("/collection1", "/v2/cores/collection1/update") :
+        baseURL + "/update/json/docs";
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/solrj/src/test/org/apache/solr/common/util/TestValidatingJsonMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestValidatingJsonMap.java b/solr/solrj/src/test/org/apache/solr/common/util/TestValidatingJsonMap.java
new file mode 100644
index 0000000..e5f8183
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/common/util/TestValidatingJsonMap.java
@@ -0,0 +1,52 @@
+/*
+ * 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.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.SolrTestCaseJ4;
+
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.common.util.ValidatingJsonMap.ENUM_OF;
+import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
+
+public class TestValidatingJsonMap extends SolrTestCaseJ4 {
+  public void testBasic() throws Exception {
+    ValidatingJsonMap m = ValidatingJsonMap.wrap(
+        makeMap("a", Boolean.TRUE,
+                "b", Boolean.FALSE,
+                "i", 10,
+                "l" , Arrays.asList("X", "Y"),
+            "c", makeMap("d", "D")));
+    assertEquals(Boolean.TRUE, m.getBool("a", Boolean.FALSE));
+    assertEquals(Boolean.FALSE, m.getBool("b", Boolean.TRUE));
+    assertEquals(new Integer(10), m.getInt("i",0));
+    try {
+      m.getList("l", ENUM_OF, ImmutableSet.of("X", "Z"));
+      fail("Must have failed with unexpected type");
+    } catch (RuntimeException e) { }
+
+    List l = m.getList("l", ENUM_OF, ImmutableSet.of("X", "Y", "Z"));
+    assertEquals(2,l.size());
+    m.getList("l", NOT_NULL);
+    assertNotNull(m.getMap("c"));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
index 0f1c8c9..11e28a6 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
@@ -51,6 +51,14 @@ public class RestTestHarness extends BaseTestHarness implements Closeable {
     return serverProvider.getBaseURL();
   }
 
+  public void setServerProvider(RESTfulServerProvider serverProvider) {
+    this.serverProvider = serverProvider;
+  }
+
+  public RESTfulServerProvider getServerProvider() {
+    return this.serverProvider;
+  }
+
   public String getAdminURL() {
     return getBaseURL().replace("/collection1", "");
   }


[3/5] lucene-solr:master: SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.commandstatus.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.commandstatus.delete.json b/solr/core/src/resources/apispec/cluster.commandstatus.delete.json
new file mode 100644
index 0000000..5576c42
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.commandstatus.delete.json
@@ -0,0 +1,10 @@
+{
+  "methods": [
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/command-status/{id}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.commandstatus.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.commandstatus.json b/solr/core/src/resources/apispec/cluster.commandstatus.json
new file mode 100644
index 0000000..a8a402b
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.commandstatus.json
@@ -0,0 +1,20 @@
+{
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/command-status"
+    ],
+    "params": {
+      "flush": {
+        "type": "boolean",
+        "default": false
+      },
+      "id":{
+        "type":"string",
+        "description": "The command id"
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.configs.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.configs.Commands.json b/solr/core/src/resources/apispec/cluster.configs.Commands.json
new file mode 100644
index 0000000..d026cd5
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.configs.Commands.json
@@ -0,0 +1,34 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/ConfigSets+API",
+  "description": "Create ConfigSets.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/configs"]
+  },
+  "commands": {
+    "create": {
+      "type" :"object",
+      "description": "Create a ConfigSet, based on another ConfigSet already in ZooKeeper.",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/ConfigSets+API#ConfigSetsAPI-create",
+      "properties": {
+        "name" :{
+          "type" :"string",
+          "description" : "The name of the ConfigSet to be created."
+        },
+        "baseConfigSet":{
+          "type" : "string",
+          "description" :"The existing ConfigSet to copy as the basis for the new one."
+        },
+        "properties" : {
+          "type":"object",
+          "description": "Additional key-value pairs, in the form of 'ConfigSetProp.<key>=<value>', as needed. These properties will override the same properties in the base ConfigSet.",
+          "additionalProperties" : true
+        }
+      },
+      "required" : ["name", "baseConfigSet"]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.configs.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.configs.delete.json b/solr/core/src/resources/apispec/cluster.configs.delete.json
new file mode 100644
index 0000000..236d457
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.configs.delete.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/ConfigSets+API#ConfigSetsAPI-delete",
+  "description": "Delete ConfigSets. The name of the ConfigSet to delete must be provided as a path parameter.",
+  "methods": [
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/configs/{name}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.configs.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.configs.json b/solr/core/src/resources/apispec/cluster.configs.json
new file mode 100644
index 0000000..9a1443a
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.configs.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/ConfigSets+API#ConfigSetsAPI-list",
+  "description": "List all ConfigSets in the cluster.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/configs"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.json b/solr/core/src/resources/apispec/cluster.json
new file mode 100644
index 0000000..0ec5b96
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.json
@@ -0,0 +1,14 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "General information about the cluster, including defined collections (with the 'cluster' endpoint), status of the overseer (with the 'cluster/overseer' endpoint), and available nodes (with the 'cluster/nodes' endpoint).",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster",
+      "/cluster/overseer",
+      "/cluster/nodes"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.nodes.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.nodes.json b/solr/core/src/resources/apispec/cluster.nodes.json
new file mode 100644
index 0000000..f992f7f
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.nodes.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "Provides general information about the available nodes of the cluster.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/nodes"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.BasicAuth.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.BasicAuth.Commands.json b/solr/core/src/resources/apispec/cluster.security.BasicAuth.Commands.json
new file mode 100644
index 0000000..da04c85
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.BasicAuth.Commands.json
@@ -0,0 +1,23 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Basic+Authentication+Plugin",
+  "description": "Modifies the configuration of Basic authentication, allowing you to add or remove users and their passwords.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authentication"
+    ]
+  },
+  "commands": {
+    "set-user": {
+      "type":"object",
+      "description": "The set-user command allows you to add users and change their passwords. Usernames and passwords are expressed as key-value pairs in a JSON object.",
+      "additionalProperties": true
+    },
+    "delete-user": {
+      "description": "Delete a user or a list of users. Passwords do not need to be provided, simply list the users in a JSON array, separated by colons.",
+      "type":"string"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.RuleBasedAuthorization.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.RuleBasedAuthorization.json b/solr/core/src/resources/apispec/cluster.security.RuleBasedAuthorization.json
new file mode 100644
index 0000000..eb9a11c
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.RuleBasedAuthorization.json
@@ -0,0 +1,129 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Rule-Based+Authorization+Plugin",
+  "description": "Defines roles for accessing Solr, and assigns users to those roles. Use this API to change user authorizations to each of Solr's components.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authorization"
+    ]
+  },
+  "commands": {
+    "set-permission": {
+      "type":"object",
+      "description": "Create a new permission, overwrite an existing permission definition, or assign a pre-defined permission to a role.",
+      "properties": {
+        "name":{
+          "type":"string",
+          "description": "The name of the permission. The name will be used to update or delete the permission later."
+        },
+        "method":{
+          "type":"string",
+          "enum":["GET", "POST", "DELETE","PUT"],
+          "description": "HTTP methods that are allowed for this permission. You could allow only GET requests, or have a role that allows PUT and POST requests. The method values that are allowed for this property are GET, POST, PUT, DELETE and HEAD."
+        },
+
+        "collection":{
+          "type":"array",
+          "items": {
+            "type": "string"
+          },
+          "description":"The collection or collections the permission will apply to. When the path that will be allowed is collection-specific, such as when setting permissions to allow use of the Schema API, omitting the collection property will allow the defined path and/or method for all collections. However, when the path is one that is non-collection-specific, such as the Collections API, the collection value must be null. In this case, two permissions may need to be created; one for collection-specific API paths allowing access to all collections, and another for non-collection-specific paths defining no collection limitations."
+        },
+
+        "path":{
+          "type":"array",
+          "items": {
+            "type": "string"
+          },
+          "description":"A request handler name, such as /update or /select. A wild card is supported, to allow for all paths as appropriate (such as, /update/*)."
+        },
+        "index": {
+          "type": "integer",
+          "description": "The index of the permission you wish to overwrite. Skip this if it is a new permission that should be created."
+        },
+        "before":{
+          "type": "integer",
+          "description":"This property allows ordering of permissions. The value for this property is the name of the permission that this new permission should be placed before in security.json."
+        },
+        "params":{
+          "type":"object",
+          "additionalProperties":true,
+          "description": "The names and values of request parameters. This property can be omitted if all request parameters are allowed, but will restrict access only to the values provided if defined."
+        },
+        "role": {
+          "type": "array",
+          "items": {
+            "type": "string",
+            "description": "The name of the role(s) to give this permission. This name will be used to map user IDs to the role to grant these permissions. The value can be wildcard such as (*), which means that any user is OK, but no user is NOT OK."
+          }
+        }
+      },
+      "required": [
+        "role"
+      ]
+    },
+    "update-permission": {
+      "type":"object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the permission. The name will be used to update or delete the permission later."
+        },
+        "method": {
+          "type": "string",
+          "description": "HTTP methods that are allowed for this permission. You could allow only GET requests, or have a role that allows PUT and POST requests. The method values that are allowed for this property are GET, POST, PUT, DELETE and HEAD."
+        },
+        "collection": {
+          "type":"array",
+          "items": {
+            "type": "string"
+          },
+          "description": "The collection or collections the permission will apply to. When the path that will be allowed is collection-specific, such as when setting permissions to allow use of the Schema API, omitting the collection property will allow the defined path and/or method for all collections. However, when the path is one that is non-collection-specific, such as the Collections API, the collection value must be null. In this case, two permissions may need to be created; one for collection-specific API paths allowing access to all collections, and another for non-collection-specific paths defining no collection limitations."
+        },
+        "path": {
+          "type":"array",
+          "items": {
+            "type": "string"
+          },
+          "description": "A request handler name, such as /update or /select. A wild card is supported, to allow for all paths as appropriate (such as, /update/*)."
+        },
+        "index": {
+          "type": "integer",
+          "description": "The index of the permission you wish to overwrite."
+        },
+        "before": {
+          "type": "integer",
+          "description": "This property allows ordering of permissions. The value for this property is the index of the permission that this new permission should be placed before in security.json."
+        },
+        "role": {
+          "type": "array",
+          "items": {
+            "type": "string",
+            "description": "The name of the role(s) to give this permission. This name will be used to map user IDs to the role to grant these permissions. The value can be wildcard such as (*), which means that any user is OK, but no user is NOT OK."
+          }
+        },
+        "params": {
+          "type": "object",
+          "additionalProperties": true,
+          "description": "The names and values of request parameters. This property can be omitted if all request parameters are allowed, but will restrict access only to the values provided if defined."
+        }
+      },
+      "required": [
+        "role",
+        "index"
+      ]
+    },
+    "delete-permission":{
+      "description":"delete a permission by its index",
+      "type":"integer"
+    },
+    "set-user-role": {
+      "type":"object",
+      "description": "A single command allows roles to be mapped to users. To remove a user's permission, you should set the role to null. The key is always a user id and the value is one or more role names.",
+      "additionalProperties":true
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.authentication.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.authentication.Commands.json b/solr/core/src/resources/apispec/cluster.security.authentication.Commands.json
new file mode 100644
index 0000000..e1f9030
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authentication.Commands.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Securing+Solr",
+  "description":"This is a placeholder output when no authentication is configured",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authentication"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.authentication.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.authentication.json b/solr/core/src/resources/apispec/cluster.security.authentication.json
new file mode 100644
index 0000000..48757c3
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authentication.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Authentication+and+Authorization+Plugins",
+  "description": "Shows the configuration for authentication, including users, classes (type of authentication) and other parameters.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authentication"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.authorization.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.authorization.Commands.json b/solr/core/src/resources/apispec/cluster.security.authorization.Commands.json
new file mode 100644
index 0000000..fe74065
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authorization.Commands.json
@@ -0,0 +1,13 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Securing+Solr",
+  "description":"This is a placeholder output when no authorization is configured",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authorization"
+    ]
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.security.authorization.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.authorization.json b/solr/core/src/resources/apispec/cluster.security.authorization.json
new file mode 100644
index 0000000..da09f8a
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authorization.json
@@ -0,0 +1,13 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Authentication+and+Authorization+Plugins",
+  "description":"Shows the configuration for authorization, including the classes (type of authorization), permissions, user-roles, and other parameters.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authorization"
+    ]
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.Commands.json b/solr/core/src/resources/apispec/collections.Commands.json
new file mode 100644
index 0000000..60ddd4e
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.Commands.json
@@ -0,0 +1,206 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
+  "description": "Create collections and collection aliases, backup or restore collections, and delete collections and aliases.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/collections",
+      "/c"
+    ]
+  },
+  "commands": {
+    "create": {
+      "type": "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
+      "description": "Create a collection.",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the collection to be created."
+        },
+        "config": {
+          "type": "string",
+          "description": "The name of the configuration set (which must already be stored in ZooKeeper) to use for this collection. If not provided, Solr will default to the collection name as the configuration set name."
+        },
+        "router": {
+          "type": "object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Shards+and+Indexing+Data+in+SolrCloud",
+          "description": "These properties define how to distribute documents across a collection's shards.",
+          "properties": {
+            "name": {
+              "type": "string",
+              "enum":["implicit","compositeId"],
+              "description": "The router implementation to use for this collection. There are two options: compositeId or implicit. The compositeId option has Solr decide how to distribute documents (with some possibilities for customization). The implicit option requires you define your own routing strategy, and puts the balancing of documents in shards entirely in your hands.",
+              "default": "compositeId"
+            },
+            "field": {
+              "type": "string",
+              "description": "A field to be used by Solr to identify the shard a document should be routed to. By default, the field defined as the unique ID for each document is used, but an alternative field can be defined with this parameter."
+            }
+          }
+        },
+        "numShards": {
+          "type": "number",
+          "description": "The number of shards to be created as part of the collection. Shards are logical partitions of a single collection. Each shard has at least one replica, but more replicas for each shard can be defined with the replicationFactor property. This is a required parameter when using the 'compositeId' router."
+        },
+        "shards": {
+          "type": "string",
+          "description": "A comma-separated list of shard names, e.g., shard-x,shard-y,shard-z. This is a required parameter when using the 'implicit' router."
+        },
+        "replicationFactor": {
+          "type": "number",
+          "description": "The number of replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard."
+        },
+        "nodeSet": {
+          "type": "array",
+          "items": {
+            "type": "string"
+          },
+          "description": "Defines nodes to spread the new collection across. If not provided, the collection will be spread across all live Solr nodes. The names to use are the 'node_name', which can be found by a request to the cluster/nodes endpoint. A special value of EMPTY will create no shards or replicas for the new collection. In this case, shards and replicas can be added later with the add-replica command available on the /collections/{collection}/shards endpoint."
+        },
+        "shuffleNodes": {
+          "type": "boolean",
+          "description": "Controls whether or not the shard-replicas created for this collection will be assigned to the nodes specified by the nodeSet property in a sequential manner, or if the list of nodes should be shuffled prior to creating individual replicas. A 'false' value makes the results of a collection creation predictable and gives more exact control over the location of the individual shard-replicas, but 'true' can be a better choice for ensuring replicas are distributed evenly across nodes. This property is ignored if nodeSet is not also specified."
+        },
+        "maxShardsPerNode": {
+          "type": "integer",
+          "description": "When creating collections, the shards and/or replicas are spread across all available, live, nodes, and two replicas of the same shard will never be on the same node. If a node is not live when the collection is created, it will not get any parts of the new collection, which could lead to too many replicas being created on a single live node. Defining maxShardsPerNode sets a limit on the number of replicas can be spread to each node. If the entire collection can not be fit into the live nodes, no collection will be created at all."
+        },
+        "autoAddReplicas": {
+          "type": "boolean",
+          "description": "When set to true, enables auto addition of replicas on shared file systems (such as HDFS). See https://cwiki.apache.org/confluence/display/solr/Running+Solr+on+HDFS for more details on settings and overrides.",
+          "default": "false"
+        },
+        "rule": {
+          "type": "array",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Rule-based+Replica+Placement",
+          "description": "Defines rules for where replicas should be located in a cluster.",
+          "items": {
+            "type": "string"
+          }
+        },
+        "snitch": {
+          "type": "array",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Rule-based+Replica+Placement",
+          "description": "",
+          "items": {
+            "type": "string"
+          }
+        },
+        "properties": {
+          "type": "object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Defining+core.properties",
+          "description": "Allows adding core.properties for the collection. Some examples of core properties you may want to modify include the config set, the node name, the data directory, among others.",
+          "additionalProperties": true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "name"
+      ]
+    },
+    "create-alias": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api4",
+      "description": "Allows one or more collections to be known by another name. If this command is used on an existing alias, the existing alias will be replaced with the new collection details.",
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The alias name to be created."
+        },
+        "collections": {
+          "type": "array",
+          "description": "The list of collections to be known as this alias.",
+          "items": {
+            "type": "string"
+          }
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "name",
+        "collections"
+      ]
+    },
+    "delete-alias": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api5",
+      "description": "Deletes a collection alias",
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the alias to delete."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required":["name"]
+    },
+    "backup-collection": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-Backup",
+      "description": "Backup Solr indexes and configurations for a specific collection. One copy of the indexes will be taken from each shard, and the config set for the collection will also be copied.",
+      "type": "object",
+      "properties": {
+        "collection": {
+          "type": "string",
+          "description": "The name of the collection to back up."
+        },
+        "name": {
+          "type": "string",
+          "description": "The name of the backup."
+        },
+        "location": {
+          "type": "string",
+          "description": "A location on a shared drive for the backup-collection command to write to. Alternately, it can be set as a cluster property with the cluster endpoint, which also supports setting a location."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "collection",
+        "name",
+        "location"
+      ]
+    },
+    "restore-collection": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-Restore",
+      "description": "Restore Solr indexes and configurations from a backup. You cannot restore into the same collection you took the backup from. The target collection must not exist before calling this command, as it will be created by the restore action. The new collection will have the same number of shards and replicas as the original collection, and all routing strategies will be retained.",
+      "type": "object",
+      "properties": {
+        "collection": {
+          "type": "string",
+          "description": "The name of the collection the backup will be restored to. This collection must not exist prior to this "
+        },
+        "name": {
+          "type": "string",
+          "description": "The name of the backup file."
+        },
+        "location": {
+          "type": "string",
+          "description": "The location on the shared drive for the restore-collection command to read from. Alternately, it can be set as a cluster property with the cluster endpoint, which also supports setting a location."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "collection",
+        "name",
+        "location"
+      ]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.Commands.json b/solr/core/src/resources/apispec/collections.collection.Commands.json
new file mode 100644
index 0000000..eb3b1da
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.json
@@ -0,0 +1,137 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "Several collection-level operations are supported with this endpoint: modify collection attributes; reload a collection; migrate documents to a different collection; rebalance collection leaders; balance properties across shards; and add or delete a replica property.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}",
+      "/c/{collection}"
+    ]
+  },
+  "commands": {
+    "modify": {
+      "#include": "collections.collection.Commands.modify"
+    },
+    "reload": {
+      "#include": "collections.collection.Commands.reload"
+    },
+    "migrate-docs":{
+      "type":"object",
+      "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api12",
+      "description": "Moves documents with a given routing key to another collection. The source collection will continue to have the same documents, but will start re-routing write requests to the new collection. This command only works on collections using the 'compositeId' type of document routing.",
+      "properties":{
+        "target":{
+          "type":"string",
+          "description":"The name of the collection to which documents will be migrated."
+        },
+        "splitKey":{
+          "type":"string",
+          "description":"The routing key prefix. For example, if uniqueKey is a!123, then you would use split.key=a! This key may span multiple shards on source and target collections. The migration will be completed shard-by-shard in a single thread."
+        },
+        "forwardTimeout":{
+          "type":"integer",
+          "description":"The timeout, in seconds, until which write requests made to the source collection for the given splitKey will be forwarded to the target shard. Once this time is up, write requests will be routed to the target collection. Any applications sending read or write requests should be modified once the migration is complete to send documents to the right collection.",
+          "default": "60"
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined. This command can be long-running, so running it asynchronously is recommended."
+        }
+      },
+      "required":["target", "splitKey"]
+    },
+    "balance-shard-unique":{
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-BalanceSliceUnique",
+      "description": "Insures a property is distributed equally across all physical nodes of a collection. If the property already exists on a replica, effort is made to leave it there. However, if it does not exist on any repica, a shard will be chosen and the property added.",
+      "properties":{
+        "property":{
+          "type":"string",
+          "description": "The property to balance across nodes. This can be entered as 'property.<property>' or simply '<property>'. If the 'property.' prefix is not defined, it will be added automatically."
+       },
+        "onlyactivenodes":{
+          "type":"boolean",
+          "description": "Normally, a property is instantiated on active nodes only. If this parameter is specified as 'false', then inactive nodes are also included for distribution.",
+          "default": "true"
+        },
+        "shardUnique":{
+          "type":"boolean",
+          "description": "There is one pre-defined property (preferredLeader) that defaults this value to 'true'. For all other properties that are balanced, this must be set to 'true' or an error message is returned."
+        }
+      },
+      "required":["property"]
+    },
+    "rebalance-leaders" :{
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-RebalanceLeaders",
+      "description": "Reassign leaders in a collection according to the preferredLeader property across active nodes. This command should be run after the preferredLeader property has been set with the balance-shards or add-replica-property commands.",
+      "properties":{
+        "maxAtOnce":{
+          "type":"number",
+          "description":"The maximum number of reassignments to have in the queue at one time. Values <=0 use the default value Integer.MAX_VALUE. When this number is reached, the process waits for one or more leaders to be successfully assigned before adding more to the queue."
+        },
+        "maxWaitSeconds":{
+          "type":"number",
+          "description":"Timeout, in seconds, when waiting for leaders to be reassigned. If maxAtOnce is less than the number of reassignments pending, this is the maximum interval for any single reassignment.",
+          "default": "60"
+        }
+      }
+    },
+    "add-replica-property": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-AddReplicaProp",
+      "description": "Assign an arbitrary property to a particular replica and give it the value specified. If the property already exists, it will be overwritten with the new value.",
+      "type": "object",
+      "properties": {
+        "shard": {
+          "type": "string",
+          "description": "The name of the shard the replica belongs to."
+        },
+        "replica": {
+          "type": "string",
+          "description": "The name of the replica."
+        },
+        "name": {
+          "type": "string",
+          "description": "The name of the property. This can be entered as 'property.<property>' or simply '<property>'. If the 'property.' prefix is not defined, it will be added automatically."
+        },
+        "value": {
+          "type": "string",
+          "description": "The value to assign to the property."
+        },
+        "shardUnique": {
+          "type": "boolean",
+          "description": "If true, setting this property in one replica will remove the property from all other replicas in that shard.",
+          "default": "false"
+        }
+      },
+      "required": [
+        "name",
+        "value",
+        "shard",
+        "replica"
+      ]
+    },
+    "delete-replica-property": {
+      "description": "Deletes an arbitrary property from a particular replica",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-DeleteReplicaProp",
+      "type": "object",
+      "properties": {
+        "shard": {
+          "type": "string",
+          "description": "The name of the shard the replica belongs to."
+        },
+        "replica": {
+          "type": "string",
+          "description": "The name of the replica."
+        },
+        "property": {
+          "type": "string",
+          "description": "The name of the property to remove."
+        }
+      },
+      "required":["shard","replica","property"]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.Commands.modify.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.Commands.modify.json b/solr/core/src/resources/apispec/collections.collection.Commands.modify.json
new file mode 100644
index 0000000..0255329
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.modify.json
@@ -0,0 +1,36 @@
+{
+  "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-modifycoll",
+  "description":"Modifies specific attributes of a collection. Multiple attributes can be changed at one time.",
+  "type": "object",
+  "properties":{
+    "rule": {
+      "type": "array",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Rule-based+Replica+Placement",
+      "description": "Modifies the rules for where replicas should be located in a cluster.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "snitch": {
+      "type": "array",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Rule-based+Replica+Placement",
+      "description": "Details of the snitch provider",
+      "items": {
+        "type": "string"
+      }
+    },
+    "autoAddReplicas": {
+      "type": "boolean",
+      "description": "When set to true, enables auto addition of replicas on shared file systems (such as HDFS). See https://cwiki.apache.org/confluence/display/solr/Running+Solr+on+HDFS for more details on settings and overrides."
+    },
+    "replicationFactor": {
+      "type": "string",
+      "description": "The number of replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard. Note that changing this value on an existing collection does not automatically add more replicas to the collection. However, it will allow add-replica commands to succeed."
+    },
+    "maxShardsPerNode": {
+      "type": "integer",
+      "description": "When creating collections, the shards and/or replicas are spread across all available, live, nodes, and two replicas of the same shard will never be on the same node. If a node is not live when the collection is created, it will not get any parts of the new collection, which could lead to too many replicas being created on a single live node. Defining maxShardsPerNode sets a limit on the number of replicas can be spread to each node. If the entire collection can not be fit into the live nodes, no collection will be created at all."
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.Commands.reload.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.Commands.reload.json b/solr/core/src/resources/apispec/collections.collection.Commands.reload.json
new file mode 100644
index 0000000..fe7e379
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.reload.json
@@ -0,0 +1,11 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api2",
+  "description": "Reloads a collection so new configuration changes can take effect and be available for use by the system.",
+  "type" : "object",
+  "properties":{
+    "async": {
+      "type": "string",
+      "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.delete.json b/solr/core/src/resources/apispec/collections.collection.delete.json
new file mode 100644
index 0000000..0ab4562
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.delete.json
@@ -0,0 +1,13 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api6",
+  "description": "Deletes a collection.",
+  "methods": [
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}",
+      "/c/{collection}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.json b/solr/core/src/resources/apispec/collections.collection.json
new file mode 100644
index 0000000..34008be
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.json
@@ -0,0 +1,19 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
+  "description": "Lists all collections, with details on shards and replicas in each collection.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}",
+      "/c/{collection}",
+      "/collections/{collection}/shards",
+      "/c/{collection}/shards",
+      "/collections/{collection}/shards/{shard}",
+      "/c/{collection}/shards/{shard}",
+      "/collections/{collection}/shards/{shard}/{replica}",
+      "/c/{collection}/shards/{shard}/{replica}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.Commands.json b/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
new file mode 100644
index 0000000..c3bf7bf
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
@@ -0,0 +1,109 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "Allows you to create a shard, split an existing shard or add a new replica.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}/shards",
+      "/c/{collection}/shards"
+    ]
+  },
+  "commands": {
+    "split": {
+      "type" : "object",
+      "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api3",
+      "description": "Splits an existing shard into two or more new shards. During this action, the existing shard will continue to contain the original data, but new data will be routed to the new shards once the split is complete. New shards will have as many replicas as the existing shards. A soft commit will be done automatically. An explicit commit request is not required because the index is automatically saved to disk during the split operation. New shards will use the original shard name as the basis for their names, adding an underscore and a number to differentiate the new shard. For example, 'shard1' would become 'shard1_0' and 'shard1_1'. Note that this operation can take a long time to complete.",
+      "properties": {
+        "shard":{
+          "type":"string",
+          "description":"The name of the shard to be split."
+        },
+        "ranges" : {
+          "description" : "A comma-separated list of hexadecimal hash ranges that will be used to split the shard into new shards containing each defined range, e.g. ranges=0-1f4,1f5-3e8,3e9-5dc. This is the only option that allows splitting a single shard into more than 2 additional shards. If neither this parameter nor splitKey are defined, the shard will be split into two equal new shards.",
+          "type":"string"
+        },
+        "splitKey":{
+          "description" : "A route key to use for splitting the index. If this is defined, the shard parameter is not required because the route key will identify the correct shard. A route key that spans more than a single shard is not supported. If neither this parameter nor ranges are defined, the shard will be split into two equal new shards.",
+          "type":"string"
+        },
+        "coreProperties":{
+          "type":"object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Defining+core.properties",
+          "description": "Allows adding core.properties for the collection. Some examples of core properties you may want to modify include the config set, the node name, the data directory, among others.",
+          "additionalProperties":true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined. This command can be long-running, so running it asynchronously is recommended."
+        }
+      }
+    },
+    "create": {
+      "type":"object",
+      "properties": {
+        "nodeSet": {
+          "description": "Defines nodes to spread the new collection across. If not provided, the collection will be spread across all live Solr nodes. The names to use are the 'node_name', which can be found by a request to the cluster/nodes endpoint.",
+          "type": "array",
+          "items": {
+            "type": "string"
+          }
+        },
+        "shard": {
+          "description": "The name of the shard to be created.",
+          "type": "string"
+        },
+        "coreProperties": {
+          "type": "object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Defining+core.properties",
+          "description": "Allows adding core.properties for the collection. Some examples of core properties you may want to modify include the config set, the node name, the data directory, among others.",
+          "additionalProperties": true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      },
+      "required":["shard"]
+    },
+    "add-replica": {
+      "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api_addreplica",
+      "description": "",
+      "type" : "object",
+      "properties": {
+        "shard": {
+          "type": "string",
+          "description": "The name of the shard in which this replica should be created. If this parameter is not specified, then '_route_' must be defined."
+        },
+        "_route_": {
+          "type": "string",
+          "description": "If the exact shard name is not known, users may pass the _route_ value and the system would identify the name of the shard. Ignored if the shard param is also specified. If the 'shard' parameter is also defined, this parameter will be ignored."
+        },
+        "node": {
+          "type": "string",
+          "description": "The name of the node where the replica should be created."
+        },
+        "instanceDir": {
+          "type": "string",
+          "description": "An optional custom instanceDir for this replica."
+        },
+        "dataDir": {
+          "type": "string",
+          "description": "An optional custom directory used to store index data for this replica."
+        },
+        "coreProperties": {
+          "type": "object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Defining+core.properties",
+          "description": "Allows adding core.properties for the collection. Some examples of core properties you may want to modify include the config set and the node name, among others.",
+          "additionalProperties": true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      },
+      "required":["shard"]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.shards.shard.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.shard.Commands.json b/solr/core/src/resources/apispec/collections.collection.shards.shard.Commands.json
new file mode 100644
index 0000000..83f7ddf
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.Commands.json
@@ -0,0 +1,24 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "Commands to force leader election and synchronize shards.",
+  "methods": [
+    "POST",
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}/shards/{shard}",
+      "/c/{collection}/shards/{shard}"
+    ]
+  },
+  "commands": {
+    "force-leader": {
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-ForceLeader",
+      "description": "In the unlikely event of a shard losing its leader, this command can be invoked to force the election of a new leader",
+      "type": "object"
+    },
+    "sync-shard": {
+      "type": "object"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.shards.shard.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.shard.delete.json b/solr/core/src/resources/apispec/collections.collection.shards.shard.delete.json
new file mode 100644
index 0000000..53c7965
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.delete.json
@@ -0,0 +1,27 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api7",
+  "description": "Deletes a shard by unloading all replicas of the shard, removing it from clusterstate.json, and by default deleting the instanceDir and dataDir. Only inactive shards or those which have no range for custom sharding will be deleted.",
+  "methods": [
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}/shards/{shard}",
+      "/c/{collection}/shards/{shard}"
+    ],
+    "params":{
+      "deleteInstanceDir":{
+        "type": "boolean",
+        "description":"By default Solr will delete the entire instanceDir of each replica that is deleted. Set this to false to prevent the instance directory from being deleted."
+      },
+      "deleteDataDir":{
+        "type":"boolean",
+        "description":"y default Solr will delete the dataDir of each replica that is deleted. Set this to false to prevent the data directory from being deleted."
+      },
+      "async": {
+        "type": "string",
+        "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined. This command can be long-running, so running it asynchronously is recommended."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.delete.json b/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.delete.json
new file mode 100644
index 0000000..a0c8ee6
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.delete.json
@@ -0,0 +1,39 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api9",
+  "description": "Deletes a replica. If the responding node is up, the core is unloaded, the entry removed from clusterstate.json, and the instanceDir and dataDir removed. If the node is not up, the entry for the replica is removed from clusterstate.json; if the nodes comes up later, the replica is automatically de-registered.",
+  "methods": [
+    "DELETE"
+  ],
+  "url": {
+    "paths": [
+      "/collections/{collection}/shards/{shard}/{replica}",
+      "/c/{collection}/shards/{shard}/{replica}"
+    ],
+    "params": {
+      "onlyIfDown": {
+        "type": "boolean",
+        "default": "false",
+        "description": "When set to 'true', no action will be taken if the replica is active."
+      },
+      "deleteIndex": {
+        "type": "boolean",
+        "default": "true",
+        "description": "By default Solr will delete the index of the replica that is deleted. Set this to false to prevent the index directory from being deleted."
+      },
+      "deleteDataDir": {
+        "type": "boolean",
+        "default": "true",
+        "description": "By default Solr will delete the dataDir of the replica that is deleted. Set this to false to prevent the data directory from being deleted."
+      },
+      "deleteInstanceDir": {
+        "type": "boolean",
+        "default": "true",
+        "description": "By default Solr will delete the entire instanceDir of the replica that is deleted. Set this to false to prevent the instance directory from being deleted."
+      },
+      "async":{
+        "type":"string",
+        "description":"Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/collections.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.json b/solr/core/src/resources/apispec/collections.json
new file mode 100644
index 0000000..49ca976
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.json
@@ -0,0 +1,13 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
+  "description": "List all available collections and their properties.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/collections",
+      "/c"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.RealtimeGet.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.RealtimeGet.json b/solr/core/src/resources/apispec/core.RealtimeGet.json
new file mode 100644
index 0000000..308870e
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.RealtimeGet.json
@@ -0,0 +1,26 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/RealTime+Get",
+  "description": "RealTime Get allows retrieving documents by ID before the documents have been committed to the index. It is useful when you need access to documents as soon as they are indexed but your commit times are high for other reasons.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/get"
+    ],
+    "params": {
+      "id": {
+        "type": "string",
+        "description": "A single document ID to retrieve."
+      },
+      "ids": {
+        "type": "string",
+        "description": "One or more document IDs to retrieve. Separate by commas if more than one ID is specified."
+      },
+      "fq":{
+        "type": "string",
+        "description": "An optional filter query to add to the query. One use case for this is security filtering, in case users or groups should not be able to retrieve the document ID requested."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.addCopyField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.addCopyField.json b/solr/core/src/resources/apispec/core.SchemaEdit.addCopyField.json
new file mode 100644
index 0000000..26c4eff
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addCopyField.json
@@ -0,0 +1,27 @@
+{
+  "documentation" : "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-AddaNewCopyFieldRule",
+  "description": "Adds a new copy field rule, to allow one field to be populated with the contents of one or more other fields.",
+  "type": "object",
+  "properties": {
+    "source": {
+      "type": "string",
+      "description": "The field to copy from."
+    },
+    "dest": {
+      "type":"array",
+      "items": {
+        "type": "string"
+      },
+      "description": "A field or an array of fields to which the source field will be copied. A wildcard for a dynamic field can be used, but only if the source field also contains a dynamic field."
+    },
+    "maxChars": {
+      "type": "integer",
+      "description": "An upper limit for the number of characters to be copied. This would be useful if index size is a concern. If a limit is not specified, the entire field will be copied."
+    }
+  },
+  "required": [
+    "source",
+    "dest"
+  ]
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.addField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.addField.json b/solr/core/src/resources/apispec/core.SchemaEdit.addField.json
new file mode 100644
index 0000000..d4a6996
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addField.json
@@ -0,0 +1,98 @@
+{
+  "documentation" :"https://cwiki.apache.org/confluence/display/solr/Schema+API",
+  "type":"object",
+  "properties":{
+    "name": {
+      "type": "string",
+      "description": "The name of the field. Names should be alphanumeric or underscore characters only, and not start with a digit. Names also cannot begin and end with an underscore, as such field names are reserved by the system."
+    },
+    "type": {
+      "type": "string",
+      "description":"The name of the fieldType for this field."
+    },
+    "defaultValue": {
+      "type": "string",
+      "description": "An optional default value that should be added automatically to any document that does not have a value for this field."
+    },
+    "indexed": {
+      "type": "boolean",
+      "description": "If true, the field will be indexed and will be available for use in queries to retrieve matching documents. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true.",
+      "default": "true"
+    },
+    "stored": {
+      "type": "boolean",
+      "description": "If true, the actual value of the field can be retrieved by queries and be displayed in results. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true.",
+      "default":"true"
+    },
+    "omitNorms": {
+      "type": "boolean",
+      "description": "If true, length normalization and index-time boosting for a field are omitted from the index. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true for all primitive field types (such as dates, strings, boolean, and numeric fields), but will default to false for non-primitive field types."
+    },
+    "omitTermFreqAndPositions": {
+      "type": "boolean",
+      "description": "If true, all term frequency, positions, and payloads will not be indexed. This means that phrase queries, proximity queries and similar queries that rely on analysis of the frequency of a query term or the position of a term to other terms will not be supported. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true for all field types that are not text fields."
+    },
+    "termVectors": {
+      "type": "boolean",
+      "description": "If true, term vectors will be stored to be able to compute similarity between two documents. This is required to use More Like This. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. Do not enable this if using the PostingsHighlighter.",
+      "default": "false"
+    },
+    "termPositions": {
+      "type": "boolean",
+      "description": "If true, term positions will be stored for use with highlighting. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. Do not enable this if using the PostingsHighlighter.",
+      "default": "false"
+    },
+    "termOffsets": {
+      "type": "boolean",
+      "description": "If true, term offsets will be stored for use with highlighting. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. Do not enable this if using the PostingsHighlighter.",
+      "default": "false"
+    },
+    "multiValued": {
+      "type": "boolean",
+      "description": "If true, a single document can have multiple values in a single field, and these values will be indexed. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false.",
+      "default": "false"
+    },
+    "sortMissingFirst": {
+      "type": "boolean",
+      "description": "If true, when sorting by the field, any documents missing a value for the field will be placed at the top of the list. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. If sortMissingFirst and sortMissingLast are both false, documents missing this field will be placed at the top when sorting in ascending order (asc) or at the bottom when sorting in descending order (desc).",
+      "default": "false"
+    },
+    "sortMissingLast": {
+      "type": "boolean",
+      "description": "If true, when sorting by the field, any documents missing a value for the field will be placed at the bottom of the list. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. If sortMissingFirst and sortMissingLast are both false, documents missing this field will be placed at the top when sorting in ascending order (asc) or at the bottom when sorting in descending order (desc).",
+      "default": "false"
+    },
+    "required": {
+      "type": "boolean",
+      "description": "If true, any document that does not have a value for the field will be rejected. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false.",
+      "default": "false"
+    },
+    "omitPositions": {
+      "type": "boolean",
+      "description": "If true, information about the position of terms in a document will not be stored in the index, which means phrase queries, proximity queries, and similar will not be supported for this field. It is similar to 'omitTermFreqAndPositions', but 'omitPositions' will allow term frequency information to be stored. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true for all field types that are not text fields."
+    },
+    "storeOffsetsWithPositions": {
+      "type": "boolean",
+      "description": "If true, term offsets will be stored with positions in the postings list in the index. This is required if using the PostingsHighlighter. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false.",
+      "default": "false"
+    },
+    "docValues": {
+      "type": "boolean",
+      "description": "If true, field values will be stored in a column-oriented docValues structure. This can be more efficient for some fields, particularly those used for faceting. More information is available from https://cwiki.apache.org/confluence/display/solr/DocValues. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true for all non-text fields (such as dates, integers, longs, etc.)."
+    },
+    "termPayloads": {
+      "type": "boolean",
+      "description": "If true, term payloads will be stored for use with highlighting. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to false. Do not enable this if using the PostingsHighlighter.",
+      "default": "false"
+    },
+    "useDocValuesAsStored": {
+      "type": "boolean",
+      "description": "If true and docValues are enabled for the field, the field will be returned when all fields are requested (using '*' with the fl parameter), even if it is not stored. If this is not defined, it will inherit the value from the fieldType. If the fieldType does not define a value, it will default to true.",
+      "default": "true"
+    }
+  },
+  "required": [
+    "name",
+    "type"
+  ]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.analyzers.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.analyzers.json b/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.analyzers.json
new file mode 100644
index 0000000..2974a60
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.analyzers.json
@@ -0,0 +1,51 @@
+{
+  "type": "object",
+  "properties": {
+    "class": {
+      "type": "string"
+    },
+    "charFilters": {
+      "type": "array",
+      "items": {
+        "type": "object",
+        "properties": {
+          "class": {
+            "type": "string"
+          }
+        },
+        "required": [
+          "class"
+        ],
+        "additionalProperties": true
+      }
+    },
+    "tokenizer": {
+      "type": "object",
+      "properties": {
+        "class": {
+          "type": "string"
+        }
+      },
+      "required": [
+        "class"
+      ],
+      "additionalProperties": true
+    },
+    "filters": {
+      "type": "array",
+      "items": {
+        "type": "object",
+        "properties": {
+          "class": {
+            "type": "string"
+          }
+        },
+        "required": [
+          "class"
+        ],
+        "additionalProperties": true
+      }
+    }
+  },
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.json b/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.json
new file mode 100644
index 0000000..e24572e
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.json
@@ -0,0 +1,53 @@
+{
+  "type":"object",
+  "properties": {
+    "name": {
+      "type": "string",
+      "description": "The name of the field type. This name is used when defining a field. It is strongly recommended that field type names consist only of alphanumeric or underscore characters and not start with a digit."
+    },
+    "class": {
+      "type": "string",
+      "description": "The class name to use for the field type. Class names do not need to be fully qualified if they are included with Solr, so instead of 'org.apache.solr.schema.TextField', you can abbreviate the name as 'solr.TextField'. Custom or third-party class names may need to be fully qualified, however."
+    },
+    "positionIncrementGap": {
+      "type": "number",
+      "description": "The distance between the values of a multivalued field. This is used to prevent inaccurate phrase matches across two separate values of the same field.",
+      "default": "0"
+    },
+    "autoGeneratePhraseQueries": {
+      "type": "boolean",
+      "description": "If true, phrase queries will automatically be generated for adjacent terms. If false, terms must also be enclosed in double-quotes to be treated as phrases.",
+      "default": "false"
+    },
+    "docValuesFormat": {
+      "type": "string",
+      "description": "Defines a custom DocValuesFormat to use for fields of this type. A custom DocValuesFormat requires that a schema-aware codec has also been configured in solrconfig.xml."
+    },
+    "postingsFormat": {
+      "type": "string",
+      "description": "Defines a custom PostingsFormat to use for fields of this type. A custom PostingsFormat requires that a schema-aware codec has also been configured in solrconfig.xml."
+    },
+    "queryAnalyzer": {
+      "description": "A query analyzer section defines how incoming queries to Solr will be analyzed for a field of this type.",
+      "#include": "core.SchemaEdit.addFieldType.analyzers"
+    },
+    "indexAnalyzer": {
+      "description": "An index analyzer section defines how incoming text in documents will be analyzed for a field of this type.",
+      "#include": "core.SchemaEdit.addFieldType.analyzers"
+    },
+    "multiTermAnalyzer": {
+      "description": "A multiterm analyzer section defines how incoming queries that results in Multi-Term expansion will be analyzed for a field of this type.",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Analyzers#Analyzers-AnalysisforMulti-TermExpansion",
+      "#include": "core.SchemaEdit.addFieldType.analyzers"
+    },
+    "analyzer": {
+      "description": "An analyzer defines how both incoming text in documents and queries are analyzed for a field of this type. If a query analyzer and an index analyzer have both been defined, a general analyzer does not need to be defined for this type.",
+      "#include": "core.SchemaEdit.addFieldType.analyzers"
+    }
+  },
+  "additionalProperties": true,
+  "required": [
+    "name",
+    "class"
+  ]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.deleteCopyField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.deleteCopyField.json b/solr/core/src/resources/apispec/core.SchemaEdit.deleteCopyField.json
new file mode 100644
index 0000000..dd6ff3a
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteCopyField.json
@@ -0,0 +1,19 @@
+{
+  "type":"object",
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-DeleteaCopyFieldRule",
+  "description": "Deletes a copy field rule. Both sides of the copy rule (source and destination) are required in order to delete the rule.",
+  "properties":{
+    "source": {
+      "type":"string",
+      "description": "The field the copy rule is defined to copy from."
+    },
+    "dest": {
+      "type": "string",
+      "description": "The field the copy rule is defined to copy to."
+    }
+  },
+  "required": [
+    "source",
+    "dest"
+  ]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.deleteDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.deleteDynamicField.json b/solr/core/src/resources/apispec/core.SchemaEdit.deleteDynamicField.json
new file mode 100644
index 0000000..9550548
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteDynamicField.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-DeleteaDynamicFieldRule",
+  "description": "Deletes a dynamic field.",
+  "type":"object",
+  "properties": {
+    "name": {
+      "type": "string",
+      "description": "The name of the dynamic field to delete."
+    }
+  },
+  "required":["name"]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.deleteField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.deleteField.json b/solr/core/src/resources/apispec/core.SchemaEdit.deleteField.json
new file mode 100644
index 0000000..6c2cb00
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteField.json
@@ -0,0 +1,12 @@
+{
+  "documentation" : "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-DeleteaField",
+  "description": "Deletes a field from the schema.",
+  "type":"object",
+  "properties":{
+    "name":{
+     "description" :"The name of the field to delete.",
+      "type" : "string"
+    }
+  },
+  "required" : ["name"]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.deleteFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.deleteFieldType.json b/solr/core/src/resources/apispec/core.SchemaEdit.deleteFieldType.json
new file mode 100644
index 0000000..712462a
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteFieldType.json
@@ -0,0 +1,14 @@
+{
+  "documentation":"https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-DeleteaFieldType",
+  "description": "Deletes a field type from the schema.",
+  "type":"object",
+  "properties": {
+    "name": {
+      "type": "string",
+      "description": "The name of the field type to delete."
+    }
+  },
+  "required": [
+    "name"
+  ]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaEdit.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.json b/solr/core/src/resources/apispec/core.SchemaEdit.json
new file mode 100644
index 0000000..bbf4082
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.json
@@ -0,0 +1,47 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+  "description": "The Schema API provides viewing, editing, adding, and deleting elements of Solr's schema. This API can only be used if Managed Schema is enabled and the schema is defined as mutable. See https://cwiki.apache.org/confluence/display/solr/Schema+Factory+Definition+in+SolrConfig for more information about enabling Managed Schema.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "$handlerName"
+    ]
+  },
+  "commands": {
+    "add-field": {
+      "#include": "core.SchemaEdit.addField"
+    },
+    "delete-field": {
+      "#include": "core.SchemaEdit.deleteField"
+    },
+    "replace-field": {
+      "#include": "core.SchemaEdit.addField"
+    },
+    "add-dynamic-field": {
+      "#include": "core.SchemaEdit.addField"
+    },
+    "delete-dynamic-field": {
+      "#include": "core.SchemaEdit.deleteDynamicField"
+    },
+    "replace-dynamic-field": {
+      "#include": "core.SchemaEdit.addField"
+    },
+    "add-field-type": {
+      "#include": "core.SchemaEdit.addFieldType"
+    },
+    "delete-field-type": {
+      "#include": "core.SchemaEdit.deleteFieldType"
+    },
+    "replace-field-type": {
+      "#include": "core.SchemaEdit.addFieldType"
+    },
+    "add-copy-field": {
+      "#include": "core.SchemaEdit.addCopyField"
+    },
+    "delete-copy-field": {
+      "#include": "core.SchemaEdit.deleteCopyField"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaRead.copyFields.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaRead.copyFields.json b/solr/core/src/resources/apispec/core.SchemaRead.copyFields.json
new file mode 100644
index 0000000..4cf822e
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.copyFields.json
@@ -0,0 +1,26 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-ListCopyFields",
+  "description": "Lists all copy fields.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "schema/copyfields"
+    ],
+    "params": {
+      "wt": {
+         "type": "string",
+         "description": "The format of the response. Valid options are xml or json."
+      },
+      "source.fl": {
+        "type": "string",
+        "description": "Comma- or space-separated list of one or more source fields to include in the response. copyField directives with all other source fields will be excluded from the response. If not specified, all copyFields will be included in the response"
+      },
+      "dest.fl": {
+        "type": "string",
+        "description": "Comma or space-separated list of one or more copyField dest (destination) fields to include in the response. copyField directives with all other dest fields will be excluded. If not specified, all copyFields will be included in the response."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaRead.dynamicFields_fieldTypes.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaRead.dynamicFields_fieldTypes.json b/solr/core/src/resources/apispec/core.SchemaRead.dynamicFields_fieldTypes.json
new file mode 100644
index 0000000..0642491
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.dynamicFields_fieldTypes.json
@@ -0,0 +1,20 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/schema/dynamicfields",
+      "/schema/dynamicfields/{name}",
+      "/schema/fieldtypes",
+      "/schema/fieldtypes/{name}"
+    ],
+    "params":{
+      "showDefaults":{
+        "type":"boolean",
+        "description":"If true, all default field properties from each field's field type will be included in the response (e.g.   tokenized  for   solr.TextField). If false, only explicitly specified field properties will be included."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaRead.fields.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaRead.fields.json b/solr/core/src/resources/apispec/core.SchemaRead.fields.json
new file mode 100644
index 0000000..3b6c787
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.fields.json
@@ -0,0 +1,34 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API#SchemaAPI-ListFields",
+  "description": "Get only the fields defined in the schema.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/schema/fields",
+      "/schema/fields/{name}"
+    ],
+    "params": {
+      "wt": {
+         "type": "string",
+         "description": "The format of the response. Valid options are xml or json.",
+         "default": "json"
+      },
+      "fl": {
+         "type": "string",
+         "description": "A comma- or space-separated list fields to return. If not specified, all fields will be returned. Note a single field can be requested by adding the field name to the endpoint."
+      },
+      "includeDynamic": {
+        "type": "boolean",
+        "description": "If true, dynamic fields will be returned in the response.",
+        "default": false
+      },
+      "showDefaults": {
+        "type": "boolean",
+        "description": "If true, all field properties from each field's field type will be included in the response, even if they are not explicitly defined on the field. If false, only explicitly defined field properties will be included.",
+        "default": false
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.SchemaRead.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaRead.json b/solr/core/src/resources/apispec/core.SchemaRead.json
new file mode 100644
index 0000000..da62c06
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.json
@@ -0,0 +1,18 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/schema",
+      "/schema/name",
+      "/schema/uniquekey",
+      "/schema/version",
+      "/schema/similarity",
+      "/schema/solrqueryparser",
+      "/schema/zkversion",
+      "/schema/solrqueryparser/defaultoperator"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.Update.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.Update.json b/solr/core/src/resources/apispec/core.Update.json
new file mode 100644
index 0000000..f9e80c1
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.Update.json
@@ -0,0 +1,17 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/update",
+      "/update/xml",
+      "/update/csv",
+      "/update/json",
+      "/update/bin",
+      "/update/json/commands"
+    ]
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json b/solr/core/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
new file mode 100644
index 0000000..731c3d8
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
@@ -0,0 +1,25 @@
+{
+  "type": "object",
+  "properties": {
+    "name": {
+      "type": "string",
+      "description": "The name of the request handler. This name will be used to update or remove the request handler later if necessary."
+    },
+    "class": {
+      "type": "string",
+      "description": "The request handler class. Class names do not need to be fully qualified if they are included with Solr, so you can abbreviate the name as 'solr.SearchHandler'. Custom or third-party class names may need to be fully qualified, however."
+    },
+    "runtimeLib": {
+      "type": "boolean",
+      "description": "An optional parameter to use a custom .jar file that has been uploaded to Solr's blobstore. This additionally requires that the .jar has also been registered with the 'add-runtimelib' command, which is one of the available commands for the Config API."
+    },
+    "startup": {
+      "type": "string",
+      "description": "Allows the request handler to only start when requested. The only option is 'lazy'.",
+      "enum": [
+        "lazy"
+      ]
+    }
+  },
+  "additionalProperties": true
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Commands.generic.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Commands.generic.json b/solr/core/src/resources/apispec/core.config.Commands.generic.json
new file mode 100644
index 0000000..9d2b01d
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Commands.generic.json
@@ -0,0 +1,19 @@
+{
+  "type": "object",
+  "properties": {
+    "name": {
+      "type": "string",
+      "description": "The name of this configuration item. This name will be used to update or remove this later if necessary."
+   },
+    "class": {
+      "type": "string",
+      "description": "The configuration item class. Class names do not need to be fully qualified if they are included with Solr, so you can abbreviate the name as 'solr.SearchHandler'. Custom or third-party class names may need to be fully qualified, however."
+   },
+    "runtimeLib": {
+      "type": "boolean",
+      "description": "An optional parameter to use a custom .jar file that has been uploaded to Solr's blobstore. This additionally requires that the .jar has also been registered with the 'add-runtimelib' command, which is one of the available commands for the Config API."
+   }
+  },
+  "required": [ "name", "class"],
+  "additionalProperties": true
+}


[4/5] lucene-solr:master: SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 148d73c..d7759ca 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.solr.api.Api;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
@@ -106,6 +107,7 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.ONLY_IF_DOW
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.REQUESTID;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARD_UNIQUE;
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
 import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
 import static org.apache.solr.common.cloud.DocCollection.RULE;
 import static org.apache.solr.common.cloud.DocCollection.SNITCH;
@@ -135,12 +137,14 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   protected final CoreContainer coreContainer;
+  private final CollectionHandlerApi v2Handler ;
 
   public CollectionsHandler() {
     super();
     // Unlike most request handlers, CoreContainer initialization
     // should happen in the constructor...
     this.coreContainer = null;
+    v2Handler = new CollectionHandlerApi(this);
   }
 
 
@@ -151,6 +155,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
    */
   public CollectionsHandler(final CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
+    v2Handler = new CollectionHandlerApi(this);
   }
 
   @Override
@@ -205,33 +210,39 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       CollectionOperation operation = CollectionOperation.get(action);
       log.info("Invoked Collection Action :{} with params {} and sendToOCPQueue={}", action.toLower(), req.getParamString(), operation.sendToOCPQueue);
 
-      SolrResponse response = null;
-      Map<String, Object> props = operation.execute(req, rsp, this);
-      String asyncId = req.getParams().get(ASYNC);
-      if (props != null) {
-        if (asyncId != null) {
-          props.put(ASYNC, asyncId);
-        }
-        props.put(QUEUE_OPERATION, operation.action.toLower());
-        ZkNodeProps zkProps = new ZkNodeProps(props);
-        if (operation.sendToOCPQueue) {
-          response = handleResponse(operation.action.toLower(), zkProps, rsp, operation.timeOut);
-        }
-        else Overseer.getStateUpdateQueue(coreContainer.getZkController().getZkClient()).offer(Utils.toJSON(props));
-        final String collectionName = zkProps.getStr(NAME);
-        if (action.equals(CollectionAction.CREATE) && asyncId == null) {
-          if (rsp.getException() == null) {
-            waitForActiveCollection(collectionName, zkProps, cores, response);
-          }
-        }
-      }
+      invokeAction(req, rsp, cores, action, operation);
     } else {
       throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param");
     }
     rsp.setHttpCaching(false);
   }
 
-
+  void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp, CoreContainer cores, CollectionAction action, CollectionOperation operation) throws Exception {
+    if (!coreContainer.isZooKeeperAware()) {
+      throw new SolrException(BAD_REQUEST,
+          "Invalid request. collections can be accessed only in SolrCloud mode");
+    }
+    SolrResponse response = null;
+    Map<String, Object> props = operation.execute(req, rsp, this);
+    String asyncId = req.getParams().get(ASYNC);
+    if (props != null) {
+      if (asyncId != null) {
+        props.put(ASYNC, asyncId);
+      }
+      props.put(QUEUE_OPERATION, operation.action.toLower());
+      ZkNodeProps zkProps = new ZkNodeProps(props);
+      if (operation.sendToOCPQueue) {
+        response = handleResponse(operation.action.toLower(), zkProps, rsp, operation.timeOut);
+      }
+      else Overseer.getStateUpdateQueue(coreContainer.getZkController().getZkClient()).offer(Utils.toJSON(props));
+      final String collectionName = zkProps.getStr(NAME);
+      if (action.equals(CollectionAction.CREATE) && asyncId == null) {
+        if (rsp.getException() == null) {
+          waitForActiveCollection(collectionName, zkProps, cores, response);
+        }
+      }
+    }
+  }
 
 
   static final Set<String> KNOWN_ROLES = ImmutableSet.of("overseer");
@@ -387,7 +398,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           COLL_CONF,
           NUM_SLICES,
           MAX_SHARDS_PER_NODE,
-          CREATE_NODE_SET, CREATE_NODE_SET_SHUFFLE,
+          CREATE_NODE_SET,
+          CREATE_NODE_SET_SHUFFLE,
           SHARDS_PROP,
           STATE_FORMAT,
           AUTO_ADD_REPLICAS,
@@ -863,7 +875,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
     }
 
-
     public static CollectionOperation get(CollectionAction action) {
       for (CollectionOperation op : values()) {
         if (op.action == action) return op;
@@ -1058,7 +1069,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
   interface CollectionOp {
     Map<String, Object> execute(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler h) throws Exception;
-    
+
   }
 
   public static final List<String> MODIFIABLE_COLL_PROPS = Arrays.asList(
@@ -1068,4 +1079,14 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       MAX_SHARDS_PER_NODE,
       AUTO_ADD_REPLICAS,
       COLL_CONF);
+
+  @Override
+  public Collection<Api> getApis() {
+    return v2Handler.getApis();
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
index f3a8dd2..5d6f02c 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
@@ -18,11 +18,13 @@ package org.apache.solr.handler.admin;
 
 import java.lang.invoke.MethodHandles;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.api.Api;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.OverseerSolrResponse;
 import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
@@ -61,6 +63,7 @@ public class ConfigSetsHandler extends RequestHandlerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   public static long DEFAULT_ZK_TIMEOUT = 300*1000;
+  private final ConfigSetsHandlerApi configSetsHandlerApi = new ConfigSetsHandlerApi(this);
 
   /**
    * Overloaded ctor to inject CoreContainer into the handler.
@@ -71,10 +74,6 @@ public class ConfigSetsHandler extends RequestHandlerBase {
     this.coreContainer = coreContainer;
   }
 
-  @Override
-  final public void init(NamedList args) {
-
-  }
 
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
@@ -96,16 +95,7 @@ public class ConfigSetsHandler extends RequestHandlerBase {
       ConfigSetAction action = ConfigSetAction.get(a);
       if (action == null)
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
-      ConfigSetOperation operation = ConfigSetOperation.get(action);
-      log.info("Invoked ConfigSet Action :{} with params {} ", action.toLower(), req.getParamString());
-      Map<String, Object> result = operation.call(req, rsp, this);
-      if (result != null) {
-        // We need to differentiate between collection and configsets actions since they currently
-        // use the same underlying queue.
-        result.put(QUEUE_OPERATION, CONFIGSETS_ACTION_PREFIX + operation.action.toLower());
-        ZkNodeProps props = new ZkNodeProps(result);
-        handleResponse(operation.action.toLower(), props, rsp, DEFAULT_ZK_TIMEOUT);
-      }
+      invokeAction(req, rsp, action);
     } else {
       throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param");
     }
@@ -113,6 +103,24 @@ public class ConfigSetsHandler extends RequestHandlerBase {
     rsp.setHttpCaching(false);
   }
 
+  void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetAction action) throws Exception {
+    ConfigSetOperation operation = ConfigSetOperation.get(action);
+    log.info("Invoked ConfigSet Action :{} with params {} ", action.toLower(), req.getParamString());
+    Map<String, Object> result = operation.call(req, rsp, this);
+    sendToZk(rsp, operation, result);
+  }
+
+  protected void sendToZk(SolrQueryResponse rsp, ConfigSetOperation operation, Map<String, Object> result)
+      throws KeeperException, InterruptedException {
+    if (result != null) {
+      // We need to differentiate between collection and configsets actions since they currently
+      // use the same underlying queue.
+      result.put(QUEUE_OPERATION, CONFIGSETS_ACTION_PREFIX + operation.action.toLower());
+      ZkNodeProps props = new ZkNodeProps(result);
+      handleResponse(operation.action.toLower(), props, rsp, DEFAULT_ZK_TIMEOUT);
+    }
+  }
+
   private void handleResponse(String operation, ZkNodeProps m,
       SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
     long time = System.nanoTime();
@@ -160,7 +168,6 @@ public class ConfigSetsHandler extends RequestHandlerBase {
   public String getDescription() {
     return "Manage SolrCloud ConfigSets";
   }
-
   @Override
   public Category getCategory() {
     return Category.ADMIN;
@@ -209,4 +216,14 @@ public class ConfigSetsHandler extends RequestHandlerBase {
       throw new SolrException(ErrorCode.SERVER_ERROR, "No such action" + action);
     }
   }
+
+  @Override
+  public Collection<Api> getApis() {
+    return configSetsHandlerApi.getApis();
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..6037bcd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.solr.client.solrj.SolrRequest;
+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;
+
+public class ConfigSetsHandlerApi extends BaseHandlerApiSupport {
+
+  final ConfigSetsHandler configSetHandler;
+
+  public ConfigSetsHandlerApi(ConfigSetsHandler configSetHandler) {
+    this.configSetHandler = configSetHandler;
+  }
+
+
+  @Override
+  protected List<ApiCommand> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(EndPoint.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);
+    }
+
+    Cmd(EndPoint endPoint, ConfigSetOperation op, SolrRequest.METHOD method, String cmdName) {
+      this.cmdName = cmdName;
+      this.endPoint = endPoint;
+      this.op = op;
+      this.method = method;
+    }
+
+    @Override
+    public String getName() {
+      return cmdName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+    @Override
+    public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
+      ((ConfigSetsHandlerApi) apiHandler).configSetHandler.invokeAction(req, rsp, op.action);
+    }
+
+  }
+  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/71abe130/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index a415d8a..275ec18 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -18,6 +18,7 @@ package org.apache.solr.handler.admin;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -28,6 +29,7 @@ import java.util.concurrent.ExecutorService;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.StringUtils;
+import org.apache.solr.api.Api;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -66,6 +68,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   protected final Map<String, Map<String, TaskObject>> requestStatusMap;
+  private final CoreAdminHandlerApi coreAdminHandlerApi;
 
   protected ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
       new DefaultSolrThreadFactory("parallelCoreAdminExecutor"));
@@ -88,6 +91,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     requestStatusMap = Collections.unmodifiableMap(map);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -103,6 +107,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     requestStatusMap = Collections.unmodifiableMap(map);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -119,6 +124,10 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     parallelExecutor = MetricUtils.instrumentedExecutorService(parallelExecutor, manager.registry(registryName),
         SolrMetricManager.mkName("parallelCoreAdminExecutor", getCategory().name(),scope, "threadPool"));
   }
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 
   /**
    * The instance of CoreContainer this handler handles. This should be the CoreContainer instance that created this
@@ -381,6 +390,11 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
 
   }
 
+  @Override
+  public Collection<Api> getApis() {
+    return coreAdminHandlerApi.getApis();
+  }
+
   static {
     for (CoreAdminOperation op : CoreAdminOperation.values())
       opMap.put(op.action.toString().toLowerCase(Locale.ROOT), op);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..9d256e6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import 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.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;
+import static org.apache.solr.handler.admin.CoreAdminOperation.MERGEINDEXES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.OVERSEEROP_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.PREPRECOVERY_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REJOINLEADERELECTION_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.RELOAD_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.RENAME_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTAPPLYUPDATES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTBUFFERUPDATES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTRECOVERY_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTSTATUS_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTSYNCSHARD_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.SPLIT_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.STATUS_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.SWAP_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.UNLOAD_OP;
+
+public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
+  private final CoreAdminHandler handler;
+
+  public CoreAdminHandlerApi(CoreAdminHandler handler) {
+    this.handler = handler;
+  }
+
+  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;
+      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;
+    }
+
+    @Override
+    public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
+      target.execute(new CoreAdminHandler.CallInfo(((CoreAdminHandlerApi) apiHandler).handler,
+          req,
+          rsp,
+          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;
+    }
+  }
+
+
+  @Override
+  protected List<ApiCommand> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(EndPoint.values());
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
index 8fdac21..c7cd052 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
@@ -16,24 +16,28 @@
  */
 package org.apache.solr.handler.admin;
 
+import java.util.Collection;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.api.ApiBag.ReqHandlerToApi;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.api.Api;
 
+import static java.util.Collections.singletonList;
+import static org.apache.solr.api.ApiBag.getSpec;
 import static org.apache.solr.common.params.CommonParams.PATH;
 
-public class InfoHandler extends RequestHandlerBase {
+public class InfoHandler extends RequestHandlerBase  {
+
   protected final CoreContainer coreContainer;
-  
-  private ThreadDumpHandler threadDumpHandler = new ThreadDumpHandler();
-  private PropertiesRequestHandler propertiesHandler = new PropertiesRequestHandler();
-  private LoggingHandler loggingHandler;
-  private SystemInfoHandler systemInfoHandler;
 
   /**
    * Overloaded ctor to inject CoreContainer into the handler.
@@ -42,9 +46,10 @@ public class InfoHandler extends RequestHandlerBase {
    */
   public InfoHandler(final CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
-    systemInfoHandler = new SystemInfoHandler(coreContainer);
-    loggingHandler = new LoggingHandler(coreContainer);
-    
+    handlers.put("threads", new ThreadDumpHandler());
+    handlers.put("properties", new PropertiesRequestHandler());
+    handlers.put("logging", new LoggingHandler(coreContainer));
+    handlers.put("system", new SystemInfoHandler(coreContainer));
   }
 
 
@@ -73,27 +78,19 @@ public class InfoHandler extends RequestHandlerBase {
     }
 
     String path = (String) req.getContext().get(PATH);
+    handle(req, rsp, path);
+  }
+
+  private void handle(SolrQueryRequest req, SolrQueryResponse rsp, String path) {
     int i = path.lastIndexOf('/');
     String name = path.substring(i + 1, path.length());
-    
-    if (name.equalsIgnoreCase("properties")) {
-      propertiesHandler.handleRequest(req, rsp);
-    } else if (name.equalsIgnoreCase("threads")) {
-      threadDumpHandler.handleRequest(req, rsp);
-    } else if (name.equalsIgnoreCase("logging")) {
-      loggingHandler.handleRequest(req, rsp);
-    }  else if (name.equalsIgnoreCase("system")) {
-      systemInfoHandler.handleRequest(req, rsp);
-    } else {
-      if (name.equalsIgnoreCase("info")) name = "";
-      throw new SolrException(ErrorCode.NOT_FOUND, "Info Handler not found: " + name);
+    RequestHandlerBase handler = handlers.get(name.toLowerCase(Locale.ROOT));
+    if(handler == null) {
+      throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No handler by name "+name + " available names are "+ handlers.keySet());
     }
-    
+    handler.handleRequest(req, rsp);
     rsp.setHttpCaching(false);
   }
-  
-  
-
 
 
   //////////////////////// SolrInfoMBeans methods //////////////////////
@@ -109,39 +106,52 @@ public class InfoHandler extends RequestHandlerBase {
   }
 
   protected PropertiesRequestHandler getPropertiesHandler() {
-    return propertiesHandler;
+    return (PropertiesRequestHandler) handlers.get("properties");
+
   }
 
   protected ThreadDumpHandler getThreadDumpHandler() {
-    return threadDumpHandler;
+    return (ThreadDumpHandler) handlers.get("threads");
   }
 
   protected LoggingHandler getLoggingHandler() {
-    return loggingHandler;
+    return (LoggingHandler) handlers.get("logging");
   }
 
   protected SystemInfoHandler getSystemInfoHandler() {
-    return systemInfoHandler;
+    return (SystemInfoHandler) handlers.get("system");
   }
 
   protected void setPropertiesHandler(PropertiesRequestHandler propertiesHandler) {
-    this.propertiesHandler = propertiesHandler;
+    handlers.put("properties", propertiesHandler);
   }
 
   protected void setThreadDumpHandler(ThreadDumpHandler threadDumpHandler) {
-    this.threadDumpHandler = threadDumpHandler;
+    handlers.put("threads", threadDumpHandler);
   }
 
   protected void setLoggingHandler(LoggingHandler loggingHandler) {
-    this.loggingHandler = loggingHandler;
+    handlers.put("logging", loggingHandler);
   }
 
   protected void setSystemInfoHandler(SystemInfoHandler systemInfoHandler) {
-    this.systemInfoHandler = systemInfoHandler;
+    handlers.put("system", systemInfoHandler);
   }
 
   @Override
   public SolrRequestHandler getSubHandler(String subPath) {
     return this;
   }
+
+  private Map<String, RequestHandlerBase> handlers = new ConcurrentHashMap<>();
+
+  @Override
+  public Collection<Api> getApis() {
+    return singletonList(new ReqHandlerToApi(this, getSpec("node.Info")));
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 1b81722..eceb4b7 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
@@ -20,12 +20,15 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.Utils;
@@ -34,10 +37,16 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthenticationPlugin;
 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.api.Api;
+import org.apache.solr.api.ApiBag.ReqHandlerToApi;
+import org.apache.solr.api.SpecProvider;
+import org.apache.solr.util.JsonSchemaValidator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -244,5 +253,66 @@ public abstract class SecurityConfHandler extends RequestHandlerBase implements
       return "SecurityConfig: version=" + version + ", data=" + Utils.toJSONString(data);
     } 
   }
+
+  private Collection<Api> apis;
+  private AuthenticationPlugin authcPlugin;
+  private AuthorizationPlugin authzPlugin;
+
+  @Override
+  public Collection<Api> getApis() {
+    if (apis == null) {
+      synchronized (this) {
+        if (apis == null) {
+          Collection<Api> apis = new ArrayList<>();
+          final SpecProvider authcCommands = ApiBag.getSpec("cluster.security.authentication.Commands");
+          final SpecProvider authzCommands = ApiBag.getSpec("cluster.security.authorization.Commands");
+          apis.add(new ReqHandlerToApi(this, ApiBag.getSpec("cluster.security.authentication")));
+          apis.add(new ReqHandlerToApi(this, ApiBag.getSpec("cluster.security.authorization")));
+          SpecProvider authcSpecProvider = () -> {
+            AuthenticationPlugin authcPlugin = cores.getAuthenticationPlugin();
+            return authcPlugin != null && authcPlugin instanceof SpecProvider ?
+                ((SpecProvider) authcPlugin).getSpec() :
+                authcCommands.getSpec();
+          };
+
+          apis.add(new ReqHandlerToApi(this, authcSpecProvider) {
+            @Override
+            public synchronized Map<String, JsonSchemaValidator> getCommandSchema() {
+              //it is possible that the Authentication plugin is modified since the last call. invalidate the
+              // the cached commandSchema
+              if(SecurityConfHandler.this.authcPlugin != cores.getAuthenticationPlugin()) commandSchema = null;
+              SecurityConfHandler.this.authcPlugin = cores.getAuthenticationPlugin();
+              return super.getCommandSchema();
+            }
+          });
+
+          SpecProvider authzSpecProvider = () -> {
+            AuthorizationPlugin authzPlugin = cores.getAuthorizationPlugin();
+            return authzPlugin != null && authzPlugin instanceof SpecProvider ?
+                ((SpecProvider) authzPlugin).getSpec() :
+                authzCommands.getSpec();
+          };
+          apis.add(new ApiBag.ReqHandlerToApi(this, authzSpecProvider) {
+            @Override
+            public synchronized Map<String, JsonSchemaValidator> getCommandSchema() {
+              //it is possible that the Authorization plugin is modified since the last call. invalidate the
+              // the cached commandSchema
+              if(SecurityConfHandler.this.authzPlugin != cores.getAuthorizationPlugin()) commandSchema = null;
+              SecurityConfHandler.this.authzPlugin = cores.getAuthorizationPlugin();
+              return super.getCommandSchema();
+            }
+          });
+
+          this.apis = ImmutableList.copyOf(apis);
+        }
+      }
+    }
+    return this.apis;
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 35d04f6..a494ec4 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
@@ -21,9 +21,13 @@ import org.apache.solr.schema.IndexSchema;
 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.util.RTimerTree;
 
 import java.security.Principal;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -98,6 +102,26 @@ public interface SolrQueryRequest extends AutoCloseable {
   public void setJSON(Map<String,Object> json);
 
   public Principal getUserPrincipal();
+
+  default String getPath() {
+    return (String) getContext().get("path");
+  }
+
+  default Map<String, String> getPathTemplateValues() {
+    return Collections.emptyMap();
+  }
+
+  default List<CommandOperation> getCommands(boolean validateInput) {
+    return Collections.emptyList();
+  }
+
+  default String getHttpMethod() {
+    return (String) getContext().get("httpMethod");
+  }
+
+  default HttpSolrCall getHttpSolrCall() {
+    return null;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 ebee2fc..4b0e4d6 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
@@ -16,8 +16,13 @@
  */
 package org.apache.solr.request;
 
+import org.apache.solr.api.ApiBag;
+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.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.schema.IndexSchema;
@@ -26,10 +31,16 @@ import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
 
 import java.io.Closeable;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.security.Principal;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 /**
  * Base implementation of <code>SolrQueryRequest</code> that provides some
  * convenience methods for accessing parameters, and manages an IndexSearcher
@@ -183,4 +194,28 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
   public Principal getUserPrincipal() {
     return null;
   }
+
+  List<CommandOperation> parsedCommands;
+
+  public List<CommandOperation> getCommands(boolean validateInput) {
+    if (parsedCommands == null) {
+      Iterable<ContentStream> contentStreams = getContentStreams();
+      if (contentStreams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No content stream");
+      for (ContentStream contentStream : contentStreams) {
+        parsedCommands = ApiBag.getCommandOperations(new InputStreamReader((InputStream) contentStream, UTF_8),
+            getValidators(), validateInput);
+      }
+
+    }
+    return CommandOperation.clone(parsedCommands);
+
+  }
+
+  protected ValidatingJsonMap getSpec() {
+    return null;
+  }
+
+  protected Map<String, JsonSchemaValidator> getValidators(){
+    return Collections.EMPTY_MAP;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 8c3b5f0..7092c09 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -18,7 +18,6 @@ package org.apache.solr.schema;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.Reader;
 import java.io.StringWriter;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
@@ -71,18 +70,10 @@ public class SchemaManager {
   /**
    * Take in a JSON command set and execute them. It tries to capture as many errors
    * as possible instead of failing at the first error it encounters
-   * @param reader The input as a Reader
    * @return List of errors. If the List is empty then the operation was successful.
    */
-  public List performOperations(Reader reader) throws Exception {
-    List<CommandOperation> ops;
-    try {
-      ops = CommandOperation.parse(reader);
-    } catch (Exception e) {
-      String msg = "Error parsing schema operations ";
-      log.warn(msg, e);
-      return Collections.singletonList(singletonMap(CommandOperation.ERR_MSGS, msg + ":" + e.getMessage()));
-    }
+  public List performOperations() throws Exception {
+    List<CommandOperation> ops = req.getCommands(false);
     List errs = CommandOperation.captureErrors(ops);
     if (!errs.isEmpty()) return errs;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 5c0717b..f1665c7 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -37,11 +37,13 @@ import org.apache.http.Header;
 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.api.SpecProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEditablePlugin {
+public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEditablePlugin , SpecProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private AuthenticationProvider authenticationProvider;
   private final static ThreadLocal<Header> authHeader = new ThreadLocal<>();
@@ -162,7 +164,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
     authHeader.remove();
   }
 
-  public interface AuthenticationProvider {
+  public interface AuthenticationProvider extends SpecProvider {
     void init(Map<String, Object> pluginConfig);
 
     boolean authenticate(String user, String pwd);
@@ -170,6 +172,10 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
     Map<String, String> getPromptHeaders();
   }
 
+  @Override
+  public ValidatingJsonMap getSpec() {
+    return authenticationProvider.getSpec();
+  }
   public boolean getBlockUnknown(){
     return blockUnknown;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 ed5a05c..a8a97ed 100644
--- a/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
@@ -27,6 +27,9 @@ import java.util.Map;
 import java.util.Set;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,7 +42,7 @@ import static org.apache.solr.handler.admin.SecurityConfHandler.getListValue;
 import static org.apache.solr.handler.admin.SecurityConfHandler.getMapValue;
 
 
-public class RuleBasedAuthorizationPlugin implements AuthorizationPlugin, ConfigEditablePlugin {
+public class RuleBasedAuthorizationPlugin implements AuthorizationPlugin, ConfigEditablePlugin, SpecProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final Map<String, Set<String>> usersVsRoles = new HashMap<>();
@@ -232,4 +235,10 @@ public class RuleBasedAuthorizationPlugin implements AuthorizationPlugin, Config
 
   private static final Map<String, AutorizationEditOperation> ops = unmodifiableMap(asList(AutorizationEditOperation.values()).stream().collect(toMap(AutorizationEditOperation::getOperationName, identity())));
 
+
+  @Override
+  public ValidatingJsonMap getSpec() {
+    return ApiBag.getSpec("cluster.security.RuleBasedAuthorization").getSpec();
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 69664fd..0cc58cd 100644
--- a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
@@ -30,7 +30,10 @@ import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.codec.binary.Base64;
+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;
 
@@ -152,5 +155,10 @@ public class Sha256AuthenticationProvider implements ConfigEditablePlugin,  Basi
     return latestConf;
   }
 
+  @Override
+  public ValidatingJsonMap getSpec() {
+    return ApiBag.getSpec("cluster.security.BasicAuth.Commands").getSpec();
+  }
+
   static final Set<String> supported_ops = ImmutableSet.of("set-user", "delete-user");
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 1f98da9..b244015 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -56,6 +56,7 @@ import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.methods.HttpPut;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.entity.InputStreamEntity;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.SolrException;
@@ -71,6 +72,8 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
@@ -97,6 +100,8 @@ 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.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -149,6 +154,13 @@ public class HttpSolrCall {
   protected String coreUrl;
   protected SolrConfig config;
   protected Map<String, Integer> invalidStates;
+  protected boolean usingAliases = false;
+
+  //The states of client that is invalid in this request
+  protected Aliases aliases = null;
+  protected String corename = "";
+  protected String origCorename = null;
+
 
   public RequestType getRequestType() {
     return requestType;
@@ -172,6 +184,16 @@ public class HttpSolrCall {
     this.retry = retry;
     this.requestType = RequestType.UNKNOWN;
     queryParams = SolrRequestParsers.parseQueryString(req.getQueryString());
+    // set a request timer which can be reused by requests if needed
+    req.setAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE, new RTimerTree());
+    // put the core container in request attribute
+    req.setAttribute("org.apache.solr.CoreContainer", cores);
+    path = req.getServletPath();
+    if (req.getPathInfo() != null) {
+      // this lets you handle /update/commit when /update is a servlet
+      path += req.getPathInfo();
+    }
+    req.setAttribute(HttpSolrCall.class.getName(), this);
   }
 
   public String getPath() {
@@ -190,21 +212,8 @@ public class HttpSolrCall {
   public SolrParams getQueryParams() {
     return queryParams;
   }
-  
-  void init() throws Exception {
-    //The states of client that is invalid in this request
-    Aliases aliases = null;
-    String corename = "";
-    String origCorename = null;
-    // set a request timer which can be reused by requests if needed
-    req.setAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE, new RTimerTree());
-    // put the core container in request attribute
-    req.setAttribute("org.apache.solr.CoreContainer", cores);
-    path = req.getServletPath();
-    if (req.getPathInfo() != null) {
-      // this lets you handle /update/commit when /update is a servlet
-      path += req.getPathInfo();
-    }
+
+  protected void init() throws Exception {
     // check for management path
     String alternate = cores.getManagementPath();
     if (alternate != null && path.startsWith(alternate)) {
@@ -259,7 +268,7 @@ public class HttpSolrCall {
           core = cores.getCore(corename);
           if (core != null) {
             path = path.substring(idx);
-          } 
+          }
         }
       }
       if (core == null) {
@@ -321,13 +330,27 @@ public class HttpSolrCall {
 
     action = PASSTHROUGH;
   }
-  
+
+  protected String lookupAliases(String collName) {
+    ZkStateReader reader = cores.getZkController().getZkStateReader();
+    aliases = reader.getAliases();
+    if (aliases != null && aliases.collectionAliasSize() > 0) {
+      usingAliases = true;
+      String alias = aliases.getCollectionAlias(collName);
+      if (alias != null) {
+        collectionsList = StrUtils.splitSmart(alias, ",", true);
+        return collectionsList.get(0);
+      }
+    }
+    return null;
+  }
+
   /**
    * Extract handler from the URL path if not set.
    * This returns true if the action is set.
    * 
    */
-  private void extractHandlerFromURLPath(SolrRequestParsers parser) throws Exception {
+  protected void extractHandlerFromURLPath(SolrRequestParsers parser) throws Exception {
     if (handler == null && path.length() > 1) { // don't match "" or "/" as valid path
       handler = core.getRequestHandler(path);
 
@@ -370,7 +393,7 @@ public class HttpSolrCall {
     }
   }
 
-  private void extractRemotePath(String corename, String origCorename, int idx) throws UnsupportedEncodingException, KeeperException, InterruptedException {
+  protected void extractRemotePath(String corename, String origCorename, int idx) throws UnsupportedEncodingException, KeeperException, InterruptedException {
     if (core == null && idx > 0) {
       coreUrl = getRemotCoreUrl(corename, origCorename);
       // don't proxy for internal update requests
@@ -468,7 +491,7 @@ public class HttpSolrCall {
               Map.Entry<String, String> entry = headers.next();
               resp.addHeader(entry.getKey(), entry.getValue());
             }
-            QueryResponseWriter responseWriter = core.getQueryResponseWriter(solrReq);
+            QueryResponseWriter responseWriter = getResponseWriter();
             if (invalidStates != null) solrReq.getContext().put(CloudSolrClient.STATE_VERSION, invalidStates);
             writeResponse(solrRsp, responseWriter, reqMethod);
           }
@@ -661,17 +684,29 @@ public class HttpSolrCall {
   private void handleAdminRequest() throws IOException {
     SolrQueryResponse solrResp = new SolrQueryResponse();
     SolrCore.preDecorateResponse(solrReq, solrResp);
-    handler.handleRequest(solrReq, solrResp);
+    handleAdmin(solrResp);
     SolrCore.postDecorateResponse(handler, solrReq, solrResp);
     if (log.isInfoEnabled() && solrResp.getToLog().size() > 0) {
       log.info(solrResp.getToLogAsString("[admin]"));
     }
     QueryResponseWriter respWriter = SolrCore.DEFAULT_RESPONSE_WRITERS.get(solrReq.getParams().get(CommonParams.WT));
-    if (respWriter == null) respWriter = SolrCore.DEFAULT_RESPONSE_WRITERS.get("standard");
+    if (respWriter == null) respWriter = getResponseWriter();
     writeResponse(solrResp, respWriter, Method.getMethod(req.getMethod()));
   }
 
-  private void processAliases(Aliases aliases,
+  protected QueryResponseWriter getResponseWriter() {
+    if (core != null) return core.getQueryResponseWriter(solrReq);
+    QueryResponseWriter respWriter = SolrCore.DEFAULT_RESPONSE_WRITERS.get(solrReq.getParams().get(CommonParams.WT));
+    if (respWriter == null) respWriter = SolrCore.DEFAULT_RESPONSE_WRITERS.get("standard");
+    return respWriter;
+
+  }
+
+  protected void handleAdmin(SolrQueryResponse solrResp) {
+    handler.handleRequest(solrReq, solrResp);
+  }
+
+  protected void processAliases(Aliases aliases,
                               List<String> collectionsList) {
     String collection = solrReq.getParams().get(COLLECTION_PROP);
     if (collection != null) {
@@ -757,7 +792,7 @@ public class HttpSolrCall {
     return result;
   }
 
-  private SolrCore getCoreByCollection(String collectionName, boolean isPreferLeader) {
+  protected SolrCore getCoreByCollection(String collectionName, boolean isPreferLeader) {
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
 
     ClusterState clusterState = zkStateReader.getClusterState();
@@ -898,6 +933,10 @@ public class HttpSolrCall {
     return null;
   }
 
+  protected Object _getHandler(){
+    return handler;
+  }
+
   private AuthorizationContext getAuthCtx() {
 
     String resource = getPath();
@@ -987,7 +1026,7 @@ public class HttpSolrCall {
 
       @Override
       public Object getHandler() {
-        return handler;
+        return _getHandler();
       }
 
       @Override
@@ -1021,6 +1060,32 @@ public class HttpSolrCall {
   static final String CONNECTION_HEADER = "Connection";
   static final String TRANSFER_ENCODING_HEADER = "Transfer-Encoding";
   static final String CONTENT_LENGTH_HEADER = "Content-Length";
+  List<CommandOperation> parsedCommands;
+
+  public List<CommandOperation> getCommands(boolean validateInput) {
+    if (parsedCommands == null) {
+      Iterable<ContentStream> contentStreams = solrReq.getContentStreams();
+      if (contentStreams == null) parsedCommands = Collections.EMPTY_LIST;
+      else {
+        for (ContentStream contentStream : contentStreams) {
+          try {
+            parsedCommands = ApiBag.getCommandOperations(contentStream.getReader(), getValidators(), validateInput);
+          } catch (IOException e) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Error reading commands");
+          }
+          break;
+        }
+      }
+    }
+    return CommandOperation.clone(parsedCommands);
+  }
+  protected ValidatingJsonMap getSpec() {
+    return null;
+  }
+
+  protected Map<String, JsonSchemaValidator> getValidators(){
+    return Collections.EMPTY_MAP;
+  }
 
   /**
    * A faster method for randomly picking items when you do not need to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 a74fa8a..00733f5 100644
--- a/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
+++ b/solr/core/src/java/org/apache/solr/servlet/ResponseUtils.java
@@ -15,8 +15,10 @@
  * limitations under the License.
  */
 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.slf4j.Logger;
 
 import java.io.PrintWriter;
@@ -48,6 +50,10 @@ public class ResponseUtils {
       errorMetadata.add(SolrException.ERROR_CLASS, ex.getClass().getName());
       errorMetadata.add(SolrException.ROOT_ERROR_CLASS, SolrException.getRootCause(ex).getClass().getName());
       info.add("metadata", errorMetadata);
+      if (ex instanceof ApiBag.ExceptionWithErrObject) {
+        ApiBag.ExceptionWithErrObject exception = (ApiBag.ExceptionWithErrObject) ex;
+        info.add(CommandOperation.ERR_MSGS, exception.getErrs() );
+      }
     }
     
     for (Throwable th = ex; th != null; th = th.getCause()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index a411bb3..ce65069 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -58,6 +58,7 @@ import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.lucene.util.Version;
+import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -93,6 +94,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   
   // Effectively immutable
   private Boolean testMode = null;
+  private boolean isV2Enabled = !"true".equals(System.getProperty("disable.v2.api", "false"));
 
   /**
    * Enum to define action that needs to be processed.
@@ -102,7 +104,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
    *  This is generally when an error is set and returned.
    * RETRY:Retry the request. In cases when a core isn't found to work with, this is set.
    */
-  enum Action {
+  public enum Action {
     PASSTHROUGH, FORWARD, RETURN, RETRY, ADMIN, REMOTEQUERY, PROCESS
   }
   
@@ -136,7 +138,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     log.trace("SolrDispatchFilter.init(): {}", this.getClass().getClassLoader());
 
     SolrRequestParsers.fileCleaningTracker = new SolrFileCleaningTracker();
-    
+
     StartupLoggingUtils.checkLogDir();
     logWelcomeBanner();
     String muteConsole = System.getProperty(SOLR_LOG_MUTECONSOLE);
@@ -380,7 +382,17 @@ public class SolrDispatchFilter extends BaseSolrFilter {
    * want to add attributes to the request and send errors differently
    */
   protected HttpSolrCall getHttpSolrCall(HttpServletRequest request, HttpServletResponse response, boolean retry) {
-    return new HttpSolrCall(this, cores, request, response, retry);
+    String path = request.getServletPath();
+    if (request.getPathInfo() != null) {
+      // this lets you handle /update/commit when /update is a servlet
+      path += request.getPathInfo();
+    }
+
+    if (isV2Enabled && (path.startsWith("/v2/") || path.equals("/v2"))) {
+      return new V2HttpCall(this, cores, request, response, false);
+    } else {
+      return new HttpSolrCall(this, cores, request, response, retry);
+    }
   }
 
   private boolean authenticateRequest(ServletRequest request, ServletResponse response, final AtomicReference<ServletRequest> wrappedRequest) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 968320e..c311d4a 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -33,6 +33,7 @@ import java.security.Principal;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -45,6 +46,7 @@ import org.apache.commons.fileupload.servlet.ServletFileUpload;
 import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.lucene.util.IOUtils;
+import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
@@ -58,6 +60,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.util.RTimerTree;
 import org.apache.solr.util.SolrFileCleaningTracker;
 
@@ -224,11 +227,33 @@ public class SolrRequestParsers
       }
     }
 
+    final HttpSolrCall httpSolrCall = req == null ? null : (HttpSolrCall) req.getAttribute(HttpSolrCall.class.getName());
     SolrQueryRequestBase q = new SolrQueryRequestBase(core, params, requestTimer) {
       @Override
       public Principal getUserPrincipal() {
         return req == null ? null : req.getUserPrincipal();
       }
+
+      @Override
+      public List<CommandOperation> getCommands(boolean validateInput) {
+        if (httpSolrCall != null) {
+          return httpSolrCall.getCommands(validateInput);
+        }
+        return Collections.emptyList();
+      }
+
+      @Override
+      public Map<String, String> getPathTemplateValues() {
+        if (httpSolrCall != null && httpSolrCall instanceof V2HttpCall) {
+          return ((V2HttpCall) httpSolrCall).getUrlParts();
+        }
+        return Collections.EMPTY_MAP;
+      }
+
+      @Override
+      public HttpSolrCall getHttpSolrCall() {
+        return httpSolrCall;
+      }
     };
     if( streams != null && streams.size() > 0 ) {
       q.setContentStreams( streams );
@@ -848,4 +873,4 @@ public class SolrRequestParsers
 
 
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
index 6b8f14f..88dfbb9 100644
--- a/solr/core/src/java/org/apache/solr/util/CommandOperation.java
+++ b/solr/core/src/java/org/apache/solr/util/CommandOperation.java
@@ -50,6 +50,10 @@ public class CommandOperation {
     this.name = operationName;
   }
 
+  public Object getCommandData() {
+    return commandData;
+  }
+
   public String getStr(String key, String def) {
     if (ROOT_OBJ.equals(key)) {
       Object obj = getRootPrimitive();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java b/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
new file mode 100644
index 0000000..1074ed8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
@@ -0,0 +1,370 @@
+/*
+ * 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.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.unmodifiableMap;
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toMap;
+
+/**A very basic and lightweight json schema parsing and data validation tool. This custom tool is created
+ * because a) we need to support non json inputs b) to avoiding double parsing (this accepts an already parsed json as a map)
+ * It validates most aspects of json schema but it is NOT A FULLY COMPLIANT JSON schema parser or validator.
+ * What is supported ?
+ * a) all types and their validation (string, boolean, array, enum,object, integer, number)
+ * b) 'required' properties, 'additionalProperties'
+ *
+ *
+ */
+
+public class JsonSchemaValidator {
+  private final SchemaNode root;
+
+  public JsonSchemaValidator(String jsonString) {
+    this((Map) Utils.fromJSONString(jsonString));
+  }
+  public JsonSchemaValidator(Map jsonSchema) {
+    root = new SchemaNode(null);
+    root.isRequired = true;
+    List<String> errs = new LinkedList<>();
+    root.validateSchema(jsonSchema, errs);
+    if(!errs.isEmpty()){
+      throw new RuntimeException("Invalid schema. "+ StrUtils.join(errs,'|'));
+    }
+  }
+
+  private static class SchemaNode {
+    final SchemaNode parent;
+    Type type;
+    Type elementType;
+    boolean isRequired = false;
+    Object validationInfo;
+    Boolean additionalProperties;
+    Map<String, SchemaNode> children;
+
+    private SchemaNode(SchemaNode parent) {
+      this.parent = parent;
+    }
+
+    private void validateSchema(Map jsonSchema, List<String> errs) {
+      Object typeStr = jsonSchema.get("type");
+      if (typeStr == null) {
+        errs.add("'type' is missing ");
+      }
+      Type type = Type.get(typeStr);
+      if (type == null) {
+        errs.add ("Unknown type " + typeStr + " in object "+ Utils.toJSONString(jsonSchema));
+        return;
+      }
+      this.type = type;
+
+      for (SchemaAttribute schemaAttribute : SchemaAttribute.values()) {
+        schemaAttribute.validateSchema(jsonSchema, this, errs);
+      }
+      jsonSchema.keySet().forEach(o -> {
+        if (!knownAttributes.containsKey(o)) errs.add("Unknown key : " + o);
+      });
+      if (!errs.isEmpty()) return;
+
+      if (type == Type.OBJECT) {
+        Map m = (Map) jsonSchema.get("properties");
+        if (m != null) {
+          for (Object o : m.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            if (e.getValue() instanceof Map) {
+              Map od = (Map) e.getValue();
+              if (children == null) children = new LinkedHashMap<>();
+              SchemaNode child = new SchemaNode(this);
+              children.put((String) e.getKey(), child);
+              child.validateSchema(od, errs);
+            } else {
+              errs.add("Invalid Object definition for field " + e.getKey());
+            }
+          }
+        } else {
+          additionalProperties = Boolean.TRUE;
+        }
+      }
+      for (SchemaAttribute attr : SchemaAttribute.values()) {
+        attr.postValidateSchema(jsonSchema, this, errs);
+      }
+
+    }
+
+    private void validate(String key, Object data, List<String> errs) {
+      if (data == null) {
+        if (isRequired) {
+          errs.add("Missing field '" + key+"'");
+          return;
+        }
+      } else {
+        type.validateData(key, data, this, errs);
+        if(!errs.isEmpty()) return;
+        if (children != null && type == Type.OBJECT) {
+          for (Map.Entry<String, SchemaNode> e : children.entrySet()) {
+            e.getValue().validate(e.getKey(), ((Map) data).get(e.getKey()), errs);
+          }
+          if (Boolean.TRUE != additionalProperties) {
+            for (Object o : ((Map) data).keySet()) {
+              if (!children.containsKey(o)) {
+                errs.add("Unknown field '" + o + "' in object : " + Utils.toJSONString(data));
+              }
+            }
+          }
+        }
+      }
+    }
+
+  }
+
+  public List<String> validateJson(Object data) {
+    List<String> errs = new LinkedList<>();
+    root.validate(null, data, errs);
+    return errs.isEmpty() ? null : errs;
+  }
+
+  /**represents an attribute in the schema definition
+   *
+   */
+  enum SchemaAttribute {
+    type(true, Type.STRING),
+    properties(false, Type.OBJECT) {
+      @Override
+      public void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
+        super.validateSchema(attrSchema, schemaNode, errors);
+        if (schemaNode.type != Type.OBJECT) return;
+        Object val = attrSchema.get(key);
+        if (val == null) {
+          Object additional = attrSchema.get(additionalProperties.key);
+          if (Boolean.TRUE.equals(additional)) schemaNode.additionalProperties =  Boolean.TRUE;
+        }
+      }
+    },
+    additionalProperties(false, Type.BOOLEAN),
+    items(false, Type.OBJECT) {
+      @Override
+      public void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
+        super.validateSchema(attrSchema, schemaNode, errors);
+        Object itemsVal = attrSchema.get(key);
+        if (itemsVal != null) {
+          if (schemaNode.type != Type.ARRAY) {
+            errors.add("Only 'array' can have 'items'");
+            return;
+          } else {
+            if (itemsVal instanceof Map) {
+              Map val = (Map) itemsVal;
+              Object value = val.get(type.key);
+              Type t = Type.get(String.valueOf(value));
+              if (t == null) {
+                errors.add("Unknown array type " + Utils.toJSONString(attrSchema));
+              } else {
+                schemaNode.elementType = t;
+              }
+            }
+          }
+        }
+      }
+    },
+    __default(false,Type.UNKNOWN),
+    description(false, Type.STRING),
+    documentation(false, Type.STRING),
+    oneOf(false, Type.ARRAY),
+    __enum(false, Type.ARRAY) {
+      @Override
+      void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
+        if (attrSchema.get(Type.ENUM._name) != null) {
+          schemaNode.elementType = schemaNode.type;
+          schemaNode.type = Type.ENUM;
+        }
+      }
+
+      @Override
+      void postValidateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errs) {
+        Object val = attrSchema.get(key);
+        if (val == null) return;
+        if (val instanceof List) {
+          List list = (List) val;
+          for (Object o : list) {
+            if (!schemaNode.elementType.validate(o)) {
+              errs.add("Invalid value : " + o + " Expected type : " + schemaNode.elementType._name);
+            }
+          }
+          if (!errs.isEmpty()) return;
+          schemaNode.validationInfo = new HashSet(list);
+        } else {
+          errs.add("'enum' should have a an array as value in Object " + Utils.toJSONString(attrSchema));
+        }
+      }
+    },
+    id(false, Type.STRING),
+    _ref(false, Type.STRING),
+    _schema(false, Type.STRING),
+    required(false, Type.ARRAY) {
+      @Override
+      public void postValidateSchema(Map attrSchema, SchemaNode attr, List<String> errors) {
+        Object val = attrSchema.get(key);
+        if (val instanceof List) {
+          List list = (List) val;
+          if (attr.children != null) {
+            for (Map.Entry<String, SchemaNode> e : attr.children.entrySet()) {
+              if (list.contains(e.getKey())) e.getValue().isRequired = true;
+            }
+          }
+        }
+      }
+    };
+
+    final String key;
+    final boolean _required;
+    final Type typ;
+
+    public String getKey() {
+      return key;
+    }
+
+    void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
+      Object val = attrSchema.get(key);
+      if (val == null) {
+        if (_required)
+          errors.add("Missing required attribute '" + key + "' in object " + Utils.toJSONString(attrSchema));
+      } else {
+        if (!typ.validate(val)) errors.add(key + " should be of type " + typ._name);
+      }
+    }
+
+    void postValidateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errs) {
+    }
+
+    SchemaAttribute(boolean required, Type type) {
+      this.key = name().replaceAll("__","").replace('_', '$');
+      this._required = required;
+      this.typ = type;
+    }
+  }
+
+  interface TypeValidator {
+    void validateData(String key, Object o, SchemaNode schemaNode, List<String> errs);
+  }
+
+  /**represents a type in json
+   *
+   */
+  enum Type {
+    STRING(o -> o instanceof String),
+    ARRAY(o -> o instanceof List, (key, o, schemaNode, errs) -> {
+      List l = o instanceof List ? (List) o : Collections.singletonList(o);
+      if (schemaNode.elementType != null) {
+        for (Object elem : l) {
+          if (!schemaNode.elementType.validate(elem)) {
+            errs.add("Expected elements of type : " + key + " but found : " + Utils.toJSONString(o));
+            break;
+          }
+        }
+      }
+    }),
+    NUMBER(o -> o instanceof Number, (key, o, schemaNode, errs) -> {
+      if (o instanceof String) {
+        try {
+          Double.parseDouble((String) o);
+        } catch (NumberFormatException e) {
+          errs.add(e.getClass().getName() + " " + e.getMessage());
+        }
+
+      }
+
+    }),
+    INTEGER(o -> o instanceof Integer, (key, o, schemaNode, errs) -> {
+      if (o instanceof String) {
+        try {
+          Integer.parseInt((String) o);
+        } catch (NumberFormatException e) {
+          errs.add(e.getClass().getName() + " " + e.getMessage());
+        }
+      }
+    }),
+    BOOLEAN(o -> o instanceof Boolean, (key, o, schemaNode, errs) -> {
+      if (o instanceof String) {
+        try {
+          Boolean.parseBoolean((String) o);
+        } catch (Exception e) {
+          errs.add(e.getClass().getName() + " " + e.getMessage());
+        }
+      }
+    }),
+    ENUM(o -> o instanceof List, (key, o, schemaNode, errs) -> {
+      if (schemaNode.validationInfo instanceof HashSet) {
+        HashSet enumVals = (HashSet) schemaNode.validationInfo;
+        if (!enumVals.contains(o)) {
+          errs.add("value of enum " + key + " must be one of" + enumVals);
+        }
+      }
+    }),
+    OBJECT(o -> o instanceof Map),
+    UNKNOWN((o -> true));
+    final String _name;
+
+    final java.util.function.Predicate typeValidator;
+    private final TypeValidator validator;
+
+    Type(java.util.function.Predicate validator) {
+      this(validator, null);
+
+    }
+
+    Type(java.util.function.Predicate validator, TypeValidator v) {
+      _name = this.name().toLowerCase(Locale.ROOT);
+      this.typeValidator = validator;
+      this.validator = v;
+    }
+
+    boolean validate(Object o) {
+      return typeValidator.test(o);
+    }
+
+    void validateData(String key, Object o, SchemaNode attr, List<String> errs) {
+      if (validator != null) {
+        validator.validateData(key, o, attr, errs);
+        return;
+      }
+      if (!typeValidator.test(o))
+        errs.add("Expected type : " + _name + " but found : " + o + "in object : " + Utils.toJSONString(o));
+    }
+
+    static Type get(Object type) {
+      for (Type t : Type.values()) {
+        if (t._name.equals(type)) return t;
+      }
+      return null;
+    }
+  }
+
+
+  static final Map<String, SchemaAttribute> knownAttributes = unmodifiableMap(asList(SchemaAttribute.values()).stream().collect(toMap(SchemaAttribute::getKey, identity())));
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/util/PathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/PathTrie.java b/solr/core/src/java/org/apache/solr/util/PathTrie.java
new file mode 100644
index 0000000..ceaa5de
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/PathTrie.java
@@ -0,0 +1,195 @@
+/*
+ * 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.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.common.util.StrUtils;
+
+import static java.util.Collections.emptyList;
+
+/**A utility class to efficiently parse/store/lookup hierarchical paths which are templatized
+ * like /collections/{collection}/shards/{shard}/{replica}
+ */
+public class PathTrie<T> {
+  private final Set<String> reserved = new HashSet<>();
+  Node root = new Node(emptyList(), null);
+
+  public PathTrie() { }
+
+  public PathTrie(Set<String> reserved) {
+    this.reserved.addAll(reserved);
+  }
+
+
+
+  public void insert(String path, Map<String, String> replacements, T o) {
+    List<String> parts = getPathSegments(path);
+    insert(parts,replacements, o);
+  }
+
+  public void insert(List<String> parts, Map<String, String> replacements, T o) {
+    if (parts.isEmpty()) {
+      root.obj = o;
+      return;
+    }
+
+    for (int i = 0; i < parts.size(); i++) {
+      String part = parts.get(i);
+      if (part.charAt(0) == '$') {
+        String replacement = replacements.get(part.substring(1));
+        if (replacement == null) {
+          throw new RuntimeException(part + " is not provided");
+        }
+        replacement = replacement.charAt(0) == '/' ? replacement.substring(1) : replacement;
+        parts.set(i, replacement);
+      }
+    }
+
+    root.insert(parts, o);
+  }
+
+  // /a/b/c will be returned as ["a","b","c"]
+  public static List<String> getPathSegments(String path) {
+    if (path == null || path.isEmpty()) return emptyList();
+    List<String> parts = new ArrayList<String>() {
+      @Override
+      public boolean add(String s) {
+        if (s == null || s.isEmpty()) return false;
+        return super.add(s);
+      }
+    };
+    StrUtils.splitSmart(path, '/', parts);
+    return parts;
+  }
+
+
+  public T lookup(String path, Map<String, String> templateValues) {
+    return root.lookup(getPathSegments(path), 0, templateValues);
+  }
+
+  public T lookup(List<String> path, Map<String, String> templateValues) {
+    return root.lookup(path, 0, templateValues);
+  }
+
+  public T lookup(String path, Map<String, String> templateValues, Set<String> paths) {
+    return root.lookup(getPathSegments(path), 0, templateValues, paths);
+  }
+
+  public static String templateName(String templateStr) {
+    return templateStr.startsWith("{") && templateStr.endsWith("}") ?
+        templateStr.substring(1, templateStr.length() - 1) :
+        null;
+
+  }
+
+  class Node {
+    String name;
+    Map<String, Node> children;
+    T obj;
+    String templateName;
+
+    Node(List<String> path, T o) {
+      if (path.isEmpty()) {
+        obj = o;
+        return;
+      }
+      String part = path.get(0);
+      templateName = templateName(part);
+      name = part;
+      if (path.isEmpty()) obj = o;
+    }
+
+
+    private synchronized void insert(List<String> path, T o) {
+      String part = path.get(0);
+      Node matchedChild = null;
+      if (children == null) children = new ConcurrentHashMap<>();
+
+      String varName = templateName(part);
+      String key = varName == null ? part : "";
+
+      matchedChild = children.get(key);
+      if (matchedChild == null) {
+        children.put(key, matchedChild = new Node(path, o));
+      }
+      if (varName != null) {
+        if (!matchedChild.templateName.equals(varName)) {
+          throw new RuntimeException("wildcard name must be " + matchedChild.templateName);
+        }
+      }
+      path.remove(0);
+      if (!path.isEmpty()) {
+        matchedChild.insert(path, o);
+      } else {
+        matchedChild.obj = o;
+      }
+
+    }
+
+
+    void findAvailableChildren(String path, Set<String> availableSubPaths) {
+      if (availableSubPaths == null) return;
+      if (children != null) {
+        for (Node node : children.values()) {
+          if (node.obj != null) {
+            String s = path + "/" + node.name;
+            availableSubPaths.add(s);
+          }
+        }
+
+        for (Node node : children.values()) {
+          node.findAvailableChildren(path + "/" + node.name, availableSubPaths);
+        }
+      }
+    }
+
+
+    public T lookup(List<String> pieces, int i, Map<String, String> templateValues) {
+      return lookup(pieces, i, templateValues, null);
+
+    }
+
+    /**
+     *
+     * @param pathSegments pieces in the url /a/b/c has pieces as 'a' , 'b' , 'c'
+     * @param index current index of the pieces that we are looking at in /a/b/c 0='a' and 1='b'
+     * @param templateVariables The mapping of template variable to its value
+     * @param availableSubPaths If not null , available sub paths will be returned in this set
+     */
+    public T lookup(List<String> pathSegments, int index, Map<String, String> templateVariables, Set<String> availableSubPaths) {
+      if (templateName != null) templateVariables.put(templateName, pathSegments.get(index - 1));
+      if (pathSegments.size() < index + 1) {
+        findAvailableChildren("", availableSubPaths);
+        return obj;
+      }
+      String piece = pathSegments.get(index);
+      if (children == null) return null;
+      Node n = children.get(piece);
+      if (n == null && !reserved.contains(piece)) n = children.get("");
+      if (n == null) return null;
+      return n.lookup(pathSegments, index + 1, templateVariables, availableSubPaths);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/ImplicitPlugins.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/ImplicitPlugins.json b/solr/core/src/resources/ImplicitPlugins.json
index 34e5c07..a1ddbe7 100644
--- a/solr/core/src/resources/ImplicitPlugins.json
+++ b/solr/core/src/resources/ImplicitPlugins.json
@@ -26,6 +26,10 @@
         "json.command": "false"
       }
     },
+    "update":{
+      "class":"solr.UpdateRequestHandlerApi",
+      "useParams": "_UPDATE_JSON_DOCS"
+    },
     "/config": {
       "useParams":"_CONFIG",
       "class": "solr.SolrConfigHandler"
@@ -159,4 +163,4 @@
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cluster.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.Commands.json b/solr/core/src/resources/apispec/cluster.Commands.json
new file mode 100644
index 0000000..8983964
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.Commands.json
@@ -0,0 +1,74 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API",
+  "description": "Cluster-wide commands to assign roles to nodes, remove role assignments, or add, edit or remove a cluster-wide property.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster"
+    ]
+  },
+  "commands": {
+    "add-role":{
+      "type":"object",
+      "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api15",
+      "description":"Assign a specific role to a node in the cluster.",
+      "properties": {
+        "role": {
+          "type": "string",
+          "description": "The name of the role. The only supported role is 'overseer'."
+
+        },
+        "node": {
+          "type": "string",
+          "description": "The name of the node. It is possible to assign a role even before that node is started."
+
+        }
+      },
+      "required": [
+        "role",
+        "node"
+      ]
+    },
+    "remove-role":{
+      "type":"object",
+      "documentation":"https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api16",
+      "description":"Unassign a role from a node in the cluster.",
+      "properties": {
+        "role": {
+          "type": "string",
+          "description": "The name of the role. The only supported role as of now is 'overseer'."
+
+        },
+        "node": {
+          "type": "string",
+          "description": "The name of the node where the role should be removed."
+        }
+      },
+      "required": [
+        "role",
+        "node"
+      ]
+    },
+    "set-property": {
+      "type": "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api11",
+      "description": "Add, edit, or delete a cluster-wide property.",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the property"
+        },
+        "val": {
+          "type": "string",
+          "description": "The value of the property. If the value is empty or null, the property is unset."
+        }
+      },
+      "required": [
+        "name",
+        "val"
+      ]
+    }
+  }
+}


[5/5] lucene-solr:master: SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new

Posted by no...@apache.org.
SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new


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

Branch: refs/heads/master
Commit: 71abe130697b0f279d6e3613145f1f8f052c7848
Parents: c8edbe8
Author: Noble Paul <no...@apache.org>
Authored: Tue Jan 31 16:27:33 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jan 31 16:27:33 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 solr/core/src/java/org/apache/solr/api/Api.java |  67 ++++
 .../src/java/org/apache/solr/api/ApiBag.java    | 354 ++++++++++++++++++
 .../java/org/apache/solr/api/ApiSupport.java    |  46 +++
 .../java/org/apache/solr/api/SpecProvider.java  |  25 ++
 .../java/org/apache/solr/api/V2HttpCall.java    | 340 +++++++++++++++++
 .../java/org/apache/solr/api/package-info.java  |  21 ++
 .../src/java/org/apache/solr/cloud/Assign.java  |  10 +-
 .../org/apache/solr/cloud/CreateShardCmd.java   |   2 +-
 .../java/org/apache/solr/core/PluginBag.java    |  71 +++-
 .../org/apache/solr/handler/BlobHandler.java    |  14 +-
 .../apache/solr/handler/DumpRequestHandler.java |  11 +
 .../apache/solr/handler/PingRequestHandler.java |   5 +
 .../apache/solr/handler/RealTimeGetHandler.java |  14 +
 .../apache/solr/handler/RequestHandlerBase.java |  12 +-
 .../org/apache/solr/handler/SchemaHandler.java  |  35 +-
 .../apache/solr/handler/SolrConfigHandler.java  |  16 +
 .../solr/handler/UpdateRequestHandler.java      |   2 +
 .../solr/handler/UpdateRequestHandlerApi.java   |  73 ++++
 .../handler/admin/BaseHandlerApiSupport.java    | 236 ++++++++++++
 .../handler/admin/CollectionHandlerApi.java     | 319 ++++++++++++++++
 .../solr/handler/admin/CollectionsHandler.java  |  69 ++--
 .../solr/handler/admin/ConfigSetsHandler.java   |  47 ++-
 .../handler/admin/ConfigSetsHandlerApi.java     | 112 ++++++
 .../solr/handler/admin/CoreAdminHandler.java    |  14 +
 .../solr/handler/admin/CoreAdminHandlerApi.java | 175 +++++++++
 .../apache/solr/handler/admin/InfoHandler.java  |  78 ++--
 .../solr/handler/admin/SecurityConfHandler.java |  70 ++++
 .../apache/solr/request/SolrQueryRequest.java   |  24 ++
 .../solr/request/SolrQueryRequestBase.java      |  35 ++
 .../org/apache/solr/schema/SchemaManager.java   |  13 +-
 .../apache/solr/security/BasicAuthPlugin.java   |  10 +-
 .../security/RuleBasedAuthorizationPlugin.java  |  11 +-
 .../security/Sha256AuthenticationProvider.java  |   8 +
 .../org/apache/solr/servlet/HttpSolrCall.java   | 115 ++++--
 .../org/apache/solr/servlet/ResponseUtils.java  |   6 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  18 +-
 .../apache/solr/servlet/SolrRequestParsers.java |  27 +-
 .../org/apache/solr/util/CommandOperation.java  |   4 +
 .../apache/solr/util/JsonSchemaValidator.java   | 370 +++++++++++++++++++
 .../src/java/org/apache/solr/util/PathTrie.java | 195 ++++++++++
 solr/core/src/resources/ImplicitPlugins.json    |   6 +-
 .../src/resources/apispec/cluster.Commands.json |  74 ++++
 .../apispec/cluster.commandstatus.delete.json   |  10 +
 .../apispec/cluster.commandstatus.json          |  20 +
 .../apispec/cluster.configs.Commands.json       |  34 ++
 .../apispec/cluster.configs.delete.json         |  12 +
 .../src/resources/apispec/cluster.configs.json  |  12 +
 solr/core/src/resources/apispec/cluster.json    |  14 +
 .../src/resources/apispec/cluster.nodes.json    |  12 +
 .../cluster.security.BasicAuth.Commands.json    |  23 ++
 ...cluster.security.RuleBasedAuthorization.json | 129 +++++++
 ...luster.security.authentication.Commands.json |  12 +
 .../cluster.security.authentication.json        |  12 +
 ...cluster.security.authorization.Commands.json |  13 +
 .../apispec/cluster.security.authorization.json |  13 +
 .../resources/apispec/collections.Commands.json | 206 +++++++++++
 .../collections.collection.Commands.json        | 137 +++++++
 .../collections.collection.Commands.modify.json |  36 ++
 .../collections.collection.Commands.reload.json |  11 +
 .../apispec/collections.collection.delete.json  |  13 +
 .../apispec/collections.collection.json         |  19 +
 .../collections.collection.shards.Commands.json | 109 ++++++
 ...ctions.collection.shards.shard.Commands.json |  24 ++
 ...lections.collection.shards.shard.delete.json |  27 ++
 ....collection.shards.shard.replica.delete.json |  39 ++
 .../core/src/resources/apispec/collections.json |  13 +
 .../src/resources/apispec/core.RealtimeGet.json |  26 ++
 .../apispec/core.SchemaEdit.addCopyField.json   |  27 ++
 .../apispec/core.SchemaEdit.addField.json       |  98 +++++
 .../core.SchemaEdit.addFieldType.analyzers.json |  51 +++
 .../apispec/core.SchemaEdit.addFieldType.json   |  53 +++
 .../core.SchemaEdit.deleteCopyField.json        |  19 +
 .../core.SchemaEdit.deleteDynamicField.json     |  12 +
 .../apispec/core.SchemaEdit.deleteField.json    |  12 +
 .../core.SchemaEdit.deleteFieldType.json        |  14 +
 .../src/resources/apispec/core.SchemaEdit.json  |  47 +++
 .../apispec/core.SchemaRead.copyFields.json     |  26 ++
 ...ore.SchemaRead.dynamicFields_fieldTypes.json |  20 +
 .../apispec/core.SchemaRead.fields.json         |  34 ++
 .../src/resources/apispec/core.SchemaRead.json  |  18 +
 .../core/src/resources/apispec/core.Update.json |  17 +
 ...g.Commands.addRequestHandler.properties.json |  25 ++
 .../apispec/core.config.Commands.generic.json   |  19 +
 .../resources/apispec/core.config.Commands.json | 215 +++++++++++
 .../core.config.Commands.runtimeLib.json        |  23 ++
 .../apispec/core.config.Params.Commands.json    |  31 ++
 .../resources/apispec/core.config.Params.json   |  13 +
 .../core/src/resources/apispec/core.config.json |  18 +
 .../src/resources/apispec/core.system.blob.json |  20 +
 .../apispec/core.system.blob.upload.json        |  12 +
 .../src/resources/apispec/cores.Commands.json   |  85 +++++
 .../src/resources/apispec/cores.Status.json     |  20 +
 .../resources/apispec/cores.core.Commands.json  | 136 +++++++
 .../apispec/cores.core.Commands.split.json      |  34 ++
 solr/core/src/resources/apispec/emptySpec.json  |  11 +
 .../src/resources/apispec/node.Commands.json    |  24 ++
 solr/core/src/resources/apispec/node.Info.json  |  11 +
 .../core/src/resources/apispec/node.invoke.json |  16 +
 .../conf/solrconfig-managed-schema.xml          |   2 +-
 .../test/org/apache/solr/api/TestPathTrie.java  |  61 +++
 .../org/apache/solr/cloud/rule/RulesTest.java   |  18 +
 .../solr/core/BlobStoreTestRequestHandler.java  |   1 +
 .../test/org/apache/solr/core/SolrCoreTest.java |   1 +
 .../apache/solr/core/TestDynamicLoading.java    |   2 +-
 .../apache/solr/core/TestSolrConfigHandler.java |  85 ++++-
 .../solr/handler/V2ApiIntegrationTest.java      |  98 +++++
 .../solr/handler/admin/TestApiFramework.java    | 219 +++++++++++
 .../solr/handler/admin/TestCollectionAPIs.java  | 231 ++++++++++++
 .../solr/handler/admin/TestConfigsApi.java      |  59 +++
 .../solr/handler/admin/TestCoreAdminApis.java   | 115 ++++++
 .../solr/rest/schema/TestBulkSchemaAPI.java     |  15 +
 .../solr/security/BasicAuthIntegrationTest.java |   4 +
 .../TestRuleBasedAuthorizationPlugin.java       |   8 +-
 .../solr/servlet/SolrRequestParserTest.java     |   4 +-
 .../org/apache/solr/util/JsonValidatorTest.java | 189 ++++++++++
 .../conf/solrconfig.xml                         |   8 +-
 .../apache/solr/client/solrj/SolrRequest.java   |  13 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |  16 +-
 .../org/apache/solr/common/cloud/Replica.java   |   3 +
 .../org/apache/solr/common/util/StrUtils.java   |  10 +-
 .../java/org/apache/solr/common/util/Utils.java |   5 +-
 .../solr/common/util/ValidatingJsonMap.java     | 349 +++++++++++++++++
 .../solrj/embedded/SolrExampleJettyTest.java    |   9 +-
 .../solr/common/util/TestValidatingJsonMap.java |  52 +++
 .../org/apache/solr/util/RestTestHarness.java   |   8 +
 126 files changed, 6715 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 27a9c7f..754c971 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -118,6 +118,9 @@ New Features
 * SOLR-9481: Authentication and Authorization plugins now work in standalone mode if security.json is placed in
   SOLR_HOME on every node. Editing config through API is supported but affects only that one node. (janhoy)
 
+* SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new
+  (noble, Steve Rowe, Cassandra Targett, Timothy Potter)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/api/Api.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/Api.java b/solr/core/src/java/org/apache/solr/api/Api.java
new file mode 100644
index 0000000..8512c89
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/Api.java
@@ -0,0 +1,67 @@
+/*
+ * 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.api;
+
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.JsonSchemaValidator;
+
+/** Every version 2 API must extend the this class. It's mostly like a request handler
+ * but it has extra methods to provide the json schema of the end point
+ *
+ */
+public abstract class Api implements SpecProvider {
+  protected SpecProvider spec;
+  protected volatile Map<String, JsonSchemaValidator> commandSchema;
+
+  protected Api(SpecProvider spec) {
+    this.spec = spec;
+  }
+
+  /**This method helps to cache the schema validator object
+   */
+  public Map<String, JsonSchemaValidator> getCommandSchema() {
+    if (commandSchema == null) {
+      synchronized (this) {
+        if(commandSchema == null) {
+          ValidatingJsonMap commands = getSpec().getMap("commands", null);
+          commandSchema = commands != null ?
+              ImmutableMap.copyOf(ApiBag.getParsedSchema(commands)) :
+              ImmutableMap.of();
+        }
+      }
+    }
+    return commandSchema;
+  }
+
+  /** The method that gets called for each request
+   */
+  public abstract void call(SolrQueryRequest req , SolrQueryResponse rsp);
+
+  /**Get the specification of the API as a Map
+   */
+  @Override
+  public ValidatingJsonMap getSpec() {
+    return spec.getSpec();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..82d6a39
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -0,0 +1,354 @@
+/*
+ * 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.api;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.handler.RequestHandlerUtils;
+import org.apache.solr.request.SolrQueryRequest;
+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.util.JsonSchemaValidator;
+import org.apache.solr.util.PathTrie;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.client.solrj.SolrRequest.SUPPORTED_METHODS;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.common.util.StrUtils.formatString;
+import static org.apache.solr.common.util.ValidatingJsonMap.ENUM_OF;
+import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
+
+public class ApiBag {
+  private final boolean isCoreSpecific;
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, PathTrie<Api>> apis = new ConcurrentHashMap<>();
+
+  public ApiBag(boolean isCoreSpecific) {
+    this.isCoreSpecific = isCoreSpecific;
+  }
+
+  public synchronized void register(Api api, Map<String, String> nameSubstitutes) {
+    try {
+      validateAndRegister(api, nameSubstitutes);
+    } catch (Exception e) {
+      log.error("Unable to register plugin:" + api.getClass().getName() + "with spec :" + Utils.toJSONString(api.getSpec()), e);
+      if (e instanceof RuntimeException) {
+        throw (RuntimeException) e;
+      } else {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+
+    }
+  }
+
+  private void validateAndRegister(Api api, Map<String, String> nameSubstitutes) {
+    ValidatingJsonMap spec = api.getSpec();
+    Api introspect = new IntrospectApi(api, isCoreSpecific);
+    List<String> methods = spec.getList("methods", ENUM_OF, SUPPORTED_METHODS);
+    for (String method : methods) {
+      PathTrie<Api> registry = apis.get(method);
+
+      if (registry == null) apis.put(method, registry = new PathTrie<>(ImmutableSet.of("_introspect")));
+      ValidatingJsonMap url = spec.getMap("url", NOT_NULL);
+      ValidatingJsonMap params = url.getMap("params", null);
+      if (params != null) {
+        for (Object o : params.keySet()) {
+          ValidatingJsonMap param = params.getMap(o.toString(), NOT_NULL);
+          param.get("type", ENUM_OF, KNOWN_TYPES);
+        }
+      }
+      List<String> paths = url.getList("paths", NOT_NULL);
+      ValidatingJsonMap parts = url.getMap("parts", null);
+      if (parts != null) {
+        Set<String> wildCardNames = getWildCardNames(paths);
+        for (Object o : parts.keySet()) {
+          if (!wildCardNames.contains(o.toString()))
+            throw new RuntimeException("" + o + " is not a valid part name");
+          ValidatingJsonMap pathMeta = parts.getMap(o.toString(), NOT_NULL);
+          pathMeta.get("type", ENUM_OF, ImmutableSet.of("enum", "string", "int", "number", "boolean"));
+        }
+      }
+      verifyCommands(api.getSpec());
+      for (String path : paths) {
+        registry.insert(path, nameSubstitutes, api);
+        registerIntrospect(nameSubstitutes, registry, path, introspect);
+      }
+    }
+  }
+
+  public static void registerIntrospect(Map<String, String> nameSubstitutes, PathTrie<Api> registry, String path, Api introspect) {
+    List<String> l = PathTrie.getPathSegments(path);
+    registerIntrospect(l, registry, nameSubstitutes, introspect);
+    int lastIdx = l.size() - 1;
+    for (int i = lastIdx; i >= 0; i--) {
+      String itemAt = l.get(i);
+      if (PathTrie.templateName(itemAt) == null) break;
+      l.remove(i);
+      if (registry.lookup(l, new HashMap<>()) != null) break;
+      registerIntrospect(l, registry, nameSubstitutes, introspect);
+    }
+  }
+
+  static void registerIntrospect(List<String> l, PathTrie<Api> registry, Map<String, String> substitutes, Api introspect) {
+    ArrayList<String> copy = new ArrayList<>(l);
+    copy.add("_introspect");
+    registry.insert(copy, substitutes, introspect);
+  }
+
+  public static class IntrospectApi extends Api {
+    Api baseApi;
+    final boolean isCoreSpecific;
+
+    public IntrospectApi(Api base, boolean isCoreSpecific) {
+      super(EMPTY_SPEC);
+      this.baseApi = base;
+      this.isCoreSpecific = isCoreSpecific;
+    }
+
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+
+      String cmd = req.getParams().get("command");
+      ValidatingJsonMap result = null;
+      if (cmd == null) {
+        result = isCoreSpecific ? ValidatingJsonMap.getDeepCopy(baseApi.getSpec(), 5, true) : baseApi.getSpec();
+      } else {
+        ValidatingJsonMap specCopy = ValidatingJsonMap.getDeepCopy(baseApi.getSpec(), 5, true);
+        ValidatingJsonMap commands = specCopy.getMap("commands", null);
+        if (commands != null) {
+          ValidatingJsonMap m = commands.getMap(cmd, null);
+          specCopy.put("commands", Collections.singletonMap(cmd, m));
+        }
+        result = specCopy;
+      }
+      if (isCoreSpecific) {
+        List<String> pieces = req.getHttpSolrCall() == null ? null : ((V2HttpCall) req.getHttpSolrCall()).pieces;
+        if (pieces != null) {
+          String prefix = "/" + pieces.get(0) + "/" + pieces.get(1);
+          List<String> paths = result.getMap("url", NOT_NULL).getList("paths", NOT_NULL);
+          result.getMap("url", NOT_NULL).put("paths",
+              paths.stream()
+                  .map(s -> prefix + s)
+                  .collect(Collectors.toList()));
+        }
+      }
+      List l = (List) rsp.getValues().get("spec");
+      if (l == null) rsp.getValues().add("spec", l = new ArrayList());
+      l.add(result);
+      RequestHandlerUtils.addExperimentalFormatWarning(rsp);
+    }
+  }
+
+  public static Map<String, JsonSchemaValidator> getParsedSchema(ValidatingJsonMap commands) {
+    Map<String, JsonSchemaValidator> validators = new HashMap<>();
+    for (Object o : commands.entrySet()) {
+      Map.Entry cmd = (Map.Entry) o;
+      try {
+        validators.put((String) cmd.getKey(), new JsonSchemaValidator((Map) cmd.getValue()));
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in api spec", e);
+      }
+    }
+    return validators;
+  }
+
+
+  private void verifyCommands(ValidatingJsonMap spec) {
+    ValidatingJsonMap commands = spec.getMap("commands", null);
+    if (commands == null) return;
+    getParsedSchema(commands);
+
+  }
+
+  private Set<String> getWildCardNames(List<String> paths) {
+    Set<String> wildCardNames = new HashSet<>();
+    for (String path : paths) {
+      List<String> p = PathTrie.getPathSegments(path);
+      for (String s : p) {
+        String wildCard = PathTrie.templateName(s);
+        if (wildCard != null) wildCardNames.add(wildCard);
+      }
+    }
+    return wildCardNames;
+  }
+
+
+  public Api lookup(String path, String httpMethod, Map<String, String> parts) {
+    if (httpMethod == null) {
+      for (PathTrie<Api> trie : apis.values()) {
+        Api api = trie.lookup(path, parts);
+        if (api != null) return api;
+      }
+      return null;
+    } else {
+      PathTrie<Api> registry = apis.get(httpMethod);
+      if (registry == null) return null;
+      return registry.lookup(path, parts);
+    }
+  }
+
+  public static SpecProvider getSpec(final String name) {
+    return () -> {
+      return ValidatingJsonMap.parse(APISPEC_LOCATION + name + ".json", APISPEC_LOCATION);
+    };
+  }
+
+  public static class ReqHandlerToApi extends Api implements PermissionNameProvider {
+    SolrRequestHandler rh;
+
+    public ReqHandlerToApi(SolrRequestHandler rh, SpecProvider spec) {
+      super(spec);
+      this.rh = rh;
+    }
+
+    @Override
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      rh.handleRequest(req, rsp);
+    }
+
+    @Override
+    public Name getPermissionName(AuthorizationContext ctx) {
+      if (rh instanceof PermissionNameProvider) {
+        return ((PermissionNameProvider) rh).getPermissionName(ctx);
+      }
+      return null;
+    }
+  }
+
+  public static List<Api> wrapRequestHandlers(final SolrRequestHandler rh, String... specs) {
+    ImmutableList.Builder<Api> b = ImmutableList.builder();
+    for (String spec : specs) b.add(new ReqHandlerToApi(rh, ApiBag.getSpec(spec)));
+    return b.build();
+  }
+
+  public static final String APISPEC_LOCATION = "apispec/";
+  public static final String INTROSPECT = "/_introspect";
+
+
+  public static final SpecProvider EMPTY_SPEC = () -> ValidatingJsonMap.EMPTY;
+  public static final String HANDLER_NAME = "handlerName";
+  public static final Set<String> KNOWN_TYPES = ImmutableSet.of("string", "boolean", "list", "int", "double", "object");
+
+  public PathTrie<Api> getRegistry(String method) {
+    return apis.get(method);
+  }
+
+  public void registerLazy(PluginBag.PluginHolder<SolrRequestHandler> holder, PluginInfo info) {
+    String specName = info.attributes.get("spec");
+    if (specName == null) specName = "emptySpec";
+    register(new LazyLoadedApi(ApiBag.getSpec(specName), holder), Collections.singletonMap(HANDLER_NAME, info.attributes.get(NAME)));
+  }
+
+  public static SpecProvider constructSpec(PluginInfo info) {
+    Object specObj = info == null ? null : info.attributes.get("spec");
+    if (specObj == null) specObj = "emptySpec";
+    if (specObj instanceof Map) {
+      Map map = (Map) specObj;
+      return () -> ValidatingJsonMap.getDeepCopy(map, 4, false);
+    } else {
+      return ApiBag.getSpec((String) specObj);
+    }
+  }
+
+  public static List<CommandOperation> getCommandOperations(Reader reader, Map<String, JsonSchemaValidator> validators, boolean validate) {
+    List<CommandOperation> parsedCommands = null;
+    try {
+      parsedCommands = CommandOperation.parse(reader);
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+    if (validators == null || !validate) {    // no validation possible because we do not have a spec
+      return parsedCommands;
+    }
+
+    List<CommandOperation> commandsCopy = CommandOperation.clone(parsedCommands);
+
+    for (CommandOperation cmd : commandsCopy) {
+      JsonSchemaValidator validator = validators.get(cmd.name);
+      if (validator == null) {
+        cmd.addError(formatString("Unknown operation ''{0}'' available ops are ''{1}''", cmd.name,
+            validators.keySet()));
+        continue;
+      } else {
+        List<String> errs = validator.validateJson(cmd.getCommandData());
+        if (errs != null) for (String err : errs) cmd.addError(err);
+      }
+
+    }
+    List<Map> errs = CommandOperation.captureErrors(commandsCopy);
+    if (!errs.isEmpty()) {
+      throw new ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error in command payload", errs);
+    }
+    return commandsCopy;
+  }
+
+  public static class ExceptionWithErrObject extends SolrException {
+    private List<Map> errs;
+
+    public ExceptionWithErrObject(ErrorCode code, String msg, List<Map> errs) {
+      super(code, msg);
+      this.errs = errs;
+    }
+
+    public List<Map> getErrs() {
+      return errs;
+    }
+  }
+
+  public static class LazyLoadedApi extends Api {
+
+    private final PluginBag.PluginHolder<SolrRequestHandler> holder;
+    private Api delegate;
+
+    protected LazyLoadedApi(SpecProvider specProvider, PluginBag.PluginHolder<SolrRequestHandler> lazyPluginHolder) {
+      super(specProvider);
+      this.holder = lazyPluginHolder;
+    }
+
+    @Override
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      if (!holder.isLoaded()) {
+        delegate = new ReqHandlerToApi(holder.get(), ApiBag.EMPTY_SPEC);
+      }
+      delegate.call(req, rsp);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/api/ApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiSupport.java b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
new file mode 100644
index 0000000..ca1e866
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
@@ -0,0 +1,46 @@
+/*
+ * 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.api;
+
+import java.util.Collection;
+
+/**The interface that is implemented by a request handler to support the V2 end point
+ *
+ */
+public interface ApiSupport {
+
+  /**It is possible to support multiple v2 apis by a single requesthandler
+   *
+   * @return the list of v2 api implementations
+   */
+  Collection<Api> getApis();
+
+  /**Whether this should be made available at the regular legacy path
+   */
+  default Boolean registerV1() {
+    return Boolean.TRUE;
+  }
+
+  /**Whether this request handler must be made available at the /v2/ path
+   */
+  default Boolean registerV2() {
+    return Boolean.FALSE;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/api/SpecProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/SpecProvider.java b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
new file mode 100644
index 0000000..c373c99
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
@@ -0,0 +1,25 @@
+
+/*
+ * 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.api;
+import org.apache.solr.common.util.ValidatingJsonMap;
+
+/**A generic interface for any class that is capable of providing its specification as a json schema
+ */
+public interface SpecProvider {
+  ValidatingJsonMap getSpec();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
new file mode 100644
index 0000000..4a053dc
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
@@ -0,0 +1,340 @@
+/*
+ * 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.api;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.servlet.HttpSolrCall;
+import org.apache.solr.servlet.SolrDispatchFilter;
+import org.apache.solr.servlet.SolrRequestParsers;
+import org.apache.solr.util.JsonSchemaValidator;
+import org.apache.solr.util.PathTrie;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.params.CommonParams.JSON;
+import static org.apache.solr.common.params.CommonParams.WT;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.PASSTHROUGH;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.PROCESS;
+import static org.apache.solr.util.PathTrie.getPathSegments;
+
+// class that handle the '/v2' path
+public class V2HttpCall extends HttpSolrCall {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private Api api;
+  List<String> pieces;
+  private String prefix;
+  HashMap<String, String> parts = new HashMap<>();
+  static final Set<String> knownPrefixes = ImmutableSet.of("cluster", "node", "collections", "cores", "c");
+
+  public V2HttpCall(SolrDispatchFilter solrDispatchFilter, CoreContainer cc,
+                    HttpServletRequest request, HttpServletResponse response, boolean retry) {
+    super(solrDispatchFilter, cc, request, response, retry);
+  }
+
+  protected void init() throws Exception {
+    String path = this.path;
+    String fullPath = path = path.substring(3);//strip off '/v2'
+    try {
+      pieces = getPathSegments(path);
+      if (pieces.size() == 0) {
+        prefix = "c";
+        path = "/c";
+      } else {
+        prefix = pieces.get(0);
+      }
+
+      boolean isCompositeApi = false;
+      if (knownPrefixes.contains(prefix)) {
+        api = getApiInfo(cores.getRequestHandlers(), path, req.getMethod(), fullPath, parts);
+        if (api != null) {
+          isCompositeApi = api instanceof CompositeApi;
+          if (!isCompositeApi) {
+            initAdminRequest(path);
+            return;
+          }
+        }
+      }
+
+      if ("c".equals(prefix) || "collections".equals(prefix)) {
+        String collectionName = origCorename = corename = pieces.get(1);
+        DocCollection collection = getDocCollection(collectionName);
+        if (collection == null) {
+           if ( ! path.endsWith(ApiBag.INTROSPECT)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such collection or alias");
+          }
+        } else {
+          boolean isPreferLeader = false;
+          if (path.endsWith("/update") || path.contains("/update/")) {
+            isPreferLeader = true;
+          }
+          core = getCoreByCollection(collection.getName(), isPreferLeader);
+          if (core == null) {
+            //this collection exists , but this node does not have a replica for that collection
+            //todo find a better way to compute remote
+            extractRemotePath(corename, origCorename, 0);
+            return;
+          }
+        }
+      } else if ("cores".equals(prefix)) {
+        origCorename = corename = pieces.get(1);
+        core = cores.getCore(corename);
+      }
+      if (core == null) {
+        log.error(">> path: '" + path + "'");
+        if (path.endsWith(ApiBag.INTROSPECT)) {
+          initAdminRequest(path);
+          return;
+        } else {
+          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no core retrieved for " + corename);
+        }
+      }
+
+      this.path = path = path.substring(prefix.length() + pieces.get(1).length() + 2);
+      Api apiInfo = getApiInfo(core.getRequestHandlers(), path, req.getMethod(), fullPath, parts);
+      if (isCompositeApi && apiInfo instanceof CompositeApi) {
+        ((CompositeApi) this.api).add(apiInfo);
+      } else {
+        api = apiInfo;
+      }
+      MDCLoggingContext.setCore(core);
+      parseRequest();
+
+      if (usingAliases) {
+        processAliases(aliases, collectionsList);
+      }
+
+      action = PROCESS;
+      // we are done with a valid handler
+    } catch (RuntimeException rte) {
+      log.error("Error in init()", rte);
+      throw rte;
+    } finally {
+      if (api == null) action = PASSTHROUGH;
+      if (solrReq != null) solrReq.getContext().put(CommonParams.PATH, path);
+    }
+  }
+
+  private void initAdminRequest(String path) throws Exception {
+    solrReq = SolrRequestParsers.DEFAULT.parse(null, path, req);
+    solrReq.getContext().put(CoreContainer.class.getName(), cores);
+    requestType = AuthorizationContext.RequestType.ADMIN;
+    action = ADMIN;
+  }
+
+  protected void parseRequest() throws Exception {
+    config = core.getSolrConfig();
+    // get or create/cache the parser for the core
+    SolrRequestParsers parser = config.getRequestParsers();
+
+    // With a valid handler and a valid core...
+
+    if (solrReq == null) solrReq = parser.parse(core, path, req);
+  }
+
+  protected DocCollection getDocCollection(String collectionName) {
+    if (!cores.isZooKeeperAware()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Solr not running in cloud mode ");
+    }
+    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
+    DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
+    if (collection == null) {
+      collectionName = corename = lookupAliases(collectionName);
+      collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
+    }
+    return collection;
+  }
+
+  public static Api getApiInfo(PluginBag<SolrRequestHandler> requestHandlers,
+                               String path, String method,
+                               String fullPath,
+                               Map<String, String> parts) {
+    fullPath = fullPath == null ? path : fullPath;
+    Api api = requestHandlers.v2lookup(path, method, parts);
+    if (api == null && path.endsWith(ApiBag.INTROSPECT)) {
+      // the particular http method does not have any ,
+      // just try if any other method has this path
+      api = requestHandlers.v2lookup(path, null, parts);
+    }
+
+    if (api == null) {
+      return getSubPathApi(requestHandlers, path, fullPath, new CompositeApi(null));
+    }
+
+    if (api instanceof ApiBag.IntrospectApi) {
+      final Map<String, Api> apis = new LinkedHashMap<>();
+      for (String m : SolrRequest.SUPPORTED_METHODS) {
+        Api x = requestHandlers.v2lookup(path, m, parts);
+        if (x != null) apis.put(m, x);
+      }
+      api = new CompositeApi(new Api(ApiBag.EMPTY_SPEC) {
+        @Override
+        public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+          String method = req.getParams().get("method");
+          Set<Api> added = new HashSet<>();
+          for (Map.Entry<String, Api> e : apis.entrySet()) {
+            if (method == null || e.getKey().equals(method)) {
+              if (!added.contains(e.getValue())) {
+                e.getValue().call(req, rsp);
+                added.add(e.getValue());
+              }
+            }
+          }
+        }
+      });
+      getSubPathApi(requestHandlers,path, fullPath, (CompositeApi) api);
+    }
+
+
+    return api;
+  }
+
+  private static CompositeApi getSubPathApi(PluginBag<SolrRequestHandler> requestHandlers, String path, String fullPath, CompositeApi compositeApi) {
+
+    String newPath = path.endsWith(ApiBag.INTROSPECT) ? path.substring(0, path.length() - ApiBag.INTROSPECT.length()) : path;
+    Map<String, Set<String>> subpaths = new LinkedHashMap<>();
+
+    getSubPaths(newPath, requestHandlers.getApiBag(), subpaths);
+    final Map<String, Set<String>> subPaths = subpaths;
+    if (subPaths.isEmpty()) return null;
+    return compositeApi.add(new Api(() -> ValidatingJsonMap.EMPTY) {
+      @Override
+      public void call(SolrQueryRequest req1, SolrQueryResponse rsp) {
+        String prefix = null;
+        prefix = fullPath.endsWith(ApiBag.INTROSPECT) ?
+            fullPath.substring(0, fullPath.length() - ApiBag.INTROSPECT.length()) :
+            fullPath;
+        LinkedHashMap<String, Set<String>> result = new LinkedHashMap<>(subPaths.size());
+        for (Map.Entry<String, Set<String>> e : subPaths.entrySet()) {
+          if (e.getKey().endsWith(ApiBag.INTROSPECT)) continue;
+          result.put(prefix + e.getKey(), e.getValue());
+        }
+
+        Map m = (Map) rsp.getValues().get("availableSubPaths");
+        if(m != null){
+          m.putAll(result);
+        } else {
+          rsp.add("availableSubPaths", result);
+        }
+      }
+    });
+  }
+
+  private static void getSubPaths(String path, ApiBag bag, Map<String, Set<String>> pathsVsMethod) {
+    for (SolrRequest.METHOD m : SolrRequest.METHOD.values()) {
+      PathTrie<Api> registry = bag.getRegistry(m.toString());
+      if (registry != null) {
+        HashSet<String> subPaths = new HashSet<>();
+        registry.lookup(path, new HashMap<>(), subPaths);
+        for (String subPath : subPaths) {
+          Set<String> supportedMethods = pathsVsMethod.get(subPath);
+          if (supportedMethods == null) pathsVsMethod.put(subPath, supportedMethods = new HashSet<>());
+          supportedMethods.add(m.toString());
+        }
+      }
+    }
+  }
+
+  public static class CompositeApi extends Api {
+    private LinkedList<Api> apis = new LinkedList<>();
+
+    public CompositeApi(Api api) {
+      super(ApiBag.EMPTY_SPEC);
+      if (api != null) apis.add(api);
+    }
+
+    @Override
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      for (Api api : apis) {
+        api.call(req, rsp);
+      }
+
+    }
+
+    public CompositeApi add(Api api) {
+      apis.add(api);
+      return this;
+    }
+  }
+
+  @Override
+  protected void handleAdmin(SolrQueryResponse solrResp) {
+    api.call(this.solrReq, solrResp);
+  }
+
+  @Override
+  protected void execute(SolrQueryResponse rsp) {
+    try {
+      api.call(solrReq, rsp);
+    } catch (RuntimeException e) {
+      throw e;
+    }
+  }
+
+  @Override
+  protected Object _getHandler() {
+    return api;
+  }
+
+  public Map<String,String> getUrlParts(){
+    return parts;
+  }
+
+  @Override
+  protected QueryResponseWriter getResponseWriter() {
+    String wt = solrReq.getParams().get(WT, JSON);
+    if (core != null) return core.getResponseWriters().get(wt);
+    return SolrCore.DEFAULT_RESPONSE_WRITERS.get(wt);
+  }
+
+  @Override
+  protected ValidatingJsonMap getSpec() {
+    return api == null ? null : api.getSpec();
+  }
+
+  @Override
+  protected Map<String, JsonSchemaValidator> getValidators() {
+    return api == null ? null : api.getCommandSchema();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/api/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/package-info.java b/solr/core/src/java/org/apache/solr/api/package-info.java
new file mode 100644
index 0000000..c3574c7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Commonly used classes for Solr V2 API.
+ */
+package org.apache.solr.api;
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index e6e08f9..ba03ccd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -146,10 +146,16 @@ public class Assign {
   // could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
   public static List<ReplicaCount> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
                                                           String shard, int numberOfNodes,
-                                                          String createNodeSetStr, CoreContainer cc) {
+                                                          Object createNodeSet, CoreContainer cc) {
     DocCollection coll = clusterState.getCollection(collectionName);
     Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
-    List<String> createNodeList = createNodeSetStr  == null ? null: StrUtils.splitSmart(createNodeSetStr, ",", true);
+    List<String> createNodeList = null;
+
+    if (createNodeSet instanceof List) {
+      createNodeList = (List) createNodeSet;
+    } else {
+      createNodeList = createNodeSet == null ? null : StrUtils.splitSmart((String) createNodeSet, ",", true);
+    }
 
      HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, createNodeList);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
index 3d5aa41..52df32b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
@@ -68,7 +68,7 @@ public class CreateShardCmd implements Cmd {
     ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
     DocCollection collection = clusterState.getCollection(collectionName);
     int repFactor = message.getInt(REPLICATION_FACTOR, collection.getInt(REPLICATION_FACTOR, 1));
-    String createNodeSetStr = message.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET);
+    Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
     List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, repFactor,
         createNodeSetStr, ocmh.overseer.getZkController().getCoreContainer());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/core/PluginBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 77e2379..ad8bdec 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -46,10 +46,15 @@ import org.apache.solr.util.SimplePostTool;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.api.ApiBag.HANDLER_NAME;
 
 /**
  * This manages the lifecycle of a set of plugin of the same type .
@@ -63,11 +68,13 @@ public class PluginBag<T> implements AutoCloseable {
   private final Class klass;
   private SolrCore core;
   private final SolrConfig.SolrPluginInfo meta;
+  private final ApiBag apiBag;
 
   /**
    * Pass needThreadSafety=true if plugins can be added and removed concurrently with lookups.
    */
   public PluginBag(Class<T> klass, SolrCore core, boolean needThreadSafety) {
+    this.apiBag = klass == SolrRequestHandler.class ? new ApiBag(core != null) : null;
     this.core = core;
     this.klass = klass;
     // TODO: since reads will dominate writes, we could also think about creating a new instance of a map each time it changes.
@@ -174,16 +181,52 @@ public class PluginBag<T> implements AutoCloseable {
    */
   public T put(String name, T plugin) {
     if (plugin == null) return null;
-    PluginHolder<T> old = put(name, new PluginHolder<T>(null, plugin));
+    PluginHolder<T> pluginHolder = new PluginHolder<>(null, plugin);
+    pluginHolder.registerAPI = false;
+    PluginHolder<T> old = put(name, pluginHolder);
     return old == null ? null : old.get();
   }
 
-
   PluginHolder<T> put(String name, PluginHolder<T> plugin) {
-    PluginHolder<T> old = registry.put(name, plugin);
-    if (plugin.pluginInfo != null && plugin.pluginInfo.isDefault()) {
-      setDefault(name);
+    Boolean registerApi = null;
+    Boolean disableHandler = null;
+    if (plugin.pluginInfo != null) {
+      String registerAt = plugin.pluginInfo.attributes.get("registerPath");
+      if (registerAt != null) {
+        List<String> strs = StrUtils.splitSmart(registerAt, ',');
+        disableHandler = !strs.contains("/");
+        registerApi = strs.contains("/v2");
+      }
+    }
+
+    if (apiBag != null) {
+      if (plugin.isLoaded()) {
+        T inst = plugin.get();
+        if (inst instanceof ApiSupport) {
+          ApiSupport apiSupport = (ApiSupport) inst;
+          if (registerApi == null) registerApi = apiSupport.registerV2();
+          if (disableHandler == null) disableHandler = !apiSupport.registerV1();
+
+          if(registerApi) {
+            Collection<Api> apis = apiSupport.getApis();
+            if (apis != null) {
+              Map<String, String> nameSubstitutes = singletonMap(HANDLER_NAME, name);
+              for (Api api : apis) {
+                apiBag.register(api, nameSubstitutes);
+              }
+            }
+          }
+
+        }
+      } else {
+        if (registerApi != null && registerApi)
+          apiBag.registerLazy((PluginHolder<SolrRequestHandler>) plugin, plugin.pluginInfo);
+      }
     }
+    if(disableHandler == null) disableHandler = Boolean.FALSE;
+    PluginHolder<T> old = null;
+    if(!disableHandler) old = registry.put(name, plugin);
+    if (plugin.pluginInfo != null && plugin.pluginInfo.isDefault()) setDefault(name);
     if (plugin.isLoaded()) registerMBean(plugin.get(), core, name);
     return old;
   }
@@ -249,7 +292,7 @@ public class PluginBag<T> implements AutoCloseable {
     return result.isLoaded();
   }
 
-  private static void registerMBean(Object inst, SolrCore core, String pluginKey) {
+  private void registerMBean(Object inst, SolrCore core, String pluginKey) {
     if (core == null) return;
     if (inst instanceof SolrInfoMBean) {
       SolrInfoMBean mBean = (SolrInfoMBean) inst;
@@ -280,6 +323,7 @@ public class PluginBag<T> implements AutoCloseable {
   public static class PluginHolder<T> implements AutoCloseable {
     private T inst;
     protected final PluginInfo pluginInfo;
+    boolean registerAPI = false;
 
     public PluginHolder(PluginInfo info) {
       this.pluginInfo = info;
@@ -321,7 +365,7 @@ public class PluginBag<T> implements AutoCloseable {
    * A class that loads plugins Lazily. When the get() method is invoked
    * the Plugin is initialized and returned.
    */
-  public static class LazyPluginHolder<T> extends PluginHolder<T> {
+  public class LazyPluginHolder<T> extends PluginHolder<T> {
     private volatile T lazyInst;
     private final SolrConfig.SolrPluginInfo pluginMeta;
     protected SolrException solrException;
@@ -516,4 +560,17 @@ public class PluginBag<T> implements AutoCloseable {
       }
     }
   }
+
+
+  public Api v2lookup(String path, String method, Map<String, String> parts) {
+    if (apiBag == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "this should not happen, looking up for v2 API at the wrong place");
+    }
+    return apiBag.lookup(path, method, parts);
+  }
+
+  public ApiBag getApiBag() {
+    return apiBag;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
index 25b3b14..f5b49ea 100644
--- a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
@@ -22,6 +22,7 @@ import java.lang.invoke.MethodHandles;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
+import java.util.Collection;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -34,6 +35,8 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldDocs;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
@@ -72,7 +75,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
 
   @Override
   public void handleRequestBody(final SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    String httpMethod = (String) req.getContext().get("httpMethod");
+    String httpMethod = req.getHttpMethod();
     String path = (String) req.getContext().get("path");
     SolrConfigHandler.setWt(req, JSON);
 
@@ -277,4 +280,13 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
     req.getCore().getRequestHandler(handler).handleRequest(r, rsp);
   }
 
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ApiBag.wrapRequestHandlers(this, "core.system.blob", "core.system.blob.upload");
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/DumpRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/DumpRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/DumpRequestHandler.java
index ecafb52..d7d5b71 100644
--- a/solr/core/src/java/org/apache/solr/handler/DumpRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/DumpRequestHandler.java
@@ -19,7 +19,9 @@ package org.apache.solr.handler;
 import java.io.IOException;
 import java.io.Reader;
 import java.util.ArrayList;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.util.ContentStream;
@@ -39,6 +41,15 @@ public class DumpRequestHandler extends RequestHandlerBase
   {
     // Show params
     rsp.add( "params", req.getParams().toNamedList() );
+    String[] parts = req.getParams().getParams("urlTemplateValues");
+    if (parts != null && parts.length > 0) {
+      Map map = new LinkedHashMap<>();
+      rsp.getValues().add("urlTemplateValues", map);
+      for (String part : parts) {
+        map.put(part, req.getPathTemplateValues().get(part));
+      }
+    }
+
     String[] returnParams = req.getParams().getParams("param");
     if(returnParams !=null) {
       NamedList params = (NamedList) rsp.getValues().get("params");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
index 04b930a..8230bf5 100644
--- a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
@@ -330,6 +330,11 @@ public class PingRequestHandler extends RequestHandlerBase implements SolrCoreAw
   }
 
   @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
+
+  @Override
   public Category getCategory() {
     return Category.ADMIN;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java b/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
index 6c9b0a9..9049318 100644
--- a/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
@@ -16,12 +16,16 @@
  */
 package org.apache.solr.handler;
 
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.handler.component.*;
 
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
+
 public class RealTimeGetHandler extends SearchHandler {
   @Override
   protected List<String> getDefaultComponents()
@@ -42,6 +46,16 @@ public class RealTimeGetHandler extends SearchHandler {
   public URL[] getDocs() {
     return null;
   }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ApiBag.wrapRequestHandlers(this, "core.RealtimeGet");
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index b70c096..3c6f5fa 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -18,7 +18,9 @@ package org.apache.solr.handler;
 
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
+import java.util.Collection;
 
+import com.google.common.collect.ImmutableList;
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Timer;
@@ -37,6 +39,9 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,7 +51,7 @@ import static org.apache.solr.core.RequestParams.USEPARAM;
 /**
  *
  */
-public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, SolrMetricProducer, NestedRequestHandler {
+public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, SolrMetricProducer, NestedRequestHandler,ApiSupport {
 
   protected NamedList initArgs = null;
   protected SolrParams defaults;
@@ -290,6 +295,11 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
     MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ImmutableList.of(new ApiBag.ReqHandlerToApi(this, ApiBag.constructSpec(pluginInfo)));
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index 9c2d45c..f3e503e 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -19,17 +19,19 @@ package org.apache.solr.handler;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
@@ -86,15 +88,12 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
         return;
       }
 
-      for (ContentStream stream : req.getContentStreams()) {
-        try {
-          List errs = new SchemaManager(req).performOperations(stream.getReader());
-          if (!errs.isEmpty()) rsp.add("errors", errs);
-        } catch (IOException e) {
-          rsp.add("errors", Collections.singletonList("Error reading input String " + e.getMessage()));
-          rsp.setException(e);
-        }
-        break;
+      try {
+        List errs = new SchemaManager(req).performOperations();
+        if (!errs.isEmpty()) rsp.add("errors", errs);
+      } catch (IOException e) {
+        rsp.add("errors", Collections.singletonList("Error reading input String " + e.getMessage()));
+        rsp.setException(e);
       }
     } else {
       handleGET(req, rsp);
@@ -260,4 +259,20 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
   public void inform(SolrCore core) {
     isImmutableConfigSet = SolrConfigHandler.getImmutable(core);
   }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ApiBag.wrapRequestHandlers(this, "core.SchemaRead",
+        "core.SchemaRead.fields",
+        "core.SchemaRead.copyFields",
+        "core.SchemaEdit",
+        "core.SchemaRead.dynamicFields_fieldTypes"
+        );
+
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 1c584b1..2660cba 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -76,6 +76,8 @@ import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -895,4 +897,18 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       return null;
     }
   }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ApiBag.wrapRequestHandlers(this,
+        "core.config",
+        "core.config.Commands",
+        "core.config.Params",
+        "core.config.Params.Commands");
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
index 6628368..fd7a754 100644
--- a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
@@ -150,6 +150,7 @@ public class UpdateRequestHandler extends ContentStreamHandlerBase implements Pe
     pathVsLoaders.put(JSON_PATH,registry.get("application/json"));
     pathVsLoaders.put(DOC_PATH,registry.get("application/json"));
     pathVsLoaders.put(CSV_PATH,registry.get("application/csv"));
+    pathVsLoaders.put(BIN_PATH,registry.get("application/javabin"));
     return registry;
   }
 
@@ -178,6 +179,7 @@ public class UpdateRequestHandler extends ContentStreamHandlerBase implements Pe
   public static final String DOC_PATH = "/update/json/docs";
   public static final String JSON_PATH = "/update/json";
   public static final String CSV_PATH = "/update/csv";
+  public static final String BIN_PATH = "/update/bin";
 
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
new file mode 100644
index 0000000..6ba3229
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+
+public class UpdateRequestHandlerApi extends UpdateRequestHandler  {
+
+
+  @Override
+  public Collection<Api> getApis() {
+    return Collections.singleton(getApiImpl());
+  }
+
+  private Api getApiImpl() {
+    return new Api(ApiBag.getSpec("core.Update")) {
+      @Override
+      public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+        String path = req.getPath();
+        String target =  mapping.get(path);
+        if(target != null) req.getContext().put("path", target);
+        try {
+          handleRequest(req, rsp);
+        } catch (RuntimeException e) {
+          throw e;
+        } catch (Exception e){
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,e );
+        }
+      }
+    };
+  }
+
+  @Override
+  public Boolean registerV1() {
+    return Boolean.FALSE;
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
+
+  private static final Map<String, String> mapping = ImmutableMap.<String,String>builder()
+      .put("/update", DOC_PATH)
+      .put(JSON_PATH, DOC_PATH)
+      .put("/update/json/commands", JSON_PATH)
+      .build();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..0e58ccc
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+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;
+import static org.apache.solr.common.util.StrUtils.splitSmart;
+
+/**
+ * This is a utility class to provide an easy mapping of request handlers which support multiple commands
+ * to the V2 API format (core admin api, collections api). This helps in automatically mapping paths
+ * to actions and old parameter names to new parameter names
+ */
+public abstract class BaseHandlerApiSupport implements ApiSupport {
+  protected final Map<SolrRequest.METHOD, Map<V2EndPoint, List<ApiCommand>>> commandsMapping;
+
+  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<>());
+      list.add(cmd);
+    }
+  }
+
+  @Override
+  public synchronized Collection<Api> getApis() {
+    ImmutableList.Builder<Api> l = ImmutableList.builder();
+    for (V2EndPoint op : getEndPoints()) l.add(getApi(op));
+    return l.build();
+  }
+
+
+  private Api getApi(final V2EndPoint op) {
+    final BaseHandlerApiSupport apiHandler = this;
+    return new Api(ApiBag.getSpec(op.getSpecName())) {
+      @Override
+      public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+        SolrParams params = req.getParams();
+        SolrRequest.METHOD method = SolrRequest.METHOD.valueOf(req.getHttpMethod());
+        List<ApiCommand> commands = commandsMapping.get(method).get(op);
+        try {
+          if (method == POST) {
+            List<CommandOperation> cmds = req.getCommands(true);
+            if (cmds.size() > 1)
+              throw new SolrException(BAD_REQUEST, "Only one command is allowed");
+            CommandOperation c = cmds.size() == 0 ? null : cmds.get(0);
+            ApiCommand command = null;
+            String commandName = c == null ? null : c.name;
+            for (ApiCommand cmd : commands) {
+              if (Objects.equals(cmd.getName(), commandName)) {
+                command = cmd;
+                break;
+              }
+            }
+
+            if (command == null) {
+              throw new SolrException(BAD_REQUEST, " no such command " + c);
+            }
+            wrapParams(req, c, command, false);
+            command.invoke(req, rsp, apiHandler);
+
+          } else {
+            if (commands == null || commands.isEmpty()) {
+              rsp.add("error", "No support for : " + method + " at :" + req.getPath());
+              return;
+            }
+            if (commands.size() > 1) {
+              for (ApiCommand command : commands) {
+                if (command.getName().equals(req.getPath())) {
+                  commands = Collections.singletonList(command);
+                  break;
+                }
+              }
+            }
+            wrapParams(req, new CommandOperation("", Collections.EMPTY_MAP), commands.get(0), true);
+            commands.get(0).invoke(req, rsp, apiHandler);
+          }
+
+        } catch (SolrException e) {
+          throw e;
+        } catch (Exception e) {
+          throw new SolrException(BAD_REQUEST, e);
+        } finally {
+          req.setParams(params);
+        }
+
+      }
+    };
+
+  }
+
+  private static void wrapParams(final SolrQueryRequest req, final CommandOperation co, final ApiCommand cmd, final boolean useRequestParams) {
+    final Map<String, String> pathValues = req.getPathTemplateValues();
+    final Map<String, Object> map = co == null || !(co.getCommandData() instanceof Map) ?
+        Collections.singletonMap("", co.getCommandData()) : co.getDataMap();
+    final SolrParams origParams = req.getParams();
+
+    req.setParams(
+        new SolrParams() {
+          @Override
+          public String get(String param) {
+            Object vals = getParams0(param);
+            if (vals == null) return null;
+            if (vals instanceof String) return (String) vals;
+            if (vals instanceof Boolean || vals instanceof Number) return String.valueOf(vals);
+            if (vals instanceof String[] && ((String[]) vals).length > 0) return ((String[]) vals)[0];
+            return null;
+          }
+
+          private Object getParams0(String param) {
+            param = cmd.getParamSubstitute(param);
+            Object o = param.indexOf('.') > 0 ?
+                Utils.getObjectByPath(map, true, splitSmart(param, '.')) :
+                map.get(param);
+            if (o == null) o = pathValues.get(param);
+            if (o == null && useRequestParams) o = origParams.getParams(param);
+            if (o instanceof List) {
+              List l = (List) o;
+              return l.toArray(new String[l.size()]);
+            }
+
+            return o;
+          }
+
+          @Override
+          public String[] getParams(String param) {
+            Object vals = getParams0(param);
+            return vals == null || vals instanceof String[] ?
+                (String[]) vals :
+                new String[]{vals.toString()};
+          }
+
+          @Override
+          public Iterator<String> getParameterNamesIterator() {
+            return cmd.getParamNames(co).iterator();
+
+          }
+
+
+        });
+
+  }
+
+
+  public static Collection<String> getParamNames(CommandOperation op, ApiCommand 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());
+      }
+    }
+  }
+
+  protected abstract List<ApiCommand> getCommands();
+
+  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;
+    }
+
+    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/71abe130/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
new file mode 100644
index 0000000..581fe46
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import 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.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;
+
+  public CollectionHandlerApi(CollectionsHandler handler) {
+    this.handler = handler;
+  }
+
+  @Override
+  protected List<ApiCommand> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(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),
+    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) {
+      @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)
+
+    ;
+    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);
+
+    }
+
+    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;
+      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 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_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;
+    }
+  }
+
+}


[2/5] lucene-solr:master: SOLR-8029: Added new style APIs and a framework for creating new APIs and mapping old APIs to new

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Commands.json b/solr/core/src/resources/apispec/core.config.Commands.json
new file mode 100644
index 0000000..256306b
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Commands.json
@@ -0,0 +1,215 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
+  "description": "The Config API enables manipulating various aspects of your solrconfig.xml using REST-like API calls. All properties set with this API update a file called configoverlay.json, but not the solrconfig.xml file itself.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/config"
+    ]
+  },
+  "commands": {
+    "set-property:": {
+      "type": "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API#ConfigAPI-Commandstomodifytheconfig",
+      "description": "Sets one or more of several pre-defined properties. These properties set cache sizes and classes, commit rules, JMX settings, and request dispatcher settings. See the documentation for the list of properties that are supported. If a property is set that already exists, it will be overwritten.",
+      "additionalProperties": true
+    },
+    "unset-property": {
+      "type":"array",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API#ConfigAPI-Commandstomodifytheconfig",
+      "description": "Removes one or more of several pre-defined properties. These properties set cache sizes and classes, commit rules, JMX settings, and request dispatcher settings. See the documentation for the list of properties that are supported. The value of the property does not need to be defined with the list of properties, only the name of the property.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-requesthandler": {
+      "#include":"core.config.Commands.addRequestHandler.properties",
+      "required": [
+        "name",
+        "class"
+      ]
+    },
+    "update-requesthandler": {
+      "#include":"core.config.Commands.addRequestHandler.properties",
+      "required": [
+        "name"
+      ]
+    },
+    "delete-requesthandler": {
+      "type": "array",
+      "description": "Deletes one or more request handlers, using the name given when the request handler was created. Define more than one request handler by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-searchcomponent": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-searchcomponent": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-searchcomponent": {
+      "type": "array",
+      "description": "Deletes one or more search components, using the name given when the search component was created. Define more than one search component by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-initparams": {
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "Name by which it is added, so that it can be updated by name"
+        }
+      },
+      "additionalProperties": true
+    },
+    "update-initparams": {
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "Name by which it is added"
+        }
+      },
+      "required": [
+        "name"
+      ],
+      "additionalProperties": true
+    },
+    "delete-initparams": {
+      "type": "array",
+      "description": "Deletes one or more init params, using the name given when the init param set was created. Define more than one init params by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-queryresponsewriter": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-queryresponsewriter": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-queryresponsewriter": {
+      "type": "array",
+      "description": "Deletes one or more query response writers, using the name given when the response writer was created. Define more than one response writer by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-queryparser": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-queryparser": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-queryparser": {
+      "type": "array",
+      "items": {
+        "type": "string"
+      },
+      "description": "Deletes one or more query parsers, using the name given when the query parser was created. Define more than one query parser by separating the list of names with commas."
+    },
+    "add-valuesourceparser": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-valuesourceparser": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-valuesourceparser": {
+      "type": "array",
+      "description": "Deletes one or more ValueSourceParsers, using the name given when the ValueSourceParser was created. Define more than one ValueSourceParsers by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-transformer": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-transformer": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-transformer": {
+      "type": "array",
+      "description": "Deletes one or more document transformers, using the name given when the document transformer was created. Define more than one document transformers by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-updateprocessor": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-updateprocessor": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-updateprocessor": {
+      "type": "array",
+      "description": "Deletes one or more update processors, using the name given when the update processor was created. Define more than one update processors by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-queryconverter": {
+      "#include": "core.config.Commands.generic"
+    },
+    "update-queryconverter": {
+      "#include": "core.config.Commands.generic"
+    },
+    "delete-queryconverter": {
+      "type": "array",
+      "description": "Deletes one or more query converters, using the name given when the query converter was created. Define more than one query converters by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-listener": {
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "Name by which it is added, so that it can be updated by name"
+        }
+      },
+      "required": [
+        "name"
+      ],
+      "additionalProperties": true
+    },
+    "update-listener": {
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "Name by which it is added"
+        }
+      },
+      "required": [
+        "name"
+      ],
+      "additionalProperties": true
+    },
+    "delete-listener": {
+      "type": "array",
+      "description": "Deletes one or more listeners, using the name given when the listener was created. Define more than one listener by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "add-runtimelib": {
+      "#include": "core.config.Commands.runtimeLib"
+    },
+    "update-runtimelib": {
+      "#include": "core.config.Commands.runtimeLib"
+    },
+    "delete-runtimelib": {
+      "type":"array",
+      "description": "Deletes one or more runtime libraries (runtimeLibs), using the name given when the runtimeLib was created. Define more than one runtimeLibs by separating the list of names with commas.",
+      "items": {
+        "type": "string"
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Commands.runtimeLib.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Commands.runtimeLib.json b/solr/core/src/resources/apispec/core.config.Commands.runtimeLib.json
new file mode 100644
index 0000000..8e2fb2d
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Commands.runtimeLib.json
@@ -0,0 +1,23 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Adding+Custom+Plugins+in+SolrCloud+Mode",
+  "description": "Allows you to register .jars that have been uploaded to the .system collection in Solr. Note that uploading the .jar must occur before using this API.",
+  "type": "object",
+  "properties": {
+    "name": {
+      "description": "The name of the .jar blob in .system collection. This is the name you provided when you uploaded it.",
+      "type": "string"
+    },
+    "version": {
+      "type": "integer",
+      "description": "The version of the blob in .system collection. Be sure to use the correct version if you have multiple versions of the same .jar uploaded."
+    },
+    "sig": {
+      "type": "string",
+      "description": "The sha1 signature of the .jar, if it was signed before uploading. If you signed the sha1 digest of your .jar file prior to uploading it to the .system collection, this is where you need to provide the signature."
+    }
+  },
+  "required": [
+    "name",
+    "version"
+  ]
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Params.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Params.Commands.json b/solr/core/src/resources/apispec/core.config.Params.Commands.json
new file mode 100644
index 0000000..474f1a6
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Params.Commands.json
@@ -0,0 +1,31 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Request+Parameters+API",
+  "description": "Create, update and delete request parameter sets (paramsets) to override or replace parameters defined in solrconfig.xml. Parameter sets are used with request handlers by setting the useParams attribute to the paramset name in the definition of the request handler or with individual requests to Solr. Parameter sets defined with this API are stored in a file params.json in ZooKeeper or on the filesystem when not using SolrCloud. Note this API does not directly update solrconfig.xml. ",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/config/params"
+    ]
+  },
+  "commands": {
+    "set:": {
+      "type":"object",
+      "description":"Add or overwrite one or more paramsets. Each paramset definition includes a paramset name, followed by key-value pairs of the parameter and value to be set.",
+      "additionalProperties": true
+    },
+    "unset": {
+      "type":"array",
+      "description": "Delete one or more paramsets.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "update": {
+      "type":"object",
+      "description": "Update one or more paramsets. This command will attempt to merge an existing paramset with the new values. Each paramset definition includes a paramset name, followed by key-value pairs of the parameters and values to be updated.",
+      "additionalProperties": true
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.Params.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.Params.json b/solr/core/src/resources/apispec/core.config.Params.json
new file mode 100644
index 0000000..cff0350
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.Params.json
@@ -0,0 +1,13 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Request+Parameters+API",
+  "description": "List all parameter sets (paramsets). Individual paramsets can be requested by paramset name.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/config/params",
+      "/config/params/{params_set}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.config.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.config.json b/solr/core/src/resources/apispec/core.config.json
new file mode 100644
index 0000000..2633fd9
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.config.json
@@ -0,0 +1,18 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
+  "description": "Gets the Solr configuration for a collection.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/config",
+      "/config/overlay",
+      "/config/query",
+      "/config/jmx",
+      "/config/requestDispatcher",
+      "/config/znodeVersion",
+      "/config/{plugin}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.system.blob.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.system.blob.json b/solr/core/src/resources/apispec/core.system.blob.json
new file mode 100644
index 0000000..96fedcf
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.system.blob.json
@@ -0,0 +1,20 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Blob+Store+API",
+  "description": "Lists blobs in the blob store (the .system collection). The list can be limited by name or name and version.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/blob",
+      "/blob/{name}",
+      "/blob/{name}/{version}"
+    ],
+    "params": {
+      "wt": {
+        "type":"string",
+        "description": "Use the value 'filestream' to get the file content. Use other response writers (such as xml, or json) to fetch only the metadata."
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/core.system.blob.upload.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.system.blob.upload.json b/solr/core/src/resources/apispec/core.system.blob.upload.json
new file mode 100644
index 0000000..854e544
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.system.blob.upload.json
@@ -0,0 +1,12 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/Blob+Store+API",
+  "description": "Uploads a blob to the blob store. Note that the blob store is a specially named collection (which must be '.system') which must be created before uploading a blob to it.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/blob/{name}"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cores.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.Commands.json b/solr/core/src/resources/apispec/cores.Commands.json
new file mode 100644
index 0000000..d6419cf
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.Commands.json
@@ -0,0 +1,85 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API",
+  "description": "Actions on non-specific cores. See the /cores/{core} endpoint for actions on specific, named, cores.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cores"
+    ]
+  },
+  "commands": {
+    "create": {
+      "type" : "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-CREATE",
+      "description": "Creates a new core. If you are using SolrCloud, you should use the Collections API instead. While a core for a SolrCloud cluster can be created with this API, it is considered an expert-level action. The Collections API has commands for creating new shards and replicas that ensure the safety of those actions within your cluster.",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The core name to create. If a core with this name already exists, an error will be returned."
+        },
+        "instanceDir": {
+          "type": "string",
+          "description": "The core instance directory, where files for this core should be stored. While this parameter is not required, if it is not defined it will default to a path relative to Solr Home that includes the name you've given the new core. This location MUST EXIST prior to creating the core, and it must include a conf directory that includes solrconfig.xml and your schema, either as a schema.xml file or using the managed schema feature."
+        },
+        "schema": {
+          "type": "string",
+          "description": "Name of the schema file to use for the core. Please note that if you are using a 'managed schema' (Solr's default behavior) then any value for this property that does not match the effective managedSchemaResourceName will be read once, backed up, and converted for managed schema use. If you are using the default name (schema.xml or the managed schema name), you do not need to define the schema file name."
+        },
+        "dataDir": {
+          "type": "string",
+          "description": "Name of the data directory relative to instanceDir. This is where the index files will be stored."
+        },
+        "config": {
+          "type": "string",
+          "description": "Name of the config file (i.e., solrconfig.xml) relative to instanceDir. If you are using the default name (solrconfig.xml), you do not need to define the config file name."
+        },
+        "configSet": {
+          "type": "string",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+Sets",
+          "description": "The name of a config set to use. The config set must already exist. The solr.xml file defines the location of the configset base directory, and configuration files can be shared between cores by defining sub-directories. The files in the named configSet will be used for the schema and config properties instead of defining them explicitly."
+        },
+        "loadOnStartup": {
+          "type": "boolean",
+          "description": "If true, the core will be loaded on startup. Set to false to enable lazy loading, where the core will only be loaded if it is referenced or called.",
+          "default": "true"
+        },
+        "transient": {
+          "type": "boolean",
+          "description": "Allows Solr to unload the core if resources are required.",
+          "default": "false"
+        },
+        "shard": {
+          "type": "string",
+          "description": "In SolrCloud mode, the shard this core should belong to."
+        },
+        "collection": {
+          "type": "string",
+          "description": "The name of the collection this core belongs to."
+        },
+        "props": {
+          "type": "object",
+          "documentation": "https://cwiki.apache.org/confluence/display/solr/Defining+core.properties",
+          "description": "Allows adding core.properties for the collection.",
+          "additionalProperties": true
+        },
+        "coreNodeName": {
+          "type": "string",
+          "description": "The replica name."
+        },
+        "numShards": {
+          "type":"number",
+          "description":"The number of shards to create for this core."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      },
+      "required": [
+        "name"
+      ]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cores.Status.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.Status.json b/solr/core/src/resources/apispec/cores.Status.json
new file mode 100644
index 0000000..155bcf5
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.Status.json
@@ -0,0 +1,20 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-STATUS",
+  "description": "Provides status and other information about the status of each core. Individual cores can be requested by core name.",
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/cores",
+      "/cores/{core}"
+    ],
+    "params": {
+      "indexInfo": {
+        "type": "boolean",
+        "description": "If true, index information will be returned, such as information about number of documents, deletions, segments, etc. In a large cluster with more than hundreds of cores, this can take a long time to retrieve. If you have a large cluster, consider setting this to false.",
+        "default": true
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cores.core.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.core.Commands.json b/solr/core/src/resources/apispec/cores.core.Commands.json
new file mode 100644
index 0000000..5049a3a
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.core.Commands.json
@@ -0,0 +1,136 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API",
+  "description": "Actions that are peformed on individual cores, such as reloading, swapping cores, renaming, and others.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cores/{core}"
+    ]
+  },
+  "commands": {
+    "reload": {
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-RELOAD",
+      "description": "Reloads a core. This is useful when you have made changes on disk such as editing the schema or solrconfig.xml files. Most APIs reload cores automatically, so this should not be necessary if changes were made with those APIs."
+    },
+    "swap": {
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-SWAP",
+      "description": "Swaps the names of two existing Solr cores. This can be used to swap new content into production. The former core can be swapped back if necessary. Using this API is not supported in SolrCloud mode.",
+      "properties": {
+        "with": {
+          "type": "string",
+          "description": "The name of the other core to be swapped (the first core name is included in the request)."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      },
+      "required": [
+        "with"
+      ]
+    },
+    "rename": {
+      "type": "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-RENAME",
+      "description": "Change the name of a core.",
+      "properties": {
+        "to": {
+          "type": "string",
+          "description": "The new name for the core."
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      },
+      "required": [
+        "to"
+      ]
+    },
+    "unload": {
+      "type": "object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-UNLOAD",
+      "description": "Removes a core. Active requests would continue to be processed, but new requests will not be sent to the new core. If a core is registered under more than one name, only the name given in the request is removed.",
+      "properties": {
+        "deleteIndex": {
+          "type": "boolean",
+          "description": "If true, the index will be removed while unloading the core.",
+          "default": "false"
+        },
+        "deleteDataDir": {
+          "type": "boolean",
+          "description": "If true, the data directory and all sub-directories will be removed while unloading the core.",
+          "default": "false"
+        },
+        "deleteInstanceDir": {
+          "type": "boolean",
+          "description": "If true, everything related to the core, including the index, data, and conf directories, will be removed while unloading the core.",
+          "default": "false"
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      }
+    },
+    "merge-indexes": {
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-MERGEINDEXES",
+      "description":"Merges one or more indexes to another index. The indexes must have completed commits, and should be locked against writes until the merge is complete to avoid index corruption. The target core (which is the core that should be used as the endpoint for this command) must exist before using this command. A commit should also be performed on this core after the merge is complete.",
+      "properties": {
+        "indexDir": {
+          "type": "array",
+          "description": "A comma-separated list index directories for each source core that will be merged with the target core.",
+          "items": {
+            "type": "string"
+          }
+        },
+        "srcCore": {
+          "type": "array",
+          "description": "A comma-separated list of the names of each source core to be merged with the target core.",
+          "items": {
+            "type": "string"
+          }
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        }
+      }
+    },
+    "split":  { "#include": "cores.core.Commands.split"},
+    "request-recovery": {
+      "type":"object",
+      "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-REQUESTRECOVERY",
+      "description": "Manually asks a core to recover by synching with a leader. It may help SolrCloud clusters where a node refuses to come back up. However, it is considered an expert-level command, and should be used very carefully."
+    },
+    "force-prepare-for-leadership": {
+      "type": "object",
+      "description": "An internal API used by the Collections API to force leader election. This should not be used directly by end-users."
+    },
+    "prep-recovery": {
+      "type": "object",
+      "additionalProperties": true,
+      "description": "An internal API used by the Collections API. This should not be used directly by end-users."
+    },
+    "request-apply-updates": {
+      "type": "object",
+      "additionalProperties": true,
+      "description": "An internal API used by the Collections API. This should not be used directly by end-users."
+    },
+    "request-sync-shard": {
+      "type": "object",
+      "additionalProperties": true,
+      "description": "An internal API used by the Collections API. This should not be used directly by end-users."
+    },
+    "request-buffer-updates": {
+      "type": "object",
+      "additionalProperties": true,
+      "description": "An internal API used by the Collections API. This should not be used directly by end-users."
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/cores.core.Commands.split.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.core.Commands.split.json b/solr/core/src/resources/apispec/cores.core.Commands.split.json
new file mode 100644
index 0000000..3aa0a91
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.core.Commands.split.json
@@ -0,0 +1,34 @@
+{
+  "documentation": "https://cwiki.apache.org/confluence/display/solr/CoreAdmin+API#CoreAdminAPI-SPLIT",
+  "description": "Allows splitting an index into two or more new indexes.",
+  "type": "object",
+  "properties": {
+    "path": {
+      "type": "array",
+      "description": "Directory path(s) in which a piece of the index will be written. This allows splitting the index before creating the cores to contain them. Note if using this approach that the indexes will not be able to receive updates until a new core has been created to handle the incoming updates. If you have already created the new cores, you should define the targetCore property instead.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "targetCore": {
+      "type": "array",
+      "description": "The target Solr core(s) to which a piece of the index will be merged (if the target core already contains data). This requires that the cores have already been created. If the cores have not yet been created, use the path property instead.",
+      "items": {
+        "type": "string"
+      }
+    },
+    "splitKey": {
+      "type":"string",
+      "description": "A route key to use for splitting the index. This parameter is optional, but should not be defined if the ranges parameter is also defined."
+    },
+    "ranges": {
+      "type": "string",
+      "description": "A comma-separated list of hexadecimal hash ranges that will be used to split the core. This parameter is optional, but should not be defined if the splitKey parameter is also defined."
+    },
+    "async": {
+      "type": "string",
+      "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined. This command can be long-running, so running it asynchronously is recommended."
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/emptySpec.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/emptySpec.json b/solr/core/src/resources/apispec/emptySpec.json
new file mode 100644
index 0000000..d95bff9
--- /dev/null
+++ b/solr/core/src/resources/apispec/emptySpec.json
@@ -0,0 +1,11 @@
+{
+  "methods": [
+    "GET",
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "$handlerName"
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/node.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/node.Commands.json b/solr/core/src/resources/apispec/node.Commands.json
new file mode 100644
index 0000000..11b3c89
--- /dev/null
+++ b/solr/core/src/resources/apispec/node.Commands.json
@@ -0,0 +1,24 @@
+{
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/node"
+    ]
+  },
+  "commands": {
+    "overseer-op": {
+      "type": "object",
+      "additionalProperties": true
+    },
+    "rejoin-leader-election": {
+      "type": "object",
+      "additionalProperties": true
+    },
+    "invoke":{
+      "type": "object",
+      "additionalProperties": true
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/node.Info.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/node.Info.json b/solr/core/src/resources/apispec/node.Info.json
new file mode 100644
index 0000000..e7752e6
--- /dev/null
+++ b/solr/core/src/resources/apispec/node.Info.json
@@ -0,0 +1,11 @@
+{
+   "description": "Provides information about system properties, threads, logging settings, and system details for a node.",
+  "methods": ["GET"],
+  "url": {
+    "paths": [
+      "/node/properties",
+      "/node/threads",
+      "/node/logging",
+      "/node/system"]
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/resources/apispec/node.invoke.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/node.invoke.json b/solr/core/src/resources/apispec/node.invoke.json
new file mode 100644
index 0000000..c8a9f69
--- /dev/null
+++ b/solr/core/src/resources/apispec/node.invoke.json
@@ -0,0 +1,16 @@
+{
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/node/invoke"
+    ],
+    "params": {
+      "class": {
+        "type": "string",
+        "description": "Name of the class that must be invoked. "
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
index 467deca..31bbbb3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
@@ -40,7 +40,7 @@
   </requestHandler>
 
 
-  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a">
+  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a" registerPath="/,/v2">
     <lst name="defaults">
       <str name="a">${my.custom.variable.a:A}</str>
       <str name="b">${my.custom.variable.b:B}</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/api/TestPathTrie.java b/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
new file mode 100644
index 0000000..d4cbf32
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
@@ -0,0 +1,61 @@
+/*
+ * 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.api;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.util.PathTrie;
+
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.api.ApiBag.HANDLER_NAME;
+
+public class TestPathTrie extends SolrTestCaseJ4 {
+
+  public void testPathTrie() {
+    PathTrie<String> pathTrie = new PathTrie<>(ImmutableSet.of("_introspect"));
+    pathTrie.insert("/", emptyMap(), "R");
+    pathTrie.insert("/aa", emptyMap(), "d");
+    pathTrie.insert("/aa/bb/{cc}/dd", emptyMap(), "a");
+    pathTrie.insert("/$handlerName/{cc}/dd", singletonMap(HANDLER_NAME, "test"), "test");
+    pathTrie.insert("/aa/bb/{cc}/{xx}", emptyMap(), "b");
+    pathTrie.insert("/aa/bb", emptyMap(), "c");
+
+    HashMap templateValues = new HashMap<>();
+    assertEquals("R", pathTrie.lookup("/", templateValues, null));
+    assertEquals("d", pathTrie.lookup("/aa", templateValues, null));
+    assertEquals("a", pathTrie.lookup("/aa/bb/hello/dd", templateValues, null));
+    templateValues.clear();
+    assertEquals("test", pathTrie.lookup("/test/hello/dd", templateValues, null));
+    assertEquals("hello", templateValues.get("cc"));
+    templateValues.clear();
+    assertEquals("b", pathTrie.lookup("/aa/bb/hello/world", templateValues, null));
+    assertEquals("hello", templateValues.get("cc"));
+    assertEquals("world", templateValues.get("xx"));
+    Set<String> subPaths =  new HashSet<>();
+    templateValues.clear();
+    pathTrie.lookup("/aa",templateValues, subPaths);
+    assertEquals(3, subPaths.size());
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
index 6955418..13649e1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
@@ -22,10 +22,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -36,6 +38,7 @@ import org.junit.rules.ExpectedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+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.COLLECTIONS_HANDLER_PATH;
 import static org.junit.matchers.JUnitMatchers.containsString;
@@ -161,6 +164,21 @@ public class RulesTest extends SolrCloudTestCase {
 
   }
 
+  @Test
+  public void testInvokeApi() throws Exception {
+    JettySolrRunner jetty = cluster.getRandomJetty(random());
+    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString())) {
+      GenericSolrRequest req =  new GenericSolrRequest(GET, "/v2/node/invoke", new ModifiableSolrParams()
+          .add("class", ImplicitSnitch.class.getName())
+          .add("cores", "1")
+          .add("freedisk", "1")
+      );
+      SimpleSolrResponse rsp = req.process(client);
+      assertNotNull(((Map) rsp.getResponse().get(ImplicitSnitch.class.getName())).get("cores"));
+      assertNotNull(((Map) rsp.getResponse().get(ImplicitSnitch.class.getName())).get("freedisk"));
+    }
+  }
+
 
   @Test
   public void testModifyColl() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java b/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
index ff5e1cb..dda1eb3 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
@@ -58,4 +58,5 @@ public class BlobStoreTestRequestHandler extends DumpRequestHandler implements R
     run();
 
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 004039c..695e869 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -112,6 +112,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
       ++ihCount; assertEquals(pathToClassMap.get("/analysis/document"), "solr.DocumentAnalysisRequestHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/analysis/field"), "solr.FieldAnalysisRequestHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/debug/dump"), "solr.DumpRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("update"), "solr.UpdateRequestHandlerApi");
     }
     assertEquals("wrong number of implicit handlers", ihCount, implicitHandlers.size());
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
index bd20b1e..8479ae4 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
@@ -88,7 +88,7 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
 
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/test1', 'class': 'org.apache.solr.core.BlobStoreTestRequestHandler' , 'runtimeLib' : true }\n" +
+        "'create-requesthandler' : { 'name' : '/test1', 'class': 'org.apache.solr.core.BlobStoreTestRequestHandler' ,registerPath: '/,/v2',  'runtimeLib' : true }\n" +
         "}";
 
     client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index c182495..021efac 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -36,12 +36,14 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.handler.DumpRequestHandler;
 import org.apache.solr.handler.TestBlobHandler;
 import org.apache.solr.handler.TestSolrConfigHandlerConcurrent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrCache;
+import org.apache.solr.util.RESTfulServerProvider;
 import org.apache.solr.util.RestTestBase;
 import org.apache.solr.util.RestTestHarness;
 import org.eclipse.jetty.servlet.ServletHolder;
@@ -82,6 +84,10 @@ public class TestSolrConfigHandler extends RestTestBase {
 
     createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
         "/solr", true, extraServlets);
+    if (random().nextBoolean()) {
+      log.info("These tests are run with V2 API");
+      restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME);
+    }
   }
 
   @After
@@ -451,7 +457,7 @@ public class TestSolrConfigHandler extends RestTestBase {
 
     map = getRespMap("/dump100?wt=json&json.nl=arrmap&initArgs=true", writeHarness);
     List initArgs = (List) map.get("initArgs");
-    assertEquals(2, initArgs.size());
+    assertTrue(initArgs.size() >= 2);
     assertTrue(((Map)initArgs.get(0)).containsKey("suggester"));
     assertTrue(((Map)initArgs.get(1)).containsKey("suggester"));
 
@@ -525,9 +531,20 @@ public class TestSolrConfigHandler extends RestTestBase {
         continue;
 
       }
-      if (Objects.equals(expected, Utils.getObjectByPath(m, false, jsonPath))) {
-        success = true;
-        break;
+      Object actual = Utils.getObjectByPath(m, false, jsonPath);
+
+      if (expected instanceof ValidatingJsonMap.PredicateWithErrMsg) {
+        ValidatingJsonMap.PredicateWithErrMsg predicate = (ValidatingJsonMap.PredicateWithErrMsg) expected;
+        if (predicate.test(actual) == null) {
+          success = true;
+          break;
+        }
+
+      } else {
+        if (Objects.equals(expected, actual)) {
+          success = true;
+          break;
+        }
       }
       Thread.sleep(100);
 
@@ -568,7 +585,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         10);
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/d', 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
+        "'create-requesthandler' : { 'name' : '/d', registerPath :'/,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
         "}";
 
     TestSolrConfigHandler.runConfigCommand(harness, "/config?wt=json", payload);
@@ -598,7 +615,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         5);
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/dump1', 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
+        "'create-requesthandler' : { 'name' : '/dump1', registerPath :'/,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
         "}";
 
     TestSolrConfigHandler.runConfigCommand(harness, "/config?wt=json", payload);
@@ -643,7 +660,7 @@ public class TestSolrConfigHandler extends RestTestBase {
 
     TestSolrConfigHandler.testForResponseElement(harness,
         null,
-        "/dump?wt=json&useParams=y",
+        "/dump1?wt=json&useParams=y",
         null,
         Arrays.asList("params", "c"),
         "CY val",
@@ -745,6 +762,60 @@ public class TestSolrConfigHandler extends RestTestBase {
         null,
         10);
 
+    payload = "{\n" +
+        "  'create-requesthandler': {\n" +
+        "    'name': 'aRequestHandler',\n" +
+        "    'registerPath': '/v2',\n" +
+        "    'class': 'org.apache.solr.handler.DumpRequestHandler',\n" +
+        "    'spec': {\n" +
+        "      'methods': [\n" +
+        "        'GET',\n" +
+        "        'POST'\n" +
+        "      ],\n" +
+        "      'url': {\n" +
+        "        'paths': [\n" +
+        "          '/something/{part1}/fixed/{part2}'\n" +
+        "        ]\n" +
+        "      }\n" +
+        "    }\n" +
+        "  }\n" +
+        "}";
+
+    TestSolrConfigHandler.runConfigCommand(harness, "/config?wt=json", payload);
+    TestSolrConfigHandler.testForResponseElement(harness,
+        null,
+        "/config/overlay?wt=json",
+        null,
+        Arrays.asList("overlay", "requestHandler", "aRequestHandler", "class"),
+        "org.apache.solr.handler.DumpRequestHandler",
+        10);
+    RESTfulServerProvider oldProvider = restTestHarness.getServerProvider();
+    restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME);
+
+    Map rsp = TestSolrConfigHandler.testForResponseElement(
+        harness,
+        null,
+        "/something/part1_Value/fixed/part2_Value?urlTemplateValues=part1&urlTemplateValues=part2",
+        null,
+        Arrays.asList("urlTemplateValues"),
+        new ValidatingJsonMap.PredicateWithErrMsg() {
+          @Override
+          public String test(Object o) {
+            if (o instanceof Map) {
+              Map m = (Map) o;
+              if ("part1_Value".equals(m.get("part1"))  && "part2_Value".equals(m.get("part2"))) return null;
+
+            }
+            return "no match";
+          }
+
+          @Override
+          public String toString() {
+            return "{part1:part1_Value, part2 : part2_Value]";
+          }
+        },
+        10);
+    restTestHarness.setServerProvider(oldProvider);
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
new file mode 100644
index 0000000..1af5d93
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.TestSolrConfigHandler;
+import org.apache.solr.util.RESTfulServerProvider;
+import org.apache.solr.util.RestTestHarness;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class V2ApiIntegrationTest extends SolrCloudTestCase {
+  private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
+
+  private static String COLL_NAME = "collection1";
+
+  private void setupHarnesses() {
+    for (final JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
+      RestTestHarness harness = new RestTestHarness(new ServerProvider(jettySolrRunner));
+      restTestHarnesses.add(harness);
+    }
+  }
+  static class ServerProvider implements RESTfulServerProvider {
+
+    final JettySolrRunner jettySolrRunner;
+    String baseurl;
+
+    ServerProvider(JettySolrRunner jettySolrRunner) {
+      this.jettySolrRunner = jettySolrRunner;
+      baseurl = jettySolrRunner.getBaseUrl().toString() + "/" + COLL_NAME;
+    }
+
+    @Override
+    public String getBaseURL() {
+      return baseurl;
+    }
+
+  }
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(2)
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-managed").resolve("conf"))
+        .configure();
+    CollectionAdminRequest.createCollection(COLL_NAME, "conf1", 1, 2)
+        .process(cluster.getSolrClient());
+  }
+
+  @Test
+  public void test() throws Exception {
+    try {
+      setupHarnesses();
+      testApis();
+
+    } finally {
+      for (RestTestHarness r : restTestHarnesses) {
+        r.close();
+      }
+    }
+  }
+
+  private void testApis() throws Exception {
+    RestTestHarness restHarness = restTestHarnesses.get(0);
+    ServerProvider serverProvider = (ServerProvider) restHarness.getServerProvider();
+    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/v2/c/"+ COLL_NAME;
+    Map result = TestSolrConfigHandler.getRespMap("/get/_introspect", restHarness);
+    assertEquals("/c/collection1/get", Utils.getObjectByPath(result, true, "/spec[0]/url/paths[0]"));
+    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/v2/collections/"+ COLL_NAME;
+    result = TestSolrConfigHandler.getRespMap("/get/_introspect", restHarness);
+    assertEquals("/collections/collection1/get", Utils.getObjectByPath(result, true, "/spec[0]/url/paths[0]"));
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..b784f87
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.V2HttpCall.CompositeApi;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginBag;
+import org.apache.solr.handler.PingRequestHandler;
+import org.apache.solr.handler.SchemaHandler;
+import org.apache.solr.handler.SolrConfigHandler;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+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.util.PathTrie;
+
+import static org.apache.solr.api.ApiBag.EMPTY_SPEC;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
+import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
+
+public class TestApiFramework extends SolrTestCaseJ4 {
+
+  public void testFramework() {
+    Map<String, Object[]> calls = new HashMap<>();
+    Map<String, Object> out = new HashMap<>();
+    CoreContainer mockCC = TestCoreAdminApis.getCoreContainerMock(calls, out);
+    PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null, false);
+    containerHandlers.put(COLLECTIONS_HANDLER_PATH, new TestCollectionAPIs.MockCollectionsHandler());
+    containerHandlers.put(CORES_HANDLER_PATH, new CoreAdminHandler(mockCC));
+    containerHandlers.put(CONFIGSETS_HANDLER_PATH, new ConfigSetsHandler(mockCC));
+    out.put("getRequestHandlers", containerHandlers);
+
+    PluginBag<SolrRequestHandler> coreHandlers = new PluginBag<>(SolrRequestHandler.class, null, false);
+    coreHandlers.put("/schema", new SchemaHandler());
+    coreHandlers.put("/config", new SolrConfigHandler());
+    coreHandlers.put("/admin/ping", new PingRequestHandler());
+
+    Map<String, String> parts = new HashMap<>();
+    String fullPath = "/collections/hello/shards";
+    Api api = V2HttpCall.getApiInfo(containerHandlers, fullPath, "POST",
+       fullPath, parts);
+    assertNotNull(api);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "POST",
+        "/commands/create", NOT_NULL));
+    assertEquals("hello", parts.get("collection"));
+
+
+    parts = new HashMap<>();
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards", "POST",
+      null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "POST",
+        "/commands/split", NOT_NULL,
+        "/commands/add-replica", NOT_NULL
+    ));
+
+
+    parts = new HashMap<>();
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1", "POST",
+        null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "POST",
+        "/commands/force-leader", NOT_NULL
+    ));
+    assertEquals("hello", parts.get("collection"));
+    assertEquals("shard1", parts.get("shard"));
+
+
+    parts = new HashMap<>();
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello", "POST",
+       null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "POST",
+        "/commands/add-replica-property", NOT_NULL,
+        "/commands/delete-replica-property", NOT_NULL
+    ));
+    assertEquals("hello", parts.get("collection"));
+
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "DELETE",
+       null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "DELETE",
+        "/url/params/onlyIfDown/type", "boolean"
+    ));
+    assertEquals("hello", parts.get("collection"));
+    assertEquals("shard1", parts.get("shard"));
+    assertEquals("replica1", parts.get("replica"));
+
+    SolrQueryResponse rsp = invoke(containerHandlers, null, "/collections/_introspect", GET, mockCC);
+
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/spec[0]/methods[0]", "DELETE",
+        "/spec[1]/methods[0]", "POST",
+        "/spec[2]/methods[0]", "GET"
+
+    ));
+
+    rsp = invoke(coreHandlers, "/schema/_introspect", "/collections/hello/schema/_introspect", GET, mockCC);
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/spec[0]/methods[0]", "POST",
+        "/spec[0]/commands", NOT_NULL,
+        "/spec[1]/methods[0]", "GET"));
+
+    rsp = invoke(coreHandlers, "/", "/collections/hello/_introspect", GET, mockCC);
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/availableSubPaths", NOT_NULL,
+        "availableSubPaths /collections/hello/config/jmx", NOT_NULL,
+        "availableSubPaths /collections/hello/schema", NOT_NULL,
+        "availableSubPaths /collections/hello/shards", NOT_NULL,
+        "availableSubPaths /collections/hello/shards/{shard}", NOT_NULL,
+        "availableSubPaths /collections/hello/shards/{shard}/{replica}", NOT_NULL
+    ));
+
+  }
+  public void testTrailingTemplatePaths(){
+    PathTrie<Api> registry =  new PathTrie<>();
+    Api api = new Api(EMPTY_SPEC) {
+      @Override
+      public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+
+      }
+    };
+    Api intropsect = new ApiBag.IntrospectApi(api,false);
+    ApiBag.registerIntrospect(Collections.emptyMap(),registry,"/c/.system/blob/{name}",intropsect);
+    ApiBag.registerIntrospect(Collections.emptyMap(), registry, "/c/.system/{x}/{name}", intropsect);
+    assertEquals(intropsect, registry.lookup("/c/.system/blob/random_string/_introspect", new HashMap<>()));
+    assertEquals(intropsect, registry.lookup("/c/.system/blob/_introspect", new HashMap<>()));
+    assertEquals(intropsect, registry.lookup("/c/.system/_introspect", new HashMap<>()));
+    assertEquals(intropsect, registry.lookup("/c/.system/v1/_introspect", new HashMap<>()));
+    assertEquals(intropsect, registry.lookup("/c/.system/v1/v2/_introspect", new HashMap<>()));
+  }
+  private SolrQueryResponse invoke(PluginBag<SolrRequestHandler> reqHandlers, String path,
+                                   String fullPath, SolrRequest.METHOD method,
+                                   CoreContainer mockCC) {
+    HashMap<String, String> parts = new HashMap<>();
+    boolean containerHandlerLookup = mockCC.getRequestHandlers() == reqHandlers;
+    path = path == null ? fullPath : path;
+    Api api = null;
+    if (containerHandlerLookup) {
+      api = V2HttpCall.getApiInfo(reqHandlers, path, "GET", fullPath, parts);
+    } else {
+      api = V2HttpCall.getApiInfo(mockCC.getRequestHandlers(), fullPath, "GET", fullPath, parts);
+      if (api == null) api = new CompositeApi(null);
+      if (api instanceof CompositeApi) {
+        CompositeApi compositeApi = (CompositeApi) api;
+        api = V2HttpCall.getApiInfo(reqHandlers, path, "GET", fullPath, parts);
+        compositeApi.add(api);
+        api = compositeApi;
+      }
+    }
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>())){
+      @Override
+      public List<CommandOperation> getCommands(boolean validateInput) {
+        return Collections.emptyList();
+      }
+    };
+
+    api.call(req,rsp);
+    return rsp;
+
+  }
+
+
+  private void assertConditions(Map root, Map conditions) {
+    for (Object o : conditions.entrySet()) {
+      Map.Entry e = (Map.Entry) o;
+      String path = (String) e.getKey();
+      List<String> parts = StrUtils.splitSmart(path, path.charAt(0) == '/' ?  '/':' ');
+      if (parts.get(0).isEmpty()) parts.remove(0);
+      Object val = Utils.getObjectByPath(root, false, parts);
+      if (e.getValue() instanceof ValidatingJsonMap.PredicateWithErrMsg) {
+        ValidatingJsonMap.PredicateWithErrMsg value = (ValidatingJsonMap.PredicateWithErrMsg) e.getValue();
+        String err = value.test(val);
+        if(err != null){
+          assertEquals(err + " for " + e.getKey() + " in :" + Utils.toJSONString(root), e.getValue(), val);
+        }
+
+      } else {
+        assertEquals("incorrect value for path " + e.getKey() + " in :" + Utils.toJSONString(root), e.getValue(), val);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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
new file mode 100644
index 0000000..7f072ec
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.io.StringReader;
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.MultiMapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+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.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+public class TestCollectionAPIs extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+
+  public void testCommands() throws Exception {
+    MockCollectionsHandler collectionsHandler = new MockCollectionsHandler();
+    ApiBag apiBag = new ApiBag(false);
+    Collection<Api> apis = collectionsHandler.getApis();
+    for (Api api : apis) apiBag.register(api, Collections.EMPTY_MAP);
+    //test a simple create collection call
+    compareOutput(apiBag, "/collections", POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2 }}", null,
+        "{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}");
+
+    //test a create collection with custom properties
+    compareOutput(apiBag, "/collections", POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2, properties:{prop1:'prop1val', prop2: prop2val} }}", null,
+        "{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create, property.prop1:prop1val, property.prop2:prop2val}");
+
+
+    compareOutput(apiBag, "/collections", POST,
+        "{create-alias:{name: aliasName , collections:[c1,c2] }}", null, "{operation : createalias, name: aliasName, collections:[c1,c2] }");
+
+    compareOutput(apiBag, "/collections", POST,
+        "{delete-alias:{ name: aliasName}}", null, "{operation : deletealias, name: aliasName}");
+
+    compareOutput(apiBag, "/collections/collName", POST,
+        "{reload:{}}", null,
+        "{name:collName, operation :reload}");
+
+    compareOutput(apiBag, "/collections/collName", DELETE,
+        null, null,
+        "{name:collName, operation :delete}");
+
+    compareOutput(apiBag, "/collections/collName/shards/shard1", DELETE,
+        null, null,
+        "{collection:collName, shard: shard1 , operation :deleteshard }");
+
+    compareOutput(apiBag, "/collections/collName/shards/shard1/replica1?deleteDataDir=true&onlyIfDown=true", DELETE,
+        null, null,
+        "{collection:collName, shard: shard1, replica :replica1 , deleteDataDir:'true', onlyIfDown: 'true', operation :deletereplica }");
+
+    compareOutput(apiBag, "/collections/collName/shards", POST,
+        "{split:{shard:shard1, ranges: '0-1f4,1f5-3e8,3e9-5dc', coreProperties : {prop1:prop1Val, prop2:prop2Val} }}", null,
+        "{collection: collName , shard : shard1, ranges :'0-1f4,1f5-3e8,3e9-5dc', operation : splitshard, property.prop1:prop1Val, property.prop2: prop2Val}"
+    );
+
+    compareOutput(apiBag, "/collections/collName/shards", POST,
+        "{add-replica:{shard: shard1, node: 'localhost_8978' , coreProperties : {prop1:prop1Val, prop2:prop2Val} }}", null,
+        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, property.prop1:prop1Val, property.prop2: prop2Val}"
+    );
+
+    compareOutput(apiBag, "/collections/collName/shards", POST,
+        "{split:{ splitKey:id12345, coreProperties : {prop1:prop1Val, prop2:prop2Val} }}", null,
+        "{collection: collName , split.key : id12345 , operation : splitshard, property.prop1:prop1Val, property.prop2: prop2Val}"
+    );
+
+    compareOutput(apiBag, "/collections/collName", POST,
+        "{add-replica-property : {name:propA , value: VALA, shard: shard1, replica:replica1}}", null,
+        "{collection: collName, shard: shard1, replica : replica1 , property : propA , operation : addreplicaprop, property.value : 'VALA'}"
+    );
+
+    compareOutput(apiBag, "/collections/collName", POST,
+        "{delete-replica-property : {property: propA , shard: shard1, replica:replica1} }", null,
+        "{collection: collName, shard: shard1, replica : replica1 , property : propA , operation : deletereplicaprop}"
+    );
+
+    compareOutput(apiBag, "/collections/collName", POST,
+        "{modify : {rule : 'replica:*,cores:<5', autoAddReplicas : false} }", null,
+        "{collection: collName, operation : modifycollection , autoAddReplicas : 'false', rule : [{replica: '*', cores : '<5' }]}"
+    );
+    compareOutput(apiBag, "/cluster", POST,
+        "{add-role : {role : overseer, node : 'localhost_8978'} }", null,
+        "{operation : addrole ,role : overseer, node : 'localhost_8978'}"
+    );
+
+    compareOutput(apiBag, "/cluster", POST,
+        "{remove-role : {role : overseer, node : 'localhost_8978'} }", null,
+        "{operation : removerole ,role : overseer, node : 'localhost_8978'}"
+    );
+
+    compareOutput(apiBag, "/collections/coll1", POST,
+        "{balance-shard-unique : {property: preferredLeader} }", null,
+        "{operation : balanceshardunique ,collection : coll1, property : preferredLeader}"
+    );
+
+    compareOutput(apiBag, "/collections/coll1", POST,
+        "{migrate-docs : {forwardTimeout: 1800, target: coll2, splitKey: 'a123!'} }", null,
+        "{operation : migrate ,collection : coll1, target.collection:coll2, forward.timeout:1800, split.key:'a123!'}"
+    );
+
+  }
+
+  static ZkNodeProps compareOutput(final ApiBag apiBag, final String path, final SolrRequest.METHOD method,
+                            final String payload, final CoreContainer cc, String expectedOutputMapJson) throws Exception {
+    Pair<SolrQueryRequest, SolrQueryResponse> ctx = makeCall(apiBag, path, method, payload, cc);
+    ZkNodeProps output = (ZkNodeProps) ctx.second().getValues().get(ZkNodeProps.class.getName());
+    Map expected = (Map) fromJSONString(expectedOutputMapJson);
+    assertMapEqual(expected, output);
+    return output;
+
+  }
+
+  public static Pair<SolrQueryRequest, SolrQueryResponse> makeCall(final ApiBag apiBag, String path,
+                                                                   final SolrRequest.METHOD method,
+                                                                   final String payload, final CoreContainer cc) throws Exception {
+    SolrParams queryParams = new MultiMapSolrParams(Collections.EMPTY_MAP);
+    if (path.indexOf('?') > 0) {
+      String queryStr = path.substring(path.indexOf('?') + 1);
+      path = path.substring(0, path.indexOf('?'));
+      queryParams = SolrRequestParsers.parseQueryString(queryStr);
+    }
+    final HashMap<String, String> parts = new HashMap<>();
+    Api api = apiBag.lookup(path, method.toString(), parts);
+    if (api == null) throw new RuntimeException("No handler at path :" + path);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, queryParams) {
+      @Override
+      public List<CommandOperation> getCommands(boolean validateInput) {
+        if (payload == null) return Collections.emptyList();
+        return ApiBag.getCommandOperations(new StringReader(payload), api.getCommandSchema(), true);
+      }
+
+      @Override
+      public Map<String, String> getPathTemplateValues() {
+        return parts;
+      }
+
+      @Override
+      public String getHttpMethod() {
+        return method.toString();
+      }
+    };
+    try {
+      api.call(req, rsp);
+    } catch (ApiBag.ExceptionWithErrObject e) {
+      throw new RuntimeException(e.getMessage() + Utils.toJSONString(e.getErrs()), e);
+
+    }
+    return new Pair<>(req, rsp);
+  }
+
+  private static void assertMapEqual(Map expected, ZkNodeProps actual) {
+    assertEquals(errorMessage(expected, actual), expected.size(), actual.getProperties().size());
+    for (Object o : expected.entrySet()) {
+      Map.Entry e = (Map.Entry) o;
+      Object actualVal = actual.get((String) e.getKey());
+      if (actualVal instanceof String[]) {
+        actualVal = Arrays.asList((String[]) actualVal);
+      }
+      assertEquals(errorMessage(expected, actual), String.valueOf(e.getValue()), String.valueOf(actualVal));
+    }
+  }
+
+  private static String errorMessage(Map expected, ZkNodeProps actual) {
+    return "expected: " + Utils.toJSONString(expected) + "\nactual: " + Utils.toJSONString(actual);
+
+  }
+
+  static class MockCollectionsHandler extends CollectionsHandler {
+    LocalSolrQueryRequest req;
+
+    MockCollectionsHandler() {
+    }
+
+    @Override
+    void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp,
+                      CoreContainer cores,
+                      CollectionParams.CollectionAction action,
+                      CollectionOperation operation) throws Exception {
+      Map<String, Object> result = operation.execute(req, rsp, this);
+      if (result != null) {
+        result.put(QUEUE_OPERATION, operation.action.toLower());
+        rsp.add(ZkNodeProps.class.getName(), new ZkNodeProps(result));
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java b/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
new file mode 100644
index 0000000..d2c96a6
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.zookeeper.KeeperException;
+
+import static java.util.Collections.EMPTY_MAP;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.handler.admin.TestCollectionAPIs.compareOutput;
+
+public class TestConfigsApi extends SolrTestCaseJ4 {
+
+
+  public void testCommands() throws Exception {
+
+    ConfigSetsHandler handler = new ConfigSetsHandler(null) {
+      @Override
+      protected void sendToZk(SolrQueryResponse rsp,
+                              ConfigSetOperation operation,
+                              Map<String, Object> result)
+          throws KeeperException, InterruptedException {
+        result.put(QUEUE_OPERATION, operation.action.toLower());
+        rsp.add(ZkNodeProps.class.getName(), new ZkNodeProps(result));
+      }
+    };
+    ApiBag apiBag = new ApiBag(false);
+    for (Api api : handler.getApis()) apiBag.register(api, EMPTY_MAP);
+    compareOutput(apiBag, "/cluster/configs/sample", DELETE, null, null,
+        "{name :sample, operation:delete}");
+
+    compareOutput(apiBag, "/cluster/configs", POST, "{create:{name : newconf, baseConfigSet: sample }}", null,
+        "{operation:create, name :newconf,  baseConfigSet: sample }");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
new file mode 100644
index 0000000..f263f5e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.easymock.EasyMock;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+import static org.easymock.EasyMock.anyBoolean;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.getCurrentArguments;
+
+public class TestCoreAdminApis extends SolrTestCaseJ4 {
+
+  public void testCalls() throws Exception {
+    Map<String, Object[]> calls = new HashMap<>();
+    CoreContainer mockCC = getCoreContainerMock(calls, new HashMap<>());
+
+    CoreAdminHandler  coreAdminHandler = new CoreAdminHandler(mockCC);
+    ApiBag apiBag = new ApiBag(false);
+    for (Api api : coreAdminHandler.getApis()) {
+      apiBag.register(api, Collections.EMPTY_MAP);
+    }
+    TestCollectionAPIs.makeCall(apiBag, "/cores", SolrRequest.METHOD.POST,
+        "{create:{name: hello, instanceDir : someDir, schema: 'schema.xml'}}", mockCC);
+    Object[] params = calls.get("create");
+    assertEquals("hello" ,params[0]);
+    assertEquals(fromJSONString("{schema : schema.xml}") ,params[2]);
+
+    TestCollectionAPIs.makeCall(apiBag, "/cores/core1", SolrRequest.METHOD.POST,
+        "{swap:{with: core2}}", mockCC);
+    params = calls.get("swap");
+    assertEquals("core1" ,params[0]);
+    assertEquals("core2" ,params[1]);
+
+    TestCollectionAPIs.makeCall(apiBag, "/cores/core1", SolrRequest.METHOD.POST,
+        "{rename:{to: core2}}", mockCC);
+    params = calls.get("swap");
+    assertEquals("core1" ,params[0]);
+    assertEquals("core2" ,params[1]);
+
+    TestCollectionAPIs.makeCall(apiBag, "/cores/core1", SolrRequest.METHOD.POST,
+        "{unload:{deleteIndex : true}}", mockCC);
+    params = calls.get("unload");
+    assertEquals("core1" ,params[0]);
+    assertEquals(Boolean.TRUE ,params[1]);
+  }
+
+  public static CoreContainer getCoreContainerMock(final Map<String, Object[]> in,Map<String,Object> out ) {
+    CoreContainer mockCC = EasyMock.createMock(CoreContainer.class);
+    EasyMock.reset(mockCC);
+    mockCC.create(anyObject(String.class), anyObject(Path.class) , anyObject(Map.class), anyBoolean());
+    EasyMock.expectLastCall().andAnswer(() -> {
+      in.put("create", getCurrentArguments());
+      return null;
+    }).anyTimes();
+    mockCC.swap(anyObject(String.class), anyObject(String.class));
+    EasyMock.expectLastCall().andAnswer(() -> {
+      in.put("swap", getCurrentArguments());
+      return null;
+    }).anyTimes();
+
+    mockCC.rename(anyObject(String.class), anyObject(String.class));
+    EasyMock.expectLastCall().andAnswer(() -> {
+      in.put("rename", getCurrentArguments());
+      return null;
+    }).anyTimes();
+
+    mockCC.unload(anyObject(String.class), anyBoolean(),
+        anyBoolean(), anyBoolean());
+    EasyMock.expectLastCall().andAnswer(() -> {
+      in.put("unload", getCurrentArguments());
+      return null;
+    }).anyTimes();
+
+    mockCC.getCoreRootDirectory();
+    EasyMock.expectLastCall().andAnswer(() -> Paths.get("coreroot")).anyTimes();
+    mockCC.getContainerProperties();
+    EasyMock.expectLastCall().andAnswer(() -> new Properties()).anyTimes();
+
+    mockCC.getRequestHandlers();
+    EasyMock.expectLastCall().andAnswer(() -> out.get("getRequestHandlers")).anyTimes();
+
+    EasyMock.replay(mockCC);
+    return mockCC;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
index d5db82e..ea8fd7b 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
@@ -27,6 +27,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.schema.SimilarityFactory;
 import org.apache.solr.search.similarities.SchemaSimilarityFactory;
+import org.apache.solr.util.RESTfulServerProvider;
 import org.apache.solr.util.RestTestBase;
 import org.apache.solr.util.RestTestHarness;
 
@@ -34,9 +35,12 @@ import org.junit.After;
 import org.junit.Before;
 import org.noggit.JSONParser;
 import org.noggit.ObjectBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.StringReader;
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -45,6 +49,8 @@ import java.util.Set;
 
 
 public class TestBulkSchemaAPI extends RestTestBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
 
   private static File tmpSolrHome;
 
@@ -58,6 +64,15 @@ public class TestBulkSchemaAPI extends RestTestBase {
 
     createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
         "/solr", true, null);
+    if (random().nextBoolean()) {
+      log.info("These tests are run with V2 API");
+      restTestHarness.setServerProvider(new RESTfulServerProvider() {
+        @Override
+        public String getBaseURL() {
+          return jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME;
+        }
+      });
+    }
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 137fcdd..397f4e8 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -85,6 +85,10 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
 
     String authcPrefix = "/admin/authentication";
     String authzPrefix = "/admin/authorization";
+    if(random().nextBoolean()){
+      authcPrefix = "/v2/cluster/security/authentication";
+      authzPrefix = "/v2/cluster/security/authorization";
+    }
 
     NamedList<Object> rsp;
     HttpClient cl = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/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 0f70d73..03656c5 100644
--- a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
@@ -312,7 +312,7 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
     perms.runCmd("{set-permission : {name: config-edit, role: admin } }", true);
     assertEquals("config-edit",  getObjectByPath(perms.conf, false, "permissions[0]/name"));
     assertEquals(1 , perms.getVal("permissions[0]/index"));
-    assertEquals("admin" ,  perms.getVal("permissions[0]/role"));
+    assertEquals("admin", perms.getVal("permissions[0]/role"));
     perms.runCmd("{set-permission : {name: config-edit, role: [admin, dev], index:2 } }", false);
     perms.runCmd("{set-permission : {name: config-edit, role: [admin, dev], index:1}}", true);
     Collection roles = (Collection) perms.getVal("permissions[0]/role");
@@ -324,19 +324,19 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
     assertEquals("x", perms.getVal("permissions[1]/collection"));
     assertEquals("/a/b", perms.getVal("permissions[1]/path"));
     perms.runCmd("{update-permission : {index : 2, method : POST }}", true);
-    assertEquals("POST" , perms.getVal("permissions[1]/method"));
+    assertEquals("POST", perms.getVal("permissions[1]/method"));
     perms.runCmd("{set-permission : {name : read, collection : y, role:[guest, dev] ,  before :2}}", true);
     assertNotNull(perms.getVal("permissions[2]"));
     assertEquals("y", perms.getVal("permissions[1]/collection"));
     assertEquals("read", perms.getVal("permissions[1]/name"));
     perms.runCmd("{delete-permission : 3}", true);
     assertTrue(captureErrors(perms.parsedCommands).isEmpty());
-    assertEquals("y",perms.getVal("permissions[1]/collection"));
+    assertEquals("y", perms.getVal("permissions[1]/collection"));
   }
 
   static class  Perms {
     Map conf =  new HashMap<>();
-    RuleBasedAuthorizationPlugin plugin = new RuleBasedAuthorizationPlugin();
+    ConfigEditablePlugin plugin = new RuleBasedAuthorizationPlugin();
     List<CommandOperation> parsedCommands;
 
     public void runCmd(String cmds, boolean failOnError) throws IOException {