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

[12/18] lucene-solr:jira/solr-5944: Updating branch by merging latest changes from master

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/api/ApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiSupport.java b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
new file mode 100644
index 0000000..ca1e866
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ApiSupport.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.api;
+
+import java.util.Collection;
+
+/**The interface that is implemented by a request handler to support the V2 end point
+ *
+ */
+public interface ApiSupport {
+
+  /**It is possible to support multiple v2 apis by a single requesthandler
+   *
+   * @return the list of v2 api implementations
+   */
+  Collection<Api> getApis();
+
+  /**Whether this should be made available at the regular legacy path
+   */
+  default Boolean registerV1() {
+    return Boolean.TRUE;
+  }
+
+  /**Whether this request handler must be made available at the /v2/ path
+   */
+  default Boolean registerV2() {
+    return Boolean.FALSE;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/api/SpecProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/SpecProvider.java b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
new file mode 100644
index 0000000..c373c99
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/SpecProvider.java
@@ -0,0 +1,25 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.api;
+import org.apache.solr.common.util.ValidatingJsonMap;
+
+/**A generic interface for any class that is capable of providing its specification as a json schema
+ */
+public interface SpecProvider {
+  ValidatingJsonMap getSpec();
+}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/api/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/package-info.java b/solr/core/src/java/org/apache/solr/api/package-info.java
new file mode 100644
index 0000000..c3574c7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Commonly used classes for Solr V2 API.
+ */
+package org.apache.solr.api;
+

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
index 4c5ae00..b891c92 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
@@ -28,12 +28,14 @@ import java.util.concurrent.TimeUnit;
 import org.apache.solr.common.NonExistentCoreException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -56,6 +58,11 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     final String collection = message.getStr(NAME);
     try {
+      // Remove the snapshots meta-data for this collection in ZK. Deleting actual index files
+      // should be taken care of as part of collection delete operation.
+      SolrZkClient zkClient = zkStateReader.getZkClient();
+      SolrSnapshotManager.cleanupCollectionLevelSnapshots(zkClient, collection);
+
       if (zkStateReader.getClusterState().getCollectionOrNull(collection) == null) {
         if (zkStateReader.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection, true)) {
           // if the collection is not in the clusterstate, but is listed in zk, do nothing, it will just

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 023e7b1..f7a8f33 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -77,6 +77,7 @@ import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
@@ -678,7 +679,16 @@ public class CoreContainer {
       }
       if (backgroundCloser != null) { // Doesn't seem right, but tests get in here without initializing the core.
         try {
-          backgroundCloser.join();
+          while (true) {
+            backgroundCloser.join(15000);
+            if (backgroundCloser.isAlive()) {
+              synchronized (solrCores.getModifyLock()) {
+                solrCores.getModifyLock().notifyAll(); // there is a race we have to protect against
+              }
+            } else {
+              break;
+            }
+          }
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           if (log.isDebugEnabled()) {
@@ -1183,15 +1193,15 @@ public class CoreContainer {
     SolrCore core = solrCores.remove(name);
     coresLocator.delete(this, cd);
 
-    // delete metrics specific to this core
-    metricManager.removeRegistry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name));
-
     if (core == null) {
       // transient core
       SolrCore.deleteUnloadedCore(cd, deleteDataDir, deleteInstanceDir);
       return;
     }
 
+    // delete metrics specific to this core
+    metricManager.removeRegistry(core.getCoreMetricManager().getRegistryName());
+
     if (zkSys.getZkController() != null) {
       // cancel recovery in cloud mode
       core.getSolrCoreState().cancelRecovery();
@@ -1217,6 +1227,9 @@ public class CoreContainer {
     SolrIdentifierValidator.validateCoreName(toName);
     try (SolrCore core = getCore(name)) {
       if (core != null) {
+        String oldRegistryName = core.getCoreMetricManager().getRegistryName();
+        String newRegistryName = SolrCoreMetricManager.createRegistryName(core.getCoreDescriptor().getCollectionName(), toName);
+        metricManager.swapRegistries(oldRegistryName, newRegistryName);
         registerCore(toName, core, true, false);
         SolrCore old = solrCores.remove(name);
         coresLocator.rename(this, old.getCoreDescriptor(), core.getCoreDescriptor());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 9dd0d8a..e4f0c5e 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -23,8 +23,10 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.store.Directory;
@@ -326,7 +328,7 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     return Collections.emptySet();
   }
 
-  public void cleanupOldIndexDirectories(final String dataDirPath, final String currentIndexDirPath) {
+  public void cleanupOldIndexDirectories(final String dataDirPath, final String currentIndexDirPath, boolean afterCoreReload) {
     File dataDir = new File(dataDirPath);
     if (!dataDir.isDirectory()) {
       log.debug("{} does not point to a valid data directory; skipping clean-up of old index directories.", dataDirPath);
@@ -347,9 +349,17 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     if (oldIndexDirs == null || oldIndexDirs.length == 0)
       return; // nothing to do (no log message needed)
 
-    log.info("Found {} old index directories to clean-up under {}", oldIndexDirs.length, dataDirPath);
-    for (File dir : oldIndexDirs) {
-
+    List<File> dirsList = Arrays.asList(oldIndexDirs);
+    Collections.sort(dirsList, Collections.reverseOrder());
+    
+    int i = 0;
+    if (afterCoreReload) {
+      log.info("Will not remove most recent old directory after reload {}", oldIndexDirs[0]);
+      i = 1;
+    }
+    log.info("Found {} old index directories to clean-up under {} afterReload={}", oldIndexDirs.length - i, dataDirPath, afterCoreReload);
+    for (; i < dirsList.size(); i++) {
+      File dir = dirsList.get(i);
       String dirToRmPath = dir.getAbsolutePath();
       try {
         if (deleteOldIndexDirectory(dirToRmPath)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
index cee7860..d5bcbb8 100644
--- a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
@@ -16,14 +16,18 @@
  */
 package org.apache.solr.core;
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 
 import org.apache.lucene.store.Directory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Directory provider for implementations that do not persist over reboots.
  * 
  */
 public abstract class EphemeralDirectoryFactory extends CachingDirectoryFactory {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   @Override
   public boolean exists(String path) throws IOException {
@@ -61,5 +65,9 @@ public abstract class EphemeralDirectoryFactory extends CachingDirectoryFactory
   public void remove(String path) throws IOException {
     // ram dir does not persist its dir anywhere
   }
+  
+  public void cleanupOldIndexDirectories(final String dataDirPath, final String currentIndexDirPath, boolean reload) {
+    // currently a no-op
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
index e1e3d6e..db953d3 100644
--- a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
@@ -21,8 +21,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.URLEncoder;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 import java.util.Locale;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
@@ -505,7 +508,7 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
   }
 
   @Override
-  public void cleanupOldIndexDirectories(final String dataDir, final String currentIndexDir) {
+  public void cleanupOldIndexDirectories(final String dataDir, final String currentIndexDir, boolean afterReload) {
 
     // Get the FileSystem object
     final Path dataDirPath = new Path(dataDir);
@@ -549,13 +552,27 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
     } catch (IOException ioExc) {
       LOG.error("Error checking for old index directories to clean-up.", ioExc);
     }
+    
+    List<Path> oldIndexPaths = new ArrayList<>(oldIndexDirs.length);
+    for (FileStatus ofs : oldIndexDirs) {
+      oldIndexPaths.add(ofs.getPath());
+    }
 
     if (oldIndexDirs == null || oldIndexDirs.length == 0)
       return; // nothing to clean-up
 
+    Collections.sort(oldIndexPaths, Collections.reverseOrder());
+    
     Set<String> livePaths = getLivePaths();
-    for (FileStatus oldDir : oldIndexDirs) {
-      Path oldDirPath = oldDir.getPath();
+    
+    int i = 0;
+    if (afterReload) {
+      LOG.info("Will not remove most recent old directory on reload {}", oldIndexDirs[0]);
+      i = 1;
+    }
+    LOG.info("Found {} old index directories to clean-up under {} afterReload={}", oldIndexDirs.length - i, dataDirPath, afterReload);
+    for (; i < oldIndexPaths.size(); i++) {
+      Path oldDirPath = oldIndexPaths.get(i);
       if (livePaths.contains(oldDirPath.toString())) {
         LOG.warn("Cannot delete directory {} because it is still being referenced in the cache.", oldDirPath);
       } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
index f441579..b567434 100644
--- a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
@@ -169,8 +169,8 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
   }
 
   @Override
-  public void cleanupOldIndexDirectories(String dataDirPath, String currentIndexDirPath) {
-    in.cleanupOldIndexDirectories(dataDirPath, currentIndexDirPath);
+  public void cleanupOldIndexDirectories(String dataDirPath, String currentIndexDirPath, boolean reload) {
+    in.cleanupOldIndexDirectories(dataDirPath, currentIndexDirPath, reload);
   }
 
   @Override

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 74238e7..9e01374 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -617,34 +617,37 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   public SolrCore reload(ConfigSet coreConfig) throws IOException {
-    solrCoreState.increfSolrCoreState();
-    final SolrCore currentCore;
-    if (!getNewIndexDir().equals(getIndexDir())) {
-      // the directory is changing, don't pass on state
-      currentCore = null;
-    } else {
-      currentCore = this;
-    }
+    // only one reload at a time
+    synchronized (getUpdateHandler().getSolrCoreState().getReloadLock()) {
+      solrCoreState.increfSolrCoreState();
+      final SolrCore currentCore;
+      if (!getNewIndexDir().equals(getIndexDir())) {
+        // the directory is changing, don't pass on state
+        currentCore = null;
+      } else {
+        currentCore = this;
+      }
 
-    boolean success = false;
-    SolrCore core = null;
-    try {
-      CoreDescriptor cd = new CoreDescriptor(coreDescriptor.getName(), coreDescriptor);
-      cd.loadExtraProperties(); //Reload the extra properties
-      core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
-          coreConfig.getIndexSchema(), coreConfig.getProperties(),
-          cd, updateHandler, solrDelPolicy, currentCore);
-      
-      // we open a new IndexWriter to pick up the latest config
-      core.getUpdateHandler().getSolrCoreState().newIndexWriter(core, false);
-      
-      core.getSearcher(true, false, null, true);
-      success = true;
-      return core;
-    } finally {
-      // close the new core on any errors that have occurred.
-      if (!success) {
-        IOUtils.closeQuietly(core);
+      boolean success = false;
+      SolrCore core = null;
+      try {
+        CoreDescriptor cd = new CoreDescriptor(coreDescriptor.getName(), coreDescriptor);
+        cd.loadExtraProperties(); //Reload the extra properties
+        core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
+            coreConfig.getIndexSchema(), coreConfig.getProperties(),
+            cd, updateHandler, solrDelPolicy, currentCore, true);
+        
+        // we open a new IndexWriter to pick up the latest config
+        core.getUpdateHandler().getSolrCoreState().newIndexWriter(core, false);
+        
+        core.getSearcher(true, false, null, true);
+        success = true;
+        return core;
+      } finally {
+        // close the new core on any errors that have occurred.
+        if (!success) {
+          IOUtils.closeQuietly(core);
+        }
       }
     }
   }
@@ -686,7 +689,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     }
   }
 
-  void initIndex(boolean reload) throws IOException {
+  void initIndex(boolean passOnPreviousState, boolean reload) throws IOException {
 
     String indexDir = getNewIndexDir();
     boolean indexExists = getDirectoryFactory().exists(indexDir);
@@ -697,7 +700,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     initIndexReaderFactory();
 
-    if (indexExists && firstTime && !reload) {
+    if (indexExists && firstTime && !passOnPreviousState) {
       final String lockType = getSolrConfig().indexConfig.lockType;
       Directory dir = directoryFactory.get(indexDir, DirContext.DEFAULT, lockType);
       try {
@@ -726,7 +729,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     }
 
 
-    cleanupOldIndexDirectories();
+    cleanupOldIndexDirectories(reload);
   }
 
 
@@ -823,7 +826,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
   public SolrCore(CoreDescriptor cd, ConfigSet coreConfig) {
     this(cd.getName(), null, coreConfig.getSolrConfig(), coreConfig.getIndexSchema(), coreConfig.getProperties(),
-        cd, null, null, null);
+        cd, null, null, null, false);
   }
 
   
@@ -843,7 +846,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   public SolrCore(String name, String dataDir, SolrConfig config,
       IndexSchema schema, NamedList configSetProperties,
       CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
-      IndexDeletionPolicyWrapper delPolicy, SolrCore prev) {
+      IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
     
     assert ObjectReleaseTracker.track(searcherExecutor); // ensure that in unclean shutdown tests we still close this
     
@@ -905,7 +908,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       this.codec = initCodec(solrConfig, this.schema);
 
       memClassLoader = new MemClassLoader(PluginBag.RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(PluginBag.RuntimeLib.class.getName())), getResourceLoader());
-      initIndex(prev != null);
+      initIndex(prev != null, reload);
 
       initWriters();
       qParserPlugins.init(createInstances(QParserPlugin.standardPlugins), this);
@@ -1533,7 +1536,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     }
 
     if (coreStateClosed) {
-
+      try {
+        cleanupOldIndexDirectories(false);
+      } catch (Exception e) {
+        SolrException.log(log, e);
+      }
+      
       try {
         directoryFactory.close();
       } catch (Throwable e) {
@@ -1542,7 +1550,6 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
           throw (Error) e;
         }
       }
-
     }
 
     if( closeHooks != null ) {
@@ -1557,6 +1564,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
          }
       }
     }
+    
     assert ObjectReleaseTracker.release(this);
   }
 
@@ -2952,16 +2960,16 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     return false;
   }
 
-  public void cleanupOldIndexDirectories() {
+  public void cleanupOldIndexDirectories(boolean reload) {
     final DirectoryFactory myDirFactory = getDirectoryFactory();
     final String myDataDir = getDataDir();
-    final String myIndexDir = getIndexDir();
+    final String myIndexDir = getNewIndexDir(); // ensure the latest replicated index is protected 
     final String coreName = getName();
     if (myDirFactory != null && myDataDir != null && myIndexDir != null) {
       Thread cleanupThread = new Thread(() -> {
         log.debug("Looking for old index directories to cleanup for core {} in {}", coreName, myDataDir);
         try {
-          myDirFactory.cleanupOldIndexDirectories(myDataDir, myIndexDir);
+          myDirFactory.cleanupOldIndexDirectories(myDataDir, myIndexDir, reload);
         } catch (Exception exc) {
           log.error("Failed to cleanup old index directories for core "+coreName, exc);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/SolrCores.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java
index 2bcea17..b25e9bb 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCores.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java
@@ -239,7 +239,9 @@ class SolrCores {
       }
       cores.put(n0, c1);
       cores.put(n1, c0);
-
+      container.getMetricManager().swapRegistries(
+          c0.getCoreMetricManager().getRegistryName(),
+          c1.getCoreMetricManager().getRegistryName());
       c0.setName(n1);
       c1.setName(n0);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java
index cb1c52c..935ef63 100644
--- a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java
+++ b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java
@@ -295,6 +295,7 @@ public class SolrSnapshotsTool implements Closeable {
       Optional<String> asyncReqId) {
     try {
       CollectionAdminRequest.Backup backup = new CollectionAdminRequest.Backup(collectionName, snapshotName);
+      backup.setCommitName(snapshotName);
       backup.setIndexBackupStrategy(CollectionAdminParams.COPY_FILES_STRATEGY);
       backup.setLocation(destPath);
       if (backupRepo.isPresent()) {
@@ -350,29 +351,29 @@ public class SolrSnapshotsTool implements Closeable {
 
     if (cmd.hasOption(CREATE) || cmd.hasOption(DELETE) || cmd.hasOption(LIST) || cmd.hasOption(DESCRIBE)
         || cmd.hasOption(PREPARE_FOR_EXPORT) || cmd.hasOption(EXPORT_SNAPSHOT)) {
-      try (SolrSnapshotsTool tool = new SolrSnapshotsTool(cmd.getOptionValue(SOLR_ZK_ENSEMBLE))) {
+      try (SolrSnapshotsTool tool = new SolrSnapshotsTool(requiredArg(options, cmd, SOLR_ZK_ENSEMBLE))) {
         if (cmd.hasOption(CREATE)) {
           String snapshotName = cmd.getOptionValue(CREATE);
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           tool.createSnapshot(collectionName, snapshotName);
 
         } else if (cmd.hasOption(DELETE)) {
           String snapshotName = cmd.getOptionValue(DELETE);
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           tool.deleteSnapshot(collectionName, snapshotName);
 
         } else if (cmd.hasOption(LIST)) {
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           tool.listSnapshots(collectionName);
 
         } else if (cmd.hasOption(DESCRIBE)) {
           String snapshotName = cmd.getOptionValue(DESCRIBE);
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           tool.describeSnapshot(collectionName, snapshotName);
 
         } else if (cmd.hasOption(PREPARE_FOR_EXPORT)) {
           String snapshotName = cmd.getOptionValue(PREPARE_FOR_EXPORT);
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           String localFsDir = requiredArg(options, cmd, TEMP_DIR);
           String hdfsOpDir = requiredArg(options, cmd, DEST_DIR);
           Optional<String> pathPrefix = Optional.ofNullable(cmd.getOptionValue(HDFS_PATH_PREFIX));
@@ -391,7 +392,7 @@ public class SolrSnapshotsTool implements Closeable {
 
         }  else if (cmd.hasOption(EXPORT_SNAPSHOT)) {
           String snapshotName = cmd.getOptionValue(EXPORT_SNAPSHOT);
-          String collectionName = cmd.getOptionValue(COLLECTION);
+          String collectionName = requiredArg(options, cmd, COLLECTION);
           String destDir = requiredArg(options, cmd, DEST_DIR);
           Optional<String> backupRepo = Optional.ofNullable(cmd.getOptionValue(BACKUP_REPO_NAME));
           Optional<String> asyncReqId = Optional.ofNullable(cmd.getOptionValue(ASYNC_REQ_ID));

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 968af61..8634aee 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -54,7 +54,6 @@ import java.util.zip.Adler32;
 import java.util.zip.Checksum;
 import java.util.zip.InflaterInputStream;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.IndexCommit;
@@ -75,6 +74,7 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.FastInputStream;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.DirectoryFactory;
@@ -182,7 +182,13 @@ public class IndexFetcher {
     useInternalCompression = INTERNAL.equals(compress);
     useExternalCompression = EXTERNAL.equals(compress);
     connTimeout = getParameter(initArgs, HttpClientUtil.PROP_CONNECTION_TIMEOUT, 30000, null);
-    soTimeout = getParameter(initArgs, HttpClientUtil.PROP_SO_TIMEOUT, 120000, null);
+    
+    // allow a master override for tests - you specify this in /replication slave section of solrconfig and some 
+    // test don't want to define this
+    soTimeout = Integer.getInteger("solr.indexfetcher.sotimeout", -1);
+    if (soTimeout == -1) {
+      soTimeout = getParameter(initArgs, HttpClientUtil.PROP_SO_TIMEOUT, 120000, null);
+    }
 
     String httpBasicAuthUser = (String) initArgs.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
     String httpBasicAuthPassword = (String) initArgs.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
@@ -325,6 +331,7 @@ public class IndexFetcher {
       }
 
       LOG.info("Slave's generation: " + commit.getGeneration());
+      LOG.info("Slave's version: " + IndexDeletionPolicyWrapper.getCommitTimestamp(commit));
 
       if (latestVersion == 0L) {
         if (forceReplication && commit.getGeneration() != 0) {
@@ -459,7 +466,7 @@ public class IndexFetcher {
             downloadConfFiles(confFilesToDownload, latestGeneration);
             if (isFullCopyNeeded) {
               successfulInstall = solrCore.modifyIndexProps(tmpIdxDirName);
-              deleteTmpIdxDir = false;
+              if (successfulInstall) deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
             }
@@ -487,7 +494,7 @@ public class IndexFetcher {
             terminateAndWaitFsyncService();
             if (isFullCopyNeeded) {
               successfulInstall = solrCore.modifyIndexProps(tmpIdxDirName);
-              deleteTmpIdxDir = false;
+              if (successfulInstall) deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
             }
@@ -565,7 +572,8 @@ public class IndexFetcher {
         try {
           logReplicationTimeAndConfFiles(null, successfulInstall);
         } catch (Exception e) {
-          LOG.error("caught", e);
+          // this can happen on shutdown, a fetch may be running in a thread after DirectoryFactory is closed
+          LOG.warn("Could not log failed replication details", e);
         }
       }
 
@@ -583,25 +591,32 @@ public class IndexFetcher {
       stop = false;
       fsyncException = null;
     } finally {
-      if (deleteTmpIdxDir && tmpIndexDir != null) {
-        try {
+      // order below is important
+      try {
+        if (tmpIndexDir != null && deleteTmpIdxDir) {
           core.getDirectoryFactory().doneWithDirectory(tmpIndexDir);
           core.getDirectoryFactory().remove(tmpIndexDir);
-        } catch (IOException e) {
-          SolrException.log(LOG, "Error removing directory " + tmpIndexDir, e);
         }
-      }
-
-      if (tmpIndexDir != null) {
-        core.getDirectoryFactory().release(tmpIndexDir);
-      }
-
-      if (indexDir != null) {
-        core.getDirectoryFactory().release(indexDir);
-      }
-
-      if (tmpTlogDir != null) {
-        delTree(tmpTlogDir);
+      } catch (Exception e) {
+        SolrException.log(LOG, e);
+      } finally {
+        try {
+          if (tmpIndexDir != null) core.getDirectoryFactory().release(tmpIndexDir);
+        } catch (Exception e) {
+          SolrException.log(LOG, e);
+        }
+        try {
+          if (indexDir != null) {
+            core.getDirectoryFactory().release(indexDir);
+          }
+        } catch (Exception e) {
+          SolrException.log(LOG, e);
+        }
+        try {
+          if (tmpTlogDir != null) delTree(tmpTlogDir);
+        } catch (Exception e) {
+          SolrException.log(LOG, e);
+        }
       }
     }
   }
@@ -863,8 +878,9 @@ public class IndexFetcher {
       String filename = (String) file.get(NAME);
       long size = (Long) file.get(SIZE);
       CompareResult compareResult = compareFile(indexDir, filename, size, (Long) file.get(CHECKSUM));
-      if (!compareResult.equal || downloadCompleteIndex
-          || filesToAlwaysDownloadIfNoChecksums(filename, size, compareResult)) {
+      boolean alwaysDownload = filesToAlwaysDownloadIfNoChecksums(filename, size, compareResult);
+      LOG.debug("Downloading file={} size={} checksum={} alwaysDownload={}", filename, size, file.get(CHECKSUM), alwaysDownload);
+      if (!compareResult.equal || downloadCompleteIndex || alwaysDownload) {
         dirFileFetcher = new DirectoryFileFetcher(tmpIndexDir, file,
             (String) file.get(NAME), FILE, latestGeneration);
         currentFile = file;
@@ -915,7 +931,7 @@ public class IndexFetcher {
             compareResult.equal = true;
             return compareResult;
           } else {
-            LOG.warn(
+            LOG.info(
                 "File {} did not match. expected length is {} and actual length is {}", filename, backupIndexFileLen, indexFileLen);
             compareResult.equal = false;
             return compareResult;
@@ -1349,15 +1365,15 @@ public class IndexFetcher {
   private class FileFetcher {
     private final FileInterface file;
     private boolean includeChecksum = true;
-    private String fileName;
-    private String saveAs;
-    private String solrParamOutput;
-    private Long indexGen;
+    private final String fileName;
+    private final String saveAs;
+    private final String solrParamOutput;
+    private final Long indexGen;
 
-    private long size;
+    private final long size;
     private long bytesDownloaded = 0;
     private byte[] buf = new byte[1024 * 1024];
-    private Checksum checksum;
+    private final Checksum checksum;
     private int errorCount = 0;
     private boolean aborted = false;
 
@@ -1369,8 +1385,11 @@ public class IndexFetcher {
       this.solrParamOutput = solrParamOutput;
       this.saveAs = saveAs;
       indexGen = latestGen;
-      if (includeChecksum)
+      if (includeChecksum) {
         checksum = new Adler32();
+      } else {
+        checksum = null;
+      }
     }
 
     public long getBytesDownloaded() {
@@ -1381,6 +1400,21 @@ public class IndexFetcher {
      * The main method which downloads file
      */
     public void fetchFile() throws Exception {
+      bytesDownloaded = 0;
+      try {
+        fetch();
+      } catch(Exception e) {
+        if (!aborted) {
+          SolrException.log(IndexFetcher.LOG, "Error fetching file, doing one retry...", e);
+          // one retry
+          fetch();
+        } else {
+          throw e;
+        }
+      }
+    }
+    
+    private void fetch() throws Exception {
       try {
         while (true) {
           final FastInputStream is = getStream();
@@ -1569,7 +1603,7 @@ public class IndexFetcher {
         return new FastInputStream(is);
       } catch (Exception e) {
         //close stream on error
-        IOUtils.closeQuietly(is);
+        org.apache.commons.io.IOUtils.closeQuietly(is);
         throw new IOException("Could not download file '" + fileName + "'", e);
       }
     }

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index b875144..cdbadc4 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -29,6 +29,8 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -1418,9 +1420,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       params = solrParams;
       delPolicy = core.getDeletionPolicy();
 
-      fileName = params.get(FILE);
-      cfileName = params.get(CONF_FILE_SHORT);
-      tlogFileName = params.get(TLOG_FILE);
+      fileName = validateFilenameOrError(params.get(FILE));
+      cfileName = validateFilenameOrError(params.get(CONF_FILE_SHORT));
+      tlogFileName = validateFilenameOrError(params.get(TLOG_FILE));
+      
       sOffset = params.get(OFFSET);
       sLen = params.get(LEN);
       compress = params.get(COMPRESSION);
@@ -1434,6 +1437,22 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       rateLimiter = new RateLimiter.SimpleRateLimiter(maxWriteMBPerSec);
     }
 
+    // Throw exception on directory traversal attempts 
+    protected String validateFilenameOrError(String filename) {
+      if (filename != null) {
+        Path filePath = Paths.get(filename);
+        filePath.forEach(subpath -> {
+          if ("..".equals(subpath.toString())) {
+            throw new SolrException(ErrorCode.FORBIDDEN, "File name cannot contain ..");
+          }
+        });
+        if (filePath.isAbsolute()) {
+          throw new SolrException(ErrorCode.FORBIDDEN, "File name must be relative");
+        }
+        return filename;
+      } else return null;
+    }
+
     protected void initWrite() throws IOException {
       if (sOffset != null) offset = Long.parseLong(sOffset);
       if (sLen != null) len = Integer.parseInt(sLen);

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

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

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