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/12 03:19:45 UTC

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

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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
deleted file mode 100644
index 9357e2d..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/V2HttpCall.java
+++ /dev/null
@@ -1,369 +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 javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.ContentStream;
-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.logging.MDCLoggingContext;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.QueryResponseWriter;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.AuthorizationContext;
-import org.apache.solr.servlet.HttpSolrCall;
-import org.apache.solr.servlet.SolrDispatchFilter;
-import org.apache.solr.servlet.SolrRequestParsers;
-import org.apache.solr.util.CommandOperation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.solr.common.params.CommonParams.JSON;
-import static org.apache.solr.common.params.CommonParams.WT;
-import static org.apache.solr.common.util.Map2.getDeepCopy;
-import static org.apache.solr.common.util.StrUtils.formatString;
-import static org.apache.solr.common.util.Map2.NOT_NULL;
-import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
-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 Api api;
-  private List<String> pieces;
-  private String prefix;
-  HashMap<String, String> parts = new HashMap<>();
-  static final Set<String> knownPrefixes = ImmutableSet.of("cluster", "node", "collections", "cores", "c");
-  static final Set<String> commonPaths4ContainerLevelAndCoreLevel = ImmutableSet.of("collections", "cores", "c");
-
-  public V2HttpCall(SolrDispatchFilter solrDispatchFilter, CoreContainer cc,
-                    HttpServletRequest request, HttpServletResponse response, boolean retry) {
-    super(solrDispatchFilter, cc, request, response, retry);
-  }
-
-  protected void init() throws Exception {
-    String path = this.path;
-    String fullPath = path = path.substring(3);//strip off '/v2'
-    try {
-      pieces = PathTrie.getParts(path);
-      if (pieces.size() == 0) {
-        prefix = "c";
-        path = "/c";
-      } else {
-        prefix = pieces.get(0);
-      }
-
-      if (knownPrefixes.contains(prefix)) {
-        api = getApiInfo(cores.getRequestHandlers(), path, req.getMethod(), cores, prefix, null, parts);
-        if (api != null) {
-          solrReq = SolrRequestParsers.DEFAULT.parse(null, path, req);
-          solrReq.getContext().put(CoreContainer.class.getName(), cores);
-          requestType = AuthorizationContext.RequestType.ADMIN;
-          action = ADMIN;
-          return;
-        }
-      }
-
-      if ("c".equals(prefix) || "collections".equals(prefix)) {
-        String collectionName = origCorename = corename = pieces.get(1);
-        DocCollection collection = getDocCollection(collectionName);
-        if (collection == null)
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such collection or alias");
-        core = getCoreByCollection(collection.getName());
-        if (core == null) {
-          //this collection exists , but this node does not have a replica for that collection
-          //todo find a better way to compute remote
-          extractRemotePath(corename, origCorename, 0);
-          return;
-        }
-
-      } else if ("cores".equals(prefix)) {
-        origCorename = corename = pieces.get(1);
-        core = cores.getCore(corename);
-      }
-      if (core == null)
-        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no core retrieved for " + corename);
-
-
-      this.path = path = path.substring(prefix.length() + pieces.get(1).length() + 2);
-      api = getApiInfo(core.getRequestHandlers(), path, req.getMethod(), cores, prefix, fullPath, parts);
-      MDCLoggingContext.setCore(core);
-      parseRequest();
-
-      if (usingAliases) {
-        processAliases(aliases, collectionsList);
-      }
-
-      action = PROCESS;
-      // we are done with a valid handler
-    } catch (RuntimeException rte) {
-      log.error("Error in init()", rte);
-      throw rte;
-    } finally {
-      if (solrReq != null) solrReq.getContext().put(CommonParams.PATH, path);
-    }
-  }
-
-  protected void parseRequest() throws Exception {
-    config = core.getSolrConfig();
-    // get or create/cache the parser for the core
-    SolrRequestParsers parser = config.getRequestParsers();
-
-    // With a valid handler and a valid core...
-
-    if (solrReq == null) solrReq = parser.parse(core, path, req);
-  }
-
-  protected DocCollection getDocCollection(String collectionName) {
-    if (!cores.isZooKeeperAware()) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Solr not running in cloud mode ");
-    }
-    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
-    DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
-    if (collection == null) {
-      collectionName = corename = lookupAliases(collectionName);
-      collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
-    }
-    return collection;
-  }
-
-  public static Api getApiInfo(PluginBag<SolrRequestHandler> requestHandlers,
-                                 String path, String method,
-                                 CoreContainer cores, String prefix, String fullPath,
-                                 Map<String, String> parts) {
-    if (fullPath == null) fullPath = path;
-    Api api;
-    boolean containerHandlerLookup = cores.getRequestHandlers() == requestHandlers;
-    api = requestHandlers.v2lookup(path, method, parts);
-    if (api == null && path.endsWith(ApiBag.INTROSPECT)) {
-      // the particular http method does not have any ,
-      // just try if any other method has this path
-      api = requestHandlers.v2lookup(path, null, parts);
-    }
-
-    if (api == null) {
-      // this is to return the user with all the subpaths for  a given 4044 request
-      // the request  begins with /collections , /cores or a /c and the current lookup is on container level handlers
-      // So the subsequent per core lookup would find a path
-      if (containerHandlerLookup && commonPaths4ContainerLevelAndCoreLevel.contains(prefix)) return null;
-
-
-      Map<String, Set<String>> subpaths = new LinkedHashMap<>();
-
-      getSubPaths(path, requestHandlers.getApiBag(), subpaths);
-      if (!containerHandlerLookup) getSubPaths(fullPath, cores.getRequestHandlers().getApiBag(), subpaths);
-
-      if (subpaths.isEmpty()) {
-        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No valid handler for path :" + path);
-      } else {
-        return getSubPathImpl(subpaths, fullPath);
-      }
-    }
-    if (api.getSpec() == ApiBag.INTROSPECT_SPEC)
-      api = mergeIntrospect(requestHandlers, path, method, parts);
-    return api;
-  }
-
-  private static void getSubPaths(String path, ApiBag bag, Map<String, Set<String>> pathsVsMethod) {
-    for (SolrRequest.METHOD m : SolrRequest.METHOD.values()) {
-      PathTrie<Api> registry = bag.getRegistry(m.toString());
-      if (registry != null) {
-        HashSet<String> subPaths = new HashSet<>();
-        registry.lookup(path, new HashMap<>(), subPaths);
-        for (String subPath : subPaths) {
-          Set<String> supportedMethods = pathsVsMethod.get(subPath);
-          if(supportedMethods == null) pathsVsMethod.put(subPath, supportedMethods = new HashSet<>());
-          supportedMethods.add(m.toString());
-        }
-      }
-    }
-  }
-
-  private static Api mergeIntrospect(PluginBag<SolrRequestHandler> requestHandlers,
-                                       String path, String method, Map<String, String> parts) {
-    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 Api(ApiBag.INTROSPECT_SPEC) {
-      @Override
-      public void call(V2RequestContext ctx) {
-        String method = ctx.getSolrRequest().getParams().get("method");
-        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);
-              added.add(e.getValue());
-            }
-          }
-        }
-      }
-    };
-    return api;
-  }
-
-  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");
-        LinkedHashMap<String, Set<String>> result = new LinkedHashMap<>(subpaths.size());
-        for (Map.Entry<String, Set<String>> e : subpaths.entrySet()) {
-          if (e.getKey().endsWith(ApiBag.INTROSPECT)) continue;
-          result.put(path + e.getKey(), e.getValue());
-        }
-        ctx.getResponse().add("availableSubPaths", result);
-      }
-    };
-  }
-
-  @Override
-  protected void handleAdmin(SolrQueryResponse solrResp) {
-    api.call(getV2RequestCtx(solrResp));
-  }
-
-  @Override
-  protected void execute(SolrQueryResponse rsp) {
-    try {
-      api.call(getV2RequestCtx(rsp));
-    } catch (RuntimeException e) {
-      //todo remove for debugging only
-      log.error("error execute()", e);
-      throw e;
-    }
-  }
-
-  @Override
-  protected QueryResponseWriter getResponseWriter() {
-    String wt = solrReq.getParams().get(WT, JSON);
-    if (core != null) return core.getResponseWriters().get(wt);
-    return SolrCore.DEFAULT_RESPONSE_WRITERS.get(wt);
-  }
-
-  public V2RequestContext getV2RequestCtx(SolrQueryResponse solrResp) {
-
-    return new V2RequestContext() {
-      List<CommandOperation> parsedCommands;
-
-      @Override
-      public SolrQueryResponse getResponse() {
-        return solrResp;
-      }
-
-      @Override
-      public CoreContainer getCoreContainer() {
-        return cores;
-      }
-
-      @Override
-      public SolrQueryRequest getSolrRequest() {
-        return solrReq;
-      }
-
-      @Override
-      public String getPath() {
-        return path;
-      }
-
-      @Override
-      public String getHttpMethod() {
-        return (String) solrReq.getContext().get("httpMethod");
-      }
-
-      @Override
-      public Map<String, String> getPathValues() {
-        return parts;
-      }
-
-      @Override
-      public List<CommandOperation> getCommands(boolean validateInput) {
-        if (parsedCommands == null) {
-          Iterable<ContentStream> contentStreams = solrReq.getContentStreams();
-          if (contentStreams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No content stream");
-          for (ContentStream contentStream : contentStreams) {
-            parsedCommands = getCommandOperations(new InputStreamReader((InputStream) contentStream, UTF_8),
-                api.getSpec(), solrResp);
-
-          }
-
-        }
-        return CommandOperation.clone(parsedCommands);
-
-      }
-
-
-    };
-  }
-
-  public static List<CommandOperation> getCommandOperations(Reader reader, Map2 spec, SolrQueryResponse rsp) {
-    List<CommandOperation> parsedCommands = null;
-    try {
-      parsedCommands = CommandOperation.parse(reader);
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
-    }
-
-    Map2 cmds = spec.getMap("commands", NOT_NULL);
-    List<CommandOperation> commandsCopy = CommandOperation.clone(parsedCommands);
-
-    for (CommandOperation cmd : commandsCopy) {
-      if (!cmds.containsKey(cmd.name)) {
-        cmd.addError(formatString("Unknown operation ''{0}'' in path ''{1}''", cmd.name,
-            spec.getMap("url", NOT_NULL).get("paths")));
-      }
-      //TODO validation
-
-    }
-    List<Map> errs = CommandOperation.captureErrors(commandsCopy);
-    if (!errs.isEmpty()) {
-      rsp.add(CommandOperation.ERR_MSGS, errs);
-      SolrException exp = new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Errors in commands");
-      rsp.setException(exp);
-      throw exp;
-    }
-    return commandsCopy;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/v2api/V2RequestContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/V2RequestContext.java b/solr/core/src/java/org/apache/solr/v2api/V2RequestContext.java
deleted file mode 100644
index 18ca61c..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/V2RequestContext.java
+++ /dev/null
@@ -1,45 +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.List;
-import java.util.Map;
-
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.CommandOperation;
-
-public interface V2RequestContext {
-
-  SolrQueryResponse getResponse();
-
-  CoreContainer getCoreContainer();
-
-  SolrQueryRequest getSolrRequest();
-
-  String getPath();
-
-
-  Map<String, String> getPathValues();
-
-  List<CommandOperation> getCommands(boolean validateInput);
-
-  String getHttpMethod();
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/api/TestPathTrie.java b/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
new file mode 100644
index 0000000..3d2b1e1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/api/TestPathTrie.java
@@ -0,0 +1,52 @@
+package org.apache.solr.api;
+
+/*
+ * 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 org.apache.solr.SolrTestCaseJ4;
+
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.api.ApiBag.HANDLER_NAME;
+
+public class TestPathTrie extends SolrTestCaseJ4 {
+
+  public void testPathTrie() {
+    PathTrie<String> pathTrie = new PathTrie<>();
+    pathTrie.insert("/", emptyMap(), "R");
+    pathTrie.insert("/aa", emptyMap(), "d");
+    pathTrie.insert("/aa/bb/{cc}/dd", emptyMap(), "a");
+    pathTrie.insert("/$handlerName/{cc}/dd", singletonMap(HANDLER_NAME, "test"), "test");
+    pathTrie.insert("/aa/bb/{cc}/{xx}", emptyMap(), "b");
+    pathTrie.insert("/aa/bb", emptyMap(), "c");
+
+    HashMap parts = new HashMap<>();
+    assertEquals("R", pathTrie.lookup("/", parts, null));
+    assertEquals("d", pathTrie.lookup("/aa", parts, null));
+    assertEquals("a", pathTrie.lookup("/aa/bb/hello/dd", parts, null));
+    assertEquals("test", pathTrie.lookup("/test/hello/dd", parts, null));
+    assertEquals("hello", parts.get("cc"));
+    assertEquals("b", pathTrie.lookup("/aa/bb/hello/world", parts, null));
+    assertEquals("hello", parts.get("cc"));
+    assertEquals("world", parts.get("xx"));
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index 53735bf..496d5df 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -35,9 +35,9 @@ 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 org.apache.solr.api.Api;
+import org.apache.solr.api.V2HttpCall;
+import org.apache.solr.api.V2RequestContext;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
 import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
index 4b538b4..45d4e96 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -35,10 +35,10 @@ 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 org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.V2HttpCall;
+import org.apache.solr.api.V2RequestContext;
 
 import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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
index b6ad4c5..1fde321 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCoreAdminApis.java
@@ -30,8 +30,8 @@ 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.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.easymock.EasyMock;
 import org.easymock.IAnswer;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/test/org/apache/solr/v2api/TestPathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/v2api/TestPathTrie.java b/solr/core/src/test/org/apache/solr/v2api/TestPathTrie.java
deleted file mode 100644
index a2f91eb..0000000
--- a/solr/core/src/test/org/apache/solr/v2api/TestPathTrie.java
+++ /dev/null
@@ -1,52 +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.HashMap;
-
-import org.apache.solr.SolrTestCaseJ4;
-
-import static java.util.Collections.emptyMap;
-import static java.util.Collections.singletonMap;
-import static org.apache.solr.v2api.ApiBag.HANDLER_NAME;
-
-public class TestPathTrie extends SolrTestCaseJ4 {
-
-  public void testPathTrie() {
-    PathTrie<String> pathTrie = new PathTrie<>();
-    pathTrie.insert("/", emptyMap(), "R");
-    pathTrie.insert("/aa", emptyMap(), "d");
-    pathTrie.insert("/aa/bb/{cc}/dd", emptyMap(), "a");
-    pathTrie.insert("/$handlerName/{cc}/dd", singletonMap(HANDLER_NAME, "test"), "test");
-    pathTrie.insert("/aa/bb/{cc}/{xx}", emptyMap(), "b");
-    pathTrie.insert("/aa/bb", emptyMap(), "c");
-
-    HashMap parts = new HashMap<>();
-    assertEquals("R", pathTrie.lookup("/", parts, null));
-    assertEquals("d", pathTrie.lookup("/aa", parts, null));
-    assertEquals("a", pathTrie.lookup("/aa/bb/hello/dd", parts, null));
-    assertEquals("test", pathTrie.lookup("/test/hello/dd", parts, null));
-    assertEquals("hello", parts.get("cc"));
-    assertEquals("b", pathTrie.lookup("/aa/bb/hello/world", parts, null));
-    assertEquals("hello", parts.get("cc"));
-    assertEquals("world", parts.get("xx"));
-
-
-  }
-}


[2/2] 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/34603800
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/34603800
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/34603800

Branch: refs/heads/apiv2
Commit: 346038000e19bad6d3a5a2b9063849b94e897848
Parents: 3cfebd5
Author: Noble Paul <no...@gmail.com>
Authored: Thu Feb 11 18:19:27 2016 -0800
Committer: Noble Paul <no...@gmail.com>
Committed: Thu Feb 11 18:19:27 2016 -0800

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/api/Api.java |  38 ++
 .../src/java/org/apache/solr/api/ApiBag.java    | 272 ++++++++++++++
 .../java/org/apache/solr/api/ApiSupport.java    |  30 ++
 .../src/java/org/apache/solr/api/PathTrie.java  | 171 +++++++++
 .../java/org/apache/solr/api/SpecLoader.java    |  25 ++
 .../java/org/apache/solr/api/SpecProvider.java  |  24 ++
 .../java/org/apache/solr/api/V2HttpCall.java    | 369 +++++++++++++++++++
 .../org/apache/solr/api/V2RequestContext.java   |  45 +++
 .../java/org/apache/solr/core/PluginBag.java    |   8 +-
 .../org/apache/solr/core/RequestHandlers.java   |   3 -
 .../apache/solr/handler/RequestHandlerBase.java |   8 +-
 .../org/apache/solr/handler/SchemaHandler.java  |   8 +-
 .../apache/solr/handler/SolrConfigHandler.java  |   8 +-
 .../solr/handler/UpdateRequestHandlerApi.java   |   6 +-
 .../handler/admin/BaseHandlerApiSupport.java    |   8 +-
 .../handler/admin/CollectionHandlerApi.java     |   2 +-
 .../solr/handler/admin/CollectionsHandler.java  |   4 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   4 +-
 .../solr/handler/admin/CoreAdminHandlerApi.java |   2 +-
 .../apache/solr/handler/admin/InfoHandler.java  |   8 +-
 .../solr/handler/admin/SecurityConfHandler.java |   8 +-
 .../apache/solr/handler/admin/V2Command.java    |   3 +-
 .../apache/solr/security/BasicAuthPlugin.java   |   3 +-
 .../security/RuleBasedAuthorizationPlugin.java  |   5 +-
 .../security/Sha256AuthenticationProvider.java  |   3 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |   7 +-
 .../src/java/org/apache/solr/v2api/Api.java     |  38 --
 .../src/java/org/apache/solr/v2api/ApiBag.java  | 272 --------------
 .../java/org/apache/solr/v2api/ApiSupport.java  |  30 --
 .../java/org/apache/solr/v2api/PathTrie.java    | 171 ---------
 .../java/org/apache/solr/v2api/SpecLoader.java  |  25 --
 .../org/apache/solr/v2api/SpecProvider.java     |  25 --
 .../java/org/apache/solr/v2api/V2HttpCall.java  | 369 -------------------
 .../org/apache/solr/v2api/V2RequestContext.java |  45 ---
 .../test/org/apache/solr/api/TestPathTrie.java  |  52 +++
 .../solr/handler/admin/TestApiFramework.java    |   6 +-
 .../solr/handler/admin/TestCollectionAPIs.java  |   8 +-
 .../solr/handler/admin/TestCoreAdminApis.java   |   4 +-
 .../org/apache/solr/v2api/TestPathTrie.java     |  52 ---
 39 files changed, 1078 insertions(+), 1091 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/Api.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/Api.java b/solr/core/src/java/org/apache/solr/api/Api.java
new file mode 100644
index 0000000..d540062
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/Api.java
@@ -0,0 +1,38 @@
+package org.apache.solr.api;
+
+/*
+ * 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/34603800/solr/core/src/java/org/apache/solr/api/ApiBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
new file mode 100644
index 0000000..71021b3
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -0,0 +1,272 @@
+package org.apache.solr.api;
+
+/*
+ * 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.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Map2;
+import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.request.SolrRequestHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.client.solrj.SolrRequest.SUPPORTED_METHODS;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.common.util.Map2.ENUM_OF;
+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<Api>> apis = new ConcurrentHashMap<>();
+
+  public static Map2 getResource(String name) {
+    InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(name);
+    if (is == null)
+      throw new RuntimeException("invalid API spec :" + name );
+    Map2 map1 = null;
+    try {
+      map1 = Map2.fromJSON(is);
+    } catch (Exception e) {
+      log.error("Error in JSON : " + name, e);
+      if (e instanceof RuntimeException) {
+        throw (RuntimeException) e;
+      }
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
+    if (map1 == null) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Empty value for " + name);
+
+    return Map2.getDeepCopy(map1, 5, false);
+  }
+
+
+
+  public synchronized void register(Api api, Map<String, String> nameSubstitutes) {
+    try {
+      validateAndRegister(api, nameSubstitutes);
+    } catch (Exception e) {
+      log.error("Unable to register plugin:" + api.getClass().getName() + "with spec :" + api.getSpec(), e);
+      if (e instanceof RuntimeException) {
+        throw (RuntimeException) e;
+      } else {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+
+    }
+  }
+
+  private void validateAndRegister(Api api, Map<String, String> nameSubstitutes) {
+    Map2 spec = api.getSpec();
+    Api introspect = getIntrospect(api);
+    List<String> methods = spec.getList("methods", ENUM_OF, SUPPORTED_METHODS);
+    for (String method : methods) {
+      PathTrie<Api> registry = apis.get(method);
+      if (registry == null) apis.put(method, registry = new PathTrie<>());
+      Map2 url = spec.getMap("url", NOT_NULL);
+      Map2 params = url.getMap("params", null);
+      if (params != null) {
+        for (Object o : params.keySet()) {
+          Map2 param = params.getMap(o.toString(), NOT_NULL);
+          param.get("type", ENUM_OF, KNOWN_TYPES);
+          param.get("description", NOT_NULL);
+        }
+      }
+      List<String> paths = url.getList("paths", NOT_NULL);
+      Map2 parts = url.getMap("parts", null);
+      if (parts != null) {
+        Set<String> wildCardNames = getWildCardNames(paths);
+        for (Object o : parts.keySet()) {
+          if (!wildCardNames.contains(o.toString()))
+            throw new RuntimeException("" + o + " is not a valid part name");
+          Map2 pathMeta = parts.getMap(o.toString(), NOT_NULL);
+          pathMeta.get("type", ENUM_OF, ImmutableSet.of("enum", "string", "int", "number", "boolean"));
+        }
+      }
+      verifyCommands(api.getSpec());
+      for (String path : paths) {
+        registry.insert(path, nameSubstitutes, api);
+        registry.insert(path + INTROSPECT, nameSubstitutes, introspect);
+      }
+    }
+  }
+
+  private Api getIntrospect(final Api baseApi) {
+    return new Api(Map2.EMPTY) {
+
+      @Override
+      public Map2 getSpec() {
+        return INTROSPECT_SPEC;
+      }
+
+      @Override
+      public void call(V2RequestContext ctx) {
+        String cmd = ctx.getSolrRequest().getParams().get("command");
+        Map2 result = null;
+        if (cmd == null) {
+          result = baseApi.getSpec();
+        } else {
+          Map2 specCopy = Map2.getDeepCopy(baseApi.getSpec(), 5, true);
+          Map2 commands = specCopy.getMap("commands", null);
+          if (commands != null) {
+            Map2 m = commands.getMap(cmd, null);
+            specCopy.put("commands", Collections.singletonMap(cmd, m));
+          }
+          result = specCopy;
+        }
+        List l = (List) ctx.getResponse().getValues().get("spec");
+        if (l == null) ctx.getResponse().getValues().add("spec", l = new ArrayList());
+        l.add(result);
+      }
+    };
+  }
+
+  private void verifyCommands(Map2 spec) {
+    Map2 commands = spec.getMap("commands", null);
+    if (commands == null) return;
+    //TODO do verify
+
+  }
+
+  private Set<String> getWildCardNames(List<String> paths) {
+    Set<String> wildCardNames = new HashSet<>();
+    for (String path : paths) {
+      List<String> p = PathTrie.getParts(path);
+      for (String s : p) {
+        String wildCard = PathTrie.wildCardName(s);
+        if (wildCard != null) wildCardNames.add(wildCard);
+      }
+    }
+    return wildCardNames;
+  }
+
+
+  public Api lookup(String path, String httpMethod, Map<String, String> parts) {
+    if (httpMethod == null) {
+      for (PathTrie<Api> trie : apis.values()) {
+        Api api = trie.lookup(path, parts);
+        if (api != null) return api;
+      }
+      return null;
+    } else {
+      PathTrie<Api> registry = apis.get(httpMethod);
+      if (registry == null) return null;
+      return registry.lookup(path, parts);
+    }
+  }
+
+  public static Map2 getSpec(String name) {
+    Map2 map = getResource(APISPEC_LOCATION + name + ".json");
+    Map2 result = map.getMap(name, NOT_NULL);
+    Map2 cmds = result.getMap("commands", null);
+    if (cmds != null) {
+      Map<String, Map2> comands2BReplaced = new Map2<>();
+      for (Object o : cmds.keySet()) {
+        Object val = cmds.get(o);
+        if (val instanceof String) {
+          String s = (String) val;
+          Map2 cmdSpec = getResource(APISPEC_LOCATION + s + ".json");
+          comands2BReplaced.put(o.toString(), cmdSpec);
+        }
+      }
+
+      if (!comands2BReplaced.isEmpty()) {
+        Map2 mapCopy = Map2.getDeepCopy(result, 4, true);
+        mapCopy.getMap("commands", NOT_NULL).putAll(comands2BReplaced);
+        result = Map2.getDeepCopy(mapCopy, 4, false);
+      }
+    }
+
+    return result;
+  }
+
+  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());
+      }
+
+      @Override
+      public Map2 getSpec() {
+        return specProvider != null ?
+            specProvider.getSpec() :
+            super.getSpec();
+      }
+    };
+  }
+
+  public static final String APISPEC_LOCATION = "apispec/";
+  public static final String INTROSPECT = "/_introspect";
+
+
+  public static final Map2 INTROSPECT_SPEC = new Map2(Collections.EMPTY_MAP);
+  public static final String HANDLER_NAME = "handlerName";
+  public static final Set<String> KNOWN_TYPES = ImmutableSet.of("string", "boolean", "list", "int", "double");
+
+  public PathTrie<Api> getRegistry(String method) {
+    return apis.get(method);
+  }
+
+  public <T> void registerLazy(PluginBag.PluginHolder<SolrRequestHandler> holder, PluginInfo info) {
+    String specName = info.attributes.get("spec");
+    if (specName == null) specName = "emptySpec";
+    Map2 spec = ApiBag.getSpec(specName);
+    register(new LazyLoadedApi(spec, holder), Collections.singletonMap(HANDLER_NAME, info.attributes.get(NAME)));
+  }
+
+  public static Map2 constructSpec(PluginInfo info) {
+    Object specObj = info == null ? null : info.attributes.get("spec");
+    if (specObj == null) specObj = "emptySpec";
+    if (specObj instanceof Map) {
+      Map map = (Map) specObj;
+      return Map2.getDeepCopy(map, 4, false);
+    } else {
+      return ApiBag.getSpec((String) specObj);
+    }
+  }
+
+  public static class LazyLoadedApi extends Api {
+
+    private final PluginBag.PluginHolder<SolrRequestHandler> holder;
+    private Api delegate;
+
+    protected LazyLoadedApi(Map2 spec, PluginBag.PluginHolder<SolrRequestHandler> lazyPluginHolder) {
+      super(spec);
+      this.holder = lazyPluginHolder;
+    }
+
+    @Override
+    public void call(V2RequestContext ctx) {
+      if (!holder.isLoaded()) {
+        delegate = wrapRequestHandler(holder.get(), null, null);
+      }
+      delegate.call(ctx);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/ApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiSupport.java b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
new file mode 100644
index 0000000..cfc2349
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
@@ -0,0 +1,30 @@
+package org.apache.solr.api;
+
+/*
+ * 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/PathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/PathTrie.java b/solr/core/src/java/org/apache/solr/api/PathTrie.java
new file mode 100644
index 0000000..9ba7403
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/PathTrie.java
@@ -0,0 +1,171 @@
+package org.apache.solr.api;
+
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.common.util.StrUtils;
+
+import static java.util.Collections.emptyList;
+
+public class PathTrie<T> {
+  Node root = new Node(emptyList(), null);
+
+
+  public void insert(String path, Map<String, String> replacements, T o) {
+    List<String> parts = getParts(path);
+    if (parts.isEmpty()) {
+      root.obj = o;
+      return;
+    }
+
+    for (int i = 0; i < parts.size(); i++) {
+      String part = parts.get(i);
+      if (part.charAt(0) == '$') {
+        String replacement = replacements.get(part.substring(1));
+        if (replacement == null) {
+          throw new RuntimeException(part + " is not provided");
+        }
+        replacement = replacement.charAt(0) == '/' ? replacement.substring(1) : replacement;
+        parts.set(i, replacement);
+      }
+    }
+
+    root.insert(parts, o);
+  }
+
+  public static List<String> getParts(String path) {
+    if (path == null || path.isEmpty()) return emptyList();
+    List<String> parts = StrUtils.splitSmart(path, '/');
+    while(true) {
+      if(parts.isEmpty()) break;
+      if ("".equals(parts.get(0))) parts.remove(0);
+      else break;
+    }
+    return parts;
+  }
+
+
+  public T lookup(String uri, Map<String, String> parts) {
+    return root.lookup(getParts(uri), 0, parts);
+  }
+
+  public T lookup(String path, Map<String, String> parts, Set<String> paths) {
+    return root.lookup(getParts(path), 0, parts, paths);
+  }
+
+  public static String wildCardName(String part) {
+    return part.startsWith("{") && part.endsWith("}") ?
+        part.substring(1, part.length() - 1) :
+        null;
+
+  }
+
+  class Node {
+    String name;
+    Map<String, Node> children;
+    T obj;
+    String varName;
+
+    Node(List<String> path, T o) {
+      if (path.isEmpty()) {
+        obj = o;
+        return;
+      }
+      String part = path.get(0);
+      varName = wildCardName(part);
+      name = part;
+      if (path.isEmpty()) obj = o;
+    }
+
+
+    private synchronized void insert(List<String> path, T o) {
+      String part = path.get(0);
+      Node matchedChild = null;
+      if (children == null) children = new ConcurrentHashMap<>();
+
+      String varName = wildCardName(part);
+      String key = varName == null ? part : "";
+
+      matchedChild = children.get(key);
+      if (matchedChild == null) {
+        children.put(key, matchedChild = new Node(path, o));
+      }
+      if (varName != null) {
+        if (!matchedChild.varName.equals(varName)) {
+          throw new RuntimeException("wildcard name must be " + matchedChild.varName);
+        }
+      }
+      path.remove(0);
+      if (!path.isEmpty()) {
+        matchedChild.insert(path, o);
+      } else {
+        matchedChild.obj = o;
+      }
+
+    }
+
+
+    void findValidChildren(String path, Set<String> availableSubPaths) {
+      if (availableSubPaths == null) return;
+      if (children != null) {
+        for (Node node : children.values()) {
+          if (node.obj != null) {
+            String s = path + "/" + node.name;
+            availableSubPaths.add(s);
+          }
+        }
+
+        for (Node node : children.values()) {
+          node.findValidChildren(path + "/" + node.name, availableSubPaths);
+        }
+      }
+    }
+
+
+    public T lookup(List<String> pieces, int i, Map<String, String> parts) {
+      return lookup(pieces, i, parts, null);
+
+    }
+
+    public T lookup(List<String> pieces, int i, Map<String, String> parts, Set<String> availableSubPaths) {
+      if (varName != null) parts.put(varName, pieces.get(i - 1));
+      if (pieces.size() < i + 1) {
+        findValidChildren("", availableSubPaths);
+        return obj;
+      }
+      String piece = pieces.get(i);
+      if (children == null) return null;
+      Node n = children.get(piece);
+      if (n == null && !reserved.contains(piece)) n = children.get("");
+      if (n == null) return null;
+      return n.lookup(pieces, i + 1, parts, availableSubPaths);
+    }
+  }
+
+  private Set<String> reserved = new HashSet<>();
+
+  {
+    reserved.add("_introspect");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/SpecLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/SpecLoader.java b/solr/core/src/java/org/apache/solr/api/SpecLoader.java
new file mode 100644
index 0000000..e30e4b3
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/SpecLoader.java
@@ -0,0 +1,25 @@
+package org.apache.solr.api;
+
+/*
+ * 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 interface SpecLoader {
+  Map2 get(String name);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/SpecProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/SpecProvider.java b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
new file mode 100644
index 0000000..1d1ad32
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
@@ -0,0 +1,24 @@
+package org.apache.solr.api;
+
+/*
+ * 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 interface SpecProvider {
+  Map2 getSpec();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
new file mode 100644
index 0000000..fe5c336
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
@@ -0,0 +1,369 @@
+package org.apache.solr.api;
+
+/*
+ * 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 javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.ContentStream;
+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.logging.MDCLoggingContext;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.servlet.HttpSolrCall;
+import org.apache.solr.servlet.SolrDispatchFilter;
+import org.apache.solr.servlet.SolrRequestParsers;
+import org.apache.solr.util.CommandOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.common.params.CommonParams.JSON;
+import static org.apache.solr.common.params.CommonParams.WT;
+import static org.apache.solr.common.util.Map2.getDeepCopy;
+import static org.apache.solr.common.util.StrUtils.formatString;
+import static org.apache.solr.common.util.Map2.NOT_NULL;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
+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 Api api;
+  private List<String> pieces;
+  private String prefix;
+  HashMap<String, String> parts = new HashMap<>();
+  static final Set<String> knownPrefixes = ImmutableSet.of("cluster", "node", "collections", "cores", "c");
+  static final Set<String> commonPaths4ContainerLevelAndCoreLevel = ImmutableSet.of("collections", "cores", "c");
+
+  public V2HttpCall(SolrDispatchFilter solrDispatchFilter, CoreContainer cc,
+                    HttpServletRequest request, HttpServletResponse response, boolean retry) {
+    super(solrDispatchFilter, cc, request, response, retry);
+  }
+
+  protected void init() throws Exception {
+    String path = this.path;
+    String fullPath = path = path.substring(3);//strip off '/v2'
+    try {
+      pieces = PathTrie.getParts(path);
+      if (pieces.size() == 0) {
+        prefix = "c";
+        path = "/c";
+      } else {
+        prefix = pieces.get(0);
+      }
+
+      if (knownPrefixes.contains(prefix)) {
+        api = getApiInfo(cores.getRequestHandlers(), path, req.getMethod(), cores, prefix, null, parts);
+        if (api != null) {
+          solrReq = SolrRequestParsers.DEFAULT.parse(null, path, req);
+          solrReq.getContext().put(CoreContainer.class.getName(), cores);
+          requestType = AuthorizationContext.RequestType.ADMIN;
+          action = ADMIN;
+          return;
+        }
+      }
+
+      if ("c".equals(prefix) || "collections".equals(prefix)) {
+        String collectionName = origCorename = corename = pieces.get(1);
+        DocCollection collection = getDocCollection(collectionName);
+        if (collection == null)
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such collection or alias");
+        core = getCoreByCollection(collection.getName());
+        if (core == null) {
+          //this collection exists , but this node does not have a replica for that collection
+          //todo find a better way to compute remote
+          extractRemotePath(corename, origCorename, 0);
+          return;
+        }
+
+      } else if ("cores".equals(prefix)) {
+        origCorename = corename = pieces.get(1);
+        core = cores.getCore(corename);
+      }
+      if (core == null)
+        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no core retrieved for " + corename);
+
+
+      this.path = path = path.substring(prefix.length() + pieces.get(1).length() + 2);
+      api = getApiInfo(core.getRequestHandlers(), path, req.getMethod(), cores, prefix, fullPath, parts);
+      MDCLoggingContext.setCore(core);
+      parseRequest();
+
+      if (usingAliases) {
+        processAliases(aliases, collectionsList);
+      }
+
+      action = PROCESS;
+      // we are done with a valid handler
+    } catch (RuntimeException rte) {
+      log.error("Error in init()", rte);
+      throw rte;
+    } finally {
+      if (solrReq != null) solrReq.getContext().put(CommonParams.PATH, path);
+    }
+  }
+
+  protected void parseRequest() throws Exception {
+    config = core.getSolrConfig();
+    // get or create/cache the parser for the core
+    SolrRequestParsers parser = config.getRequestParsers();
+
+    // With a valid handler and a valid core...
+
+    if (solrReq == null) solrReq = parser.parse(core, path, req);
+  }
+
+  protected DocCollection getDocCollection(String collectionName) {
+    if (!cores.isZooKeeperAware()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Solr not running in cloud mode ");
+    }
+    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
+    DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
+    if (collection == null) {
+      collectionName = corename = lookupAliases(collectionName);
+      collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
+    }
+    return collection;
+  }
+
+  public static Api getApiInfo(PluginBag<SolrRequestHandler> requestHandlers,
+                                 String path, String method,
+                                 CoreContainer cores, String prefix, String fullPath,
+                                 Map<String, String> parts) {
+    if (fullPath == null) fullPath = path;
+    Api api;
+    boolean containerHandlerLookup = cores.getRequestHandlers() == requestHandlers;
+    api = requestHandlers.v2lookup(path, method, parts);
+    if (api == null && path.endsWith(ApiBag.INTROSPECT)) {
+      // the particular http method does not have any ,
+      // just try if any other method has this path
+      api = requestHandlers.v2lookup(path, null, parts);
+    }
+
+    if (api == null) {
+      // this is to return the user with all the subpaths for  a given 4044 request
+      // the request  begins with /collections , /cores or a /c and the current lookup is on container level handlers
+      // So the subsequent per core lookup would find a path
+      if (containerHandlerLookup && commonPaths4ContainerLevelAndCoreLevel.contains(prefix)) return null;
+
+
+      Map<String, Set<String>> subpaths = new LinkedHashMap<>();
+
+      getSubPaths(path, requestHandlers.getApiBag(), subpaths);
+      if (!containerHandlerLookup) getSubPaths(fullPath, cores.getRequestHandlers().getApiBag(), subpaths);
+
+      if (subpaths.isEmpty()) {
+        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No valid handler for path :" + path);
+      } else {
+        return getSubPathImpl(subpaths, fullPath);
+      }
+    }
+    if (api.getSpec() == ApiBag.INTROSPECT_SPEC)
+      api = mergeIntrospect(requestHandlers, path, method, parts);
+    return api;
+  }
+
+  private static void getSubPaths(String path, ApiBag bag, Map<String, Set<String>> pathsVsMethod) {
+    for (SolrRequest.METHOD m : SolrRequest.METHOD.values()) {
+      PathTrie<Api> registry = bag.getRegistry(m.toString());
+      if (registry != null) {
+        HashSet<String> subPaths = new HashSet<>();
+        registry.lookup(path, new HashMap<>(), subPaths);
+        for (String subPath : subPaths) {
+          Set<String> supportedMethods = pathsVsMethod.get(subPath);
+          if(supportedMethods == null) pathsVsMethod.put(subPath, supportedMethods = new HashSet<>());
+          supportedMethods.add(m.toString());
+        }
+      }
+    }
+  }
+
+  private static Api mergeIntrospect(PluginBag<SolrRequestHandler> requestHandlers,
+                                       String path, String method, Map<String, String> parts) {
+    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 Api(ApiBag.INTROSPECT_SPEC) {
+      @Override
+      public void call(V2RequestContext ctx) {
+        String method = ctx.getSolrRequest().getParams().get("method");
+        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);
+              added.add(e.getValue());
+            }
+          }
+        }
+      }
+    };
+    return api;
+  }
+
+  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");
+        LinkedHashMap<String, Set<String>> result = new LinkedHashMap<>(subpaths.size());
+        for (Map.Entry<String, Set<String>> e : subpaths.entrySet()) {
+          if (e.getKey().endsWith(ApiBag.INTROSPECT)) continue;
+          result.put(path + e.getKey(), e.getValue());
+        }
+        ctx.getResponse().add("availableSubPaths", result);
+      }
+    };
+  }
+
+  @Override
+  protected void handleAdmin(SolrQueryResponse solrResp) {
+    api.call(getV2RequestCtx(solrResp));
+  }
+
+  @Override
+  protected void execute(SolrQueryResponse rsp) {
+    try {
+      api.call(getV2RequestCtx(rsp));
+    } catch (RuntimeException e) {
+      //todo remove for debugging only
+      log.error("error execute()", e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected QueryResponseWriter getResponseWriter() {
+    String wt = solrReq.getParams().get(WT, JSON);
+    if (core != null) return core.getResponseWriters().get(wt);
+    return SolrCore.DEFAULT_RESPONSE_WRITERS.get(wt);
+  }
+
+  public V2RequestContext getV2RequestCtx(SolrQueryResponse solrResp) {
+
+    return new V2RequestContext() {
+      List<CommandOperation> parsedCommands;
+
+      @Override
+      public SolrQueryResponse getResponse() {
+        return solrResp;
+      }
+
+      @Override
+      public CoreContainer getCoreContainer() {
+        return cores;
+      }
+
+      @Override
+      public SolrQueryRequest getSolrRequest() {
+        return solrReq;
+      }
+
+      @Override
+      public String getPath() {
+        return path;
+      }
+
+      @Override
+      public String getHttpMethod() {
+        return (String) solrReq.getContext().get("httpMethod");
+      }
+
+      @Override
+      public Map<String, String> getPathValues() {
+        return parts;
+      }
+
+      @Override
+      public List<CommandOperation> getCommands(boolean validateInput) {
+        if (parsedCommands == null) {
+          Iterable<ContentStream> contentStreams = solrReq.getContentStreams();
+          if (contentStreams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No content stream");
+          for (ContentStream contentStream : contentStreams) {
+            parsedCommands = getCommandOperations(new InputStreamReader((InputStream) contentStream, UTF_8),
+                api.getSpec(), solrResp);
+
+          }
+
+        }
+        return CommandOperation.clone(parsedCommands);
+
+      }
+
+
+    };
+  }
+
+  public static List<CommandOperation> getCommandOperations(Reader reader, Map2 spec, SolrQueryResponse rsp) {
+    List<CommandOperation> parsedCommands = null;
+    try {
+      parsedCommands = CommandOperation.parse(reader);
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+
+    Map2 cmds = spec.getMap("commands", NOT_NULL);
+    List<CommandOperation> commandsCopy = CommandOperation.clone(parsedCommands);
+
+    for (CommandOperation cmd : commandsCopy) {
+      if (!cmds.containsKey(cmd.name)) {
+        cmd.addError(formatString("Unknown operation ''{0}'' in path ''{1}''", cmd.name,
+            spec.getMap("url", NOT_NULL).get("paths")));
+      }
+      //TODO validation
+
+    }
+    List<Map> errs = CommandOperation.captureErrors(commandsCopy);
+    if (!errs.isEmpty()) {
+      rsp.add(CommandOperation.ERR_MSGS, errs);
+      SolrException exp = new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Errors in commands");
+      rsp.setException(exp);
+      throw exp;
+    }
+    return commandsCopy;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/api/V2RequestContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/V2RequestContext.java b/solr/core/src/java/org/apache/solr/api/V2RequestContext.java
new file mode 100644
index 0000000..02782b6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/V2RequestContext.java
@@ -0,0 +1,45 @@
+package org.apache.solr.api;
+
+/*
+ * 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.List;
+import java.util.Map;
+
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.CommandOperation;
+
+public interface V2RequestContext {
+
+  SolrQueryResponse getResponse();
+
+  CoreContainer getCoreContainer();
+
+  SolrQueryRequest getSolrRequest();
+
+  String getPath();
+
+
+  Map<String, String> getPathValues();
+
+  List<CommandOperation> getCommands(boolean validateInput);
+
+  String getHttpMethod();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 5d627c0..00091a7 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -43,15 +43,15 @@ 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.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.v2api.ApiBag.HANDLER_NAME;
+import static org.apache.solr.api.ApiBag.HANDLER_NAME;
 
 /**
  * This manages the lifecycle of a set of plugin of the same type .

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
index 841e161..a74153f 100644
--- a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
+++ b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
@@ -23,12 +23,9 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.v2api.*;
-import org.apache.solr.v2api.ApiBag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 0883100..b812661 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -39,14 +39,14 @@ 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.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.core.RequestParams.USEPARAM;
-import static org.apache.solr.v2api.ApiBag.wrapRequestHandler;
+import static org.apache.solr.api.ApiBag.wrapRequestHandler;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 4e06297..0fba1df 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -40,15 +40,15 @@ 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.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static 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;
+import static org.apache.solr.api.ApiBag.wrapRequestHandler;
 
 public class SchemaHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 7f6bd85..50e3fba 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -74,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.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -91,7 +91,7 @@ import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_CLASS;
 import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_NAME;
 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;
+import static org.apache.solr.api.ApiBag.wrapRequestHandler;
 
 public class SolrConfigHandler extends RequestHandlerBase implements ApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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
index bf43163..6109ceb 100644
--- a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandlerApi.java
@@ -24,9 +24,9 @@ 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;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.V2RequestContext;
 
 
 public class UpdateRequestHandlerApi extends UpdateRequestHandler  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
index 9a575ed..6112491 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
@@ -34,10 +34,10 @@ 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 org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
+import org.apache.solr.api.V2RequestContext;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
index 1c1f213..60f6bd2 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -29,7 +29,7 @@ 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 org.apache.solr.api.V2RequestContext;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 8e8279e..e6b7956 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.Api;
-import org.apache.solr.v2api.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiSupport;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 8fbc519..f234743 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
@@ -47,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.Api;
-import org.apache.solr.v2api.ApiSupport;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
index ae3eff0..493908d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@ -27,7 +27,7 @@ import java.util.Map;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.V2RequestContext;
+import org.apache.solr.api.V2RequestContext;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
 import static org.apache.solr.handler.admin.CoreAdminOperation.*;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 4186eb2..a40579c 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
@@ -30,10 +30,10 @@ 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.ApiSupport;
-import org.apache.solr.v2api.V2RequestContext;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
+import org.apache.solr.api.V2RequestContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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 8d7c42d..db3a0c6 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
@@ -41,10 +41,10 @@ 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.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
+import org.apache.solr.api.SpecProvider;
 import org.apache.zookeeper.KeeperException;
 
 public class SecurityConfHandler extends RequestHandlerBase implements ApiSupport {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/handler/admin/V2Command.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/V2Command.java b/solr/core/src/java/org/apache/solr/handler/admin/V2Command.java
index 5ddf996..cd66653 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/V2Command.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/V2Command.java
@@ -19,11 +19,10 @@ package org.apache.solr.handler.admin;
 
 
 import java.util.Collection;
-import java.util.Map;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.V2RequestContext;
+import org.apache.solr.api.V2RequestContext;
 
 public interface V2Command<T> {
   String getName();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index a041262..0f0ec7d 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -39,10 +39,9 @@ import org.apache.http.Header;
 import org.apache.http.auth.BasicUserPrincipal;
 import org.apache.http.message.BasicHeader;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.Lookup;
 import org.apache.solr.common.util.Map2;
 import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.SpecProvider;
+import org.apache.solr.api.SpecProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java b/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
index a3f1f95..eddfd2c 100644
--- a/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/RuleBasedAuthorizationPlugin.java
@@ -35,12 +35,11 @@ import java.util.function.Predicate;
 import com.google.common.collect.ImmutableSet;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CollectionParams;
-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.util.CommandOperation;
-import org.apache.solr.v2api.ApiBag;
-import org.apache.solr.v2api.SpecProvider;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.SpecProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
index 78d4e80..0244554 100644
--- a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
@@ -31,10 +31,9 @@ import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.codec.binary.Base64;
-import org.apache.solr.common.util.Lookup;
 import org.apache.solr.common.util.Map2;
 import org.apache.solr.util.CommandOperation;
-import org.apache.solr.v2api.ApiBag;
+import org.apache.solr.api.ApiBag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 0f7c4d1..7491514 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -25,11 +25,8 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import java.io.ByteArrayInputStream;
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
@@ -39,14 +36,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.Lookup;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.SolrCore;
@@ -55,7 +50,7 @@ import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
-import org.apache.solr.v2api.V2HttpCall;
+import org.apache.solr.api.V2HttpCall;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/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
deleted file mode 100644
index 48109ce..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/Api.java
+++ /dev/null
@@ -1,38 +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.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/34603800/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
deleted file mode 100644
index 239c197..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/ApiBag.java
+++ /dev/null
@@ -1,272 +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.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.google.common.collect.ImmutableSet;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.Map2;
-import org.apache.solr.core.PluginBag;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.request.SolrRequestHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.client.solrj.SolrRequest.SUPPORTED_METHODS;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.common.util.Map2.ENUM_OF;
-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<Api>> apis = new ConcurrentHashMap<>();
-
-  public static Map2 getResource(String name) {
-    InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(name);
-    if (is == null)
-      throw new RuntimeException("invalid API spec :" + name );
-    Map2 map1 = null;
-    try {
-      map1 = Map2.fromJSON(is);
-    } catch (Exception e) {
-      log.error("Error in JSON : " + name, e);
-      if (e instanceof RuntimeException) {
-        throw (RuntimeException) e;
-      }
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-    }
-    if (map1 == null) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Empty value for " + name);
-
-    return Map2.getDeepCopy(map1, 5, false);
-  }
-
-
-
-  public synchronized void register(Api api, Map<String, String> nameSubstitutes) {
-    try {
-      validateAndRegister(api, nameSubstitutes);
-    } catch (Exception e) {
-      log.error("Unable to register plugin:" + api.getClass().getName() + "with spec :" + api.getSpec(), e);
-      if (e instanceof RuntimeException) {
-        throw (RuntimeException) e;
-      } else {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      }
-
-    }
-  }
-
-  private void validateAndRegister(Api api, Map<String, String> nameSubstitutes) {
-    Map2 spec = api.getSpec();
-    Api introspect = getIntrospect(api);
-    List<String> methods = spec.getList("methods", ENUM_OF, SUPPORTED_METHODS);
-    for (String method : methods) {
-      PathTrie<Api> registry = apis.get(method);
-      if (registry == null) apis.put(method, registry = new PathTrie<>());
-      Map2 url = spec.getMap("url", NOT_NULL);
-      Map2 params = url.getMap("params", null);
-      if (params != null) {
-        for (Object o : params.keySet()) {
-          Map2 param = params.getMap(o.toString(), NOT_NULL);
-          param.get("type", ENUM_OF, KNOWN_TYPES);
-          param.get("description", NOT_NULL);
-        }
-      }
-      List<String> paths = url.getList("paths", NOT_NULL);
-      Map2 parts = url.getMap("parts", null);
-      if (parts != null) {
-        Set<String> wildCardNames = getWildCardNames(paths);
-        for (Object o : parts.keySet()) {
-          if (!wildCardNames.contains(o.toString()))
-            throw new RuntimeException("" + o + " is not a valid part name");
-          Map2 pathMeta = parts.getMap(o.toString(), NOT_NULL);
-          pathMeta.get("type", ENUM_OF, ImmutableSet.of("enum", "string", "int", "number", "boolean"));
-        }
-      }
-      verifyCommands(api.getSpec());
-      for (String path : paths) {
-        registry.insert(path, nameSubstitutes, api);
-        registry.insert(path + INTROSPECT, nameSubstitutes, introspect);
-      }
-    }
-  }
-
-  private Api getIntrospect(final Api baseApi) {
-    return new Api(Map2.EMPTY) {
-
-      @Override
-      public Map2 getSpec() {
-        return INTROSPECT_SPEC;
-      }
-
-      @Override
-      public void call(V2RequestContext ctx) {
-        String cmd = ctx.getSolrRequest().getParams().get("command");
-        Map2 result = null;
-        if (cmd == null) {
-          result = baseApi.getSpec();
-        } else {
-          Map2 specCopy = Map2.getDeepCopy(baseApi.getSpec(), 5, true);
-          Map2 commands = specCopy.getMap("commands", null);
-          if (commands != null) {
-            Map2 m = commands.getMap(cmd, null);
-            specCopy.put("commands", Collections.singletonMap(cmd, m));
-          }
-          result = specCopy;
-        }
-        List l = (List) ctx.getResponse().getValues().get("spec");
-        if (l == null) ctx.getResponse().getValues().add("spec", l = new ArrayList());
-        l.add(result);
-      }
-    };
-  }
-
-  private void verifyCommands(Map2 spec) {
-    Map2 commands = spec.getMap("commands", null);
-    if (commands == null) return;
-    //TODO do verify
-
-  }
-
-  private Set<String> getWildCardNames(List<String> paths) {
-    Set<String> wildCardNames = new HashSet<>();
-    for (String path : paths) {
-      List<String> p = PathTrie.getParts(path);
-      for (String s : p) {
-        String wildCard = PathTrie.wildCardName(s);
-        if (wildCard != null) wildCardNames.add(wildCard);
-      }
-    }
-    return wildCardNames;
-  }
-
-
-  public Api lookup(String path, String httpMethod, Map<String, String> parts) {
-    if (httpMethod == null) {
-      for (PathTrie<Api> trie : apis.values()) {
-        Api api = trie.lookup(path, parts);
-        if (api != null) return api;
-      }
-      return null;
-    } else {
-      PathTrie<Api> registry = apis.get(httpMethod);
-      if (registry == null) return null;
-      return registry.lookup(path, parts);
-    }
-  }
-
-  public static Map2 getSpec(String name) {
-    Map2 map = getResource(APISPEC_LOCATION + name + ".json");
-    Map2 result = map.getMap(name, NOT_NULL);
-    Map2 cmds = result.getMap("commands", null);
-    if (cmds != null) {
-      Map<String, Map2> comands2BReplaced = new Map2<>();
-      for (Object o : cmds.keySet()) {
-        Object val = cmds.get(o);
-        if (val instanceof String) {
-          String s = (String) val;
-          Map2 cmdSpec = getResource(APISPEC_LOCATION + s + ".json");
-          comands2BReplaced.put(o.toString(), cmdSpec);
-        }
-      }
-
-      if (!comands2BReplaced.isEmpty()) {
-        Map2 mapCopy = Map2.getDeepCopy(result, 4, true);
-        mapCopy.getMap("commands", NOT_NULL).putAll(comands2BReplaced);
-        result = Map2.getDeepCopy(mapCopy, 4, false);
-      }
-    }
-
-    return result;
-  }
-
-  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());
-      }
-
-      @Override
-      public Map2 getSpec() {
-        return specProvider != null ?
-            specProvider.getSpec() :
-            super.getSpec();
-      }
-    };
-  }
-
-  public static final String APISPEC_LOCATION = "apispec/";
-  public static final String INTROSPECT = "/_introspect";
-
-
-  public static final Map2 INTROSPECT_SPEC = new Map2(Collections.EMPTY_MAP);
-  public static final String HANDLER_NAME = "handlerName";
-  public static final Set<String> KNOWN_TYPES = ImmutableSet.of("string", "boolean", "list", "int", "double");
-
-  public PathTrie<Api> getRegistry(String method) {
-    return apis.get(method);
-  }
-
-  public <T> void registerLazy(PluginBag.PluginHolder<SolrRequestHandler> holder, PluginInfo info) {
-    String specName = info.attributes.get("spec");
-    if (specName == null) specName = "emptySpec";
-    Map2 spec = ApiBag.getSpec(specName);
-    register(new LazyLoadedApi(spec, holder), Collections.singletonMap(HANDLER_NAME, info.attributes.get(NAME)));
-  }
-
-  public static Map2 constructSpec(PluginInfo info) {
-    Object specObj = info == null ? null : info.attributes.get("spec");
-    if (specObj == null) specObj = "emptySpec";
-    if (specObj instanceof Map) {
-      Map map = (Map) specObj;
-      return Map2.getDeepCopy(map, 4, false);
-    } else {
-      return ApiBag.getSpec((String) specObj);
-    }
-  }
-
-  public static class LazyLoadedApi extends Api {
-
-    private final PluginBag.PluginHolder<SolrRequestHandler> holder;
-    private Api delegate;
-
-    protected LazyLoadedApi(Map2 spec, PluginBag.PluginHolder<SolrRequestHandler> lazyPluginHolder) {
-      super(spec);
-      this.holder = lazyPluginHolder;
-    }
-
-    @Override
-    public void call(V2RequestContext ctx) {
-      if (!holder.isLoaded()) {
-        delegate = wrapRequestHandler(holder.get(), null, null);
-      }
-      delegate.call(ctx);
-    }
-  }
-
-}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/v2api/PathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/PathTrie.java b/solr/core/src/java/org/apache/solr/v2api/PathTrie.java
deleted file mode 100644
index 5740ae5..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/PathTrie.java
+++ /dev/null
@@ -1,171 +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.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.solr.common.util.StrUtils;
-
-import static java.util.Collections.emptyList;
-
-public class PathTrie<T> {
-  Node root = new Node(emptyList(), null);
-
-
-  public void insert(String path, Map<String, String> replacements, T o) {
-    List<String> parts = getParts(path);
-    if (parts.isEmpty()) {
-      root.obj = o;
-      return;
-    }
-
-    for (int i = 0; i < parts.size(); i++) {
-      String part = parts.get(i);
-      if (part.charAt(0) == '$') {
-        String replacement = replacements.get(part.substring(1));
-        if (replacement == null) {
-          throw new RuntimeException(part + " is not provided");
-        }
-        replacement = replacement.charAt(0) == '/' ? replacement.substring(1) : replacement;
-        parts.set(i, replacement);
-      }
-    }
-
-    root.insert(parts, o);
-  }
-
-  public static List<String> getParts(String path) {
-    if (path == null || path.isEmpty()) return emptyList();
-    List<String> parts = StrUtils.splitSmart(path, '/');
-    while(true) {
-      if(parts.isEmpty()) break;
-      if ("".equals(parts.get(0))) parts.remove(0);
-      else break;
-    }
-    return parts;
-  }
-
-
-  public T lookup(String uri, Map<String, String> parts) {
-    return root.lookup(getParts(uri), 0, parts);
-  }
-
-  public T lookup(String path, Map<String, String> parts, Set<String> paths) {
-    return root.lookup(getParts(path), 0, parts, paths);
-  }
-
-  public static String wildCardName(String part) {
-    return part.startsWith("{") && part.endsWith("}") ?
-        part.substring(1, part.length() - 1) :
-        null;
-
-  }
-
-  class Node {
-    String name;
-    Map<String, Node> children;
-    T obj;
-    String varName;
-
-    Node(List<String> path, T o) {
-      if (path.isEmpty()) {
-        obj = o;
-        return;
-      }
-      String part = path.get(0);
-      varName = wildCardName(part);
-      name = part;
-      if (path.isEmpty()) obj = o;
-    }
-
-
-    private synchronized void insert(List<String> path, T o) {
-      String part = path.get(0);
-      Node matchedChild = null;
-      if (children == null) children = new ConcurrentHashMap<>();
-
-      String varName = wildCardName(part);
-      String key = varName == null ? part : "";
-
-      matchedChild = children.get(key);
-      if (matchedChild == null) {
-        children.put(key, matchedChild = new Node(path, o));
-      }
-      if (varName != null) {
-        if (!matchedChild.varName.equals(varName)) {
-          throw new RuntimeException("wildcard name must be " + matchedChild.varName);
-        }
-      }
-      path.remove(0);
-      if (!path.isEmpty()) {
-        matchedChild.insert(path, o);
-      } else {
-        matchedChild.obj = o;
-      }
-
-    }
-
-
-    void findValidChildren(String path, Set<String> availableSubPaths) {
-      if (availableSubPaths == null) return;
-      if (children != null) {
-        for (Node node : children.values()) {
-          if (node.obj != null) {
-            String s = path + "/" + node.name;
-            availableSubPaths.add(s);
-          }
-        }
-
-        for (Node node : children.values()) {
-          node.findValidChildren(path + "/" + node.name, availableSubPaths);
-        }
-      }
-    }
-
-
-    public T lookup(List<String> pieces, int i, Map<String, String> parts) {
-      return lookup(pieces, i, parts, null);
-
-    }
-
-    public T lookup(List<String> pieces, int i, Map<String, String> parts, Set<String> availableSubPaths) {
-      if (varName != null) parts.put(varName, pieces.get(i - 1));
-      if (pieces.size() < i + 1) {
-        findValidChildren("", availableSubPaths);
-        return obj;
-      }
-      String piece = pieces.get(i);
-      if (children == null) return null;
-      Node n = children.get(piece);
-      if (n == null && !reserved.contains(piece)) n = children.get("");
-      if (n == null) return null;
-      return n.lookup(pieces, i + 1, parts, availableSubPaths);
-    }
-  }
-
-  private Set<String> reserved = new HashSet<>();
-
-  {
-    reserved.add("_introspect");
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/v2api/SpecLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/SpecLoader.java b/solr/core/src/java/org/apache/solr/v2api/SpecLoader.java
deleted file mode 100644
index c27d17b..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/SpecLoader.java
+++ /dev/null
@@ -1,25 +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.Map2;
-
-public interface SpecLoader {
-  Map2 get(String name);
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34603800/solr/core/src/java/org/apache/solr/v2api/SpecProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/v2api/SpecProvider.java b/solr/core/src/java/org/apache/solr/v2api/SpecProvider.java
deleted file mode 100644
index 1cb8336..0000000
--- a/solr/core/src/java/org/apache/solr/v2api/SpecProvider.java
+++ /dev/null
@@ -1,25 +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 interface SpecProvider {
-  Map2 getSpec();
-}