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 2016/02/11 22:11:07 UTC

[1/3] lucene-solr git commit: SOLR-8029 renamed class/dirs and removed the 'v2' part

Repository: lucene-solr
Updated Branches:
  refs/heads/apiv2 292fe4a19 -> 3cfebd53f


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..53735bf
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -0,0 +1,184 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+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.params.MapSolrParams;
+import org.apache.solr.common.util.Predicate;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+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.SolrRequestHandler;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.V2HttpCall;
+import org.apache.solr.v2api.V2RequestContext;
+
+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.CORES_HANDLER_PATH;
+import static org.apache.solr.common.util.Map2.NOT_NULL;
+import static org.apache.solr.handler.admin.TestCollectionAPIs.getV2RequestContext;
+
+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));
+    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, "GET",
+        mockCC, "collections", fullPath, parts);
+    assertNotNull(api);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "GET",
+        "/methods[1]", "POST",
+        "/commands/create", NOT_NULL));
+    assertEquals("hello", parts.get("collection"));
+
+    parts = new HashMap<>();
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1", "GET",
+        mockCC, "collections", null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "POST",
+        "/methods[1]", "GET",
+        "/methods[2]", "DELETE",
+        "/commands/split", NOT_NULL,
+        "/commands/add-replica", NOT_NULL,
+        "/commands/force-leader", NOT_NULL
+    ));
+    assertEquals("hello", parts.get("collection"));
+    assertEquals("shard1", parts.get("shard"));
+
+
+    parts = new HashMap<>();
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "GET",
+        mockCC, "collections", null, parts);
+    assertConditions(api.getSpec(), Utils.makeMap(
+        "/methods[0]", "GET",
+        "/methods[1]", "POST",
+        "/commands/set", NOT_NULL
+    ));
+    assertEquals("hello", parts.get("collection"));
+    assertEquals("shard1", parts.get("shard"));
+    assertEquals("replica1", parts.get("replica"));
+
+    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "DELETE",
+        mockCC, "collections", 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, "/collections/_introspect", GET, mockCC);
+
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/spec[0]/methods[0]", "POST",
+        "/spec[0]/methods[1]", "GET"));
+
+    rsp = invoke(coreHandlers, "/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", 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
+    ));
+
+    rsp = invoke(coreHandlers,"/collections/hello/schema",SolrRequest.METHOD.POST, mockCC);
+
+
+
+  }
+
+  private SolrQueryResponse invoke(PluginBag<SolrRequestHandler> reqHandlers, String path, SolrRequest.METHOD method,
+                                   CoreContainer mockCC) {
+    HashMap<String, String> parts = new HashMap<>();
+    boolean containerHandlerLookup = mockCC.getRequestHandlers() == reqHandlers;
+    String fullPath = path;
+    String prefix = null;
+    if (!containerHandlerLookup) {
+      int idx = path.indexOf('/', 1);
+      prefix = path.substring(1, idx);
+      if (idx > 0) idx = path.indexOf('/', idx + 1);
+      path = idx == -1 ? "/" : path.substring(idx);
+    }
+
+    Api api = V2HttpCall.getApiInfo(reqHandlers, path, "GET", mockCC, prefix, fullPath, parts);
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>()));
+    V2RequestContext ctx = getV2RequestContext( path, method, null, mockCC, parts, api, req);
+    api.call(ctx);
+    return ctx.getResponse();
+
+  }
+
+
+  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 Predicate) {
+        Predicate value = (Predicate) 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/3cfebd53/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..4b538b4
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -0,0 +1,153 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+
+import java.io.StringReader;
+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.MapSolrParams;
+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.util.CommandOperation;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.ApiBag;
+import org.apache.solr.v2api.V2HttpCall;
+import org.apache.solr.v2api.V2RequestContext;
+
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+
+public class TestCollectionAPIs extends SolrTestCaseJ4 {
+
+  public void testCreate() throws Exception{
+    MockCollectionsHandler collectionsHandler = new MockCollectionsHandler();
+    ApiBag apiBag = new ApiBag();
+    Collection<Api> apis = collectionsHandler.getApis();
+    for (Api api : apis) apiBag.register(api, Collections.EMPTY_MAP);
+    //test a simple create collection call
+    V2RequestContext ctx = makeCall(apiBag, "/collections", SolrRequest.METHOD.POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2 }}", null);
+    assertMapEqual((Map) Utils.fromJSONString("{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}"),
+        (ZkNodeProps) ctx.getSolrRequest().getContext().get(ZkNodeProps.class.getName()));
+
+    //test a create collection with custom properties
+    ctx = makeCall(apiBag, "/collections", SolrRequest.METHOD.POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2, properties:{prop1:'prop1val', prop2: prop2val} }}", null);
+
+    assertMapEqual(
+        (Map) Utils.fromJSONString("{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create, property.prop1:prop1val, property.prop2:prop2val}"),
+        (ZkNodeProps) ctx.getSolrRequest().getContext().get(ZkNodeProps.class.getName()));
+
+  }
+
+  public static V2RequestContext makeCall(final ApiBag apiBag, final String path, final SolrRequest.METHOD method,
+                                    final String payload, final CoreContainer cc) throws Exception {
+    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);
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>()));
+    V2RequestContext ctx = getV2RequestContext(path, method, payload, cc, parts, api, req);
+    api.call(ctx);
+    return ctx;
+  }
+
+  public static V2RequestContext getV2RequestContext(final String path,
+                                                     final SolrRequest.METHOD method,
+                                                     final String payload,
+                                                     final CoreContainer cc,
+                                                     final HashMap<String, String> parts,
+                                                     final Api api,
+                                                     final LocalSolrQueryRequest req) {
+    return new V2RequestContext() {
+      SolrQueryResponse rsp = new SolrQueryResponse();
+
+      @Override
+      public SolrQueryResponse getResponse() {
+        return rsp;
+      }
+
+      @Override
+      public CoreContainer getCoreContainer() {
+        return cc;
+      }
+
+      @Override
+      public SolrQueryRequest getSolrRequest() {
+        return req;
+      }
+
+      @Override
+      public String getPath() {
+        return path;
+      }
+
+      @Override
+      public Map<String, String> getPathValues() {
+        return parts;
+      }
+
+      @Override
+      public List<CommandOperation> getCommands(boolean validateInput) {
+        return V2HttpCall.getCommandOperations(new StringReader(payload), api.getSpec(), rsp);
+      }
+
+      @Override
+      public String getHttpMethod() {
+        return method.toString();
+      }
+    };
+  }
+
+  private void assertMapEqual(Map expected, ZkNodeProps actual) {
+    assertEquals(expected.size(), actual.getProperties().size());
+    for (Object o : expected.entrySet()) {
+      Map.Entry e = (Map.Entry) o;
+      assertEquals(e.getValue(), actual.get((String) e.getKey()));
+    }
+  }
+
+  static class MockCollectionsHandler extends CollectionsHandler {
+    LocalSolrQueryRequest req;
+
+    MockCollectionsHandler() {
+
+    }
+
+
+    @Override
+    protected void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp, CollectionOperation operation) throws Exception {
+      Map<String, Object> result = operation.call(req, rsp, this);
+      if (result != null) {
+        result.put(QUEUE_OPERATION, operation.action.toLower());
+        req.getContext().put(ZkNodeProps.class.getName(),new ZkNodeProps(result) );
+      }
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..b6ad4c5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
@@ -0,0 +1,102 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+
+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.core.PluginBag;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.ApiBag;
+import org.easymock.EasyMock;
+import org.easymock.IAnswer;
+
+import static org.easymock.EasyMock.getCurrentArguments;
+
+public class TestCoreAdminApis extends SolrTestCaseJ4 {
+
+  public void testCreate() throws Exception {
+    Map<String, Object[]> calls = new HashMap<>();
+    CoreContainer mockCC = getCoreContainerMock(calls, new HashMap<>());
+
+    CoreAdminHandler  coreAdminHandler = new CoreAdminHandler(mockCC);
+    ApiBag apiBag = new ApiBag();
+    for (Api api : coreAdminHandler.getApis()) {
+      apiBag.register(api, Collections.EMPTY_MAP);
+    }
+    TestCollectionAPIs.makeCall(apiBag, "/cores", SolrRequest.METHOD.POST,
+        "{create:{name: hello, instanceDir : someDir, config : 'solrconfig.xml', schema: 'schema.xml'}}", mockCC);
+    Object[] create = calls.get("create");
+    assertEquals("hello" ,create[0]);
+    HashMap expected = new HashMap();
+    expected.put("schema", "schema.xml");
+    expected.put("config", "solrconfig.xml");
+    assertEquals(expected ,create[2]);
+
+  }
+
+  public static CoreContainer getCoreContainerMock(final Map<String, Object[]> in,Map<String,Object> out ) {
+    CoreContainer mockCC = EasyMock.createMock(CoreContainer.class);
+    EasyMock.reset(mockCC);
+    mockCC.create(EasyMock.anyObject(String.class), EasyMock.anyObject(Path.class ) ,EasyMock.anyObject(Map.class));
+    EasyMock.expectLastCall().andAnswer(new IAnswer<SolrCore>() {
+      @Override
+      public SolrCore answer() throws Throwable {
+        in.put("create", getCurrentArguments());
+        return null;
+      }
+    }).anyTimes();
+
+    mockCC.getCoreRootDirectory();
+    EasyMock.expectLastCall().andAnswer(new IAnswer<Path>() {
+      @Override
+      public Path answer() throws Throwable {
+        return Paths.get("coreroot");
+      }
+    }).anyTimes();
+    mockCC.getContainerProperties();
+    EasyMock.expectLastCall().andAnswer(new IAnswer<Properties>() {
+      @Override
+      public Properties answer() throws Throwable {
+        return new Properties();
+      }
+    }).anyTimes();
+
+    mockCC.getRequestHandlers();
+    EasyMock.expectLastCall().andAnswer(new IAnswer<PluginBag>() {
+      @Override
+      public PluginBag answer() throws Throwable {
+        return (PluginBag)out.get("getRequestHandlers");
+      }
+    }).anyTimes();
+
+    EasyMock.replay(mockCC);
+    return mockCC;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/test/org/apache/solr/handler/admin/TestV2CollectionAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestV2CollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestV2CollectionAPIs.java
deleted file mode 100644
index 8f87352..0000000
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestV2CollectionAPIs.java
+++ /dev/null
@@ -1,160 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.StringReader;
-import java.nio.ByteBuffer;
-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.MapSolrParams;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-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.SolrDispatchFilter;
-import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2HttpCall;
-import org.apache.solr.v2api.V2RequestContext;
-import org.apache.zookeeper.server.ByteBufferInputStream;
-
-import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
-
-public class TestV2CollectionAPIs extends SolrTestCaseJ4 {
-
-  public void testCreate() throws Exception{
-    MockCollectionsHandler collectionsHandler = new MockCollectionsHandler();
-    ApiBag apiBag = new ApiBag();
-    Collection<V2Api> apis = collectionsHandler.getApis();
-    for (V2Api api : apis) apiBag.register(api, Collections.EMPTY_MAP);
-    //test a simple create collection call
-    V2RequestContext ctx = makeCall(apiBag, "/collections", SolrRequest.METHOD.POST,
-        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2 }}", null);
-    assertMapEqual((Map) Utils.fromJSONString("{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}"),
-        (ZkNodeProps) ctx.getSolrRequest().getContext().get(ZkNodeProps.class.getName()));
-
-    //test a create collection with custom properties
-    ctx = makeCall(apiBag, "/collections", SolrRequest.METHOD.POST,
-        "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2, properties:{prop1:'prop1val', prop2: prop2val} }}", null);
-
-    assertMapEqual(
-        (Map) Utils.fromJSONString("{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create, property.prop1:prop1val, property.prop2:prop2val}"),
-        (ZkNodeProps) ctx.getSolrRequest().getContext().get(ZkNodeProps.class.getName()));
-
-  }
-
-  public static V2RequestContext makeCall(final ApiBag apiBag, final String path, final SolrRequest.METHOD method,
-                                    final String payload, final CoreContainer cc) throws Exception {
-    final HashMap<String, String> parts = new HashMap<>();
-    V2Api api = apiBag.lookup(path, method.toString(), parts);
-    if (api == null) throw new RuntimeException("No handler at path :" + path);
-    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<String, String>()));
-    V2RequestContext ctx = getV2RequestContext(path, method, payload, cc, parts, api, req);
-    api.call(ctx);
-    return ctx;
-  }
-
-  public static V2RequestContext getV2RequestContext(final String path,
-                                                     final SolrRequest.METHOD method,
-                                                     final String payload,
-                                                     final CoreContainer cc,
-                                                     final HashMap<String, String> parts,
-                                                     final V2Api api,
-                                                     final LocalSolrQueryRequest req) {
-    return new V2RequestContext() {
-      SolrQueryResponse rsp = new SolrQueryResponse();
-
-      @Override
-      public SolrQueryResponse getResponse() {
-        return rsp;
-      }
-
-      @Override
-      public CoreContainer getCoreContainer() {
-        return cc;
-      }
-
-      @Override
-      public SolrQueryRequest getSolrRequest() {
-        return req;
-      }
-
-      @Override
-      public String getPath() {
-        return path;
-      }
-
-      @Override
-      public Map<String, String> getPathValues() {
-        return parts;
-      }
-
-      @Override
-      public List<CommandOperation> getCommands(boolean validateInput) {
-        return V2HttpCall.getCommandOperations(new StringReader(payload), api.getSpec(), rsp);
-      }
-
-      @Override
-      public String getHttpMethod() {
-        return method.toString();
-      }
-    };
-  }
-
-  private void assertMapEqual(Map expected, ZkNodeProps actual) {
-    assertEquals(expected.size(), actual.getProperties().size());
-    for (Object o : expected.entrySet()) {
-      Map.Entry e = (Map.Entry) o;
-      assertEquals(e.getValue(), actual.get((String) e.getKey()));
-    }
-  }
-
-  static class MockCollectionsHandler extends CollectionsHandler {
-    LocalSolrQueryRequest req;
-
-    MockCollectionsHandler() {
-
-    }
-
-
-    @Override
-    protected void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp, CollectionOperation operation) throws Exception {
-      Map<String, Object> result = operation.call(req, rsp, this);
-      if (result != null) {
-        result.put(QUEUE_OPERATION, operation.action.toLower());
-        req.getContext().put(ZkNodeProps.class.getName(),new ZkNodeProps(result) );
-      }
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/test/org/apache/solr/handler/admin/TestV2CoreAdminAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestV2CoreAdminAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestV2CoreAdminAPIs.java
deleted file mode 100644
index aeee949..0000000
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestV2CoreAdminAPIs.java
+++ /dev/null
@@ -1,104 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-
-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.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.PluginBag;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.easymock.EasyMock;
-import org.easymock.IAnswer;
-
-import static org.easymock.EasyMock.getCurrentArguments;
-
-public class TestV2CoreAdminAPIs extends SolrTestCaseJ4 {
-
-  public void testCreate() throws Exception {
-    Map<String, Object[]> calls = new HashMap<>();
-    CoreContainer mockCC = getCoreContainerMock(calls, new HashMap<>());
-
-    CoreAdminHandler  coreAdminHandler = new CoreAdminHandler(mockCC);
-    ApiBag apiBag = new ApiBag();
-    for (V2Api v2Api : coreAdminHandler.getApis()) {
-      apiBag.register(v2Api, Collections.EMPTY_MAP);
-    }
-    TestV2CollectionAPIs.makeCall(apiBag,"/cores", SolrRequest.METHOD.POST ,
-        "{create:{name: hello, instanceDir : someDir, config : 'solrconfig.xml', schema: 'schema.xml'}}", mockCC);
-    Object[] create = calls.get("create");
-    assertEquals("hello" ,create[0]);
-    HashMap expected = new HashMap();
-    expected.put("schema", "schema.xml");
-    expected.put("config", "solrconfig.xml");
-    assertEquals(expected ,create[2]);
-
-  }
-
-  public static CoreContainer getCoreContainerMock(final Map<String, Object[]> in,Map<String,Object> out ) {
-    CoreContainer mockCC = EasyMock.createMock(CoreContainer.class);
-    EasyMock.reset(mockCC);
-    mockCC.create(EasyMock.anyObject(String.class), EasyMock.anyObject(Path.class ) ,EasyMock.anyObject(Map.class));
-    EasyMock.expectLastCall().andAnswer(new IAnswer<SolrCore>() {
-      @Override
-      public SolrCore answer() throws Throwable {
-        in.put("create", getCurrentArguments());
-        return null;
-      }
-    }).anyTimes();
-
-    mockCC.getCoreRootDirectory();
-    EasyMock.expectLastCall().andAnswer(new IAnswer<Path>() {
-      @Override
-      public Path answer() throws Throwable {
-        return Paths.get("coreroot");
-      }
-    }).anyTimes();
-    mockCC.getContainerProperties();
-    EasyMock.expectLastCall().andAnswer(new IAnswer<Properties>() {
-      @Override
-      public Properties answer() throws Throwable {
-        return new Properties();
-      }
-    }).anyTimes();
-
-    mockCC.getRequestHandlers();
-    EasyMock.expectLastCall().andAnswer(new IAnswer<PluginBag>() {
-      @Override
-      public PluginBag answer() throws Throwable {
-        return (PluginBag)out.get("getRequestHandlers");
-      }
-    }).anyTimes();
-
-    EasyMock.replay(mockCC);
-    return mockCC;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/test/org/apache/solr/handler/admin/TestV2Framework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestV2Framework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestV2Framework.java
deleted file mode 100644
index 81104c8..0000000
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestV2Framework.java
+++ /dev/null
@@ -1,187 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-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.params.MapSolrParams;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-import org.apache.solr.common.util.Predicate;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-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.SolrRequestHandler;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.v2api.PathTrie;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2HttpCall;
-import org.apache.solr.v2api.V2RequestContext;
-
-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.CORES_HANDLER_PATH;
-import static org.apache.solr.common.util.Map2.NOT_NULL;
-import static org.apache.solr.handler.admin.TestV2CollectionAPIs.getV2RequestContext;
-
-public class TestV2Framework extends SolrTestCaseJ4 {
-
-  public void testFramework() {
-    Map<String, Object[]> calls = new HashMap<>();
-    Map<String, Object> out = new HashMap<>();
-    CoreContainer mockCC = TestV2CoreAdminAPIs.getCoreContainerMock(calls, out);
-    PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null, false);
-    containerHandlers.put(COLLECTIONS_HANDLER_PATH, new TestV2CollectionAPIs.MockCollectionsHandler());
-    containerHandlers.put(CORES_HANDLER_PATH, new CoreAdminHandler(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";
-    V2Api api = V2HttpCall.getApiInfo(containerHandlers, fullPath, "GET",
-        mockCC, "collections", fullPath, parts);
-    assertNotNull(api);
-    assertConditions(api.getSpec(), Utils.makeMap(
-        "/methods[0]", "GET",
-        "/methods[1]", "POST",
-        "/commands/create", NOT_NULL));
-    assertEquals("hello", parts.get("collection"));
-
-    parts = new HashMap<>();
-    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1", "GET",
-        mockCC, "collections", null, parts);
-    assertConditions(api.getSpec(), Utils.makeMap(
-        "/methods[0]", "POST",
-        "/methods[1]", "GET",
-        "/methods[2]", "DELETE",
-        "/commands/split", NOT_NULL,
-        "/commands/add-replica", NOT_NULL,
-        "/commands/force-leader", NOT_NULL
-    ));
-    assertEquals("hello", parts.get("collection"));
-    assertEquals("shard1", parts.get("shard"));
-
-
-    parts = new HashMap<>();
-    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "GET",
-        mockCC, "collections", null, parts);
-    assertConditions(api.getSpec(), Utils.makeMap(
-        "/methods[0]", "GET",
-        "/methods[1]", "POST",
-        "/commands/set", NOT_NULL
-    ));
-    assertEquals("hello", parts.get("collection"));
-    assertEquals("shard1", parts.get("shard"));
-    assertEquals("replica1", parts.get("replica"));
-
-    api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "DELETE",
-        mockCC, "collections", 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, "/collections/_introspect", GET, mockCC);
-
-    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
-        "/spec[0]/methods[0]", "POST",
-        "/spec[0]/methods[1]", "GET"));
-
-    rsp = invoke(coreHandlers, "/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", 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
-    ));
-
-    rsp = invoke(coreHandlers,"/collections/hello/schema",SolrRequest.METHOD.POST, mockCC);
-
-
-
-  }
-
-  private SolrQueryResponse invoke(PluginBag<SolrRequestHandler> reqHandlers, String path, SolrRequest.METHOD method,
-                                   CoreContainer mockCC) {
-    HashMap<String, String> parts = new HashMap<>();
-    boolean containerHandlerLookup = mockCC.getRequestHandlers() == reqHandlers;
-    String fullPath = path;
-    String prefix = null;
-    if (!containerHandlerLookup) {
-      int idx = path.indexOf('/', 1);
-      prefix = path.substring(1, idx);
-      if (idx > 0) idx = path.indexOf('/', idx + 1);
-      path = idx == -1 ? "/" : path.substring(idx);
-    }
-
-    V2Api api = V2HttpCall.getApiInfo(reqHandlers, path, "GET", mockCC, prefix, fullPath, parts);
-    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>()));
-    V2RequestContext ctx = getV2RequestContext( path, method, null, mockCC, parts, api, req);
-    api.call(ctx);
-    return ctx.getResponse();
-
-  }
-
-
-  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 Predicate) {
-        Predicate value = (Predicate) 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);
-      }
-    }
-
-  }
-
-
-}


