You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/05/30 09:30:11 UTC

[3/3] lucene-solr:feature/autoscaling_solr7: SOLR-9735: Initial port of autoscaling work for Solr 7

SOLR-9735: Initial port of autoscaling work for Solr 7


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

Branch: refs/heads/feature/autoscaling_solr7
Commit: e5d8ed397ab8db3268e1de86ca5ee5fe53dc04cc
Parents: cb97ad7
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue May 30 14:59:25 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue May 30 14:59:25 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  48 +-
 .../org/apache/solr/cloud/ZkController.java     |   1 +
 .../cloud/autoscaling/AutoScalingHandler.java   | 319 ++++++++++++
 .../solr/cloud/autoscaling/package-info.java    |  21 +
 .../solr/cloud/rule/ServerSnitchContext.java    |   7 +-
 .../org/apache/solr/core/CoreContainer.java     |  30 +-
 .../apache/solr/core/SolrResourceLoader.java    |   3 +-
 .../org/apache/solr/handler/BlobHandler.java    |   2 +-
 .../solr/handler/RequestHandlerUtils.java       |  13 +
 .../org/apache/solr/handler/SchemaHandler.java  |   2 +-
 .../apache/solr/handler/SolrConfigHandler.java  |  11 +-
 .../solr/handler/admin/SecurityConfHandler.java |   4 +-
 .../solr/response/TextResponseWriter.java       |  13 +-
 .../solr/security/PermissionNameProvider.java   |   2 +
 .../resources/apispec/autoscaling.Commands.json |  47 ++
 .../autoscaling/AutoScalingHandlerTest.java     | 326 ++++++++++++
 .../solr/cloud/autoscaling/TestPolicyCloud.java |  98 ++++
 .../solr/client/solrj/impl/CloudSolrClient.java |  15 +-
 .../solrj/impl/SolrClientDataProvider.java      | 262 ++++++++++
 .../impl/ZkClientClusterStateProvider.java      |  11 +-
 .../solr/client/solrj/request/V2Request.java    |   4 +-
 .../cloud/autoscaling/AddReplicaSuggester.java  |  69 +++
 .../org/apache/solr/cloud/autoscaling/Cell.java |  57 ++
 .../apache/solr/cloud/autoscaling/Clause.java   | 335 ++++++++++++
 .../cloud/autoscaling/ClusterDataProvider.java  |  52 ++
 .../cloud/autoscaling/MoveReplicaSuggester.java |  83 +++
 .../apache/solr/cloud/autoscaling/Operand.java  | 155 ++++++
 .../apache/solr/cloud/autoscaling/Policy.java   | 508 ++++++++++++++++++
 .../solr/cloud/autoscaling/PolicyHelper.java    |  98 ++++
 .../solr/cloud/autoscaling/Preference.java      |  84 +++
 .../org/apache/solr/cloud/autoscaling/Row.java  | 115 ++++
 .../solr/cloud/autoscaling/package-info.java    |  22 +
 .../org/apache/solr/common/IteratorWriter.java  |  21 +-
 .../java/org/apache/solr/common/MapWriter.java  |  10 +
 .../apache/solr/common/cloud/DocCollection.java |   5 +
 .../apache/solr/common/cloud/SolrZkClient.java  |  15 +
 .../apache/solr/common/cloud/ZkStateReader.java |   1 +
 .../solr/common/cloud/rule/ImplicitSnitch.java  |  15 +-
 .../solr/common/cloud/rule/SnitchContext.java   |   4 -
 .../apache/solr/common/params/CommonParams.java |   6 +-
 .../solr/common/util/CommandOperation.java      |  50 +-
 .../apache/solr/common/util/JavaBinCodec.java   |  15 +-
 .../java/org/apache/solr/common/util/Utils.java |  71 ++-
 .../solr/cloud/autoscaling/TestPolicy.java      | 520 +++++++++++++++++++
 44 files changed, 3453 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c413cf8..7a3c81f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -64,7 +64,7 @@ Upgrading from Solr 6.x
   registries as hierarchical MBeans. This behavior can be also disabled by specifying a SolrJmxReporter
   configuration with a boolean init arg "enabled" set to "false". For a more fine-grained control users
   should explicitly specify at least one SolrJmxReporter configuration.
-  
+
 * The sow (split-on-whitespace) request param now defaults to false (true in previous versions).
   This affects the edismax and standard/"lucene" query parsers: if the sow param is not specified,
   query text will not be split on whitespace before analysis. See
@@ -102,19 +102,25 @@ New Features
 * SOLR-10431: Make it possible to invoke v2 api calls using SolrJ (Cao Manh Dat, Noble Paul, shalin)
 
 * SOLR-10233: Add support for different replica types, that can handle updates differently:
-  - NRT: Writes updates to transaction log and indexes locally. Replicas of type “NRT” support NRT 
-         (soft commits) and RTG. Any NRT replica can become a leader. This was the only type supported 
+  - NRT: Writes updates to transaction log and indexes locally. Replicas of type “NRT” support NRT
+         (soft commits) and RTG. Any NRT replica can become a leader. This was the only type supported
          in SolrCloud until now and it’s the default type.
