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

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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
new file mode 100644
index 0000000..6037bcd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandlerApi.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.CREATE_OP;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.DELETE_OP;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.ConfigSetOperation.LIST_OP;
+
+public class ConfigSetsHandlerApi extends BaseHandlerApiSupport {
+
+  final ConfigSetsHandler configSetHandler;
+
+  public ConfigSetsHandlerApi(ConfigSetsHandler configSetHandler) {
+    this.configSetHandler = configSetHandler;
+  }
+
+
+  @Override
+  protected List<ApiCommand> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(EndPoint.values());
+  }
+
+  enum Cmd implements ApiCommand {
+    LIST(EndPoint.LIST_CONFIG, LIST_OP, GET),
+    CREATE(EndPoint.CONFIG_COMMANDS, CREATE_OP, POST, "create"),
+    DEL(EndPoint.CONFIG_DEL, DELETE_OP, DELETE)
+    ;
+    private final EndPoint endPoint;
+    private final ConfigSetOperation op;
+    private final SolrRequest.METHOD method;
+    private final String cmdName;
+
+    Cmd(EndPoint endPoint, ConfigSetOperation op, SolrRequest.METHOD method) {
+      this(endPoint, op, method, null);
+    }
+
+    Cmd(EndPoint endPoint, ConfigSetOperation op, SolrRequest.METHOD method, String cmdName) {
+      this.cmdName = cmdName;
+      this.endPoint = endPoint;
+      this.op = op;
+      this.method = method;
+    }
+
+    @Override
+    public String getName() {
+      return cmdName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+    @Override
+    public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
+      ((ConfigSetsHandlerApi) apiHandler).configSetHandler.invokeAction(req, rsp, op.action);
+    }
+
+  }
+  enum EndPoint implements V2EndPoint {
+    LIST_CONFIG("cluster.configs"),
+    CONFIG_COMMANDS("cluster.configs.Commands"),
+    CONFIG_DEL("cluster.configs.delete");
+
+    public final String spec;
+
+    EndPoint(String spec) {
+      this.spec = spec;
+    }
+
+    @Override
+    public String getSpecName() {
+      return spec;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index a415d8a..275ec18 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -18,6 +18,7 @@ package org.apache.solr.handler.admin;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -28,6 +29,7 @@ import java.util.concurrent.ExecutorService;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.StringUtils;
+import org.apache.solr.api.Api;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -66,6 +68,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   protected final Map<String, Map<String, TaskObject>> requestStatusMap;
+  private final CoreAdminHandlerApi coreAdminHandlerApi;
 
   protected ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
       new DefaultSolrThreadFactory("parallelCoreAdminExecutor"));
@@ -88,6 +91,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     requestStatusMap = Collections.unmodifiableMap(map);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -103,6 +107,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     map.put(COMPLETED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     map.put(FAILED, Collections.synchronizedMap(new LinkedHashMap<String, TaskObject>()));
     requestStatusMap = Collections.unmodifiableMap(map);
+    coreAdminHandlerApi = new CoreAdminHandlerApi(this);
   }
 
 
@@ -119,6 +124,10 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     parallelExecutor = MetricUtils.instrumentedExecutorService(parallelExecutor, manager.registry(registryName),
         SolrMetricManager.mkName("parallelCoreAdminExecutor", getCategory().name(),scope, "threadPool"));
   }
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
 
   /**
    * The instance of CoreContainer this handler handles. This should be the CoreContainer instance that created this
@@ -381,6 +390,11 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
 
   }
 
+  @Override
+  public Collection<Api> getApis() {
+    return coreAdminHandlerApi.getApis();
+  }
+
   static {
     for (CoreAdminOperation op : CoreAdminOperation.values())
       opMap.put(op.action.toString().toLowerCase(Locale.ROOT), op);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
new file mode 100644
index 0000000..9d256e6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_COMMANDS;
+import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_STATUS;
+import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEAPIS;
+import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEINVOKE;
+import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.PER_CORE_COMMANDS;
+import static org.apache.solr.handler.admin.CoreAdminOperation.CREATE_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.FORCEPREPAREFORLEADERSHIP_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.INVOKE_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.MERGEINDEXES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.OVERSEEROP_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.PREPRECOVERY_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REJOINLEADERELECTION_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.RELOAD_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.RENAME_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTAPPLYUPDATES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTBUFFERUPDATES_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTRECOVERY_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTSTATUS_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.REQUESTSYNCSHARD_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.SPLIT_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.STATUS_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.SWAP_OP;
+import static org.apache.solr.handler.admin.CoreAdminOperation.UNLOAD_OP;
+
+public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
+  private final CoreAdminHandler handler;
+
+  public CoreAdminHandlerApi(CoreAdminHandler handler) {
+    this.handler = handler;
+  }
+
+  enum Cmd implements ApiCommand {
+    CREATE(CORES_COMMANDS, POST, CREATE_OP, null, ImmutableMap.of("config", "configSet")),
+    UNLOAD(PER_CORE_COMMANDS, POST, UNLOAD_OP, null, null),
+    RELOAD(PER_CORE_COMMANDS, POST, RELOAD_OP, null, null),
+    STATUS(CORES_STATUS, GET, STATUS_OP),
+    SWAP(PER_CORE_COMMANDS, POST, SWAP_OP, null, ImmutableMap.of("other", "with")),
+    RENAME(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
+    MERGEINDEXES(PER_CORE_COMMANDS, POST, MERGEINDEXES_OP, "merge-indexes", null),
+    SPLIT(PER_CORE_COMMANDS, POST, SPLIT_OP, null, ImmutableMap.of("split.key", "splitKey")),
+    PREPRECOVERY(PER_CORE_COMMANDS, POST, PREPRECOVERY_OP, "prep-recovery", null),
+    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
+    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, REQUESTSYNCSHARD_OP, "request-sync-shard", null),
+    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, REQUESTBUFFERUPDATES_OP, "request-buffer-updates", null),
+    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, REQUESTAPPLYUPDATES_OP, "request-apply-updates", null),
+    REQUESTSTATUS(PER_CORE_COMMANDS, POST, REQUESTSTATUS_OP, null, null),
+    OVERSEEROP(NODEAPIS, POST, OVERSEEROP_OP, "overseer-op", null),
+    REJOINLEADERELECTION(NODEAPIS, POST, REJOINLEADERELECTION_OP, "rejoin-leader-election", null),
+    INVOKE(NODEINVOKE, GET, INVOKE_OP, null, null),
+    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, FORCEPREPAREFORLEADERSHIP_OP, "force-prepare-for-leadership", null);
+
+    public final String commandName;
+    public final BaseHandlerApiSupport.V2EndPoint endPoint;
+    public final SolrRequest.METHOD method;
+    public final Map<String, String> paramstoAttr;
+    final CoreAdminOperation target;
+
+
+    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target) {
+      this.endPoint = endPoint;
+      this.method = method;
+      this.target = target;
+      commandName = null;
+      paramstoAttr = Collections.EMPTY_MAP;
+
+    }
+
+
+    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target, String commandName,
+        Map<String, String> paramstoAttr) {
+      this.commandName = commandName == null ? target.action.toString().toLowerCase(Locale.ROOT) : commandName;
+      this.endPoint = endPoint;
+      this.method = method;
+      this.target = target;
+      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : paramstoAttr;
+    }
+
+    @Override
+    public String getName() {
+      return commandName;
+    }
+
+    @Override
+    public SolrRequest.METHOD getHttpMethod() {
+      return method;
+    }
+
+    @Override
+    public V2EndPoint getEndPoint() {
+      return endPoint;
+    }
+
+    @Override
+    public String getParamSubstitute(String param) {
+      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+    }
+
+    @Override
+    public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
+      target.execute(new CoreAdminHandler.CallInfo(((CoreAdminHandlerApi) apiHandler).handler,
+          req,
+          rsp,
+          target));
+
+    }
+
+  }
+
+
+
+  enum EndPoint implements BaseHandlerApiSupport.V2EndPoint {
+    CORES_STATUS("cores.Status"),
+    CORES_COMMANDS("cores.Commands"),
+    PER_CORE_COMMANDS("cores.core.Commands"),
+    NODEINVOKE("node.invoke"),
+    NODEAPIS("node.Commands")
+    ;
+
+    final String specName;
+
+    EndPoint(String specName) {
+      this.specName = specName;
+    }
+
+    @Override
+    public String getSpecName() {
+      return specName;
+    }
+  }
+
+
+  @Override
+  protected List<ApiCommand> getCommands() {
+    return Arrays.asList(Cmd.values());
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(EndPoint.values());
+  }
+
+
+}

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71abe130/solr/core/src/java/org/apache/solr/util/PathTrie.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/PathTrie.java b/solr/core/src/java/org/apache/solr/util/PathTrie.java
new file mode 100644
index 0000000..ceaa5de
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/PathTrie.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.common.util.StrUtils;
+
+import static java.util.Collections.emptyList;
+
+/**A utility class to efficiently parse/store/lookup hierarchical paths which are templatized
+ * like /collections/{collection}/shards/{shard}/{replica}
+ */
+public class PathTrie<T> {
+  private final Set<String> reserved = new HashSet<>();
+  Node root = new Node(emptyList(), null);
+
+  public PathTrie() { }
+
+  public PathTrie(Set<String> reserved) {
+    this.reserved.addAll(reserved);
+  }
+
+
+
+  public void insert(String path, Map<String, String> replacements, T o) {
+    List<String> parts = getPathSegments(path);
+    insert(parts,replacements, o);
+  }
+
+  public void insert(List<String> parts, Map<String, String> replacements, T o) {
+    if (parts.isEmpty()) {
+      root.obj = o;
+      return;
+    }
+
+    for (int i = 0; i < parts.size(); i++) {
+      String part = parts.get(i);
+      if (part.charAt(0) == '$') {
+        String replacement = replacements.get(part.substring(1));
+        if (replacement == null) {
+          throw new RuntimeException(part + " is not provided");
+        }
+        replacement = replacement.charAt(0) == '/' ? replacement.substring(1) : replacement;
+        parts.set(i, replacement);
+      }
+    }
+
+    root.insert(parts, o);
+  }
+
+  // /a/b/c will be returned as ["a","b","c"]
+  public static List<String> getPathSegments(String path) {
+    if (path == null || path.isEmpty()) return emptyList();
+    List<String> parts = new ArrayList<String>() {
+      @Override
+      public boolean add(String s) {
+        if (s == null || s.isEmpty()) return false;
+        return super.add(s);
+      }
+    };
+    StrUtils.splitSmart(path, '/', parts);
+    return parts;
+  }
+
+
+  public T lookup(String path, Map<String, String> templateValues) {
+    return root.lookup(getPathSegments(path), 0, templateValues);
+  }
+
+  public T lookup(List<String> path, Map<String, String> templateValues) {
+    return root.lookup(path, 0, templateValues);
+  }
+
+  public T lookup(String path, Map<String, String> templateValues, Set<String> paths) {
+    return root.lookup(getPathSegments(path), 0, templateValues, paths);
+  }
+
+  public static String templateName(String templateStr) {
+    return templateStr.startsWith("{") && templateStr.endsWith("}") ?
+        templateStr.substring(1, templateStr.length() - 1) :
+        null;
+
+  }
+
+  class Node {
+    String name;
+    Map<String, Node> children;
+    T obj;
+    String templateName;
+
+    Node(List<String> path, T o) {
+      if (path.isEmpty()) {
+        obj = o;
+        return;
+      }
+      String part = path.get(0);
+      templateName = templateName(part);
+      name = part;
+      if (path.isEmpty()) obj = o;
+    }
+
+
+    private synchronized void insert(List<String> path, T o) {
+      String part = path.get(0);
+      Node matchedChild = null;
+      if (children == null) children = new ConcurrentHashMap<>();
+
+      String varName = templateName(part);
+      String key = varName == null ? part : "";
+
+      matchedChild = children.get(key);
+      if (matchedChild == null) {
+        children.put(key, matchedChild = new Node(path, o));
+      }
+      if (varName != null) {
+        if (!matchedChild.templateName.equals(varName)) {
+          throw new RuntimeException("wildcard name must be " + matchedChild.templateName);
+        }
+      }
+      path.remove(0);
+      if (!path.isEmpty()) {
+        matchedChild.insert(path, o);
+      } else {
+        matchedChild.obj = o;
+      }
+
+    }
+
+
+    void findAvailableChildren(String path, Set<String> availableSubPaths) {
+      if (availableSubPaths == null) return;
+      if (children != null) {
+        for (Node node : children.values()) {
+          if (node.obj != null) {
+            String s = path + "/" + node.name;
+            availableSubPaths.add(s);
+          }
+        }
+
+        for (Node node : children.values()) {
+          node.findAvailableChildren(path + "/" + node.name, availableSubPaths);
+        }
+      }
+    }
+
+
+    public T lookup(List<String> pieces, int i, Map<String, String> templateValues) {
+      return lookup(pieces, i, templateValues, null);
+
+    }
+
+    /**
+     *
+     * @param pathSegments pieces in the url /a/b/c has pieces as 'a' , 'b' , 'c'
+     * @param index current index of the pieces that we are looking at in /a/b/c 0='a' and 1='b'
+     * @param templateVariables The mapping of template variable to its value
+     * @param availableSubPaths If not null , available sub paths will be returned in this set
+     */
+    public T lookup(List<String> pathSegments, int index, Map<String, String> templateVariables, Set<String> availableSubPaths) {
+      if (templateName != null) templateVariables.put(templateName, pathSegments.get(index - 1));
+      if (pathSegments.size() < index + 1) {
+        findAvailableChildren("", availableSubPaths);
+        return obj;
+      }
+      String piece = pathSegments.get(index);
+      if (children == null) return null;
+      Node n = children.get(piece);
+      if (n == null && !reserved.contains(piece)) n = children.get("");
+      if (n == null) return null;
+      return n.lookup(pathSegments, index + 1, templateVariables, availableSubPaths);
+    }
+  }
+
+}

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

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