[2/3] lucene-solr git commit: SOLR-8029 renamed class/dirs and removed the 'v2' part

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/v2api/V2Api.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/V2Api.java b/solr/core/src/java/org/apache/solr/v2api/V2Api.java
deleted file mode 100644
index 031a1cd..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/V2Api.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.solr.v2api;
-
-/*
- * 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.
- */
-
-
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-
-public abstract class V2Api implements SpecProvider {
-  protected Map2 spec;
-
-  protected V2Api(Map2 spec) {
-    this.spec = spec;
-  }
-
-
-  public abstract void call(V2RequestContext ctx);
-
-  @Override
-  public Map2 getSpec() {
-    return spec;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/v2api/V2ApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/V2ApiSupport.java b/solr/core/src/java/org/apache/solr/v2api/V2ApiSupport.java
deleted file mode 100644
index fa86b7b..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/V2ApiSupport.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.apache.solr.v2api;
-
-/*
- * 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.
- */
-
-import java.util.Collection;
-
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-
-public interface V2ApiSupport {
-
-  Collection<V2Api> getApis();
-
-
-  default boolean registerAutomatically() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java
index 185f208..9357e2d 100644
--- a/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java
+++ b/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java
@@ -68,7 +68,7 @@ import static org.apache.solr.servlet.SolrDispatchFilter.Action.PROCESS;
 
 public class V2HttpCall extends HttpSolrCall {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private V2Api api;
+  private Api api;
   private List<String> pieces;
   private String prefix;
   HashMap<String, String> parts = new HashMap<>();
@@ -166,12 +166,12 @@ public class V2HttpCall extends HttpSolrCall {
     return collection;
   }
 
-  public static V2Api getApiInfo(PluginBag<SolrRequestHandler> requestHandlers,
+  public static Api getApiInfo(PluginBag<SolrRequestHandler> requestHandlers,
                                  String path, String method,
                                  CoreContainer cores, String prefix, String fullPath,
                                  Map<String, String> parts) {
     if (fullPath == null) fullPath = path;
-    V2Api api;
+    Api api;
     boolean containerHandlerLookup = cores.getRequestHandlers() == requestHandlers;
     api = requestHandlers.v2lookup(path, method, parts);
     if (api == null && path.endsWith(ApiBag.INTROSPECT)) {
@@ -205,7 +205,7 @@ public class V2HttpCall extends HttpSolrCall {
 
   private static void getSubPaths(String path, ApiBag bag, Map<String, Set<String>> pathsVsMethod) {
     for (SolrRequest.METHOD m : SolrRequest.METHOD.values()) {
-      PathTrie<V2Api> registry = bag.getRegistry(m.toString());
+      PathTrie<Api> registry = bag.getRegistry(m.toString());
       if (registry != null) {
         HashSet<String> subPaths = new HashSet<>();
         registry.lookup(path, new HashMap<>(), subPaths);
@@ -218,20 +218,20 @@ public class V2HttpCall extends HttpSolrCall {
     }
   }
 
-  private static V2Api mergeIntrospect(PluginBag<SolrRequestHandler> requestHandlers,
+  private static Api mergeIntrospect(PluginBag<SolrRequestHandler> requestHandlers,
                                        String path, String method, Map<String, String> parts) {
-    V2Api api;
-    final Map<String, V2Api> apis = new LinkedHashMap<>();
+    Api api;
+    final Map<String, Api> apis = new LinkedHashMap<>();
     for (String m : SolrRequest.SUPPORTED_METHODS) {
       api = requestHandlers.v2lookup(path, m, parts);
       if (api != null) apis.put(m, api);
     }
-    api = new V2Api(ApiBag.INTROSPECT_SPEC) {
+    api = new Api(ApiBag.INTROSPECT_SPEC) {
       @Override
       public void call(V2RequestContext ctx) {
         String method = ctx.getSolrRequest().getParams().get("method");
-        Set<V2Api> added = new HashSet<>();
-        for (Map.Entry<String, V2Api> e : apis.entrySet()) {
+        Set<Api> added = new HashSet<>();
+        for (Map.Entry<String, Api> e : apis.entrySet()) {
           if (method == null || e.getKey().equals(ctx.getHttpMethod())) {
             if (!added.contains(e.getValue())) {
               e.getValue().call(ctx);
@@ -244,8 +244,8 @@ public class V2HttpCall extends HttpSolrCall {
     return api;
   }
 
-  private static V2Api getSubPathImpl(final Map<String, Set<String>> subpaths, String path) {
-    return new V2Api(Map2.EMPTY) {
+  private static Api getSubPathImpl(final Map<String, Set<String>> subpaths, String path) {
+    return new Api(Map2.EMPTY) {
       @Override
       public void call(V2RequestContext ctx) {
         ctx.getResponse().add("msg", "Invalid path, try the following");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..bb52f98
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.json
@@ -0,0 +1,17 @@
+{
+  "cluster": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
+    "methods": ["GET"],
+    "url": {
+      "paths": [
+        "/cluster",
+        "/cluster/overseer",
+        "/cluster/nodes",
+        "/cluster/command-status",
+        "/cluster/command-status/{id}",
+        "/cluster/command-status/{id}"
+      ]
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/cluster.security.BasicAuth.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.security.BasicAuth.json b/solr/core/src/resources/apispec/cluster.security.BasicAuth.json
new file mode 100644
index 0000000..44d536c
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.BasicAuth.json
@@ -0,0 +1,25 @@
+{
+  "cluster.security.BasicAuth": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/cluster/security/authentication"
+      ]
+    },
+    "commands": {
+      "set-user": {
+        "description": "The set-user command allows you to add users and change their passwords",
+        "properties": {},
+        "additionalProperties": true
+      },
+      "delete-user": {
+        "description": "",
+        "type": "list"
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..4d28fec
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.RuleBasedAuthorization.json
@@ -0,0 +1,56 @@
+{
+  "cluster.security.RuleBasedAuthorization": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/cluster/security/authorization"
+      ]
+    },
+    "commands": {
+      "set-permission": {
+        "description": "create a new permission, overwrite an existing permission definition, or assign a pre-defined permission to a role.",
+        "properties": {
+          "name":{
+            "type":"string",
+            "description":""
+          },
+          "method":{
+            "type":"string",
+            "description":""
+          },
+
+          "collection":{
+            "type":"list",
+            "description":""
+          },
+
+          "path":{
+            "type":"list",
+            "description":""
+          },
+          "before":{
+            "type":"string",
+            "description":""
+          },
+          "params":{
+            "type":"object",
+            "properties":{},
+            "additionalProperties":true,
+            "description":""
+          }
+        },
+        "required":["name","role"]
+      },
+      "set-user-role": {
+        "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",
+        "properties":{},
+        "additionalProperties":true
+
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..4f55c4f
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authentication.json
@@ -0,0 +1,15 @@
+{
+  "cluster.security.authentication": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/cluster/security/authentication"
+      ]
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..c5b08ea
--- /dev/null
+++ b/solr/core/src/resources/apispec/cluster.security.authorization.json
@@ -0,0 +1,15 @@
+{
+  "cluster.security.authorization": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/cluster/security/authorization"
+      ]
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..c5a194d
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.Commands.json
@@ -0,0 +1,82 @@
+{
+  "collections.Commands": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
+    "methods": [
+      "POST",
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/collections",
+        "/c"
+      ]
+    },
+    "commands": {
+      "create": {
+        "properties": {
+          "name": {
+            "type": "string",
+            "description": ""
+          },
+          "config": {
+            "type": "string",
+            "description": ""
+          },
+          "router": {
+            "type": "object",
+            "properties": {
+              "name": {
+              },
+              "field": {
+              }
+            }
+          },
+          "numShards": {
+            "type": "string",
+            "description": ""
+          },
+          "shards": {
+            "type": "string",
+            "description": ""
+          },
+          "replicationFactor": {
+            "type": "string",
+            "description": ""
+          },
+          "createNodeSet": {
+            "type": "string",
+            "description": ""
+          },
+          "autoAddReplicas": {
+            "type": "boolean",
+            "description": ""
+          },
+          "rule": {
+            "type": "list",
+            "description":""
+
+          },
+          "snitch": {
+            "type": "list",
+            "description":""
+          }
+        },
+        "required":["name"]
+      },
+      "create-alias":{
+        "properties": {
+          "name": {
+            "type": "string",
+            "description": "The alias name to be created"
+          },
+          "collections" :{
+            "type":"list",
+            "description":"The list of collections to be aliased"
+          }
+        },
+        "required" : ["name","collections"]
+      },
+      "delete-alias":{}
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..9566787
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.json
@@ -0,0 +1,19 @@
+{
+  "collections.collection.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "DELETE"
+    ],
+    "url": {
+      "paths": [
+        "/collections/{collection}",
+        "/c/{collection}"
+      ]
+    },
+    "commands": {
+      "modify": "collections.collection.Commands.modify",
+      "reload": "collections.collection.Commands.reload"
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..b78e79d
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.modify.json
@@ -0,0 +1,6 @@
+{
+  "additionalProperties":true,
+  "properties":{
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..7d53e5b
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.Commands.reload.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties" : true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..97bb22d
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
@@ -0,0 +1,21 @@
+{
+  "collections.collection.shards.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "GET",
+      "POST"
+    ],
+    "url": {
+      "paths": [
+        "/collections/{collection}/shards",
+        "/c/{collection}/shards"
+      ]
+    },
+    "commands": {
+      "create": {
+        "properties":{},
+        "additionalProperties":true
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..fceaeeb
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.Commands.json
@@ -0,0 +1,28 @@
+{
+  "collections.collection.shards.shard.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST",
+      "GET",
+      "DELETE"
+    ],
+    "url": {
+      "paths": [
+        "/collections/{collection}/shards/{shard}",
+        "/c/{collection}/shards/{shard}"
+      ]
+    },
+    "commands": {
+      "split": {
+        "properties": {},
+        "additionalProperties": true
+      },
+      "add-replica": {
+        "additionalProperties": true
+      },
+      "force-leader": {
+        "additionalProperties": true
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.Commands.json b/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.Commands.json
new file mode 100644
index 0000000..e7a43d5
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.Commands.json
@@ -0,0 +1,20 @@
+{
+  "collections.collection.shards.shard.replica.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "GET",
+      "POST"
+    ],
+    "url": {
+      "paths": [
+        "/collections/{collection}/shards/{shard}/{replica}",
+        "/c/{collection}/shards/{shard}/{replica}"
+      ]
+    },
+    "commands": {
+      "set": {
+        "additionalProperties": true
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..6f1243d
--- /dev/null
+++ b/solr/core/src/resources/apispec/collections.collection.shards.shard.replica.delete.json
@@ -0,0 +1,23 @@
+{
+  "collections.collection.shards.shard.replica.delete": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "DELETE"
+    ],
+    "url": {
+      "paths": [
+        "/collections/{collection}/shards/{shard}/{replica}",
+        "/c/{collection}/shards/{shard}/{replica}"
+      ],
+      "params":{
+        "onlyIfDown" : {
+          "type":"boolean",
+          "default":false,
+          "description":""
+        }
+      }
+
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.ConfigEdit.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.ConfigEdit.json b/solr/core/src/resources/apispec/core.ConfigEdit.json
new file mode 100644
index 0000000..0146809
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.ConfigEdit.json
@@ -0,0 +1,115 @@
+{
+  "core.ConfigEdit": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": [
+      "POST"
+    ],
+    "url": {
+      "paths": [
+        "$handlerName",
+        "$handlerName/params"
+      ]
+    },
+    "commands": {
+      "add-requesthandler": {
+        "additionalProperties": true
+      },
+      "update-requesthandler": {
+        "additionalProperties": true
+      },
+      "delete-requesthandler": {
+        "additionalProperties": true
+      },
+      "add-searchcomponent": {
+        "additionalProperties": true
+      },
+      "update-searchcomponent": {
+        "additionalProperties": true
+      },
+      "delete-searchcomponent": {
+        "additionalProperties": true
+      },
+      "add-initparams": {
+        "additionalProperties": true
+      },
+      "update-initparams": {
+        "additionalProperties": true
+      },
+      "delete-initparams": {
+        "additionalProperties": true
+      },
+      "add-queryresponsewriter": {
+        "additionalProperties": true
+      },
+      "update-queryresponsewriter": {
+        "additionalProperties": true
+      },
+      "delete-queryresponsewriter": {
+        "additionalProperties": true
+      },
+      "add-queryparser": {
+        "additionalProperties": true
+      },
+      "update-queryparser": {
+        "additionalProperties": true
+      },
+      "delete-queryparser": {
+        "additionalProperties": true
+      },
+      "add-valuesourceparser": {
+        "additionalProperties": true
+      },
+      "update-valuesourceparser": {
+        "additionalProperties": true
+      },
+      "delete-valuesourceparser": {
+        "additionalProperties": true
+      },
+      "add-transformer": {
+        "additionalProperties": true
+      },
+      "update-transformer": {
+        "additionalProperties": true
+      },
+      "delete-transformer": {
+        "additionalProperties": true
+      },
+      "add-updateprocessor": {
+        "additionalProperties": true
+      },
+      "update-updateprocessor": {
+        "additionalProperties": true
+      },
+      "delete-updateprocessor": {
+        "additionalProperties": true
+      },
+      "add-queryconverter": {
+        "additionalProperties": true
+      },
+      "update-queryconverter": {
+        "additionalProperties": true
+      },
+      "delete-queryconverter": {
+        "additionalProperties": true
+      },
+      "add-listener": {
+        "additionalProperties": true
+      },
+      "update-listener": {
+        "additionalProperties": true
+      },
+      "delete-listener": {
+        "additionalProperties": true
+      },
+      "add-runtimelib": {
+        "additionalProperties": true
+      },
+      "update-runtimelib": {
+        "additionalProperties": true
+      },
+      "delete-runtimelib": {
+        "additionalProperties": true
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.ConfigRead.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.ConfigRead.json b/solr/core/src/resources/apispec/core.ConfigRead.json
new file mode 100644
index 0000000..b86a5e4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.ConfigRead.json
@@ -0,0 +1,19 @@
+{
+  "core.ConfigRead": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
+    "methods": ["GET"],
+    "url": {
+      "paths": [
+        "$handlerName",
+        "$handlerName/overlay",
+        "$handlerName/params",
+        "$handlerName/params",
+        "$handlerName/query",
+        "$handlerName/jmx",
+        "$handlerName/requestDispatcher",
+        "$handlerName/znodeVersion",
+        "$handlerName/{plugin}"
+      ] },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..20a1cfe
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.RealtimeGet.json
@@ -0,0 +1,23 @@
+{
+  "core.RealtimeGet": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": ["GET"],
+    "url": {
+      "path": "/get",
+      "paths": [
+        "$handlerName",
+        "$handlerName/versions",
+        "$handlerName/updates"],
+      "params":{
+        "id" : {
+          "type":"string",
+          "description" :"one or more ids. Separate by commas if there are more than one"
+        },
+        "ids" : {
+          "type":"string",
+          "description" :"one or more ids. Separate by commas if there are more than one"
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..782efe2
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addCopyField.json
@@ -0,0 +1,4 @@
+{
+  "additionalProperties":true
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.SchemaEdit.addDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.addDynamicField.json b/solr/core/src/resources/apispec/core.SchemaEdit.addDynamicField.json
new file mode 100644
index 0000000..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addDynamicField.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..bc28113
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addField.json
@@ -0,0 +1,4 @@
+{
+  "properties":{},
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..bc28113
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.addFieldType.json
@@ -0,0 +1,4 @@
+{
+  "properties":{},
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..bc28113
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteCopyField.json
@@ -0,0 +1,4 @@
+{
+  "properties":{},
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteDynamicField.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteField.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.deleteFieldType.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..59cdd5f
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.json
@@ -0,0 +1,22 @@
+{
+  "core.SchemaEdit": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": ["POST"],
+    "url": {
+      "paths": ["$handlerName"]
+    },
+    "commands":{
+      "add-field":"core.SchemaEdit.addField",
+      "delete-field":"core.SchemaEdit.deleteField",
+      "replace-field":"core.SchemaEdit.replaceField",
+      "add-dynamic-field":"core.SchemaEdit.addDynamicField",
+      "delete-dynamic-field":"core.SchemaEdit.deleteDynamicField",
+      "replace-dynamic-field":"core.SchemaEdit.replaceDynamicField",
+      "add-field-type":"core.SchemaEdit.addFieldType",
+      "delete-field-type":"core.SchemaEdit.deleteFieldType",
+      "replace-field-type":"core.SchemaEdit.replaceFieldType",
+      "add-copy-field":"core.SchemaEdit.addCopyField",
+      "delete-copy-field":"core.SchemaEdit.deleteCopyField"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.SchemaEdit.replaceDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.replaceDynamicField.json b/solr/core/src/resources/apispec/core.SchemaEdit.replaceDynamicField.json
new file mode 100644
index 0000000..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.replaceDynamicField.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.SchemaEdit.replaceField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.replaceField.json b/solr/core/src/resources/apispec/core.SchemaEdit.replaceField.json
new file mode 100644
index 0000000..bc28113
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.replaceField.json
@@ -0,0 +1,4 @@
+{
+  "properties":{},
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/core.SchemaEdit.replaceFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/core.SchemaEdit.replaceFieldType.json b/solr/core/src/resources/apispec/core.SchemaEdit.replaceFieldType.json
new file mode 100644
index 0000000..2fe97f4
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaEdit.replaceFieldType.json
@@ -0,0 +1,3 @@
+{
+  "additionalProperties":true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..338affd
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.copyFields.json
@@ -0,0 +1,23 @@
+{
+  "core.SchemaRead.copyFields": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": [
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "$handlerName/copyfields",
+        "$handlerName/copyfields/{name}"
+      ],
+      "params": {
+        "source.fl" :{
+          "type":"string",
+          "description":"Comma- or space-separated list of one or more copyField source fields to include in the response - copyField directives with all other source fields will be excluded from the response. If not specified, all copyField-s will be included in the response"},
+        "dest.fl" :{
+          "type":"string",
+          "description":"Comma- or space-separated list of one or more copyField dest fields to include in the response - copyField directives with all other dest fields will be excluded. If not specified, all copyField-s will be included in the response."}
+      }
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..1b6f924
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.fields.json
@@ -0,0 +1,31 @@
+{
+  "core.SchemaRead.fields": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": [
+      "GET"
+    ],
+    "url": {
+      "paths": [
+        "/$handlerName/fields",
+        "/$handlerName/fields/{name}",
+        "/$handlerName/dynamicfields",
+        "/$handlerName/dynamicfields/{name}",
+        "/$handlerName/fieldtypes/",
+        "/$handlerName/fieldtypes/{name}"
+      ],
+      "params": {
+        "includeDynamic" :{
+          "type":"boolean",
+          "description":"If true, and if the fl query parameter is specified or the fieldname path parameter is used, matching dynamic fields are included in the response and identified with the dynamicBase property. If neither the fl query parameter nor the fieldname path parameter is specified, the includeDynamic query parameter is ignored. If false, matching dynamic fields will not be returned.",
+          "default":false
+        },
+        "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.",
+          "default":false
+        }
+      }
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..b76070b
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.SchemaRead.json
@@ -0,0 +1,25 @@
+{
+  "core.SchemaRead": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr$handlerName+API",
+    "methods": ["GET"],
+    "url": {
+      "paths": [
+        "$handlerName",
+        "$handlerName/name",
+        "$handlerName/uniquekey",
+        "$handlerName/version",
+        "$handlerName/similarity",
+        "$handlerName/solrqueryparser",
+        "$handlerName/zkversion",
+        "$handlerName/zkversion",
+        "$handlerName/solrqueryparser/defaultoperator",
+        "$handlerName/name",
+        "$handlerName/version",
+        "$handlerName/uniquekey",
+        "$handlerName/similarity",
+        "$handlerName/similarity"
+      ]
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..cf2c2ce
--- /dev/null
+++ b/solr/core/src/resources/apispec/core.Update.json
@@ -0,0 +1,19 @@
+{
+  "core.Update": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": [
+      "POST"
+    ],
+    "url": {
+      "path": "/update",
+      "paths": [
+        "/update",
+        "/update/xml",
+        "/update/csv",
+        "/update/json",
+        "/update/json/commands"
+      ]
+
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..784d9bf
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.Commands.json
@@ -0,0 +1,67 @@
+{
+  "cores.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST"
+    ],
+    "url": {
+      "path": "/cores",
+      "paths": [
+        "/cores"
+      ]
+    },
+    "commands": {
+      "create": {
+        "properties": {
+          "name": {
+            "type": "string",
+            "description": "The core name"
+          },
+          "instanceDir":{
+            "type": "string",
+            "description": "The core instance dir"
+          },
+          "schema": {
+            "type": "string",
+            "description": "The core name"
+          },
+          "dataDir": {
+            "type": "string",
+            "description": "The core name"
+          },
+          "configSet": {
+            "type": "string",
+            "description": "The configset name"
+          },
+          "loadOnStartup": {
+            "type": "boolean",
+            "description": "Load the core on startup"
+          },
+          "transient": {
+            "type": "boolean",
+            "description": "The core may be unloaded if required"
+          },
+          "shard": {
+            "type": "string",
+            "description": "In SolrCloud mode, which shard does this core belong to"
+          },
+          "collection": {
+            "type": "string",
+            "description": "The collection where this core belongs to"
+          },
+          "props":{
+            "type":"object",
+            "additionalProperties": true
+          },
+          "coreNodeName": {
+            "type":"string",
+            "description":"The replica name"
+          },
+          "numShards": {
+          }
+        },
+        "required":["name","configset"]
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..0bf41aa
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.Status.json
@@ -0,0 +1,23 @@
+{
+  "cores.Status": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "GET"
+    ],
+    "url": {
+      "path": "/cores",
+      "paths": [
+        "/cores",
+        "/cores/{core}/status"
+      ],
+      "params": {
+        "indexInfo": {
+          "type": "boolean",
+          "description": "return index info",
+          "default": false
+        }
+      }
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..c8181b2
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.core.Commands.json
@@ -0,0 +1,36 @@
+{
+  "cores.core.Commands": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "POST"
+    ],
+    "url": {
+      "path": "/cores/{core}",
+      "paths": [
+        "/cores/{core}"
+      ]
+    },
+    "commands": {
+      "reload": {
+        "properties": {}
+      },
+      "unload":{
+        "properties": {}
+      },
+      "swap": {
+        "properties": {
+          "with" : {
+            "type":"string",
+            "description":"The other core name"
+          }
+        },
+        "required":["with"]
+      },
+      "merge-indexes": {
+        "properties": {}
+      },
+      "request-recovery": {},
+      "split":"cores.core.Commands.split"
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/cores.core.Commands.requestRecovery.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.core.Commands.requestRecovery.json b/solr/core/src/resources/apispec/cores.core.Commands.requestRecovery.json
new file mode 100644
index 0000000..f5fff5b
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.core.Commands.requestRecovery.json
@@ -0,0 +1,4 @@
+{
+  "properties": {},
+  "additionalProperties": true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..8f2512d
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.core.Commands.split.json
@@ -0,0 +1,3 @@
+{
+  "properties":{}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/apispec/cores.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cores.json b/solr/core/src/resources/apispec/cores.json
new file mode 100644
index 0000000..4157df1
--- /dev/null
+++ b/solr/core/src/resources/apispec/cores.json
@@ -0,0 +1,22 @@
+{
+  "cores": {
+    "documentation": "https://cwiki.apache.org",
+    "methods": [
+      "GET"
+    ],
+    "url": {
+      "path": "/cores",
+      "paths": [
+        "/cores"
+      ],
+      "params": {
+        "indexInfo": {
+          "type": "boolean",
+          "description": "return index info",
+          "default": false
+        }
+      }
+    },
+    "body": null
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..0c5125b
--- /dev/null
+++ b/solr/core/src/resources/apispec/emptySpec.json
@@ -0,0 +1,15 @@
+{
+  "emptySpec": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": [
+      "GET",
+      "POST"
+    ],
+    "url": {
+      "path": "$handlerName",
+      "paths": [
+        "$handlerName"
+      ]
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..687b6b6
--- /dev/null
+++ b/solr/core/src/resources/apispec/node.Info.json
@@ -0,0 +1,14 @@
+{
+  "node.Info": {
+    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
+    "methods": ["GET"],
+    "url": {
+      "path": "/node",
+      "paths": [
+        "/node/properties",
+        "/node/threads",
+        "/node/logging",
+        "/node/system"]
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cluster.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cluster.json b/solr/core/src/resources/v2apispec/cluster.json
deleted file mode 100644
index bb52f98..0000000
--- a/solr/core/src/resources/v2apispec/cluster.json
+++ /dev/null
@@ -1,17 +0,0 @@
-{
-  "cluster": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
-    "methods": ["GET"],
-    "url": {
-      "paths": [
-        "/cluster",
-        "/cluster/overseer",
-        "/cluster/nodes",
-        "/cluster/command-status",
-        "/cluster/command-status/{id}",
-        "/cluster/command-status/{id}"
-      ]
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cluster.security.BasicAuth.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cluster.security.BasicAuth.json b/solr/core/src/resources/v2apispec/cluster.security.BasicAuth.json
deleted file mode 100644
index 44d536c..0000000
--- a/solr/core/src/resources/v2apispec/cluster.security.BasicAuth.json
+++ /dev/null
@@ -1,25 +0,0 @@
-{
-  "cluster.security.BasicAuth": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/cluster/security/authentication"
-      ]
-    },
-    "commands": {
-      "set-user": {
-        "description": "The set-user command allows you to add users and change their passwords",
-        "properties": {},
-        "additionalProperties": true
-      },
-      "delete-user": {
-        "description": "",
-        "type": "list"
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cluster.security.RuleBasedAuthorization.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cluster.security.RuleBasedAuthorization.json b/solr/core/src/resources/v2apispec/cluster.security.RuleBasedAuthorization.json
deleted file mode 100644
index 4d28fec..0000000
--- a/solr/core/src/resources/v2apispec/cluster.security.RuleBasedAuthorization.json
+++ /dev/null
@@ -1,56 +0,0 @@
-{
-  "cluster.security.RuleBasedAuthorization": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/cluster/security/authorization"
-      ]
-    },
-    "commands": {
-      "set-permission": {
-        "description": "create a new permission, overwrite an existing permission definition, or assign a pre-defined permission to a role.",
-        "properties": {
-          "name":{
-            "type":"string",
-            "description":""
-          },
-          "method":{
-            "type":"string",
-            "description":""
-          },
-
-          "collection":{
-            "type":"list",
-            "description":""
-          },
-
-          "path":{
-            "type":"list",
-            "description":""
-          },
-          "before":{
-            "type":"string",
-            "description":""
-          },
-          "params":{
-            "type":"object",
-            "properties":{},
-            "additionalProperties":true,
-            "description":""
-          }
-        },
-        "required":["name","role"]
-      },
-      "set-user-role": {
-        "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",
-        "properties":{},
-        "additionalProperties":true
-
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cluster.security.authentication.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cluster.security.authentication.json b/solr/core/src/resources/v2apispec/cluster.security.authentication.json
deleted file mode 100644
index 4f55c4f..0000000
--- a/solr/core/src/resources/v2apispec/cluster.security.authentication.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
-  "cluster.security.authentication": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/cluster/security/authentication"
-      ]
-    }
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cluster.security.authorization.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cluster.security.authorization.json b/solr/core/src/resources/v2apispec/cluster.security.authorization.json
deleted file mode 100644
index c5b08ea..0000000
--- a/solr/core/src/resources/v2apispec/cluster.security.authorization.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
-  "cluster.security.authorization": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/cluster/security/authorization"
-      ]
-    }
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.Commands.json b/solr/core/src/resources/v2apispec/collections.Commands.json
deleted file mode 100644
index c5a194d..0000000
--- a/solr/core/src/resources/v2apispec/collections.Commands.json
+++ /dev/null
@@ -1,82 +0,0 @@
-{
-  "collections.Commands": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api1",
-    "methods": [
-      "POST",
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/collections",
-        "/c"
-      ]
-    },
-    "commands": {
-      "create": {
-        "properties": {
-          "name": {
-            "type": "string",
-            "description": ""
-          },
-          "config": {
-            "type": "string",
-            "description": ""
-          },
-          "router": {
-            "type": "object",
-            "properties": {
-              "name": {
-              },
-              "field": {
-              }
-            }
-          },
-          "numShards": {
-            "type": "string",
-            "description": ""
-          },
-          "shards": {
-            "type": "string",
-            "description": ""
-          },
-          "replicationFactor": {
-            "type": "string",
-            "description": ""
-          },
-          "createNodeSet": {
-            "type": "string",
-            "description": ""
-          },
-          "autoAddReplicas": {
-            "type": "boolean",
-            "description": ""
-          },
-          "rule": {
-            "type": "list",
-            "description":""
-
-          },
-          "snitch": {
-            "type": "list",
-            "description":""
-          }
-        },
-        "required":["name"]
-      },
-      "create-alias":{
-        "properties": {
-          "name": {
-            "type": "string",
-            "description": "The alias name to be created"
-          },
-          "collections" :{
-            "type":"list",
-            "description":"The list of collections to be aliased"
-          }
-        },
-        "required" : ["name","collections"]
-      },
-      "delete-alias":{}
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.Commands.json b/solr/core/src/resources/v2apispec/collections.collection.Commands.json
deleted file mode 100644
index 9566787..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.Commands.json
+++ /dev/null
@@ -1,19 +0,0 @@
-{
-  "collections.collection.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "DELETE"
-    ],
-    "url": {
-      "paths": [
-        "/collections/{collection}",
-        "/c/{collection}"
-      ]
-    },
-    "commands": {
-      "modify": "collections.collection.Commands.modify",
-      "reload": "collections.collection.Commands.reload"
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.Commands.modify.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.Commands.modify.json b/solr/core/src/resources/v2apispec/collections.collection.Commands.modify.json
deleted file mode 100644
index b78e79d..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.Commands.modify.json
+++ /dev/null
@@ -1,6 +0,0 @@
-{
-  "additionalProperties":true,
-  "properties":{
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.Commands.reload.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.Commands.reload.json b/solr/core/src/resources/v2apispec/collections.collection.Commands.reload.json
deleted file mode 100644
index 7d53e5b..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.Commands.reload.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties" : true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.shards.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.shards.Commands.json b/solr/core/src/resources/v2apispec/collections.collection.shards.Commands.json
deleted file mode 100644
index 97bb22d..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.shards.Commands.json
+++ /dev/null
@@ -1,21 +0,0 @@
-{
-  "collections.collection.shards.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "GET",
-      "POST"
-    ],
-    "url": {
-      "paths": [
-        "/collections/{collection}/shards",
-        "/c/{collection}/shards"
-      ]
-    },
-    "commands": {
-      "create": {
-        "properties":{},
-        "additionalProperties":true
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.shards.shard.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.Commands.json b/solr/core/src/resources/v2apispec/collections.collection.shards.shard.Commands.json
deleted file mode 100644
index fceaeeb..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.Commands.json
+++ /dev/null
@@ -1,28 +0,0 @@
-{
-  "collections.collection.shards.shard.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST",
-      "GET",
-      "DELETE"
-    ],
-    "url": {
-      "paths": [
-        "/collections/{collection}/shards/{shard}",
-        "/c/{collection}/shards/{shard}"
-      ]
-    },
-    "commands": {
-      "split": {
-        "properties": {},
-        "additionalProperties": true
-      },
-      "add-replica": {
-        "additionalProperties": true
-      },
-      "force-leader": {
-        "additionalProperties": true
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.Commands.json b/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.Commands.json
deleted file mode 100644
index e7a43d5..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.Commands.json
+++ /dev/null
@@ -1,20 +0,0 @@
-{
-  "collections.collection.shards.shard.replica.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "GET",
-      "POST"
-    ],
-    "url": {
-      "paths": [
-        "/collections/{collection}/shards/{shard}/{replica}",
-        "/c/{collection}/shards/{shard}/{replica}"
-      ]
-    },
-    "commands": {
-      "set": {
-        "additionalProperties": true
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.delete.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.delete.json b/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.delete.json
deleted file mode 100644
index 6f1243d..0000000
--- a/solr/core/src/resources/v2apispec/collections.collection.shards.shard.replica.delete.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "collections.collection.shards.shard.replica.delete": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "DELETE"
-    ],
-    "url": {
-      "paths": [
-        "/collections/{collection}/shards/{shard}/{replica}",
-        "/c/{collection}/shards/{shard}/{replica}"
-      ],
-      "params":{
-        "onlyIfDown" : {
-          "type":"boolean",
-          "default":false,
-          "description":""
-        }
-      }
-
-    }
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.ConfigEdit.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.ConfigEdit.json b/solr/core/src/resources/v2apispec/core.ConfigEdit.json
deleted file mode 100644
index 0146809..0000000
--- a/solr/core/src/resources/v2apispec/core.ConfigEdit.json
+++ /dev/null
@@ -1,115 +0,0 @@
-{
-  "core.ConfigEdit": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": [
-      "POST"
-    ],
-    "url": {
-      "paths": [
-        "$handlerName",
-        "$handlerName/params"
-      ]
-    },
-    "commands": {
-      "add-requesthandler": {
-        "additionalProperties": true
-      },
-      "update-requesthandler": {
-        "additionalProperties": true
-      },
-      "delete-requesthandler": {
-        "additionalProperties": true
-      },
-      "add-searchcomponent": {
-        "additionalProperties": true
-      },
-      "update-searchcomponent": {
-        "additionalProperties": true
-      },
-      "delete-searchcomponent": {
-        "additionalProperties": true
-      },
-      "add-initparams": {
-        "additionalProperties": true
-      },
-      "update-initparams": {
-        "additionalProperties": true
-      },
-      "delete-initparams": {
-        "additionalProperties": true
-      },
-      "add-queryresponsewriter": {
-        "additionalProperties": true
-      },
-      "update-queryresponsewriter": {
-        "additionalProperties": true
-      },
-      "delete-queryresponsewriter": {
-        "additionalProperties": true
-      },
-      "add-queryparser": {
-        "additionalProperties": true
-      },
-      "update-queryparser": {
-        "additionalProperties": true
-      },
-      "delete-queryparser": {
-        "additionalProperties": true
-      },
-      "add-valuesourceparser": {
-        "additionalProperties": true
-      },
-      "update-valuesourceparser": {
-        "additionalProperties": true
-      },
-      "delete-valuesourceparser": {
-        "additionalProperties": true
-      },
-      "add-transformer": {
-        "additionalProperties": true
-      },
-      "update-transformer": {
-        "additionalProperties": true
-      },
-      "delete-transformer": {
-        "additionalProperties": true
-      },
-      "add-updateprocessor": {
-        "additionalProperties": true
-      },
-      "update-updateprocessor": {
-        "additionalProperties": true
-      },
-      "delete-updateprocessor": {
-        "additionalProperties": true
-      },
-      "add-queryconverter": {
-        "additionalProperties": true
-      },
-      "update-queryconverter": {
-        "additionalProperties": true
-      },
-      "delete-queryconverter": {
-        "additionalProperties": true
-      },
-      "add-listener": {
-        "additionalProperties": true
-      },
-      "update-listener": {
-        "additionalProperties": true
-      },
-      "delete-listener": {
-        "additionalProperties": true
-      },
-      "add-runtimelib": {
-        "additionalProperties": true
-      },
-      "update-runtimelib": {
-        "additionalProperties": true
-      },
-      "delete-runtimelib": {
-        "additionalProperties": true
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.ConfigRead.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.ConfigRead.json b/solr/core/src/resources/v2apispec/core.ConfigRead.json
deleted file mode 100644
index b86a5e4..0000000
--- a/solr/core/src/resources/v2apispec/core.ConfigRead.json
+++ /dev/null
@@ -1,19 +0,0 @@
-{
-  "core.ConfigRead": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Config+API",
-    "methods": ["GET"],
-    "url": {
-      "paths": [
-        "$handlerName",
-        "$handlerName/overlay",
-        "$handlerName/params",
-        "$handlerName/params",
-        "$handlerName/query",
-        "$handlerName/jmx",
-        "$handlerName/requestDispatcher",
-        "$handlerName/znodeVersion",
-        "$handlerName/{plugin}"
-      ] },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.RealtimeGet.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.RealtimeGet.json b/solr/core/src/resources/v2apispec/core.RealtimeGet.json
deleted file mode 100644
index 20a1cfe..0000000
--- a/solr/core/src/resources/v2apispec/core.RealtimeGet.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "core.RealtimeGet": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": ["GET"],
-    "url": {
-      "path": "/get",
-      "paths": [
-        "$handlerName",
-        "$handlerName/versions",
-        "$handlerName/updates"],
-      "params":{
-        "id" : {
-          "type":"string",
-          "description" :"one or more ids. Separate by commas if there are more than one"
-        },
-        "ids" : {
-          "type":"string",
-          "description" :"one or more ids. Separate by commas if there are more than one"
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.addCopyField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.addCopyField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.addCopyField.json
deleted file mode 100644
index 782efe2..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.addCopyField.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "additionalProperties":true
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.addDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.addDynamicField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.addDynamicField.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.addDynamicField.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.addField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.addField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.addField.json
deleted file mode 100644
index bc28113..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.addField.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "properties":{},
-  "additionalProperties": true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.addFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.addFieldType.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.addFieldType.json
deleted file mode 100644
index bc28113..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.addFieldType.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "properties":{},
-  "additionalProperties": true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteCopyField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteCopyField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteCopyField.json
deleted file mode 100644
index bc28113..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteCopyField.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "properties":{},
-  "additionalProperties": true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteDynamicField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteDynamicField.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteDynamicField.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteField.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteField.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteFieldType.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteFieldType.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.deleteFieldType.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.json
deleted file mode 100644
index 59cdd5f..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.json
+++ /dev/null
@@ -1,22 +0,0 @@
-{
-  "core.SchemaEdit": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": ["POST"],
-    "url": {
-      "paths": ["$handlerName"]
-    },
-    "commands":{
-      "add-field":"core.SchemaEdit.addField",
-      "delete-field":"core.SchemaEdit.deleteField",
-      "replace-field":"core.SchemaEdit.replaceField",
-      "add-dynamic-field":"core.SchemaEdit.addDynamicField",
-      "delete-dynamic-field":"core.SchemaEdit.deleteDynamicField",
-      "replace-dynamic-field":"core.SchemaEdit.replaceDynamicField",
-      "add-field-type":"core.SchemaEdit.addFieldType",
-      "delete-field-type":"core.SchemaEdit.deleteFieldType",
-      "replace-field-type":"core.SchemaEdit.replaceFieldType",
-      "add-copy-field":"core.SchemaEdit.addCopyField",
-      "delete-copy-field":"core.SchemaEdit.deleteCopyField"
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceDynamicField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceDynamicField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceDynamicField.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceDynamicField.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceField.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceField.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceField.json
deleted file mode 100644
index bc28113..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceField.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "properties":{},
-  "additionalProperties": true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceFieldType.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceFieldType.json b/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceFieldType.json
deleted file mode 100644
index 2fe97f4..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaEdit.replaceFieldType.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "additionalProperties":true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaRead.copyFields.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaRead.copyFields.json b/solr/core/src/resources/v2apispec/core.SchemaRead.copyFields.json
deleted file mode 100644
index 338affd..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaRead.copyFields.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "core.SchemaRead.copyFields": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": [
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "$handlerName/copyfields",
-        "$handlerName/copyfields/{name}"
-      ],
-      "params": {
-        "source.fl" :{
-          "type":"string",
-          "description":"Comma- or space-separated list of one or more copyField source fields to include in the response - copyField directives with all other source fields will be excluded from the response. If not specified, all copyField-s will be included in the response"},
-        "dest.fl" :{
-          "type":"string",
-          "description":"Comma- or space-separated list of one or more copyField dest fields to include in the response - copyField directives with all other dest fields will be excluded. If not specified, all copyField-s will be included in the response."}
-      }
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaRead.fields.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaRead.fields.json b/solr/core/src/resources/v2apispec/core.SchemaRead.fields.json
deleted file mode 100644
index 1b6f924..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaRead.fields.json
+++ /dev/null
@@ -1,31 +0,0 @@
-{
-  "core.SchemaRead.fields": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": [
-      "GET"
-    ],
-    "url": {
-      "paths": [
-        "/$handlerName/fields",
-        "/$handlerName/fields/{name}",
-        "/$handlerName/dynamicfields",
-        "/$handlerName/dynamicfields/{name}",
-        "/$handlerName/fieldtypes/",
-        "/$handlerName/fieldtypes/{name}"
-      ],
-      "params": {
-        "includeDynamic" :{
-          "type":"boolean",
-          "description":"If true, and if the fl query parameter is specified or the fieldname path parameter is used, matching dynamic fields are included in the response and identified with the dynamicBase property. If neither the fl query parameter nor the fieldname path parameter is specified, the includeDynamic query parameter is ignored. If false, matching dynamic fields will not be returned.",
-          "default":false
-        },
-        "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.",
-          "default":false
-        }
-      }
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.SchemaRead.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.SchemaRead.json b/solr/core/src/resources/v2apispec/core.SchemaRead.json
deleted file mode 100644
index b76070b..0000000
--- a/solr/core/src/resources/v2apispec/core.SchemaRead.json
+++ /dev/null
@@ -1,25 +0,0 @@
-{
-  "core.SchemaRead": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr$handlerName+API",
-    "methods": ["GET"],
-    "url": {
-      "paths": [
-        "$handlerName",
-        "$handlerName/name",
-        "$handlerName/uniquekey",
-        "$handlerName/version",
-        "$handlerName/similarity",
-        "$handlerName/solrqueryparser",
-        "$handlerName/zkversion",
-        "$handlerName/zkversion",
-        "$handlerName/solrqueryparser/defaultoperator",
-        "$handlerName/name",
-        "$handlerName/version",
-        "$handlerName/uniquekey",
-        "$handlerName/similarity",
-        "$handlerName/similarity"
-      ]
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/core.Update.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/core.Update.json b/solr/core/src/resources/v2apispec/core.Update.json
deleted file mode 100644
index cf2c2ce..0000000
--- a/solr/core/src/resources/v2apispec/core.Update.json
+++ /dev/null
@@ -1,19 +0,0 @@
-{
-  "core.Update": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": [
-      "POST"
-    ],
-    "url": {
-      "path": "/update",
-      "paths": [
-        "/update",
-        "/update/xml",
-        "/update/csv",
-        "/update/json",
-        "/update/json/commands"
-      ]
-
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.Commands.json b/solr/core/src/resources/v2apispec/cores.Commands.json
deleted file mode 100644
index 784d9bf..0000000
--- a/solr/core/src/resources/v2apispec/cores.Commands.json
+++ /dev/null
@@ -1,67 +0,0 @@
-{
-  "cores.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST"
-    ],
-    "url": {
-      "path": "/cores",
-      "paths": [
-        "/cores"
-      ]
-    },
-    "commands": {
-      "create": {
-        "properties": {
-          "name": {
-            "type": "string",
-            "description": "The core name"
-          },
-          "instanceDir":{
-            "type": "string",
-            "description": "The core instance dir"
-          },
-          "schema": {
-            "type": "string",
-            "description": "The core name"
-          },
-          "dataDir": {
-            "type": "string",
-            "description": "The core name"
-          },
-          "configSet": {
-            "type": "string",
-            "description": "The configset name"
-          },
-          "loadOnStartup": {
-            "type": "boolean",
-            "description": "Load the core on startup"
-          },
-          "transient": {
-            "type": "boolean",
-            "description": "The core may be unloaded if required"
-          },
-          "shard": {
-            "type": "string",
-            "description": "In SolrCloud mode, which shard does this core belong to"
-          },
-          "collection": {
-            "type": "string",
-            "description": "The collection where this core belongs to"
-          },
-          "props":{
-            "type":"object",
-            "additionalProperties": true
-          },
-          "coreNodeName": {
-            "type":"string",
-            "description":"The replica name"
-          },
-          "numShards": {
-          }
-        },
-        "required":["name","configset"]
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.Status.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.Status.json b/solr/core/src/resources/v2apispec/cores.Status.json
deleted file mode 100644
index 0bf41aa..0000000
--- a/solr/core/src/resources/v2apispec/cores.Status.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "cores.Status": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "GET"
-    ],
-    "url": {
-      "path": "/cores",
-      "paths": [
-        "/cores",
-        "/cores/{core}/status"
-      ],
-      "params": {
-        "indexInfo": {
-          "type": "boolean",
-          "description": "return index info",
-          "default": false
-        }
-      }
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.core.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.core.Commands.json b/solr/core/src/resources/v2apispec/cores.core.Commands.json
deleted file mode 100644
index c8181b2..0000000
--- a/solr/core/src/resources/v2apispec/cores.core.Commands.json
+++ /dev/null
@@ -1,36 +0,0 @@
-{
-  "cores.core.Commands": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "POST"
-    ],
-    "url": {
-      "path": "/cores/{core}",
-      "paths": [
-        "/cores/{core}"
-      ]
-    },
-    "commands": {
-      "reload": {
-        "properties": {}
-      },
-      "unload":{
-        "properties": {}
-      },
-      "swap": {
-        "properties": {
-          "with" : {
-            "type":"string",
-            "description":"The other core name"
-          }
-        },
-        "required":["with"]
-      },
-      "merge-indexes": {
-        "properties": {}
-      },
-      "request-recovery": {},
-      "split":"cores.core.Commands.split"
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.core.Commands.requestRecovery.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.core.Commands.requestRecovery.json b/solr/core/src/resources/v2apispec/cores.core.Commands.requestRecovery.json
deleted file mode 100644
index f5fff5b..0000000
--- a/solr/core/src/resources/v2apispec/cores.core.Commands.requestRecovery.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
-  "properties": {},
-  "additionalProperties": true
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.core.Commands.split.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.core.Commands.split.json b/solr/core/src/resources/v2apispec/cores.core.Commands.split.json
deleted file mode 100644
index 8f2512d..0000000
--- a/solr/core/src/resources/v2apispec/cores.core.Commands.split.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "properties":{}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/cores.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/cores.json b/solr/core/src/resources/v2apispec/cores.json
deleted file mode 100644
index 4157df1..0000000
--- a/solr/core/src/resources/v2apispec/cores.json
+++ /dev/null
@@ -1,22 +0,0 @@
-{
-  "cores": {
-    "documentation": "https://cwiki.apache.org",
-    "methods": [
-      "GET"
-    ],
-    "url": {
-      "path": "/cores",
-      "paths": [
-        "/cores"
-      ],
-      "params": {
-        "indexInfo": {
-          "type": "boolean",
-          "description": "return index info",
-          "default": false
-        }
-      }
-    },
-    "body": null
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/emptySpec.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/emptySpec.json b/solr/core/src/resources/v2apispec/emptySpec.json
deleted file mode 100644
index 0c5125b..0000000
--- a/solr/core/src/resources/v2apispec/emptySpec.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
-  "emptySpec": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": [
-      "GET",
-      "POST"
-    ],
-    "url": {
-      "path": "$handlerName",
-      "paths": [
-        "$handlerName"
-      ]
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/resources/v2apispec/node.Info.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/v2apispec/node.Info.json b/solr/core/src/resources/v2apispec/node.Info.json
deleted file mode 100644
index 687b6b6..0000000
--- a/solr/core/src/resources/v2apispec/node.Info.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "node.Info": {
-    "documentation": "https://cwiki.apache.org/confluence/display/solr/Schema+API",
-    "methods": ["GET"],
-    "url": {
-      "path": "/node",
-      "paths": [
-        "/node/properties",
-        "/node/threads",
-        "/node/logging",
-        "/node/system"]
-    }
-  }
-}
\ No newline at end of file


[3/3] lucene-solr git commit: SOLR-8029 renamed class/dirs and removed the 'v2' part

Posted by no...@apache.org.
SOLR-8029 renamed class/dirs and removed the 'v2' part


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

Branch: refs/heads/apiv2
Commit: 3cfebd53f66198ec1d738452a688ad4f67dc8436
Parents: 292fe4a
Author: Noble Paul <no...@gmail.com>
Authored: Thu Feb 11 13:10:44 2016 -0800
Committer: Noble Paul <no...@gmail.com>
Committed: Thu Feb 11 13:10:44 2016 -0800

----------------------------------------------------------------------
 .../org/apache/solr/core/ImplicitPlugins.java   |   4 +-
 .../java/org/apache/solr/core/PluginBag.java    |  12 +-
 .../apache/solr/handler/RequestHandlerBase.java |  10 +-
 .../org/apache/solr/handler/SchemaHandler.java  |  10 +-
 .../apache/solr/handler/SolrConfigHandler.java  |  10 +-
 .../solr/handler/UpdateRequestHandlerApi.java   |  68 +++++
 .../solr/handler/V2UpdateRequestHandler.java    |  70 ------
 .../handler/admin/BaseHandlerApiSupport.java    | 204 +++++++++++++++
 .../handler/admin/CollectionHandlerApi.java     | 248 ++++++++++++++++++
 .../solr/handler/admin/CollectionsHandler.java  |  14 +-
 .../solr/handler/admin/CoreAdminHandler.java    |  19 +-
 .../solr/handler/admin/CoreAdminHandlerApi.java | 170 +++++++++++++
 .../apache/solr/handler/admin/InfoHandler.java  |  12 +-
 .../solr/handler/admin/SecurityConfHandler.java |  11 +-
 .../solr/handler/admin/V2BaseHandler.java       | 207 ---------------
 .../solr/handler/admin/V2CollectionHandler.java | 252 -------------------
 .../solr/handler/admin/V2CoreAdminHandler.java  | 173 -------------
 .../src/java/org/apache/solr/v2api/Api.java     |  38 +++
 .../src/java/org/apache/solr/v2api/ApiBag.java  |  35 ++-
 .../java/org/apache/solr/v2api/ApiSupport.java  |  30 +++
 .../src/java/org/apache/solr/v2api/V2Api.java   |  39 ---
 .../org/apache/solr/v2api/V2ApiSupport.java     |  33 ---
 .../java/org/apache/solr/v2api/V2HttpCall.java  |  24 +-
 solr/core/src/resources/apispec/cluster.json    |  17 ++
 .../apispec/cluster.security.BasicAuth.json     |  25 ++
 ...cluster.security.RuleBasedAuthorization.json |  56 +++++
 .../cluster.security.authentication.json        |  15 ++
 .../apispec/cluster.security.authorization.json |  15 ++
 .../resources/apispec/collections.Commands.json |  82 ++++++
 .../collections.collection.Commands.json        |  19 ++
 .../collections.collection.Commands.modify.json |   6 +
 .../collections.collection.Commands.reload.json |   3 +
 .../collections.collection.shards.Commands.json |  21 ++
 ...ctions.collection.shards.shard.Commands.json |  28 +++
 ...ollection.shards.shard.replica.Commands.json |  20 ++
 ....collection.shards.shard.replica.delete.json |  23 ++
 .../src/resources/apispec/core.ConfigEdit.json  | 115 +++++++++
 .../src/resources/apispec/core.ConfigRead.json  |  19 ++
 .../src/resources/apispec/core.RealtimeGet.json |  23 ++
 .../apispec/core.SchemaEdit.addCopyField.json   |   4 +
 .../core.SchemaEdit.addDynamicField.json        |   3 +
 .../apispec/core.SchemaEdit.addField.json       |   4 +
 .../apispec/core.SchemaEdit.addFieldType.json   |   4 +
 .../core.SchemaEdit.deleteCopyField.json        |   4 +
 .../core.SchemaEdit.deleteDynamicField.json     |   3 +
 .../apispec/core.SchemaEdit.deleteField.json    |   3 +
 .../core.SchemaEdit.deleteFieldType.json        |   3 +
 .../src/resources/apispec/core.SchemaEdit.json  |  22 ++
 .../core.SchemaEdit.replaceDynamicField.json    |   3 +
 .../apispec/core.SchemaEdit.replaceField.json   |   4 +
 .../core.SchemaEdit.replaceFieldType.json       |   3 +
 .../apispec/core.SchemaRead.copyFields.json     |  23 ++
 .../apispec/core.SchemaRead.fields.json         |  31 +++
 .../src/resources/apispec/core.SchemaRead.json  |  25 ++
 .../core/src/resources/apispec/core.Update.json |  19 ++
 .../src/resources/apispec/cores.Commands.json   |  67 +++++
 .../src/resources/apispec/cores.Status.json     |  23 ++
 .../resources/apispec/cores.core.Commands.json  |  36 +++
 .../cores.core.Commands.requestRecovery.json    |   4 +
 .../apispec/cores.core.Commands.split.json      |   3 +
 solr/core/src/resources/apispec/cores.json      |  22 ++
 solr/core/src/resources/apispec/emptySpec.json  |  15 ++
 solr/core/src/resources/apispec/node.Info.json  |  14 ++
 solr/core/src/resources/v2apispec/cluster.json  |  17 --
 .../v2apispec/cluster.security.BasicAuth.json   |  25 --
 ...cluster.security.RuleBasedAuthorization.json |  56 -----
 .../cluster.security.authentication.json        |  15 --
 .../cluster.security.authorization.json         |  15 --
 .../v2apispec/collections.Commands.json         |  82 ------
 .../collections.collection.Commands.json        |  19 --
 .../collections.collection.Commands.modify.json |   6 -
 .../collections.collection.Commands.reload.json |   3 -
 .../collections.collection.shards.Commands.json |  21 --
 ...ctions.collection.shards.shard.Commands.json |  28 ---
 ...ollection.shards.shard.replica.Commands.json |  20 --
 ....collection.shards.shard.replica.delete.json |  23 --
 .../resources/v2apispec/core.ConfigEdit.json    | 115 ---------
 .../resources/v2apispec/core.ConfigRead.json    |  19 --
 .../resources/v2apispec/core.RealtimeGet.json   |  23 --
 .../v2apispec/core.SchemaEdit.addCopyField.json |   4 -
 .../core.SchemaEdit.addDynamicField.json        |   3 -
 .../v2apispec/core.SchemaEdit.addField.json     |   4 -
 .../v2apispec/core.SchemaEdit.addFieldType.json |   4 -
 .../core.SchemaEdit.deleteCopyField.json        |   4 -
 .../core.SchemaEdit.deleteDynamicField.json     |   3 -
 .../v2apispec/core.SchemaEdit.deleteField.json  |   3 -
 .../core.SchemaEdit.deleteFieldType.json        |   3 -
 .../resources/v2apispec/core.SchemaEdit.json    |  22 --
 .../core.SchemaEdit.replaceDynamicField.json    |   3 -
 .../v2apispec/core.SchemaEdit.replaceField.json |   4 -
 .../core.SchemaEdit.replaceFieldType.json       |   3 -
 .../v2apispec/core.SchemaRead.copyFields.json   |  23 --
 .../v2apispec/core.SchemaRead.fields.json       |  31 ---
 .../resources/v2apispec/core.SchemaRead.json    |  25 --
 .../src/resources/v2apispec/core.Update.json    |  19 --
 .../src/resources/v2apispec/cores.Commands.json |  67 -----
 .../src/resources/v2apispec/cores.Status.json   |  23 --
 .../v2apispec/cores.core.Commands.json          |  36 ---
 .../cores.core.Commands.requestRecovery.json    |   4 -
 .../v2apispec/cores.core.Commands.split.json    |   3 -
 solr/core/src/resources/v2apispec/cores.json    |  22 --
 .../core/src/resources/v2apispec/emptySpec.json |  15 --
 .../core/src/resources/v2apispec/node.Info.json |  14 --
 .../solr/handler/admin/TestApiFramework.java    | 184 ++++++++++++++
 .../solr/handler/admin/TestCollectionAPIs.java  | 153 +++++++++++
 .../solr/handler/admin/TestCoreAdminApis.java   | 102 ++++++++
 .../handler/admin/TestV2CollectionAPIs.java     | 160 ------------
 .../solr/handler/admin/TestV2CoreAdminAPIs.java | 104 --------
 .../solr/handler/admin/TestV2Framework.java     | 187 --------------
 109 files changed, 2100 insertions(+), 2141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java b/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java
index 627e5d1..d1b44f0 100644
--- a/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java
+++ b/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java
@@ -34,7 +34,7 @@ import org.apache.solr.handler.SchemaHandler;
 import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.handler.StreamHandler;
 import org.apache.solr.handler.UpdateRequestHandler;
-import org.apache.solr.handler.V2UpdateRequestHandler;
+import org.apache.solr.handler.UpdateRequestHandlerApi;
 import org.apache.solr.handler.admin.LoggingHandler;
 import org.apache.solr.handler.admin.LukeRequestHandler;
 import org.apache.solr.handler.admin.PluginInfoHandler;
@@ -67,7 +67,7 @@ public class ImplicitPlugins {
     implicits.add(createPluginInfoWithDefaults(UpdateRequestHandler.CSV_PATH, UpdateRequestHandler.class, singletonMap("update.contentType", "application/csv")));
     implicits.add(createPluginInfoWithDefaults(UpdateRequestHandler.DOC_PATH, UpdateRequestHandler.class, makeMap("update.contentType", "application/json", "json.command", "false")));
 
-    Map attrs = Utils.makeMap("legacy", "false", NAME, "/update/v2", "class", V2UpdateRequestHandler.class.getName());
+    Map attrs = Utils.makeMap("legacy", "false", NAME, "/update/v2", "class", UpdateRequestHandlerApi.class.getName());
     implicits.add(new PluginInfo(SolrRequestHandler.TYPE, attrs, new NamedList(), null));
 
     //solrconfighandler

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 cec8b9d..5d627c0 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -43,9 +43,9 @@ import org.apache.solr.util.CryptoKeys;
 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.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -193,11 +193,11 @@ public class PluginBag<T> implements AutoCloseable {
     if (apiBag != null) {
       if (plugin.isLoaded()) {
         T inst = plugin.get();
-        if (inst instanceof V2ApiSupport && ((V2ApiSupport) inst).registerAutomatically()) {
-          Collection<V2Api> apis = ((V2ApiSupport) inst).getApis();
+        if (inst instanceof ApiSupport && ((ApiSupport) inst).registerAutomatically()) {
+          Collection<Api> apis = ((ApiSupport) inst).getApis();
           if (apis != null) {
             Map<String, String> nameSubstitutes = singletonMap(HANDLER_NAME, name);
-            for (V2Api api : apis) {
+            for (Api api : apis) {
               apiBag.register(api, nameSubstitutes);
             }
           }
@@ -515,7 +515,7 @@ public class PluginBag<T> implements AutoCloseable {
   }
 
 
-  public V2Api v2lookup(String path, String method, Map<String, String> parts) {
+  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");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 c16e02f..0883100 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -25,8 +25,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.google.common.collect.ImmutableList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.SuppressForbidden;
@@ -41,9 +39,9 @@ import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.stats.Snapshot;
 import org.apache.solr.util.stats.Timer;
 import org.apache.solr.util.stats.TimerContext;
+import org.apache.solr.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +51,7 @@ import static org.apache.solr.v2api.ApiBag.wrapRequestHandler;
 /**
  *
  */
-public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, NestedRequestHandler, V2ApiSupport {
+public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, NestedRequestHandler, ApiSupport {
 
   protected NamedList initArgs = null;
   protected SolrParams defaults;
@@ -286,7 +284,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   }
 
   @Override
-  public Collection<V2Api> getApis() {
+  public Collection<Api> getApis() {
     return ImmutableList.of(
         wrapRequestHandler(this, ApiBag.constructSpec(pluginInfo), null)
     );

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 94d8276..4e06297 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -31,8 +31,6 @@ import com.google.common.collect.ImmutableList;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.request.SolrQueryRequest;
@@ -42,9 +40,9 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SchemaManager;
 import org.apache.solr.schema.ZkIndexSchemaReader;
+import org.apache.solr.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,7 +50,7 @@ import static org.apache.solr.common.params.CommonParams.JSON;
 import static org.apache.solr.core.ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG;
 import static org.apache.solr.v2api.ApiBag.wrapRequestHandler;
 
-public class SchemaHandler extends RequestHandlerBase implements V2ApiSupport {
+public class SchemaHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean isImmutableConfigSet = false;
 
@@ -192,7 +190,7 @@ public class SchemaHandler extends RequestHandlerBase implements V2ApiSupport {
   }
 
   @Override
-  public Collection<V2Api> getApis() {
+  public Collection<Api> getApis() {
     return ImmutableList.of(
         wrapRequestHandler(this, ApiBag.getSpec("core.SchemaRead"), null) ,
         wrapRequestHandler(this, ApiBag.getSpec("core.SchemaEdit"), null));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 b7b2145..7f6bd85 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -57,8 +57,6 @@ 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.ExecutorUtil;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
@@ -76,9 +74,9 @@ import org.apache.solr.schema.SchemaManager;
 import org.apache.solr.util.CommandOperation;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.RTimer;
+import org.apache.solr.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -95,7 +93,7 @@ import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_NAME_IN_OVERLAY
 import static org.apache.solr.schema.FieldType.CLASS_NAME;
 import static org.apache.solr.v2api.ApiBag.wrapRequestHandler;
 
-public class SolrConfigHandler extends RequestHandlerBase implements V2ApiSupport {
+public class SolrConfigHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   public static final String CONFIGSET_EDITING_DISABLED_ARG = "disable.configEdit";
   public static final boolean configEditing_disabled = Boolean.getBoolean(CONFIGSET_EDITING_DISABLED_ARG);
@@ -813,7 +811,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements V2ApiSuppor
   }
 
   @Override
-  public Collection<V2Api> getApis() {
+  public Collection<Api> getApis() {
     return ImmutableList.of(
         wrapRequestHandler(this, ApiBag.getSpec("core.ConfigRead"), null),
         wrapRequestHandler(this, ApiBag.getSpec("core.ConfigEdit"), null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..bf43163
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
@@ -0,0 +1,68 @@
+package org.apache.solr.handler;
+
+/*
+ * 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.
+ */
+
+
+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.v2api.Api;
+import org.apache.solr.v2api.ApiBag;
+import org.apache.solr.v2api.V2RequestContext;
+
+
+public class UpdateRequestHandlerApi extends UpdateRequestHandler  {
+
+
+  @Override
+  public boolean registerAutomatically() {
+    return true;
+  }
+
+  @Override
+  public Collection<Api> getApis() {
+    return Collections.singleton(getApiImpl());
+  }
+
+  private Api getApiImpl() {
+    return new Api(ApiBag.getSpec("core.Update")) {
+      @Override
+      public void call(V2RequestContext ctx) {
+        String path = ctx.getPath();
+        String target =  mapping.get(path);
+        if(target != null) ctx.getSolrRequest().getContext().put("path", target);
+        try {
+          handleRequest(ctx.getSolrRequest(), ctx.getResponse());
+        } catch (RuntimeException e) {
+          throw e;
+        } catch (Exception e){
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,e );
+        }
+      }
+    };
+  }
+
+  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/3cfebd53/solr/core/src/java/org/apache/solr/handler/V2UpdateRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/V2UpdateRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/V2UpdateRequestHandler.java
deleted file mode 100644
index 2f89647..0000000
--- a/solr/core/src/java/org/apache/solr/handler/V2UpdateRequestHandler.java
+++ /dev/null
@@ -1,70 +0,0 @@
-package org.apache.solr.handler;
-
-/*
- * 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.
- */
-
-
-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.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
-import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2RequestContext;
-
-
-public class V2UpdateRequestHandler extends UpdateRequestHandler  {
-
-
-  @Override
-  public boolean registerAutomatically() {
-    return true;
-  }
-
-  @Override
-  public Collection<V2Api> getApis() {
-    return Collections.singleton(getApiImpl());
-  }
-
-  private V2Api getApiImpl() {
-    return new V2Api(ApiBag.getSpec("core.Update")) {
-      @Override
-      public void call(V2RequestContext ctx) {
-        String path = ctx.getPath();
-        String target =  mapping.get(path);
-        if(target != null) ctx.getSolrRequest().getContext().put("path", target);
-        try {
-          handleRequest(ctx.getSolrRequest(), ctx.getResponse());
-        } catch (RuntimeException e) {
-          throw e;
-        } catch (Exception e){
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,e );
-        }
-      }
-    };
-  }
-
-  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/3cfebd53/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..9a575ed
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
@@ -0,0 +1,204 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+
+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.Map2;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.CommandOperation;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.ApiBag;
+import org.apache.solr.v2api.ApiSupport;
+import org.apache.solr.v2api.V2RequestContext;
+
+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;
+
+public abstract class BaseHandlerApiSupport implements ApiSupport {
+  protected final Map<SolrRequest.METHOD, Map<V2EndPoint, List<V2Command>>> commandsMapping;
+
+  protected BaseHandlerApiSupport() {
+    commandsMapping = new HashMap<>();
+    for (V2Command cmd : getCommands()) {
+      Map<V2EndPoint, List<V2Command>> m = commandsMapping.get(cmd.getHttpMethod());
+      if (m == null) commandsMapping.put(cmd.getHttpMethod(), m = new HashMap<>());
+      List<V2Command> 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 Map2 spec = ApiBag.getSpec(op.getSpecName());
+    return new Api(spec) {
+      @Override
+      public void call(V2RequestContext ctx) {
+        SolrParams params = ctx.getSolrRequest().getParams();
+        SolrRequest.METHOD method = SolrRequest.METHOD.valueOf(ctx.getHttpMethod());
+        List<V2Command> commands = commandsMapping.get(method).get(op);
+        try {
+          if (method == POST) {
+            List<CommandOperation> cmds = ctx.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);
+            V2Command command = null;
+            String commandName = c == null ? null : c.name;
+            for (V2Command cmd : commands) {
+              if (Objects.equals(cmd.getName(), commandName)) {
+                command = cmd;
+                break;
+              }
+            }
+
+            if (command == null) {
+              throw new SolrException(BAD_REQUEST, " no such command " + c);
+            }
+            wrapParams(ctx, c, command, false);
+            invokeCommand(ctx, command, c);
+
+          } else {
+            if (commands == null || commands.isEmpty()) {
+              ctx.getResponse().add("error", "No support for : " + method + " at :" + ctx.getPath());
+              return;
+            }
+            wrapParams(ctx, new CommandOperation("", Collections.EMPTY_MAP), commands.get(0), true);
+            invokeUrl(commands.get(0), ctx);
+          }
+
+        } catch (SolrException e) {
+          throw e;
+        } catch (Exception e) {
+          throw new SolrException(BAD_REQUEST, e);
+        } finally {
+          ctx.getSolrRequest().setParams(params);
+        }
+
+      }
+    };
+
+  }
+
+  private static void wrapParams(final V2RequestContext ctx, final CommandOperation co, final V2Command cmd, final boolean useRequestParams) {
+    final Map<String, String> pathValues = ctx.getPathValues();
+    final Map<String, Object> map = co == null || !(co.getCommandData() instanceof Map) ?
+        Collections.emptyMap() : co.getDataMap();
+    final SolrParams origParams = ctx.getSolrRequest().getParams();
+
+    ctx.getSolrRequest().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 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, V2Command 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 void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception;
+
+  protected abstract void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception;
+
+  protected abstract List<V2Command> getCommands();
+
+  protected abstract List<V2EndPoint> getEndPoints();
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..1c1f213
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -0,0 +1,248 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+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.cloud.OverseerCollectionMessageHandler;
+import org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation;
+import org.apache.solr.util.CommandOperation;
+import org.apache.solr.v2api.V2RequestContext;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation.*;
+
+
+public class CollectionHandlerApi extends BaseHandlerApiSupport {
+  private final CollectionsHandler handler;
+
+  public CollectionHandlerApi(CollectionsHandler handler) {
+    this.handler = handler;
+  }
+
+  @Override
+  protected List<V2Command> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception {
+    ((Cmd) command).command(ctx, c, this);
+  }
+
+  @Override
+  protected void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception {
+    ((Cmd) command).GET(ctx, this);
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(EndPoint.values());
+  }
+
+
+
+  enum Cmd implements V2Command<CollectionHandlerApi> {
+    GET_COLLECTIONS(EndPoint.COLLECTIONS, GET, LIST_OP),
+    GET_A_COLLECTION(EndPoint.PER_COLLECTION, GET, LIST_OP),
+    CREATE_COLLECTION(EndPoint.COLLECTIONS,
+        POST,
+        CREATE_OP,
+        CREATE_OP.action.toLower(),
+        ImmutableMap.of(
+            OverseerCollectionMessageHandler.COLL_CONF, "config")){
+      @Override
+      public Collection<String> getParamNames(CommandOperation op) {
+        Collection<String> names = super.getParamNames(op);
+        Collection<String> result = new ArrayList<>(names.size());
+        for (String paramName : names) {
+          if(paramName.startsWith("properties.")){
+            result.add(paramName.replace("properties.", "property."));
+          } else {
+            result.add(paramName);
+          }
+        }
+        return result;
+      }
+
+      @Override
+      public String getParamSubstitute(String param) {
+        return param.startsWith("property.")? param.replace("property.", "properties.") : super.getParamSubstitute(param);
+      }
+    },
+
+    DELETE_COLL(EndPoint.PER_COLLECTION,
+        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")),
+
+    MIGRATE_DOCS(EndPoint.PER_COLLECTION,
+        POST,
+        MIGRATE_OP,
+        "migrate-docs",
+        ImmutableMap.of("split.key", "splitKey",
+            "target.collection", "target",
+            "forward.timeout", "forwardTimeout"
+        )),
+    CREATE_ALIAS(EndPoint.COLLECTIONS,
+        POST,
+        CREATEALIAS_OP,
+        "create-alias",
+        null),
+
+    DELETE_ALIAS(EndPoint.COLLECTIONS,
+        POST,
+        CREATEALIAS_OP,
+        "delete-alias",
+        ImmutableMap.of(NAME, "")),
+    CREATE_SHARD(EndPoint.PER_COLLECTION_SHARDS,
+        POST,
+        CREATESHARD_OP,
+        "create",
+        null),
+
+    SPLIT_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
+        POST,
+        SPLITSHARD_OP,
+        "split",
+        ImmutableMap.of(
+            "split.key", "splitKey")),
+    DELETE_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
+        DELETE,
+        DELETESHARD_OP),
+
+
+    CREATE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD,
+        POST,
+        ADDREPLICA_OP,
+        "create-replica",
+        null),
+
+    DELETE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE,
+        DELETE,
+        DELETEREPLICA_OP),
+
+    SYNC_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
+        POST,
+        SYNCSHARD_OP,
+        "synch-shard",
+        null),;
+    public final String commandName;
+    public final EndPoint endPoint;
+    public final SolrRequest.METHOD method;
+    public final CollectionOperation target;
+    public final Map<String, String> paramstoAttr;
+
+    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.commandName = 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 void command(V2RequestContext ctx, CommandOperation c, CollectionHandlerApi handler) throws Exception {
+      handler.handler.invokeAction(ctx.getSolrRequest(),ctx.getResponse(),target);
+    }
+
+    @Override
+    public void GET(V2RequestContext ctx, CollectionHandlerApi handler) throws Exception {
+      handler.handler.invokeAction(ctx.getSolrRequest(), ctx.getResponse(), target);
+    }
+
+    @Override
+    public Collection<String> getParamNames(CommandOperation op) {
+      return BaseHandlerApiSupport.getParamNames(op, this);
+    }
+
+    @Override
+    public String getParamSubstitute(String param) {
+      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+    }
+
+
+  }
+
+  enum EndPoint implements V2EndPoint {
+    CLUSTER("collections.Commands"),
+    COLLECTIONS("collections.Commands"),
+    PER_COLLECTION("collections.collection.Commands"),
+    PER_COLLECTION_SHARDS("collections.collection.shards.Commands"),
+    PER_COLLECTION_PER_SHARD("collections.collection.shards.Commands"),
+    PER_COLLECTION_PER_SHARD_REPLICAS("collections.collection.shards.shard.Commands"),
+    PER_COLLECTION_PER_SHARD_PER_REPLICA("collections.collection.shards.shard.replica.Commands"),
+    PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE("collections.collection.shards.shard.replica.delete");
+    final String specName;
+
+
+    EndPoint(String specName) {
+      this.specName = specName;
+    }
+
+    @Override
+    public String getSpecName() {
+      return specName;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 d0f74c0..8e8279e 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
@@ -133,8 +133,8 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.ApiSupport;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -143,17 +143,17 @@ import org.slf4j.LoggerFactory;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
-public class CollectionsHandler extends RequestHandlerBase implements V2ApiSupport {
+public class CollectionsHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
-  private final V2CollectionHandler v2Handler ;
+  private final CollectionHandlerApi v2Handler ;
 
   public CollectionsHandler() {
     super();
     // Unlike most request handlers, CoreContainer initialization
     // should happen in the constructor...
     this.coreContainer = null;
-    v2Handler = new V2CollectionHandler(this);
+    v2Handler = new CollectionHandlerApi(this);
   }
 
 
@@ -164,7 +164,7 @@ public class CollectionsHandler extends RequestHandlerBase implements V2ApiSuppo
    */
   public CollectionsHandler(final CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
-    v2Handler = new V2CollectionHandler(this);
+    v2Handler = new CollectionHandlerApi(this);
   }
 
 
@@ -897,7 +897,7 @@ public class CollectionsHandler extends RequestHandlerBase implements V2ApiSuppo
 
 
   @Override
-  public Collection<V2Api> getApis() {
+  public Collection<Api> getApis() {
     return v2Handler.getApis();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 276658d..8fbc519 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
@@ -37,12 +37,9 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
@@ -50,8 +47,8 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.Api;
+import org.apache.solr.v2api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -63,11 +60,11 @@ import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.STAT
  *
  * @since solr 1.3
  */
-public class CoreAdminHandler extends RequestHandlerBase implements V2ApiSupport {
+public class CoreAdminHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   protected final Map<String, Map<String, TaskObject>> requestStatusMap;
-  private final V2CoreAdminHandler v2CoreAdminHandler;
+  private final CoreAdminHandlerApi coreAdminHandlerApi;
 
   protected final ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
       new DefaultSolrThreadFactory("parallelCoreAdminExecutor"));
@@ -90,7 +87,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements V2ApiSupport
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<>()));
     requestStatusMap = Collections.unmodifiableMap(map);
-    v2CoreAdminHandler = new V2CoreAdminHandler(this);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -106,7 +103,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements V2ApiSupport
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     requestStatusMap = Collections.unmodifiableMap(map);
-    v2CoreAdminHandler = new V2CoreAdminHandler(this);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -366,8 +363,8 @@ public class CoreAdminHandler extends RequestHandlerBase implements V2ApiSupport
   }
 
   @Override
-  public Collection<V2Api> getApis() {
-    return v2CoreAdminHandler.getApis();
+  public Collection<Api> getApis() {
+    return coreAdminHandlerApi.getApis();
   }
 
   static {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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..ae3eff0
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@ -0,0 +1,170 @@
+package org.apache.solr.handler.admin;
+
+/*
+ * 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.
+ */
+
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.util.CommandOperation;
+import org.apache.solr.v2api.V2RequestContext;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
+import static org.apache.solr.handler.admin.CoreAdminOperation.*;
+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.PER_CORE_COMMANDS;
+
+public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
+  private final CoreAdminHandler handler;
+
+  public CoreAdminHandlerApi(CoreAdminHandler handler) {
+    this.handler = handler;
+  }
+
+  enum Cmd implements V2Command<CoreAdminHandlerApi> {
+    CREATE(CORES_COMMANDS, POST, CREATE_OP, null, null),
+    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, null),
+    RENAME(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
+    MERGEINDEXES(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
+    SPLIT(PER_CORE_COMMANDS, POST, SPLIT_OP, null, null),
+    PREPRECOVERY(PER_CORE_COMMANDS, POST, PREPRECOVERY_OP, null, null),
+    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
+    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
+    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, REQUESTBUFFERUPDATES_OP, null, null),
+    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, REQUESTAPPLYUPDATES_OP, null, null),
+    REQUESTSTATUS(PER_CORE_COMMANDS, POST, REQUESTSTATUS_OP, null, null),
+    OVERSEEROP(PER_CORE_COMMANDS, POST, OVERSEEROP_OP, null, null),
+    REJOINLEADERELECTION(PER_CORE_COMMANDS, POST, REJOINLEADERELECTION_OP, null, null),
+    INVOKE(PER_CORE_COMMANDS, POST, INVOKE_OP, null, null),
+    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, FORCEPREPAREFORLEADERSHIP_OP, null, null);
+
+    public final String commandName;
+    public final EndPoint 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 void command(V2RequestContext ctx, CommandOperation c, CoreAdminHandlerApi coreAdminHandlerApi) throws Exception {
+      target.call(new CoreAdminHandler.CallInfo(coreAdminHandlerApi.handler,ctx.getSolrRequest(),ctx.getResponse(),target ));
+
+    }
+
+    @Override
+    public void GET(V2RequestContext ctx, CoreAdminHandlerApi handler) throws Exception {
+      target.call(new CoreAdminHandler.CallInfo(handler.handler,ctx.getSolrRequest(),ctx.getResponse(),target ));
+
+    }
+
+    @Override
+    public Collection<String> getParamNames(CommandOperation op) {
+      return BaseHandlerApiSupport.getParamNames(op, this);
+    }
+
+    @Override
+    public String getParamSubstitute(String param) {
+      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+    }
+  }
+
+
+
+  enum EndPoint implements V2EndPoint {
+    CORES_STATUS("cores.Status"),
+    CORES_COMMANDS("cores.Commands"),
+    PER_CORE_COMMANDS("cores.core.Commands");
+
+    final String specName;
+
+    EndPoint(String specName) {
+      this.specName = specName;
+    }
+
+    @Override
+    public String getSpecName() {
+      return specName;
+    }
+  }
+
+  @Override
+  protected void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception {
+    ((Cmd) command).command(ctx, c, this);
+  }
+
+  @Override
+  protected void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception {
+    command.GET(ctx, this);
+  }
+
+  @Override
+  protected List<V2Command> 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/3cfebd53/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 a06e3d5..4186eb2 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
@@ -24,24 +24,22 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.Lookup;
-import org.apache.solr.common.util.Map2;
 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.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
+import org.apache.solr.v2api.ApiSupport;
 import org.apache.solr.v2api.V2RequestContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.PATH;
 
-public class InfoHandler extends RequestHandlerBase implements V2ApiSupport {
+public class InfoHandler extends RequestHandlerBase implements ApiSupport {
   protected static Logger log = LoggerFactory.getLogger(InfoHandler.class);
   protected final CoreContainer coreContainer;
 
@@ -147,8 +145,8 @@ public class InfoHandler extends RequestHandlerBase implements V2ApiSupport {
   private Map<String, RequestHandlerBase> handlers = new ConcurrentHashMap<>();
 
   @Override
-  public Collection<V2Api> getApis() {
-    return Collections.singletonList(new V2Api(ApiBag.getSpec("node.Info")) {
+  public Collection<Api> getApis() {
+    return Collections.singletonList(new Api(ApiBag.getSpec("node.Info")) {
       @Override
       public void call(V2RequestContext ctx) {
         handle(ctx.getSolrRequest(), ctx.getResponse(), ctx.getPath());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/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 ea2c9ef..8d7c42d 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
@@ -30,7 +30,6 @@ import com.google.common.collect.ImmutableList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader.ConfigData;
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.Lookup;
 import org.apache.solr.common.util.Map2;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
@@ -42,13 +41,13 @@ import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationPlugin;
 import org.apache.solr.security.ConfigEditablePlugin;
 import org.apache.solr.util.CommandOperation;
+import org.apache.solr.v2api.Api;
 import org.apache.solr.v2api.ApiBag;
+import org.apache.solr.v2api.ApiSupport;
 import org.apache.solr.v2api.SpecProvider;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
 import org.apache.zookeeper.KeeperException;
 
-public class SecurityConfHandler extends RequestHandlerBase implements V2ApiSupport {
+public class SecurityConfHandler extends RequestHandlerBase implements ApiSupport {
   private CoreContainer cores;
 
   public SecurityConfHandler(CoreContainer coreContainer) {
@@ -174,14 +173,14 @@ public class SecurityConfHandler extends RequestHandlerBase implements V2ApiSupp
 
 
   @Override
-  public Collection<V2Api> getApis() {
+  public Collection<Api> getApis() {
     return
     ImmutableList.of(
         getApi("authentication"),
         getApi("authorization"));
   }
 
-  private V2Api getApi( String type) {
+  private Api getApi( String type) {
     return ApiBag.wrapRequestHandler(this, null, new SpecProvider() {
       @Override
       public Map2 getSpec() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/handler/admin/V2BaseHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/V2BaseHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/V2BaseHandler.java
deleted file mode 100644
index 3b28c80..0000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/V2BaseHandler.java
+++ /dev/null
@@ -1,207 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-
-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.Lookup;
-import org.apache.solr.common.util.Map2;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2ApiSupport;
-import org.apache.solr.v2api.V2RequestContext;
-
-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;
-
-public abstract class V2BaseHandler implements V2ApiSupport {
-  protected final Map<SolrRequest.METHOD, Map<V2EndPoint, List<V2Command>>> commandsMapping;
-
-  protected V2BaseHandler() {
-    commandsMapping = new HashMap<>();
-    for (V2Command cmd : getCommands()) {
-      Map<V2EndPoint, List<V2Command>> m = commandsMapping.get(cmd.getHttpMethod());
-      if (m == null) commandsMapping.put(cmd.getHttpMethod(), m = new HashMap<>());
-      List<V2Command> list = m.get(cmd.getEndPoint());
-      if (list == null) m.put(cmd.getEndPoint(), list = new ArrayList<>());
-      list.add(cmd);
-    }
-  }
-
-  @Override
-  public synchronized Collection<V2Api> getApis() {
-    ImmutableList.Builder<V2Api> l = ImmutableList.builder();
-    for (V2EndPoint op : getEndPoints()) l.add(getApi(op));
-    return l.build();
-  }
-
-
-  private V2Api getApi(final V2EndPoint op) {
-    final Map2 spec = ApiBag.getSpec(op.getSpecName());
-    return new V2Api(spec) {
-      @Override
-      public void call(V2RequestContext ctx) {
-        SolrParams params = ctx.getSolrRequest().getParams();
-        SolrRequest.METHOD method = SolrRequest.METHOD.valueOf(ctx.getHttpMethod());
-        List<V2Command> commands = commandsMapping.get(method).get(op);
-        try {
-          if (method == POST) {
-            List<CommandOperation> cmds = ctx.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);
-            V2Command command = null;
-            String commandName = c == null ? null : c.name;
-            for (V2Command cmd : commands) {
-              if (Objects.equals(cmd.getName(), commandName)) {
-                command = cmd;
-                break;
-              }
-            }
-
-            if (command == null) {
-              throw new SolrException(BAD_REQUEST, " no such command " + c);
-            }
-            wrapParams(ctx, c, command, false);
-            invokeCommand(ctx, command, c);
-
-          } else {
-            if (commands == null || commands.isEmpty()) {
-              ctx.getResponse().add("error", "No support for : " + method + " at :" + ctx.getPath());
-              return;
-            }
-            wrapParams(ctx, new CommandOperation("", Collections.EMPTY_MAP), commands.get(0), true);
-            invokeUrl(commands.get(0), ctx);
-          }
-
-        } catch (SolrException e) {
-          throw e;
-        } catch (Exception e) {
-          throw new SolrException(BAD_REQUEST, e);
-        } finally {
-          ctx.getSolrRequest().setParams(params);
-        }
-
-      }
-    };
-
-  }
-
-  private static void wrapParams(final V2RequestContext ctx, final CommandOperation co, final V2Command cmd, final boolean useRequestParams) {
-    final Map<String, String> pathValues = ctx.getPathValues();
-    final Map<String, Object> map = co == null || !(co.getCommandData() instanceof Map) ?
-        Collections.emptyMap() : co.getDataMap();
-    final SolrParams origParams = ctx.getSolrRequest().getParams();
-
-    ctx.getSolrRequest().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 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, V2Command 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 void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception;
-
-  protected abstract void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception;
-
-  protected abstract List<V2Command> getCommands();
-
-  protected abstract List<V2EndPoint> getEndPoints();
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/handler/admin/V2CollectionHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/V2CollectionHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/V2CollectionHandler.java
deleted file mode 100644
index d89bf5a..0000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/V2CollectionHandler.java
+++ /dev/null
@@ -1,252 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-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 java.util.Objects;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.cloud.OverseerCollectionMessageHandler;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation;
-import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.V2RequestContext;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation.*;
-
-
-public class V2CollectionHandler extends V2BaseHandler {
-  private final CollectionsHandler handler;
-
-  public V2CollectionHandler(CollectionsHandler handler) {
-    this.handler = handler;
-  }
-
-  @Override
-  protected List<V2Command> getCommands() {
-    return Arrays.asList(Cmd.values());
-  }
-
-  @Override
-  protected void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception {
-    ((Cmd) command).command(ctx, c, this);
-  }
-
-  @Override
-  protected void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception {
-    ((Cmd) command).GET(ctx, this);
-  }
-
-  @Override
-  protected List<V2EndPoint> getEndPoints() {
-    return Arrays.asList(EndPoint.values());
-  }
-
-
-
-  enum Cmd implements V2Command<V2CollectionHandler> {
-    GET_COLLECTIONS(EndPoint.COLLECTIONS, GET, LIST_OP),
-    GET_A_COLLECTION(EndPoint.PER_COLLECTION, GET, LIST_OP),
-    CREATE_COLLECTION(EndPoint.COLLECTIONS,
-        POST,
-        CREATE_OP,
-        CREATE_OP.action.toLower(),
-        ImmutableMap.of(
-            OverseerCollectionMessageHandler.COLL_CONF, "config")){
-      @Override
-      public Collection<String> getParamNames(CommandOperation op) {
-        Collection<String> names = super.getParamNames(op);
-        Collection<String> result = new ArrayList<>(names.size());
-        for (String paramName : names) {
-          if(paramName.startsWith("properties.")){
-            result.add(paramName.replace("properties.", "property."));
-          } else {
-            result.add(paramName);
-          }
-        }
-        return result;
-      }
-
-      @Override
-      public String getParamSubstitute(String param) {
-        return param.startsWith("property.")? param.replace("property.", "properties.") : super.getParamSubstitute(param);
-      }
-    },
-
-    DELETE_COLL(EndPoint.PER_COLLECTION,
-        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")),
-
-    MIGRATE_DOCS(EndPoint.PER_COLLECTION,
-        POST,
-        MIGRATE_OP,
-        "migrate-docs",
-        ImmutableMap.of("split.key", "splitKey",
-            "target.collection", "target",
-            "forward.timeout", "forwardTimeout"
-        )),
-    CREATE_ALIAS(EndPoint.COLLECTIONS,
-        POST,
-        CREATEALIAS_OP,
-        "create-alias",
-        null),
-
-    DELETE_ALIAS(EndPoint.COLLECTIONS,
-        POST,
-        CREATEALIAS_OP,
-        "delete-alias",
-        ImmutableMap.of(NAME, "")),
-    CREATE_SHARD(EndPoint.PER_COLLECTION_SHARDS,
-        POST,
-        CREATESHARD_OP,
-        "create",
-        null),
-
-    SPLIT_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
-        POST,
-        SPLITSHARD_OP,
-        "split",
-        ImmutableMap.of(
-            "split.key", "splitKey")),
-    DELETE_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
-        DELETE,
-        DELETESHARD_OP),
-
-
-    CREATE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD,
-        POST,
-        ADDREPLICA_OP,
-        "create-replica",
-        null),
-
-    DELETE_REPLICA(EndPoint.PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE,
-        DELETE,
-        DELETEREPLICA_OP),
-
-    SYNC_SHARD(EndPoint.PER_COLLECTION_PER_SHARD,
-        POST,
-        SYNCSHARD_OP,
-        "synch-shard",
-        null),;
-    public final String commandName;
-    public final EndPoint endPoint;
-    public final SolrRequest.METHOD method;
-    public final CollectionOperation target;
-    public final Map<String, String> paramstoAttr;
-
-    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.commandName = 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 void command(V2RequestContext ctx, CommandOperation c, V2CollectionHandler handler) throws Exception {
-      handler.handler.invokeAction(ctx.getSolrRequest(),ctx.getResponse(),target);
-    }
-
-    @Override
-    public void GET(V2RequestContext ctx, V2CollectionHandler handler) throws Exception {
-      handler.handler.invokeAction(ctx.getSolrRequest(), ctx.getResponse(), target);
-    }
-
-    @Override
-    public Collection<String> getParamNames(CommandOperation op) {
-      return V2BaseHandler.getParamNames(op,this);
-    }
-
-    @Override
-    public String getParamSubstitute(String param) {
-      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
-    }
-
-
-  }
-
-  enum EndPoint implements V2EndPoint {
-    CLUSTER("collections.Commands"),
-    COLLECTIONS("collections.Commands"),
-    PER_COLLECTION("collections.collection.Commands"),
-    PER_COLLECTION_SHARDS("collections.collection.shards.Commands"),
-    PER_COLLECTION_PER_SHARD("collections.collection.shards.Commands"),
-    PER_COLLECTION_PER_SHARD_REPLICAS("collections.collection.shards.shard.Commands"),
-    PER_COLLECTION_PER_SHARD_PER_REPLICA("collections.collection.shards.shard.replica.Commands"),
-    PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE("collections.collection.shards.shard.replica.delete");
-    final String specName;
-
-
-    EndPoint(String specName) {
-      this.specName = specName;
-    }
-
-    @Override
-    public String getSpecName() {
-      return specName;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/handler/admin/V2CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/V2CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/V2CoreAdminHandler.java
deleted file mode 100644
index 9c9523a..0000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/V2CoreAdminHandler.java
+++ /dev/null
@@ -1,173 +0,0 @@
-package org.apache.solr.handler.admin;
-
-/*
- * 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.
- */
-
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.V2Api;
-import org.apache.solr.v2api.V2RequestContext;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
-import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
-import static org.apache.solr.handler.admin.CoreAdminOperation.*;
-import static org.apache.solr.handler.admin.V2CoreAdminHandler.EndPoint.CORES_COMMANDS;
-import static org.apache.solr.handler.admin.V2CoreAdminHandler.EndPoint.CORES_STATUS;
-import static org.apache.solr.handler.admin.V2CoreAdminHandler.EndPoint.PER_CORE_COMMANDS;
-
-public class V2CoreAdminHandler extends V2BaseHandler {
-  private final CoreAdminHandler handler;
-
-  public V2CoreAdminHandler(CoreAdminHandler handler) {
-    this.handler = handler;
-  }
-
-  enum Cmd implements V2Command<V2CoreAdminHandler> {
-    CREATE(CORES_COMMANDS, POST, CREATE_OP, null, null),
-    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, null),
-    RENAME(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
-    MERGEINDEXES(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
-    SPLIT(PER_CORE_COMMANDS, POST, SPLIT_OP, null, null),
-    PREPRECOVERY(PER_CORE_COMMANDS, POST, PREPRECOVERY_OP, null, null),
-    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
-    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
-    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, REQUESTBUFFERUPDATES_OP, null, null),
-    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, REQUESTAPPLYUPDATES_OP, null, null),
-    REQUESTSTATUS(PER_CORE_COMMANDS, POST, REQUESTSTATUS_OP, null, null),
-    OVERSEEROP(PER_CORE_COMMANDS, POST, OVERSEEROP_OP, null, null),
-    REJOINLEADERELECTION(PER_CORE_COMMANDS, POST, REJOINLEADERELECTION_OP, null, null),
-    INVOKE(PER_CORE_COMMANDS, POST, INVOKE_OP, null, null),
-    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, FORCEPREPAREFORLEADERSHIP_OP, null, null);
-
-    public final String commandName;
-    public final EndPoint 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 void command(V2RequestContext ctx, CommandOperation c, V2CoreAdminHandler v2CoreAdminHandler) throws Exception {
-      target.call(new CoreAdminHandler.CallInfo(v2CoreAdminHandler.handler,ctx.getSolrRequest(),ctx.getResponse(),target ));
-
-    }
-
-    @Override
-    public void GET(V2RequestContext ctx, V2CoreAdminHandler handler) throws Exception {
-      target.call(new CoreAdminHandler.CallInfo(handler.handler,ctx.getSolrRequest(),ctx.getResponse(),target ));
-
-    }
-
-    @Override
-    public Collection<String> getParamNames(CommandOperation op) {
-      return V2BaseHandler.getParamNames(op,this);
-    }
-
-    @Override
-    public String getParamSubstitute(String param) {
-      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
-    }
-  }
-
-
-
-  enum EndPoint implements V2EndPoint {
-    CORES_STATUS("cores.Status"),
-    CORES_COMMANDS("cores.Commands"),
-    PER_CORE_COMMANDS("cores.core.Commands");
-
-    final String specName;
-
-    EndPoint(String specName) {
-      this.specName = specName;
-    }
-
-    @Override
-    public String getSpecName() {
-      return specName;
-    }
-  }
-
-  @Override
-  protected void invokeCommand(V2RequestContext ctx, V2Command command, CommandOperation c) throws Exception {
-    ((Cmd) command).command(ctx, c, this);
-  }
-
-  @Override
-  protected void invokeUrl(V2Command command, V2RequestContext ctx) throws Exception {
-    command.GET(ctx, this);
-  }
-
-  @Override
-  protected List<V2Command> 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/3cfebd53/solr/core/src/java/org/apache/solr/v2api/Api.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/Api.java b/solr/core/src/java/org/apache/solr/v2api/Api.java
new file mode 100644
index 0000000..48109ce
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/v2api/Api.java
@@ -0,0 +1,38 @@
+package org.apache.solr.v2api;
+
+/*
+ * 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.
+ */
+
+
+import org.apache.solr.common.util.Map2;
+
+public abstract class Api implements SpecProvider {
+  protected Map2 spec;
+
+  protected Api(Map2 spec) {
+    this.spec = spec;
+  }
+
+
+  public abstract void call(V2RequestContext ctx);
+
+  @Override
+  public Map2 getSpec() {
+    return spec;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/v2api/ApiBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/ApiBag.java b/solr/core/src/java/org/apache/solr/v2api/ApiBag.java
index 9967999..239c197 100644
--- a/solr/core/src/java/org/apache/solr/v2api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/v2api/ApiBag.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.Lookup;
 import org.apache.solr.common.util.Map2;
 import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
@@ -44,7 +43,7 @@ import static org.apache.solr.common.util.Map2.NOT_NULL;
 public class ApiBag {
   private static final Logger log = LoggerFactory.getLogger(ApiBag.class);
 
-  private final Map<String, PathTrie<V2Api>> apis = new ConcurrentHashMap<>();
+  private final Map<String, PathTrie<Api>> apis = new ConcurrentHashMap<>();
 
   public static Map2 getResource(String name) {
     InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(name);
@@ -67,7 +66,7 @@ public class ApiBag {
 
 
 
-  public synchronized void register(V2Api api, Map<String, String> nameSubstitutes) {
+  public synchronized void register(Api api, Map<String, String> nameSubstitutes) {
     try {
       validateAndRegister(api, nameSubstitutes);
     } catch (Exception e) {
@@ -81,12 +80,12 @@ public class ApiBag {
     }
   }
 
-  private void validateAndRegister(V2Api api, Map<String, String> nameSubstitutes) {
+  private void validateAndRegister(Api api, Map<String, String> nameSubstitutes) {
     Map2 spec = api.getSpec();
-    V2Api introspect = getIntrospect(api);
+    Api introspect = getIntrospect(api);
     List<String> methods = spec.getList("methods", ENUM_OF, SUPPORTED_METHODS);
     for (String method : methods) {
-      PathTrie<V2Api> registry = apis.get(method);
+      PathTrie<Api> registry = apis.get(method);
       if (registry == null) apis.put(method, registry = new PathTrie<>());
       Map2 url = spec.getMap("url", NOT_NULL);
       Map2 params = url.getMap("params", null);
@@ -116,8 +115,8 @@ public class ApiBag {
     }
   }
 
-  private V2Api getIntrospect(final V2Api baseApi) {
-    return new V2Api(Map2.EMPTY) {
+  private Api getIntrospect(final Api baseApi) {
+    return new Api(Map2.EMPTY) {
 
       @Override
       public Map2 getSpec() {
@@ -166,15 +165,15 @@ public class ApiBag {
   }
 
 
-  public V2Api lookup(String path, String httpMethod, Map<String, String> parts) {
+  public Api lookup(String path, String httpMethod, Map<String, String> parts) {
     if (httpMethod == null) {
-      for (PathTrie<V2Api> trie : apis.values()) {
-        V2Api api = trie.lookup(path, parts);
+      for (PathTrie<Api> trie : apis.values()) {
+        Api api = trie.lookup(path, parts);
         if (api != null) return api;
       }
       return null;
     } else {
-      PathTrie<V2Api> registry = apis.get(httpMethod);
+      PathTrie<Api> registry = apis.get(httpMethod);
       if (registry == null) return null;
       return registry.lookup(path, parts);
     }
@@ -205,8 +204,8 @@ public class ApiBag {
     return result;
   }
 
-  public static V2Api wrapRequestHandler(final SolrRequestHandler rh, final Map2 spec, SpecProvider specProvider) {
-    return new V2Api(spec) {
+  public static Api wrapRequestHandler(final SolrRequestHandler rh, final Map2 spec, SpecProvider specProvider) {
+    return new Api(spec) {
       @Override
       public void call(V2RequestContext ctx) {
         rh.handleRequest(ctx.getSolrRequest(), ctx.getResponse());
@@ -221,7 +220,7 @@ public class ApiBag {
     };
   }
 
-  public static final String APISPEC_LOCATION = "v2apispec/";
+  public static final String APISPEC_LOCATION = "apispec/";
   public static final String INTROSPECT = "/_introspect";
 
 
@@ -229,7 +228,7 @@ public class ApiBag {
   public static final String HANDLER_NAME = "handlerName";
   public static final Set<String> KNOWN_TYPES = ImmutableSet.of("string", "boolean", "list", "int", "double");
 
-  public PathTrie<V2Api> getRegistry(String method) {
+  public PathTrie<Api> getRegistry(String method) {
     return apis.get(method);
   }
 
@@ -251,10 +250,10 @@ public class ApiBag {
     }
   }
 
-  public static class LazyLoadedApi extends V2Api {
+  public static class LazyLoadedApi extends Api {
 
     private final PluginBag.PluginHolder<SolrRequestHandler> holder;
-    private V2Api delegate;
+    private Api delegate;
 
     protected LazyLoadedApi(Map2 spec, PluginBag.PluginHolder<SolrRequestHandler> lazyPluginHolder) {
       super(spec);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cfebd53/solr/core/src/java/org/apache/solr/v2api/ApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/ApiSupport.java b/solr/core/src/java/org/apache/solr/v2api/ApiSupport.java
new file mode 100644
index 0000000..e11cb4b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/v2api/ApiSupport.java
@@ -0,0 +1,30 @@
+package org.apache.solr.v2api;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+
+public interface ApiSupport {
+
+  Collection<Api> getApis();
+
+
+  default boolean registerAutomatically() {
+    return true;
+  }
+}