-  - TLOG: Writes to transaction log, but not to index, uses replication to copy segment files from the 
-          shard leader. Any TLOG replica can become leader (by first applying all local transaction log 
+  - TLOG: Writes to transaction log, but not to index, uses replication to copy segment files from the
+          shard leader. Any TLOG replica can become leader (by first applying all local transaction log
           elements). If a replica is of type TLOG but is also the leader, it will behave as a NRT. This
           is exactly what was added in SOLR-9835 (non-realtime replicas), just the API and naming changes.
-  - PULL: Doesn’t index or writes to transaction log, just replicates from the shard leader. PULL replicas 
-          can’t become shard leaders (i.e., if there are only PULL replicas in the collection at some point, 
-          updates will fail same as if there is no leaders, queries continue to work), so they don’t even 
+  - PULL: Doesn’t index or writes to transaction log, just replicates from the shard leader. PULL replicas
+          can’t become shard leaders (i.e., if there are only PULL replicas in the collection at some point,
+          updates will fail same as if there is no leaders, queries continue to work), so they don’t even
           participate in elections.
   (Tomás Fernández Löbbe)
 
+* SOLR-10373: Implement read API for autoscaling configuration at /admin/autoscaling or
+  /cluster/autoscaling paths. (shalin)
+
+* SOLR-10677: Expose a diagnostics API to return nodes sorted by load in descending order and
+  any policy violations. (shalin)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
@@ -198,6 +204,8 @@ Other Changes
 
 * SOLR-10755: delete/refactor many solrj deprecations (hossman)
 
+* SOLR-10764: AutoScalingHandler should validate policy and preferences before updating zookeeper. (shalin)
+
 ==================  6.7.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
@@ -230,7 +238,7 @@ New Features
 
 * SOLR-10721: Provide a way to know when Core Discovery is finished and when all async cores are done loading
   (Erick Erickson)
-  
+
 * SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory. (Steve Rowe)
 
 * SOLR-10479: Adds support for HttpShardHandlerFactory.loadBalancerRequests(MinimumAbsolute|MaximumFraction)
@@ -257,8 +265,8 @@ Other Changes
 
 * SOLR-10400: Replace (instanceof TrieFooField || instanceof FooPointField) constructs with
   FieldType.getNumberType() or SchemaField.getSortField() where appropriate. (hossman, Steve Rowe)
-  
-* SOLR-10438: Assign explicit useDocValuesAsStored values to all points field types in 
+
+* SOLR-10438: Assign explicit useDocValuesAsStored values to all points field types in
   schema-point.xml/TestPointFields. (hossman, Steve Rowe)
   
 * LUCENE-7705: Allow CharTokenizer-derived tokenizers and KeywordTokenizer to configure the max token length.
@@ -335,7 +343,7 @@ New Features
 
 * SOLR-10507: Core Admin status command to emit collection details of each core (noble)
 
-* SOLR-10521: introducing sort=childfield(field) asc for searching by {!parent} (Mikhail Khludnev) 
+* SOLR-10521: introducing sort=childfield(field) asc for searching by {!parent} (Mikhail Khludnev)
 
 * SOLR-9596: Add Solr support for SimpleTextCodec, via <codecFactory class="solr.SimpleTextCodecFactory"/>
   in solrconfig.xml (per-field specification in the schema is not possible). (Steve Rowe)
@@ -394,7 +402,7 @@ Optimizations
 * SOLR-10499: facet.heatmap is now significantly faster when the docset (base query) matches everything and there are no
   deleted docs.  It's also faster when the docset matches a small fraction of the index or none. (David Smiley)
 
-* SOLR-9217: Reduced heap consumption for filter({!join ... score=...}) 
+* SOLR-9217: Reduced heap consumption for filter({!join ... score=...})
   (Andrey Kudryavtsev, Gopikannan Venugopalsamy via Mikhail Khludnev)
 
 * SOLR-10548: JSON Facet API now uses hyper-log-log++ for determining the number of buckets
@@ -423,8 +431,8 @@ Bug Fixes
 
 * SOLR-10264: Fixes multi-term synonym parsing in ManagedSynonymFilterFactory.
   (Jörg Rathlev, Steve Rowe, Christine Poerschke)
-  
-* SOLR-8807: fix Spellcheck "collateMaxCollectDocs" parameter to work with queries that have the 
+
+* SOLR-8807: fix Spellcheck "collateMaxCollectDocs" parameter to work with queries that have the
   CollpasingQParserPlugin applied.  (James Dyer)
 
 * SOLR-10474: TestPointFields.testPointFieldReturn() depends on order of unsorted hits. (Steve Rowe)
@@ -434,7 +442,7 @@ Bug Fixes
 
 * SOLR-10047: Mismatched Docvalues segments cause exception in Sorting/Faceting. Solr now uninverts per segment
   to avoid such exceptions. (Keith Laban via shalin)
-  
+
 * SOLR-10472: Fixed uninversion (aka: FieldCache) bugs with the numeric PointField classes, and CurrencyField (hossman)
 
 * SOLR-5127: Multiple highlight fields and wildcards are now supported e.g. hl.fl=title,text_*
@@ -446,13 +454,13 @@ Bug Fixes
   when there was a mincount > 1.  This has been corrected by changing numBuckets cardinality processing to
   ignore mincount > 1 for non-distributed requests. (yonik)
 
-* SOLR-10520: child.facet.field doubled counts at least when rows>0. (Dr. Oleg Savrasov via Mikhail Khludnev) 
+* SOLR-10520: child.facet.field doubled counts at least when rows>0. (Dr. Oleg Savrasov via Mikhail Khludnev)
 
 * SOLR-10480: Full pagination in JSON Facet API using offset does not work. (yonik)
 
 * SOLR-10526: facet.heatmap didn't honor facet exclusions ('ex') for distributed search. (David Smiley)
 
-* SOLR-10500: nested child docs are adopted by neighbour when several parents come in update/json/docs 
+* SOLR-10500: nested child docs are adopted by neighbour when several parents come in update/json/docs
   (Alexey Suprun,noble via Mikhail Khludnev)
 
 * SOLR-10316: Unloading a core can remove a ZK SolrCore registration entry for the wrong SolrCore. (Mark Miller)
@@ -478,7 +486,7 @@ Bug Fixes
   and accept "TO" as endpoints in range queries. (hossman, Steve Rowe)
 
 * SOLR-10735: Windows script (solr.cmd) didn't work properly with directory containing spaces. Adding quotations
-  to fix (Uwe Schindler, janhoy, Tomas Fernandez-Lobbe, Ishan Chattopadhyaya) 
+  to fix (Uwe Schindler, janhoy, Tomas Fernandez-Lobbe, Ishan Chattopadhyaya)
 
 Ref Guide
 ----------------------
@@ -578,7 +586,7 @@ Bug Fixes
 
 * SOLR-10404: The fetch() streaming expression wouldn't work if a value included query syntax chars (like :+-).
   Fixed, and enhanced the generated query to not pollute the queryCache. (David Smiley)
-  
+
 * SOLR-10423: Disable graph query production via schema configuration <fieldtype ... enableGraphQueries="false">.
   This fixes broken queries for ShingleFilter-containing query-time analyzers when request param sow=false.
   (Steve Rowe)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index cb8175e..b859599 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -668,6 +668,7 @@ public class ZkController {
     byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
     cmdExecutor.ensureExists(ZkStateReader.CLUSTER_STATE, emptyJson, CreateMode.PERSISTENT, zkClient);
     cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
+    cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
   }
 
   private void init(CurrentCoreDescriptorProvider registerOnReconnect) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
new file mode 100644
index 0000000..5946bf4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud.autoscaling;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.RequestHandlerBase;
+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.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
+import static org.apache.solr.common.params.CommonParams.JSON;
+
+/**
+ * Handler for /cluster/autoscaling
+ */
+public class AutoScalingHandler extends RequestHandlerBase implements PermissionNameProvider {
+  public static final String HANDLER_PATH = "/admin/autoscaling";
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static ImmutableSet<String> singletonCommands = ImmutableSet.of("set-cluster-preferences", "set-cluster-policy");
+  protected final CoreContainer container;
+  private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
+
+  public AutoScalingHandler(CoreContainer container) {
+    this.container = container;
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    try {
+      String httpMethod = (String) req.getContext().get("httpMethod");
+      RequestHandlerUtils.setWt(req, JSON);
+
+      if ("GET".equals(httpMethod)) {
+        String path = (String) req.getContext().get("path");
+        if (path == null) path = "/cluster/autoscaling";
+        List<String> parts = StrUtils.splitSmart(path, '/');
+        if (parts.get(0).isEmpty()) parts.remove(0);
+
+        if (parts.size() < 2 || parts.size() > 3) {
+          // invalid
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
+        }
+
+        Map<String, Object> map = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
+        if (parts.size() == 2) {
+          rsp.getValues().addAll(map);
+        } else if (parts.size() == 3 && "diagnostics".equals(parts.get(2))) {
+          handleDiagnostics(rsp, map);
+        }
+      } else {
+        if (req.getContentStreams() == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands specified for autoscaling");
+        }
+        List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), rsp.getValues(), singletonCommands);
+        if (ops == null) {
+          // errors have already been added to the response so there's nothing left to do
+          return;
+        }
+        for (CommandOperation op : ops) {
+          switch (op.name) {
+            case "set-policy":
+              handleSetPolicies(req, rsp, op);
+              break;
+            case "remove-policy":
+              handleRemovePolicy(req, rsp, op);
+              break;
+            case "set-cluster-preferences":
+              handleSetClusterPreferences(req, rsp, op);
+              break;
+            case "set-cluster-policy":
+              handleSetClusterPolicy(req, rsp, op);
+              break;
+            default:
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command: " + op.name);
+          }
+        }
+      }
+    } catch (Exception e) {
+      rsp.getValues().add("result", "failure");
+      throw e;
+    } finally {
+      RequestHandlerUtils.addExperimentalFormatWarning(rsp);
+    }
+  }
+
+  private void handleDiagnostics(SolrQueryResponse rsp, Map<String, Object> autoScalingConf) throws IOException {
+    Policy policy = new Policy(autoScalingConf);
+    try (CloudSolrClient build = new CloudSolrClient.Builder()
+        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
+        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
+      Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
+      List<Row> sorted = session.getSorted();
+      List<Clause.Violation> violations = session.getViolations();
+
+      List<Preference> clusterPreferences = policy.getClusterPreferences();
+
+      List<Map<String, Object>> sortedNodes = new ArrayList<>(sorted.size());
+      for (Row row : sorted) {
+        Map<String, Object> map = Utils.makeMap("node", row.node);
+        for (Cell cell : row.cells) {
+          for (Preference clusterPreference : clusterPreferences) {
+            Policy.SortParam name = clusterPreference.name;
+            if (cell.name.equalsIgnoreCase(name.name())) {
+              map.put(name.name(), cell.val);
+              break;
+            }
+          }
+        }
+        sortedNodes.add(map);
+      }
+
+      Map<String, Object> map = new HashMap<>(2);
+      map.put("sortedNodes", sortedNodes);
+
+      map.put("violations", violations);
+      rsp.getValues().add("diagnostics", map);
+    }
+  }
+
+  private void handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
+    List clusterPolicy = (List) op.getCommandData();
+    if (clusterPolicy == null || !(clusterPolicy instanceof List)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "A list of cluster policies was not found");
+    }
+    zkSetClusterPolicy(container.getZkController().getZkStateReader(), clusterPolicy);
+    rsp.getValues().add("result", "success");
+  }
+
+  private void handleSetClusterPreferences(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
+    List preferences = (List) op.getCommandData();
+    if (preferences == null || !(preferences instanceof List)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "A list of cluster preferences not found");
+    }
+    zkSetPreferences(container.getZkController().getZkStateReader(), preferences);
+    rsp.getValues().add("result", "success");
+  }
+
+  private void handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
+    String policyName = (String) op.getCommandData();
+
+    if (policyName.trim().length() == 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The policy name cannot be empty");
+    }
+    Map<String, Object> autoScalingConf = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
+    Map<String, Object> policies = (Map<String, Object>) autoScalingConf.get("policies");
+    if (policies == null || !policies.containsKey(policyName)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No policy exists with name: " + policyName);
+    }
+
+    zkSetPolicies(container.getZkController().getZkStateReader(), policyName, null);
+    rsp.getValues().add("result", "success");
+  }
+
+  private void handleSetPolicies(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
+    Map<String, Object> policies = op.getDataMap();
+    for (Map.Entry<String, Object> policy : policies.entrySet()) {
+      String policyName = policy.getKey();
+      if (policyName == null || policyName.trim().length() == 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The policy name cannot be null or empty");
+      }
+    }
+
+    zkSetPolicies(container.getZkController().getZkStateReader(), null, policies);
+
+    rsp.getValues().add("result", "success");
+  }
+
+  private void zkSetPolicies(ZkStateReader reader, String policyBeRemoved, Map<String, Object> newPolicies) throws KeeperException, InterruptedException, IOException {
+    while (true) {
+      Stat stat = new Stat();
+      ZkNodeProps loaded = null;
+      byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
+      loaded = ZkNodeProps.load(data);
+      Map<String, Object> policies = (Map<String, Object>) loaded.get("policies");
+      if (policies == null) policies = new HashMap<>(1);
+      if (newPolicies != null) {
+        policies.putAll(newPolicies);
+      } else {
+        policies.remove(policyBeRemoved);
+      }
+      loaded = loaded.plus("policies", policies);
+      verifyAutoScalingConf(loaded.getProperties());
+      try {
+        reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
+      } catch (KeeperException.BadVersionException bve) {
+        // somebody else has changed the configuration so we must retry
+        continue;
+      }
+      break;
+    }
+  }
+
+  private void zkSetPreferences(ZkStateReader reader, List preferences) throws KeeperException, InterruptedException, IOException {
+    while (true) {
+      Stat stat = new Stat();
+      ZkNodeProps loaded = null;
+      byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
+      loaded = ZkNodeProps.load(data);
+      loaded = loaded.plus("cluster-preferences", preferences);
+      verifyAutoScalingConf(loaded.getProperties());
+      try {
+        reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
+      } catch (KeeperException.BadVersionException bve) {
+        // somebody else has changed the configuration so we must retry
+        continue;
+      }
+      break;
+    }
+  }
+
+  private void zkSetClusterPolicy(ZkStateReader reader, List clusterPolicy) throws KeeperException, InterruptedException, IOException {
+    while (true) {
+      Stat stat = new Stat();
+      ZkNodeProps loaded = null;
+      byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
+      loaded = ZkNodeProps.load(data);
+      loaded = loaded.plus("cluster-policy", clusterPolicy);
+      verifyAutoScalingConf(loaded.getProperties());
+      try {
+        reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
+      } catch (KeeperException.BadVersionException bve) {
+        // somebody else has changed the configuration so we must retry
+        continue;
+      }
+      break;
+    }
+  }
+
+  private void verifyAutoScalingConf(Map<String, Object> autoScalingConf) throws IOException {
+    try (CloudSolrClient build = new CloudSolrClient.Builder()
+        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
+        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
+      Policy policy = new Policy(autoScalingConf);
+      Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
+      log.debug("Verified autoscaling configuration");
+    }
+  }
+
+  private Map<String, Object> zkReadAutoScalingConf(ZkStateReader reader) throws KeeperException, InterruptedException {
+    byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    ZkNodeProps loaded = ZkNodeProps.load(data);
+    return loaded.getProperties();
+  }
+
+  @Override
+  public String getDescription() {
+    return "A handler for autoscaling configuration";
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    switch (request.getHttpMethod()) {
+      case "GET":
+        return Name.AUTOSCALING_READ_PERM;
+      case "POST":
+        return Name.AUTOSCALING_WRITE_PERM;
+      default:
+        return null;
+    }
+  }
+
+  @Override
+  public Collection<Api> getApis() {
+    return ApiBag.wrapRequestHandlers(this, "autoscaling.Commands");
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
+
+  @Override
+  public SolrRequestHandler getSubHandler(String path) {
+    if (path.equals("/diagnostics")) return this;
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java
new file mode 100644
index 0000000..d3447aa
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/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.
+ */
+
+/**
+ * Package for classes related to autoscaling
+ */
+package org.apache.solr.cloud.autoscaling;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java b/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
index 2d526fd..446c80f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
@@ -74,13 +74,12 @@ public class ServerSnitchContext extends SnitchContext {
 
   public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
     if (callback == null) callback = this;
-    String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(node);
     params.add("class", klas);
     params.add(ACTION, INVOKE.toString());
     //todo batch all requests to the same server
 
     try {
-      SimpleSolrResponse rsp = invoke(coreContainer.getUpdateShardHandler(), url, CommonParams.CORES_HANDLER_PATH, params);
+      SimpleSolrResponse rsp = invoke(node, CommonParams.CORES_HANDLER_PATH, params);
       Map<String, Object> returnedVal = (Map<String, Object>) rsp.getResponse().get(klas);
       if(exception == null){
 //        log this
@@ -94,8 +93,10 @@ public class ServerSnitchContext extends SnitchContext {
     }
   }
 
-  public SimpleSolrResponse invoke(UpdateShardHandler shardHandler,  final String url, String path, SolrParams params)
+  public SimpleSolrResponse invoke(String solrNode, String path, SolrParams params)
       throws IOException, SolrServerException {
+    String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(solrNode);
+    UpdateShardHandler shardHandler = coreContainer.getUpdateShardHandler();
     GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
     try (HttpSolrClient client = new HttpSolrClient.Builder(url).withHttpClient(shardHandler.getHttpClient())
         .withResponseParser(new BinaryResponseParser()).build()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/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 505a235..37842f8 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -60,6 +60,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.AuthSchemeRegistryProvider;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.CredentialsProviderProvider;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
@@ -150,7 +151,7 @@ public class CoreContainer {
   private UpdateShardHandler updateShardHandler;
 
   private TransientSolrCoreCacheFactory transientCoreCache;
-  
+
   private ExecutorService coreContainerWorkExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(
       new DefaultSolrThreadFactory("coreContainerWorkExecutor") );
 
@@ -193,6 +194,8 @@ public class CoreContainer {
   public final static long INITIAL_CORE_LOAD_COMPLETE = 0x4L;
   private volatile long status = 0L;
 
+  protected AutoScalingHandler autoScalingHandler;
+
   private enum CoreInitFailedAction { fromleader, none }
 
   /**
@@ -528,6 +531,9 @@ public class CoreContainer {
     metricsCollectorHandler = createHandler(MetricsCollectorHandler.HANDLER_PATH, MetricsCollectorHandler.class.getName(), MetricsCollectorHandler.class);
     // may want to add some configuration here in the future
     metricsCollectorHandler.init(null);
+
+    autoScalingHandler = createHandler(AutoScalingHandler.HANDLER_PATH, AutoScalingHandler.class.getName(), AutoScalingHandler.class);
+
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
     securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
@@ -587,7 +593,7 @@ public class CoreContainer {
       }
       checkForDuplicateCoreNames(cds);
       status |= CORE_DISCOVERY_COMPLETE;
-      
+
       for (final CoreDescriptor cd : cds) {
         if (cd.isTransient() || !cd.isLoadOnStartup()) {
           getTransientCacheHandler().addTransientDescriptor(cd.getName(), cd);
@@ -663,7 +669,7 @@ public class CoreContainer {
     }
     return transientCoreCache.getTransientSolrCoreCache();
   }
-  
+
   public void securityNodeChanged() {
     log.info("Security node changed, reloading security.json");
     reloadSecurityProperties();
@@ -832,7 +838,7 @@ public class CoreContainer {
     if( core == null ) {
       throw new RuntimeException( "Can not register a null core." );
     }
-    
+
     if (isShutDown) {
       core.close();
       throw new IllegalStateException("This CoreContainer has been closed");
@@ -1114,7 +1120,7 @@ public class CoreContainer {
   /**
    * get a list of all the cores that are currently loaded
    * @return a list of al lthe available core names in either permanent or transient core lists.
-   * 
+   *
    * Note: this implies that the core is loaded
    */
   public Collection<String> getAllCoreNames() {
@@ -1163,12 +1169,12 @@ public class CoreContainer {
     if (ret == null) {
       oldDesc.loadExtraProperties(); // there may be changes to extra properties that we need to pick up.
       return oldDesc;
-      
+
     }
     // The CloudDescriptor bit here is created in a very convoluted way, requiring access to private methods
     // in ZkController. When reloading, this behavior is identical to what used to happen where a copy of the old
     // CoreDescriptor was just re-used.
-    
+
     if (ret.getCloudDescriptor() != null) {
       ret.getCloudDescriptor().reload(oldDesc.getCloudDescriptor());
     }
@@ -1186,7 +1192,7 @@ public class CoreContainer {
   public void reload(String name) {
     SolrCore core = solrCores.getCoreFromAnyList(name, false);
     if (core != null) {
-      
+
       // The underlying core properties files may have changed, we don't really know. So we have a (perhaps) stale
       // CoreDescriptor and we need to reload it from the disk files
       CoreDescriptor cd = reloadCoreDescriptor(core.getCoreDescriptor());
@@ -1206,7 +1212,7 @@ public class CoreContainer {
             if (!cd.getCloudDescriptor().isLeader()) {
               getZkController().startReplicationFromLeader(newCore.getName(), true);
             }
-            
+
           }
         }
       } catch (SolrCoreState.CoreIsClosedException e) {
@@ -1293,7 +1299,7 @@ public class CoreContainer {
       // cancel recovery in cloud mode
       core.getSolrCoreState().cancelRecovery();
       if (core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.PULL
-          || core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.TLOG) { 
+          || core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.TLOG) {
         // Stop replication if this is part of a pull/tlog replica before closing the core
         zkSys.getZkController().stopReplicationFromLeader(name);
       }
@@ -1385,10 +1391,10 @@ public class CoreContainer {
     // This is a bit of awkwardness where SolrCloud and transient cores don't play nice together. For transient cores,
     // we have to allow them to be created at any time there hasn't been a core load failure (use reload to cure that).
     // But for TestConfigSetsAPI.testUploadWithScriptUpdateProcessor, this needs to _not_ try to load the core if
-    // the core is null and there was an error. If you change this, be sure to run both TestConfiSetsAPI and 
+    // the core is null and there was an error. If you change this, be sure to run both TestConfiSetsAPI and
     // TestLazyCores
     if (desc == null || zkSys.getZkController() != null) return null;
-    
+
     // This will put an entry in pending core ops if the core isn't loaded
     core = solrCores.waitAddPendingCoreOps(name);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index 2e679cf..17cdbbc 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -90,7 +90,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   static final String[] packages = {
       "", "analysis.", "schema.", "handler.", "search.", "update.", "core.", "response.", "request.",
       "update.processor.", "util.", "spelling.", "handler.component.", "handler.dataimport.",
-      "spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.","handler.admin."
+      "spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.","handler.admin.",
+      "cloud.autoscaling."
   };
   private static final java.lang.String SOLR_CORE_NAME = "solr.core.name";
   private static Set<String> loggedOnce = new ConcurrentSkipListSet<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/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 052fcfe..ca0529a 100644
--- a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
@@ -80,7 +80,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
   public void handleRequestBody(final SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     String httpMethod = req.getHttpMethod();
     String path = (String) req.getContext().get("path");
-    SolrConfigHandler.setWt(req, JSON);
+    RequestHandlerUtils.setWt(req, JSON);
 
     List<String> pieces = StrUtils.splitSmart(path, '/');
     String blobName = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java
index 99f8315..4441024 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.*;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
@@ -119,4 +120,16 @@ public class RequestHandlerUtils
     }
     return false;
   }
+
+  /**
+   * @since 6.7
+   */
+  public static void setWt(SolrQueryRequest req, String wt) {
+    SolrParams params = req.getParams();
+    if (params.get(CommonParams.WT) != null) return;//wt is set by user
+    Map<String, String> map = new HashMap<>(1);
+    map.put(CommonParams.WT, wt);
+    map.put("indent", "true");
+    req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/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 41285be..e3e292b 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -76,7 +76,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
 
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    SolrConfigHandler.setWt(req, JSON);
+    RequestHandlerUtils.setWt(req, JSON);
     String httpMethod = (String) req.getContext().get("httpMethod");
     if ("POST".equals(httpMethod)) {
       if (isImmutableConfigSet) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/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 46646ad..92a773a 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -124,7 +124,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
 
-    setWt(req, CommonParams.JSON);
+    RequestHandlerUtils.setWt(req, CommonParams.JSON);
     String httpMethod = (String) req.getContext().get("httpMethod");
     Command command = new Command(req, rsp, httpMethod);
     if ("POST".equals(httpMethod)) {
@@ -673,15 +673,6 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
     return null;
   }
 
-  public static void setWt(SolrQueryRequest req, String wt) {
-    SolrParams params = req.getParams();
-    if (params.get(CommonParams.WT) != null) return;//wt is set by user
-    Map<String, String> map = new HashMap<>(1);
-    map.put(CommonParams.WT, wt);
-    map.put("indent", "true");
-    req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
-  }
-
   @Override
   public SolrRequestHandler getSubHandler(String path) {
     if (subPaths.contains(path)) return this;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/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 36a3b57..63b9318 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
@@ -34,7 +34,7 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.handler.SolrConfigHandler;
+import org.apache.solr.handler.RequestHandlerUtils;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthenticationPlugin;
@@ -74,7 +74,7 @@ public abstract class SecurityConfHandler extends RequestHandlerBase implements
 
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    SolrConfigHandler.setWt(req, CommonParams.JSON);
+    RequestHandlerUtils.setWt(req, CommonParams.JSON);
     String httpMethod = (String) req.getContext().get("httpMethod");
     String path = (String) req.getContext().get("path");
     String key = path.substring(path.lastIndexOf('/')+1);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
index 8bef94a..261daee 100644
--- a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
@@ -26,6 +26,9 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexableField;
@@ -144,6 +147,8 @@ public abstract class TextResponseWriter implements PushWriter {
       writeNumber(name, (Number) val);
     } else if (val instanceof Boolean) {
       writeBool(name, (Boolean) val);
+    } else if (val instanceof AtomicBoolean)  {
+      writeBool(name, ((AtomicBoolean) val).get());
     } else if (val instanceof Date) {
       writeDate(name, (Date) val);
     } else if (val instanceof Document) {
@@ -221,13 +226,17 @@ public abstract class TextResponseWriter implements PushWriter {
     } else if (val instanceof Float) {
       // we pass the float instead of using toString() because
       // it may need special formatting. same for double.
-      writeFloat(name, ((Float)val).floatValue());
+      writeFloat(name, val.floatValue());
     } else if (val instanceof Double) {
-      writeDouble(name, ((Double) val).doubleValue());
+      writeDouble(name, val.doubleValue());
     } else if (val instanceof Short) {
       writeInt(name, val.toString());
     } else if (val instanceof Byte) {
       writeInt(name, val.toString());
+    } else if (val instanceof AtomicInteger) {
+      writeInt(name, ((AtomicInteger) val).get());
+    } else if (val instanceof AtomicLong) {
+      writeLong(name, ((AtomicLong) val).get());
     } else {
       // default... for debugging only
       writeStr(name, val.getClass().getName() + ':' + val.toString(), true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index 9f797487..dc98d26 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -47,6 +47,8 @@ public interface PermissionNameProvider {
     SECURITY_EDIT_PERM("security-edit", null),
     SECURITY_READ_PERM("security-read", null),
     METRICS_READ_PERM("metrics-read", null),
+    AUTOSCALING_READ_PERM("autoscaling-read", null),
+    AUTOSCALING_WRITE_PERM("autoscaling-write", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/resources/apispec/autoscaling.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/autoscaling.Commands.json b/solr/core/src/resources/apispec/autoscaling.Commands.json
new file mode 100644
index 0000000..5ff89ef
--- /dev/null
+++ b/solr/core/src/resources/apispec/autoscaling.Commands.json
@@ -0,0 +1,47 @@
+{
+  "documentation": "TODO NOCOMMIT",
+  "description": "The Scaling API provides API for adding cluster level scaling rules, triggers and event listeners",
+  "methods": [
+    "GET",
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/autoscaling",
+      "/cluster/autoscaling/diagnostics"
+    ]
+  },
+  "commands": {
+    "set-policy" : {
+      "type":"object",
+      "description": "The set-policy command allows you to add and update policies that apply to collections",
+    /*  "patternProperties": {
+        "^.+$": {
+          "type": "array"
+        }
+      },*/
+      "additionalProperties": true
+    },
+    "set-cluster-policy" : {
+      "type" : "array",
+      "description" : "The set-cluster-policy command allows you to add and update cluster-level policy that acts as the base for all collection level policies, if any"
+    },
+    "set-cluster-preferences" : {
+      "type" : "array",
+      "description" : "The set-cluster-preferences command allows you to add and update cluster-level preferences that are used to sort nodes for selection in cluster operations"
+    },
+    "remove-policy": {
+      "description": "Remove a policy",
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the policy to be removed"
+        }
+      },
+      "required": [
+        "name"
+      ]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
new file mode 100644
index 0000000..0eaf548
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -0,0 +1,326 @@
+/*
+ * 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.cloud.autoscaling;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
+
+/**
+ * Test for AutoScalingHandler
+ */
+public class AutoScalingHandlerTest extends SolrCloudTestCase {
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // clear any persisted auto scaling configuration
+    zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
+  }
+
+  @Test
+  public void testPolicyAndPreferences() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    // add multiple policies
+    String setPolicyCommand =  "{'set-policy': {" +
+        "    'xyz':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'!overseer', 'replica':0}" +
+        "    ]," +
+        "    'policy1':[" +
+        "      {'cores':'<2', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
+        "    ]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
+    NamedList<Object> response = null;
+    try {
+      response = solrClient.request(req);
+      fail("Adding a policy with 'cores' attribute should not have succeeded.");
+    } catch (HttpSolrClient.RemoteSolrException e) {
+      // expected
+      assertTrue(e.getMessage().contains("cores is only allowed in 'cluster-policy'"));
+    } catch (Exception e) {
+      throw e;
+    }
+
+    setPolicyCommand =  "{'set-policy': {" +
+        "    'xyz':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'!overseer', 'replica':0}" +
+        "    ]," +
+        "    'policy1':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
+        "    ]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    ZkNodeProps loaded = ZkNodeProps.load(data);
+    Map<String, Object> policies = (Map<String, Object>) loaded.get("policies");
+    assertNotNull(policies);
+    assertNotNull(policies.get("xyz"));
+    assertNotNull(policies.get("policy1"));
+
+    // update default policy
+    setPolicyCommand = "{'set-policy': {" +
+        "    'xyz':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
+        "    ]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    policies = (Map<String, Object>) loaded.get("policies");
+    List conditions = (List) policies.get("xyz");
+    assertEquals(1, conditions.size());
+
+    // remove policy
+    String removePolicyCommand = "{remove-policy : policy1}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, removePolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    policies = (Map<String, Object>) loaded.get("policies");
+    assertNull(policies.get("policy1"));
+
+    // set preferences
+    String setPreferencesCommand = "{" +
+        " 'set-cluster-preferences': [" +
+        "        {'minimize': 'cores', 'precision': 3}," +
+        "        {'maximize': 'freedisk','precision': 100}," +
+        "        {'minimize': 'sysLoadAvg','precision': 10}]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    List preferences = (List) loaded.get("cluster-preferences");
+    assertEquals(3, preferences.size());
+
+    // set preferences
+    setPreferencesCommand = "{" +
+        " 'set-cluster-preferences': [" +
+        "        {'minimize': 'sysLoadAvg','precision': 10}]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    preferences = (List) loaded.get("cluster-preferences");
+    assertEquals(1, preferences.size());
+
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'!overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    List clusterPolicy = (List) loaded.get("cluster-policy");
+    assertNotNull(clusterPolicy);
+    assertEquals(3, clusterPolicy.size());
+  }
+
+  @Test
+  public void testReadApi() throws Exception  {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setPreferencesCommand = "{" +
+        " 'set-cluster-preferences': [" +
+        "        {'minimize': 'cores', 'precision': 3}," +
+        "        {'maximize': 'freedisk','precision': 100}," +
+        "        {'minimize': 'sysLoadAvg','precision': 10}," +
+        "        {'minimize': 'heapUsage','precision': 10}]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setPolicyCommand =  "{'set-policy': {" +
+        "    'xyz':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]," +
+        "    'policy1':[" +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
+        "    ]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+//    SolrQuery query = new SolrQuery().setParam(CommonParams.QT, path);
+    req = createAutoScalingRequest(SolrRequest.METHOD.GET, null);
+    response = solrClient.request(req);
+
+    List<Map> clusterPrefs = (List<Map>) response.get("cluster-preferences");
+    assertNotNull(clusterPrefs);
+    assertEquals(4, clusterPrefs.size());
+
+    List<Map> clusterPolicy = (List<Map>) response.get("cluster-policy");
+    assertNotNull(clusterPolicy);
+    assertEquals(3, clusterPolicy.size());
+
+    Map policies = (Map) response.get("policies");
+    assertNotNull(policies);
+    assertEquals(2, policies.size());
+    assertNotNull(policies.get("xyz"));
+    assertNotNull(policies.get("policy1"));
+
+    req = createAutoScalingRequest(SolrRequest.METHOD.GET, "/diagnostics", null);
+    response = solrClient.request(req);
+
+    Map<String, Object> diagnostics = (Map<String, Object>) response.get("diagnostics");
+    List sortedNodes = (List) diagnostics.get("sortedNodes");
+    assertNotNull(sortedNodes);
+
+    assertEquals(2, sortedNodes.size());
+    String[] sortedNodeNames = new String[2];
+    for (int i = 0; i < 2; i++) {
+      Map node = (Map) sortedNodes.get(i);
+      assertNotNull(node);
+      assertEquals(5, node.size());
+      assertNotNull(sortedNodeNames[i] = (String) node.get("node"));
+      assertNotNull(node.get("cores"));
+      assertEquals(0, node.get("cores"));
+      assertNotNull(node.get("freedisk"));
+      assertNotNull(node.get("sysLoadAvg"));
+      assertNotNull(node.get("heapUsage"));
+    }
+
+    List<Map<String, Object>> violations = (List<Map<String, Object>>) diagnostics.get("violations");
+    assertNotNull(violations);
+    assertEquals(0, violations.size());
+
+    violations = (List<Map<String, Object>>) diagnostics.get("violations");
+    assertNotNull(violations);
+    assertEquals(0, violations.size());
+
+    // lets create a collection which violates the rule replicas < 2
+    CollectionAdminRequest.Create create = CollectionAdminRequest.Create.createCollection("readApiTestViolations", 1, 6);
+    create.setMaxShardsPerNode(10);
+    CollectionAdminResponse adminResponse = create.process(solrClient);
+    assertTrue(adminResponse.isSuccess());
+
+    // get the diagnostics output again
+    req = createAutoScalingRequest(SolrRequest.METHOD.GET, "/diagnostics", null);
+    response = solrClient.request(req);
+    diagnostics = (Map<String, Object>) response.get("diagnostics");
+    sortedNodes = (List) diagnostics.get("sortedNodes");
+    assertNotNull(sortedNodes);
+
+    violations = (List<Map<String, Object>>) diagnostics.get("violations");
+    assertNotNull(violations);
+    assertEquals(2, violations.size());
+    for (Map<String, Object> violation : violations) {
+      assertEquals("readApiTestViolations", violation.get("collection"));
+      assertEquals("shard1", violation.get("shard"));
+      assertEquals(Utils.makeMap("replica", "3", "delta", -1), violation.get("violation"));
+      assertNotNull(violation.get("clause"));
+    }
+  }
+
+  static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String message) {
+    return createAutoScalingRequest(m, null, message);
+  }
+
+  static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String subPath, String message) {
+    boolean useV1 = random().nextBoolean();
+    String path = useV1 ? "/admin/autoscaling" : "/cluster/autoscaling";
+    path += subPath != null ? subPath : "";
+    return useV1
+        ? new AutoScalingRequest(m, path, message)
+        : new V2Request.Builder(path).withMethod(m).withPayload(message).build();
+  }
+
+  static class AutoScalingRequest extends SolrRequest {
+    protected final String message;
+
+    public AutoScalingRequest(METHOD m, String path, String message) {
+      super(m, path);
+      this.message = message;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      return null;
+    }
+
+    @Override
+    public Collection<ContentStream> getContentStreams() throws IOException {
+      return message != null ? Collections.singletonList(new ContentStreamBase.StringStream(message)) : null;
+    }
+
+    @Override
+    protected SolrResponse createResponse(SolrClient client) {
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
new file mode 100644
index 0000000..731a83d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.autoscaling;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.OverseerTaskProcessor;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@LuceneTestCase.Slow
+public class TestPolicyCloud extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  @org.junit.Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(5)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @After
+  public void removeCollections() throws Exception {
+    cluster.deleteAllCollections();
+  }
+
+
+  public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
+        .process(cluster.getSolrClient());
+    DocCollection rulesCollection = getCollectionState("policiesTest");
+    SolrClientDataProvider provider = new SolrClientDataProvider(cluster.getSolrClient());
+    Map<String, Object> val = provider.getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
+        "freedisk",
+        "cores",
+        "heapUsage",
+        "sysLoadAvg"));
+    assertNotNull(val.get("freedisk"));
+    assertNotNull(val.get("heapUsage"));
+    assertNotNull(val.get("sysLoadAvg"));
+    assertTrue(((Number) val.get("cores")).intValue() > 0);
+    assertTrue("freedisk value is " + ((Number) val.get("freedisk")).longValue(), ((Number) val.get("freedisk")).longValue() > 0);
+    assertTrue("heapUsage value is " + ((Number) val.get("heapUsage")).longValue(), ((Number) val.get("heapUsage")).longValue() > 0);
+    assertTrue("sysLoadAvg value is " + ((Number) val.get("sysLoadAvg")).longValue(), ((Number) val.get("sysLoadAvg")).longValue() > 0);
+    String overseerNode = OverseerTaskProcessor.getLeaderNode(cluster.getZkClient());
+    cluster.getSolrClient().request(CollectionAdminRequest.addRole(overseerNode, "overseer"));
+    for (int i = 0; i < 10; i++) {
+      Map<String, Object> data = cluster.getSolrClient().getZkStateReader().getZkClient().getJson(ZkStateReader.ROLES, true);
+      if (i >= 9 && data == null) {
+        throw new RuntimeException("NO overseer node created");
+      }
+      Thread.sleep(100);
+    }
+    val = provider.getNodeValues(overseerNode, Arrays.asList(
+        "nodeRole",
+        "ip_1", "ip_2", "ip_3", "ip_4",
+        "sysprop.java.version",
+        "sysprop.java.vendor"));
+    assertEquals("overseer", val.get("nodeRole"));
+    assertNotNull(val.get("ip_1"));
+    assertNotNull(val.get("ip_2"));
+    assertNotNull(val.get("ip_3"));
+    assertNotNull(val.get("ip_4"));
+    assertNotNull(val.get("sysprop.java.version"));
+    assertNotNull(val.get("sysprop.java.vendor"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 1271655..11f6b26 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -58,7 +58,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.ToleratedUpdateError;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ClusterState.CollectionRef;
 import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.CollectionStateWatcher;
 import org.apache.solr.common.cloud.DocCollection;
@@ -83,8 +83,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
 
-import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
 import static org.apache.solr.common.params.CommonParams.ID;
+import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
 
 /**
  * SolrJ client class to communicate with SolrCloud.
@@ -312,7 +312,7 @@ public class CloudSolrClient extends SolrClient {
     assert seconds > 0;
     this.collectionStateCache.timeToLive = seconds * 1000L;
   }
-  
+
   public ResponseParser getParser() {
     return lbClient.getParser();
   }
@@ -347,6 +347,7 @@ public class CloudSolrClient extends SolrClient {
   public ZkStateReader getZkStateReader() {
     if (stateProvider instanceof ZkClientClusterStateProvider) {
       ZkClientClusterStateProvider provider = (ZkClientClusterStateProvider) stateProvider;
+      stateProvider.connect();
       return provider.zkStateReader;
     }
     throw new IllegalStateException("This has no Zk stateReader");
@@ -430,7 +431,7 @@ public class CloudSolrClient extends SolrClient {
     throw new IllegalArgumentException("This client does not use ZK");
 
   }
-  
+
   /**
    * Block until a collection state matches a predicate, or a timeout
    *
@@ -1210,7 +1211,7 @@ public class CloudSolrClient extends SolrClient {
           && !cacheEntry.shoulRetry()) return col;
     }
 
-    ClusterState.CollectionRef ref = getCollectionRef(collection);
+    CollectionRef ref = getCollectionRef(collection);
     if (ref == null) {
       //no such collection exists
       return null;
@@ -1245,7 +1246,7 @@ public class CloudSolrClient extends SolrClient {
     }
   }
 
-  ClusterState.CollectionRef getCollectionRef(String collection) {
+  CollectionRef getCollectionRef(String collection) {
     return stateProvider.getState(collection);
   }
 
@@ -1407,7 +1408,7 @@ public class CloudSolrClient extends SolrClient {
       this.solrUrls.add(solrUrl);
       return this;
     }
-    
+
     /**
      * Provide a list of Solr URL to be used when configuring {@link CloudSolrClient} instances.
      * One of the provided values will be used to fetch the list of live Solr

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientDataProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientDataProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientDataProvider.java
new file mode 100644
index 0000000..8bca7dc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientDataProvider.java
@@ -0,0 +1,262 @@
+/*
+ * 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.client.solrj.impl;
+
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+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 org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
+import org.apache.solr.cloud.autoscaling.ClusterDataProvider;
+import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.cloud.rule.RemoteCallback;
+import org.apache.solr.common.cloud.rule.SnitchContext;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class that implements {@link ClusterStateProvider} accepting a SolrClient
+ */
+public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
+
+  private final CloudSolrClient solrClient;
+  private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>();
+  private Set<String> liveNodes;
+  private Map<String, Object> snitchSession = new HashMap<>();
+  private Map<String, Map> nodeVsTags = new HashMap<>();
+
+  public SolrClientDataProvider(CloudSolrClient solrClient) {
+    this.solrClient = solrClient;
+    ZkStateReader zkStateReader = solrClient.getZkStateReader();
+    ClusterState clusterState = zkStateReader.getClusterState();
+    this.liveNodes = clusterState.getLiveNodes();
+    Map<String, ClusterState.CollectionRef> all = clusterState.getCollectionStates();
+    all.forEach((collName, ref) -> {
+      DocCollection coll = ref.get();
+      if (coll == null) return;
+      coll.forEachReplica((shard, replica) -> {
+        Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.get(replica.getNodeName());
+        if (nodeData == null) data.put(replica.getNodeName(), nodeData = new HashMap<>());
+        Map<String, List<ReplicaInfo>> collData = nodeData.get(collName);
+        if (collData == null) nodeData.put(collName, collData = new HashMap<>());
+        List<ReplicaInfo> replicas = collData.get(shard);
+        if (replicas == null) collData.put(shard, replicas = new ArrayList<>());
+        replicas.add(new ReplicaInfo(replica.getName(), collName, shard, new HashMap<>()));
+      });
+    });
+  }
+
+  @Override
+  public String getPolicyNameByCollection(String coll) {
+    ClusterState.CollectionRef state = solrClient.getClusterStateProvider().getState(coll);
+    return state == null || state.get() == null ? null : (String) state.get().getProperties().get("policy");
+  }
+
+  @Override
+  public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
+    AutoScalingSnitch snitch = new AutoScalingSnitch();
+    ClientSnitchCtx ctx = new ClientSnitchCtx(null, node, snitchSession, solrClient);
+    snitch.getTags(node, new HashSet<>(tags), ctx);
+    nodeVsTags.put(node, ctx.getTags());
+    return ctx.getTags();
+  }
+
+  @Override
+  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+    return data.getOrDefault(node, Collections.emptyMap());//todo fill other details
+  }
+
+  @Override
+  public Collection<String> getNodes() {
+    return liveNodes;
+  }
+
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    ew.put("liveNodes", liveNodes);
+    ew.put("replicaInfo", Utils.getDeepCopy(data, 5));
+    ew.put("nodeValues", nodeVsTags);
+
+  }
+
+  static class ClientSnitchCtx
+      extends SnitchContext {
+    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+    ZkClientClusterStateProvider zkClientClusterStateProvider;
+    CloudSolrClient solrClient;
+
+    public ClientSnitchCtx(SnitchInfo perSnitch,
+                           String node, Map<String, Object> session,
+                           CloudSolrClient solrClient) {
+      super(perSnitch, node, session);
+      this.solrClient = solrClient;
+      this.zkClientClusterStateProvider = (ZkClientClusterStateProvider) solrClient.getClusterStateProvider();
+    }
+
+
+    public Map getZkJson(String path) {
+      try {
+        byte[] data = zkClientClusterStateProvider.getZkStateReader().getZkClient().getData(path, null, new Stat(), true);
+        if (data == null) return null;
+        return (Map) Utils.fromJSON(data);
+      } catch (Exception e) {
+        log.warn("Unable to read from ZK path : " + path, e);
+        return null;
+      }
+    }
+
+    public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
+
+    }
+
+    public SimpleSolrResponse invoke(String solrNode, String path, SolrParams params)
+        throws IOException, SolrServerException {
+      String url = zkClientClusterStateProvider.getZkStateReader().getBaseUrlForNodeName(solrNode);
+
+      GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
+      try (HttpSolrClient client = new HttpSolrClient.Builder()
+          .withHttpClient(solrClient.getHttpClient())
+          .withBaseSolrUrl(url)
+          .withResponseParser(new BinaryResponseParser())
+          .build()) {
+        NamedList<Object> rsp = client.request(request);
+        request.response.nl = rsp;
+        return request.response;
+      }
+    }
+
+  }
+
+  //uses metrics API to get node information
+  static class AutoScalingSnitch extends ImplicitSnitch {
+    @Override
+    protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
+      ClientSnitchCtx snitchContext = (ClientSnitchCtx) ctx;
+      readSysProps(solrNode, requestedTags, snitchContext);
+      Set<String> groups = new HashSet<>();
+      List<String> prefixes = new ArrayList<>();
+      if (requestedTags.contains(DISK)) {
+        groups.add("solr.node");
+        prefixes.add("CONTAINER.fs.usableSpace");
+      }
+      if (requestedTags.contains(CORES)) {
+        groups.add("solr.core");
+        prefixes.add("CORE.coreName");
+      }
+      if (requestedTags.contains(SYSLOADAVG)) {
+        groups.add("solr.jvm");
+        prefixes.add("os.systemLoadAverage");
+      }
+      if (requestedTags.contains(HEAPUSAGE)) {
+        groups.add("solr.jvm");
+        prefixes.add("memory.heap.usage");
+      }
+      if (groups.isEmpty() || prefixes.isEmpty()) return;
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.add("group", StrUtils.join(groups, ','));
+      params.add("prefix", StrUtils.join(prefixes, ','));
+
+      try {
+        SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
+        Map m = rsp.nl.asMap(4);
+        if (requestedTags.contains(DISK)) {
+          Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.node/CONTAINER.fs.usableSpace");
+          if (n != null) ctx.getTags().put(DISK, n.doubleValue() / 1024.0d / 1024.0d / 1024.0d);
+        }
+        if (requestedTags.contains(CORES)) {
+          int count = 0;
+          Map cores = (Map) m.get("metrics");
+          for (Object o : cores.keySet()) {
+            if (o.toString().startsWith("solr.core.")) count++;
+          }
+          ctx.getTags().put(CORES, count);
+        }
+        if (requestedTags.contains(SYSLOADAVG)) {
+          Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.jvm/os.systemLoadAverage");
+          if (n != null) ctx.getTags().put(SYSLOADAVG, n.doubleValue() * 100.0d);
+        }
+        if (requestedTags.contains(HEAPUSAGE)) {
+          Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.jvm/memory.heap.usage");
+          if (n != null) ctx.getTags().put(HEAPUSAGE, n.doubleValue() * 100.0d);
+        }
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+      }
+    }
+
+    private void readSysProps(String solrNode, Set<String> requestedTags, ClientSnitchCtx snitchContext) {
+      List<String> prefixes = null;
+      ModifiableSolrParams params;
+      List<String> sysProp = null;
+      for (String tag : requestedTags) {
+        if (!tag.startsWith(SYSPROP)) continue;
+        if (sysProp == null) {
+          prefixes = new ArrayList<>();
+          sysProp = new ArrayList<>();
+          prefixes.add("system.properties");
+        }
+        sysProp.add(tag.substring(SYSPROP.length()));
+      }
+
+      if (sysProp == null) return;
+      params = new ModifiableSolrParams();
+      params.add("prefix", StrUtils.join(prefixes, ','));
+      for (String s : sysProp) params.add("property", s);
+      try {
+        SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
+        Map m = rsp.nl.asMap(6);
+        for (String s : sysProp) {
+          Object v = Utils.getObjectByPath(m, true,
+              Arrays.asList("metrics", "solr.jvm", "system.properties", s));
+          if (v != null) snitchContext.getTags().put("sysprop." + s, v);
+        }
+
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
index c997289..1875c50 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
@@ -39,10 +39,16 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
 
 
   ZkStateReader zkStateReader;
+  private boolean closeZkStateReader = true;
   String zkHost;
   int zkConnectTimeout = 10000;
   int zkClientTimeout = 10000;
 
+  public ZkClientClusterStateProvider(ZkStateReader zkStateReader) {
+    this.zkStateReader = zkStateReader;
+    this.closeZkStateReader =  false;
+  }
+
   public ZkClientClusterStateProvider(Collection<String> zkHosts, String chroot) {
     zkHost = buildZkHostString(zkHosts,chroot);
   }
@@ -55,6 +61,9 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
   public ClusterState.CollectionRef getState(String collection) {
     return zkStateReader.getClusterState().getCollectionRef(collection);
   }
+  public ZkStateReader getZkStateReader(){
+    return zkStateReader;
+  }
 
   @Override
   public Set<String> liveNodes() {
@@ -151,7 +160,7 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
 
   @Override
   public void close() throws IOException {
-    if (zkStateReader != null) {
+    if (zkStateReader != null && closeZkStateReader) {
       synchronized (this) {
         if (zkStateReader != null)
           zkStateReader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d8ed39/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
index 4e0a795..6cc2314 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
@@ -102,7 +102,9 @@ public class V2Request extends SolrRequest {
      * @return builder object
      */
     public Builder withPayload(String payload) {
-      this.payload = new ByteArrayInputStream(payload.getBytes(StandardCharsets.UTF_8));
+      if (payload != null) {
+        this.payload = new ByteArrayInputStream(payload.getBytes(StandardCharsets.UTF_8));
+      }
       return this;
     }