You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/12/14 11:56:48 UTC

[1/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Repository: lucene-solr
Updated Branches:
  refs/heads/master d66d9549d -> d6d2e3b2e


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
index c285452..3041a13 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
@@ -16,15 +16,15 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Map;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.cloud.ClusterState;
 
-public interface ClusterStateProvider extends Closeable {
+public interface ClusterStateProvider extends SolrCloseable {
 
   /**
    * Obtain the state of the collection (cluster status).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
index d617d16..b623157 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +91,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public CollectionRef getState(String collection) {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         ClusterState cs = fetchClusterState(client, collection, null);
         return cs.getCollectionRef(collection);
@@ -102,7 +103,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
           return null;
         }
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
@@ -160,7 +161,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
     if (TimeUnit.SECONDS.convert((System.nanoTime() - liveNodesTimestamp), TimeUnit.NANOSECONDS) > getCacheTimeout()) {
       for (String nodeName: liveNodes) {
         try (HttpSolrClient client = new HttpSolrClient.Builder().
-            withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+            withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
             withHttpClient(httpClient).build()) {
           Set<String> liveNodes = fetchLiveNodes(client);
           this.liveNodes = (liveNodes);
@@ -168,7 +169,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
           return liveNodes;
         } catch (Exception e) {
           log.warn("Attempt to fetch live_nodes from " +
-              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+              Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
         }
       }
       throw new RuntimeException("Tried fetching live_nodes using all the node names we knew of, i.e. " + liveNodes +". However, "
@@ -209,7 +210,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
         TimeUnit.SECONDS.convert((System.nanoTime() - aliasesTimestamp), TimeUnit.NANOSECONDS) > getCacheTimeout()) {
       for (String nodeName: liveNodes) {
         try (HttpSolrClient client = new HttpSolrClient.Builder().
-            withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+            withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
             withHttpClient(httpClient).build()) {
 
           Map<String, List<String>> aliases = new CollectionAdminRequest.ListAliases().process(client).getAliasesAsLists();
@@ -226,7 +227,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
             return aliases;
           }
           log.warn("Attempt to fetch cluster state from " +
-              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+              Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
         }
       }
 
@@ -244,13 +245,13 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public ClusterState getClusterState() throws IOException {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         ClusterState cs = fetchClusterState(client, null, null);
         return cs;
       } catch (SolrServerException | RemoteSolrException | IOException e) {
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
@@ -264,14 +265,14 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public Map<String, Object> getClusterProperties() {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         Map<String, Object> clusterProperties = new HashMap<>();
         fetchClusterState(client, null, clusterProperties);
         return clusterProperties;
       } catch (SolrServerException | RemoteSolrException | IOException e) {
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
index c67cdc8..4fe345b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
@@ -45,6 +45,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -89,6 +90,11 @@ public class SolrClientCloudManager implements SolrCloudManager {
   }
 
   @Override
+  public TimeSource getTimeSource() {
+    return TimeSource.NANO_TIME;
+  }
+
+  @Override
   public ClusterStateProvider getClusterStateProvider() {
     return solrClient.getClusterStateProvider();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index f2782a3..66f54dd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -67,6 +67,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
   public static SolrClientNodeStateProvider INST;
 
 
+
   private final CloudSolrClient solrClient;
   private final ZkStateReader zkStateReader;
   private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
@@ -155,6 +156,11 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     }
   }
 
+  @Override
+  public void close() throws IOException {
+
+  }
+
   //uses metrics API to get node information
   static class AutoScalingSnitch extends ImplicitSnitch {
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
index 7664ce2..b0cc710 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
@@ -61,9 +61,9 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     try {
-      return zkClient.getChildren(path, null, true);
+      return zkClient.getChildren(path, watcher, true);
     } catch (KeeperException.NoNodeException e) {
       throw new NoSuchElementException(path);
     } catch (InterruptedException e) {
@@ -72,6 +72,11 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+    return listData(path, null);
+  }
+
+  @Override
   public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     Stat stat = new Stat();
     try {
@@ -96,9 +101,22 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    try {
+      zkClient.makePath(path, data, createMode, null, failOnExists, true);
+    } catch (KeeperException.NodeExistsException e) {
+      throw new AlreadyExistsException(path);
+    } catch (InterruptedException e) {
+      throw e;
+    }
+  }
+
+  @Override
+  public String createData(String path, byte[] data, CreateMode mode) throws NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
     try {
       return zkClient.create(path, data, mode, true);
+    } catch (KeeperException.NoNodeException e) {
+      throw new NoSuchElementException(path);
     } catch (KeeperException.NodeExistsException e) {
       throw new AlreadyExistsException(path);
     } catch (InterruptedException e) {
@@ -107,11 +125,13 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public void removeData(String path, int version) throws NoSuchElementException, BadVersionException, IOException, KeeperException, InterruptedException {
     try {
       zkClient.delete(path, version, true);
     } catch (KeeperException.NoNodeException e) {
       throw new NoSuchElementException(path);
+    } catch (KeeperException.BadVersionException e) {
+      throw new BadVersionException(version, path);
     } catch (InterruptedException e) {
       throw e;
     }
@@ -163,4 +183,8 @@ public class ZkDistribStateManager implements DistribStateManager {
     return new AutoScalingConfig(map);
   }
 
+  @Override
+  public void close() throws IOException {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index bfe33c5..4333a00 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -92,6 +92,13 @@ public class ClusterState implements JSONWriter.Writable {
   }
 
   /**
+   * Returns the zNode version that was used to construct this instance.
+   */
+  public int getZNodeVersion() {
+    return znodeVersion;
+  }
+
+  /**
    * Returns true if the specified collection name exists, false otherwise.
    *
    * Implementation note: This method resolves the collection reference by calling
@@ -196,7 +203,10 @@ public class ClusterState implements JSONWriter.Writable {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
+    sb.append("znodeVersion: " + znodeVersion);
+    sb.append("\n");
     sb.append("live nodes:" + liveNodes);
+    sb.append("\n");
     sb.append("collections:" + collectionStates);
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index 16a4c60..02beb97 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -124,6 +124,17 @@ public class Replica extends ZkNodeProps {
     type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    Replica replica = (Replica) o;
+
+    return name.equals(replica.name);
+  }
+
   public String getName() {
     return name;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 4c2be1a..8ab7ecb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -17,9 +17,7 @@
 package org.apache.solr.common.cloud;
 
 import java.io.Closeable;
-import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
-import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -79,6 +77,7 @@ public class ZkStateReader implements Closeable {
   public static final String CORE_NODE_NAME_PROP = "core_node_name";
   public static final String ROLES_PROP = "roles";
   public static final String STATE_PROP = "state";
+  /**  SolrCore name. */
   public static final String CORE_NAME_PROP = "core";
   public static final String COLLECTION_PROP = "collection";
   public static final String ELECTION_NODE_PROP = "election_node";
@@ -949,21 +948,7 @@ public class ZkStateReader implements Closeable {
    * @lucene.experimental
    */
   public String getBaseUrlForNodeName(final String nodeName) {
-    return getBaseUrlForNodeName(nodeName, getClusterProperty(URL_SCHEME, "http"));
-  }
-
-  public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
-    final int _offset = nodeName.indexOf("_");
-    if (_offset < 0) {
-      throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
-    }
-    final String hostAndPort = nodeName.substring(0,_offset);
-    try {
-      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
-      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
-    } catch (UnsupportedEncodingException e) {
-      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
-    }
+    return Utils.getBaseUrlForNodeName(nodeName, getClusterProperty(URL_SCHEME, "http"));
   }
 
   /** Watches a single collection's format2 state.json. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
new file mode 100644
index 0000000..ca535cb
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
@@ -0,0 +1,161 @@
+/*
+ * 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.common.util;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Source of time. NOTE: depending on implementation returned values may not be related in any way to the
+ * current Epoch or calendar time, and they may even be negative - but the API guarantees that they are
+ * always monotonically increasing.
+ */
+public abstract class TimeSource {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /** Implementation that uses {@link System#currentTimeMillis()}. */
+  public static final class CurrentTimeSource extends TimeSource {
+
+    @Override
+    @SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
+    public long getTime() {
+      return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      return toUnit.convert(value, fromUnit);
+    }
+  }
+
+  /** Implementation that uses {@link System#nanoTime()}. */
+  public static final class NanoTimeSource extends TimeSource {
+
+    @Override
+    public long getTime() {
+      return System.nanoTime();
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      return toUnit.convert(value, fromUnit);
+    }
+  }
+
+  /** Implementation that uses {@link #NANO_TIME} accelerated by a double multiplier. */
+  public static final class SimTimeSource extends TimeSource {
+
+    final double multiplier;
+    long start;
+
+    /**
+     * Create a simulated time source that runs faster than real time by a multipler.
+     * @param multiplier must be greater than 0.0
+     */
+    public SimTimeSource(double multiplier) {
+      this.multiplier = multiplier;
+      start = NANO_TIME.getTime();
+    }
+
+    public void advance(long delta) {
+      start = getTime() + delta;
+    }
+
+    @Override
+    public long getTime() {
+      return start + Math.round((double)(NANO_TIME.getTime() - start) * multiplier);
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      ms = Math.round((double)ms / multiplier);
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      long nano = Math.round((double)TimeUnit.NANOSECONDS.convert(value, fromUnit) / multiplier);
+      return toUnit.convert(nano, TimeUnit.NANOSECONDS);
+    }
+  }
+
+  /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
+  public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
+
+  /** This instance uses {@link NanoTimeSource} for generating timestamps. */
+  public static final TimeSource NANO_TIME = new NanoTimeSource();
+
+  private static Map<String, SimTimeSource> simTimeSources = new ConcurrentHashMap<>();
+
+  /**
+   * Obtain an instance of time source.
+   * @param type supported types: <code>currentTime</code>, <code>nanoTime</code> and accelerated
+   *             time with a double factor in the form of <code>simTime:FACTOR</code>, eg.
+   *             <code>simTime:2.5</code>
+   * @return one of the supported types
+   */
+  public static TimeSource get(String type) {
+    if (type == null) {
+      return NANO_TIME;
+    } else if (type.equals("currentTime")) {
+      return CURRENT_TIME;
+    } else if (type.equals("nanoTime")) {
+      return NANO_TIME;
+    } else if (type.startsWith("simTime")) {
+      return simTimeSources.computeIfAbsent(type, t -> {
+        String[] parts = t.split(":");
+        double mul = 1.0;
+        if (parts.length != 2) {
+          log.warn("Invalid simTime specification, assuming multiplier==1.0: '" + type + "'");
+        } else {
+          try {
+            mul = Double.parseDouble(parts[1]);
+          } catch (Exception e) {
+            log.warn("Invalid simTime specification, assuming multiplier==1.0: '" + type + "'");
+          }
+        }
+        return new SimTimeSource(mul);
+      });
+    } else {
+      throw new UnsupportedOperationException("Unsupported time source type '" + type + "'.");
+    }
+  }
+
+  /**
+   * Return a time value, in nanosecond unit.
+   */
+  public abstract long getTime();
+
+  public abstract void sleep(long ms) throws InterruptedException;
+
+  public abstract long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index 93af8c3..4ab24d2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -21,7 +21,9 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
 import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
+import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
@@ -445,12 +447,25 @@ public class Utils {
     }
   }
 
-  public static long time(TimeUnit unit) {
-    return unit.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
+  public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
+    final int _offset = nodeName.indexOf("_");
+    if (_offset < 0) {
+      throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
+    }
+    final String hostAndPort = nodeName.substring(0,_offset);
+    try {
+      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
+      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
+    }
   }
 
-  public static long timeElapsed(long start, TimeUnit unit) {
-    return unit.convert(System.nanoTime() - NANOSECONDS.convert(start, unit), NANOSECONDS);
+  public static long time(TimeSource timeSource, TimeUnit unit) {
+    return unit.convert(timeSource.getTime(), TimeUnit.NANOSECONDS);
   }
 
+  public static long timeElapsed(TimeSource timeSource, long start, TimeUnit unit) {
+    return unit.convert(timeSource.getTime() - NANOSECONDS.convert(start, unit), NANOSECONDS);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
index cda9961..7dc14de 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
@@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
@@ -146,7 +147,7 @@ abstract public class SolrExampleTestsBase extends SolrJettyTestBase {
     Assert.assertEquals(1, rsp.getResults().getNumFound());
     
     // check if the doc has been deleted every 250 ms for 30 seconds
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     do {
       Thread.sleep(250); // wait 250 ms
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
index b3a6518..84aff76 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
@@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -228,7 +229,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
   
   // wait maximum ms for serverName to come back up
   private void waitForServer(int maxSeconds, LBHttpSolrClient client, int nServers, String serverName) throws Exception {
-    final TimeOut timeout = new TimeOut(maxSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(maxSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (! timeout.hasTimedOut()) {
       QueryResponse resp;
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
index 396234b..e89f10e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
@@ -47,6 +47,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.junit.Test;
@@ -1164,6 +1165,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
       public ObjectCache getObjectCache() {
         return objectCache;
       }
+
+      @Override
+      public TimeSource getTimeSource() {
+        return TimeSource.NANO_TIME;
+      }
+
+      @Override
+      public void close() throws IOException {
+
+      }
+
       @Override
       public ClusterStateProvider getClusterStateProvider() {
         return new DelegatingClusterStateProvider(null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index fb79cf5..a597f3d 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -71,6 +71,7 @@ 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.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.Diagnostics;
@@ -1797,7 +1798,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       List<Integer> numShardsNumReplicaList,
       List<String> nodesAllowedToRunShards) throws Exception {
     // check for an expectedSlices new collection - we poll the state
-    final TimeOut timeout = new TimeOut(120, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(120, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     boolean success = false;
     String checkResult = "Didnt get to perform a single check";
     while (! timeout.hasTimedOut()) {
@@ -1856,7 +1857,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
  public static void waitForNon403or404or503(HttpSolrClient collectionClient)
       throws Exception {
     SolrException exp = null;
-    final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (! timeout.hasTimedOut()) {
       boolean missing = false;
@@ -2207,7 +2208,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   static RequestStatusState getRequestStateAfterCompletion(String requestId, int waitForSeconds, SolrClient client)
       throws IOException, SolrServerException {
     RequestStatusState state = null;
-    final TimeOut timeout = new TimeOut(waitForSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(waitForSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (!timeout.hasTimedOut())  {
       state = getRequestState(requestId, client);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index b8873fa..abb6255 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.SolrDispatchFilter;
@@ -698,7 +699,7 @@ public class ChaosMonkey {
    * @param zkStateReader current state reader
    */
   public static void wait(long runLength, String collectionName, ZkStateReader zkStateReader) throws InterruptedException {
-    TimeOut t = new TimeOut(runLength, TimeUnit.MILLISECONDS);
+    TimeOut t = new TimeOut(runLength, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (!t.hasTimedOut()) {
       Thread.sleep(Math.min(1000, t.timeLeft(TimeUnit.MILLISECONDS)));
       logCollectionStateSummary(collectionName, zkStateReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
index f437863..75418c6 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import com.google.common.util.concurrent.AtomicLongMap;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -531,7 +532,7 @@ public class ZkTestServer {
   }
   
   public static boolean waitForServerDown(String hp, long timeoutMs) {
-    final TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    final TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (true) {
       try {
         HostPort hpobj = parseHostPortList(hp).get(0);


[4/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
new file mode 100644
index 0000000..a96a1d5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
@@ -0,0 +1,267 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+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 java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link NodeStateProvider}.
+ * Note: in order to setup node-level metrics use {@link #simSetNodeValues(String, Map)}. However, in order
+ * to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)}.
+ */
+public class SimNodeStateProvider implements NodeStateProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, Map<String, Object>> nodeValues = new ConcurrentHashMap<>();
+  private final SimClusterStateProvider clusterStateProvider;
+  private final SimDistribStateManager stateManager;
+  private final LiveNodesSet liveNodesSet;
+
+  public SimNodeStateProvider(LiveNodesSet liveNodesSet, SimDistribStateManager stateManager,
+                              SimClusterStateProvider clusterStateProvider,
+                              Map<String, Map<String, Object>> nodeValues) {
+    this.liveNodesSet = liveNodesSet;
+    this.stateManager = stateManager;
+    this.clusterStateProvider = clusterStateProvider;
+    if (nodeValues != null) {
+      this.nodeValues.putAll(nodeValues);
+    }
+  }
+
+  // -------- simulator setup methods ------------
+
+  /**
+   * Get a node value
+   * @param node node id
+   * @param key property name
+   * @return property value or null if property or node doesn't exist.
+   */
+  public Object simGetNodeValue(String node, String key) {
+    Map<String, Object> values = nodeValues.get(node);
+    if (values == null) {
+      return null;
+    }
+    return values.get(key);
+  }
+
+  /**
+   * Set node values.
+   * NOTE: if values contain 'nodeRole' key then /roles.json is updated.
+   * @param node node id
+   * @param values values.
+   */
+  public void simSetNodeValues(String node, Map<String, Object> values) {
+    Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    existing.clear();
+    if (values != null) {
+      existing.putAll(values);
+    }
+    if (values == null || values.isEmpty() || values.containsKey("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Set a node value, replacing any previous value.
+   * NOTE: if key is 'nodeRole' then /roles.json is updated.
+   * @param node node id
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetNodeValue(String node, String key, Object value) {
+    Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    if (value == null) {
+      existing.remove(key);
+    } else {
+      existing.put(key, value);
+    }
+    if (key.equals("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Add a node value, creating a list of values if necessary.
+   * NOTE: if key is 'nodeRole' then /roles.json is updated.
+   * @param node node id
+   * @param key property name
+   * @param value property value.
+   */
+  public void simAddNodeValue(String node, String key, Object value) {
+    Map<String, Object> values = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    Object existing = values.get(key);
+    if (existing == null) {
+      values.put(key, value);
+    } else if (existing instanceof Set) {
+      ((Set)existing).add(value);
+    } else {
+      Set<Object> vals = new HashSet<>();
+      vals.add(existing);
+      vals.add(value);
+      values.put(key, vals);
+    }
+    if (key.equals("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Remove node values. If values contained a 'nodeRole' key then
+   * /roles.json is updated.
+   * @param node node id
+   */
+  public void simRemoveNodeValues(String node) {
+    Map<String, Object> values = nodeValues.remove(node);
+    if (values != null && values.containsKey("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Get all node values.
+   */
+  public Map<String, Map<String, Object>> simGetAllNodeValues() {
+    return nodeValues;
+  }
+
+  private synchronized void saveRoles() {
+    final Map<String, Set<String>> roles = new HashMap<>();
+    nodeValues.forEach((n, values) -> {
+      String nodeRole = (String)values.get("nodeRole");
+      if (nodeRole != null) {
+        roles.computeIfAbsent(nodeRole, role -> new HashSet<>()).add(n);
+      }
+    });
+    try {
+      stateManager.setData(ZkStateReader.ROLES, Utils.toJSON(roles), -1);
+    } catch (Exception e) {
+      throw new RuntimeException("Unexpected exception saving roles " + roles, e);
+    }
+  }
+
+  /**
+   * Simulate getting replica metrics values. This uses per-replica properties set in
+   * {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)} and
+   * similar methods.
+   * @param node node id
+   * @param tags metrics names
+   * @return map of metrics names / values
+   */
+  public Map<String, Object> getReplicaMetricsValues(String node, Collection<String> tags) {
+    List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
+    if (replicas == null || replicas.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> values = new HashMap<>();
+    for (String tag : tags) {
+      String[] parts = tag.split(":");
+      if (parts.length < 3 || !parts[0].equals("metrics")) {
+        LOG.warn("Invalid metrics: tag: " + tag);
+        continue;
+      }
+      if (!parts[1].startsWith("solr.core.")) {
+        // skip - this is probably solr.node or solr.jvm metric
+        continue;
+      }
+      String[] collParts = parts[1].substring(10).split("\\.");
+      if (collParts.length != 3) {
+        LOG.warn("Invalid registry name: " + parts[1]);
+        continue;
+      }
+      String collection = collParts[0];
+      String shard = collParts[1];
+      String replica = collParts[2];
+      String key = parts.length > 3 ? parts[2] + ":" + parts[3] : parts[2];
+      replicas.forEach(r -> {
+        if (r.getCollection().equals(collection) && r.getShard().equals(shard) && r.getCore().endsWith(replica)) {
+          Object value = r.getVariables().get(key);
+          if (value != null) {
+            values.put(tag, value);
+          } else {
+            value = r.getVariables().get(tag);
+            if (value != null) {
+              values.put(tag, value);
+            }
+          }
+        }
+      });
+    }
+    return values;
+  }
+
+  // ---------- interface methods -------------
+
+  @Override
+  public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
+    LOG.trace("-- requested values for " + node + ": " + tags);
+    if (!liveNodesSet.contains(node)) {
+      nodeValues.remove(node);
+      return Collections.emptyMap();
+    }
+    if (tags.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> result = new HashMap<>();
+    Map<String, Object> metrics = getReplicaMetricsValues(node, tags.stream().filter(s -> s.startsWith("metrics:solr.core.")).collect(Collectors.toList()));
+    result.putAll(metrics);
+    Map<String, Object> values = nodeValues.get(node);
+    if (values == null) {
+      return result;
+    }
+    result.putAll(values.entrySet().stream().filter(e -> tags.contains(e.getKey())).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
+    return result;
+  }
+
+  @Override
+  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+    List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
+    if (replicas == null || replicas.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Map<String, List<ReplicaInfo>>> res = new HashMap<>();
+    // TODO: probably needs special treatment for "metrics:solr.core..." tags
+    for (ReplicaInfo r : replicas) {
+      Map<String, List<ReplicaInfo>> perCollection = res.computeIfAbsent(r.getCollection(), s -> new HashMap<>());
+      List<ReplicaInfo> perShard = perCollection.computeIfAbsent(r.getShard(), s -> new ArrayList<>());
+      perShard.add(r);
+    }
+    return res;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
new file mode 100644
index 0000000..be7209b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -0,0 +1,251 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CollectionStatePredicate;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
+
+/**
+ * Base class for simulated test cases. Tests that use this class should configure the simulated cluster
+ * in <code>@BeforeClass</code> like this:
+ * <pre>
+ *   @BeforeClass
+ *   public static void setupCluster() throws Exception {
+ *     cluster = configureCluster(5, TimeSource.get("simTime:50"));
+ *   }
+ * </pre>
+ */
+public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int DEFAULT_TIMEOUT = 90;
+
+  /** The cluster. */
+  protected static SimCloudManager cluster;
+
+  protected static void configureCluster(int nodeCount, TimeSource timeSource) throws Exception {
+    cluster = SimCloudManager.createCluster(nodeCount, timeSource);
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws Exception {
+    if (cluster != null) {
+      cluster.close();
+    }
+    cluster = null;
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    if (cluster != null) {
+      // clear any persisted auto scaling configuration
+      cluster.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1);
+      // clean any persisted trigger state or events
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+      cluster.getSimClusterStateProvider().simDeleteAllCollections();
+      cluster.simClearSystemCollection();
+      cluster.getSimClusterStateProvider().simResetLeaderThrottle();
+      cluster.simRestartOverseer(null);
+    }
+  }
+
+  @Before
+  public void checkClusterConfiguration() {
+    if (cluster == null)
+      throw new RuntimeException("SimCloudManager not configured - have you called configureCluster()?");
+  }
+
+  protected void removeChildren(String path) throws Exception {
+    if (!cluster.getDistribStateManager().hasData(path)) {
+      return;
+    }
+    List<String> children = cluster.getDistribStateManager().listData(path);
+    for (String c : children) {
+      if (cluster.getDistribStateManager().hasData(path + "/" + c)) {
+        try {
+          cluster.getDistribStateManager().removeData(path + "/" + c, -1);
+        } catch (NoSuchElementException e) {
+          // ignore
+        }
+      }
+    }
+  }
+
+  /* Cluster helper methods ************************************/
+
+  /**
+   * Get the collection state for a particular collection
+   */
+  protected DocCollection getCollectionState(String collectionName) throws IOException {
+    return cluster.getClusterStateProvider().getClusterState().getCollection(collectionName);
+  }
+
+  /**
+   * Wait for a particular collection state to appear in the cluster client's state reader
+   *
+   * This is a convenience method using the {@link #DEFAULT_TIMEOUT}
+   *
+   * @param message     a message to report on failure
+   * @param collection  the collection to watch
+   * @param predicate   a predicate to match against the collection state
+   */
+  protected long waitForState(String message, String collection, CollectionStatePredicate predicate) {
+    AtomicReference<DocCollection> state = new AtomicReference<>();
+    AtomicReference<Set<String>> liveNodesLastSeen = new AtomicReference<>();
+    try {
+      return waitForState(collection, DEFAULT_TIMEOUT, TimeUnit.SECONDS, (n, c) -> {
+        state.set(c);
+        liveNodesLastSeen.set(n);
+        return predicate.matches(n, c);
+      });
+    } catch (Exception e) {
+      throw new AssertionError(message + "\n" + "Live Nodes: " + liveNodesLastSeen.get() + "\nLast available state: " + state.get(), e);
+    }
+  }
+
+  /**
+   * Block until a CollectionStatePredicate returns true, or the wait times out
+   *
+   * Note that the predicate may be called again even after it has returned true, so
+   * implementors should avoid changing state within the predicate call itself.
+   *
+   * @param collection the collection to watch
+   * @param wait       how long to wait
+   * @param unit       the units of the wait parameter
+   * @param predicate  the predicate to call on state changes
+   * @return number of milliseconds elapsed
+   * @throws InterruptedException on interrupt
+   * @throws TimeoutException on timeout
+   * @throws IOException on watcher register / unregister error
+   */
+  public long waitForState(final String collection, long wait, TimeUnit unit, CollectionStatePredicate predicate)
+      throws InterruptedException, TimeoutException, IOException {
+    TimeOut timeout = new TimeOut(wait, unit, cluster.getTimeSource());
+    long timeWarn = timeout.timeLeft(TimeUnit.MILLISECONDS) / 4;
+    while (!timeout.hasTimedOut()) {
+      ClusterState state = cluster.getClusterStateProvider().getClusterState();
+      DocCollection coll = state.getCollectionOrNull(collection);
+      // due to the way we manage collections in SimClusterStateProvider a null here
+      // can mean that a collection is still being created but has no replicas
+      if (coll == null) { // does not yet exist?
+        timeout.sleep(50);
+        continue;
+      }
+      if (predicate.matches(state.getLiveNodes(), coll)) {
+        log.trace("-- predicate matched with state {}", state);
+        return timeout.timeElapsed(TimeUnit.MILLISECONDS);
+      }
+      timeout.sleep(50);
+      if (timeout.timeLeft(TimeUnit.MILLISECONDS) < timeWarn) {
+        log.trace("-- still not matching predicate: {}", state);
+      }
+    }
+    throw new TimeoutException();
+  }
+
+  /**
+   * Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
+   * number of shards and replicas
+   */
+  public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) {
+    return (liveNodes, collectionState) -> {
+      if (collectionState == null)
+        return false;
+      if (collectionState.getSlices().size() != expectedShards)
+        return false;
+      for (Slice slice : collectionState) {
+        int activeReplicas = 0;
+        for (Replica replica : slice) {
+          if (replica.isActive(liveNodes))
+            activeReplicas++;
+        }
+        if (activeReplicas != expectedReplicas)
+          return false;
+      }
+      return true;
+    };
+  }
+
+  /**
+   * Get a (reproducibly) random shard from a {@link DocCollection}
+   */
+  protected static Slice getRandomShard(DocCollection collection) {
+    List<Slice> shards = new ArrayList<>(collection.getActiveSlices());
+    if (shards.size() == 0)
+      fail("Couldn't get random shard for collection as it has no shards!\n" + collection.toString());
+    Collections.shuffle(shards, random());
+    return shards.get(0);
+  }
+
+  /**
+   * Get a (reproducibly) random replica from a {@link Slice}
+   */
+  protected static Replica getRandomReplica(Slice slice) {
+    List<Replica> replicas = new ArrayList<>(slice.getReplicas());
+    if (replicas.size() == 0)
+      fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
+    Collections.shuffle(replicas, random());
+    return replicas.get(0);
+  }
+
+  /**
+   * Get a (reproducibly) random replica from a {@link Slice} matching a predicate
+   */
+  protected static Replica getRandomReplica(Slice slice, Predicate<Replica> matchPredicate) {
+    List<Replica> replicas = new ArrayList<>(slice.getReplicas());
+    if (replicas.size() == 0)
+      fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
+    Collections.shuffle(replicas, random());
+    for (Replica replica : replicas) {
+      if (matchPredicate.test(replica))
+        return replica;
+    }
+    fail("Couldn't get random replica that matched conditions\n" + slice.toString());
+    return null;  // just to keep the compiler happy - fail will always throw an Exception
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
new file mode 100644
index 0000000..396edea
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
@@ -0,0 +1,221 @@
+/*
+ * 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.sim;
+
+import java.lang.invoke.MethodHandles;
+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 java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.Watcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test compares the cluster state of a real cluster and a simulated one.
+ */
+public class TestClusterStateProvider extends SolrCloudTestCase {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static int NODE_COUNT = 3;
+  private static boolean simulated;
+
+  private static SolrCloudManager cloudManager;
+
+  private static Collection<String> liveNodes;
+  private static Map<String, Object> clusterProperties;
+  private static AutoScalingConfig autoScalingConfig;
+  private static Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicas;
+  private static Map<String, Map<String, Object>> nodeValues;
+  private static ClusterState realState;
+
+  // set up a real cluster as the source of test data
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    simulated = random().nextBoolean();
+    LOG.info("####### Using simulated components? " + simulated);
+
+    configureCluster(NODE_COUNT)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 2, 0, 1)
+        .process(cluster.getSolrClient());
+    init();
+  }
+
+  @AfterClass
+  public static void closeCloudManager() throws Exception {
+    if (simulated && cloudManager != null) {
+      cloudManager.close();
+    }
+  }
+
+  private static void init() throws Exception {
+    SolrCloudManager realManager = cluster.getJettySolrRunner(cluster.getJettySolrRunners().size() - 1).getCoreContainer()
+        .getZkController().getSolrCloudManager();
+    liveNodes = realManager.getClusterStateProvider().getLiveNodes();
+    clusterProperties = realManager.getClusterStateProvider().getClusterProperties();
+    autoScalingConfig = realManager.getDistribStateManager().getAutoScalingConfig();
+    replicas = new HashMap<>();
+    nodeValues = new HashMap<>();
+    liveNodes.forEach(n -> {
+      replicas.put(n, realManager.getNodeStateProvider().getReplicaInfo(n, Collections.emptySet()));
+      nodeValues.put(n, realManager.getNodeStateProvider().getNodeValues(n, ImplicitSnitch.tags));
+    });
+    realState = realManager.getClusterStateProvider().getClusterState();
+
+    if (simulated) {
+      // initialize simulated provider
+      SimCloudManager simCloudManager = new SimCloudManager(TimeSource.get("simTime:10"));
+      simCloudManager.getSimClusterStateProvider().simSetClusterProperties(clusterProperties);
+      simCloudManager.getSimDistribStateManager().simSetAutoScalingConfig(autoScalingConfig);
+      nodeValues.forEach((n, values) -> {
+        simCloudManager.getSimNodeStateProvider().simSetNodeValues(n, values);
+      });
+      simCloudManager.getSimClusterStateProvider().simSetClusterState(realState);
+      ClusterState simState = simCloudManager.getClusterStateProvider().getClusterState();
+      assertClusterStateEquals(realState, simState);
+      cloudManager = simCloudManager;
+    } else {
+      cloudManager = realManager;
+    }
+  }
+
+  private static void assertClusterStateEquals(ClusterState one, ClusterState two) {
+    assertEquals(one.getLiveNodes(), two.getLiveNodes());
+    assertEquals(one.getCollectionsMap().keySet(), two.getCollectionsMap().keySet());
+    one.forEachCollection(oneColl -> {
+      DocCollection twoColl = two.getCollection(oneColl.getName());
+      Map<String, Slice> oneSlices = oneColl.getSlicesMap();
+      Map<String, Slice> twoSlices = twoColl.getSlicesMap();
+      assertEquals(oneSlices.keySet(), twoSlices.keySet());
+      oneSlices.forEach((s, slice) -> {
+        Slice sTwo = twoSlices.get(s);
+        for (Replica oneReplica : slice.getReplicas()) {
+          Replica twoReplica = sTwo.getReplica(oneReplica.getName());
+          assertNotNull(twoReplica);
+          assertEquals(oneReplica, twoReplica);
+        }
+      });
+    });
+  }
+
+  private String addNode() throws Exception {
+    JettySolrRunner solr = cluster.startJettySolrRunner();
+    String nodeId = solr.getNodeName();
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimClusterStateProvider().simAddNode(nodeId);
+    }
+    return nodeId;
+  }
+
+  private String deleteNode() throws Exception {
+    String nodeId = cluster.getJettySolrRunner(0).getNodeName();
+    cluster.stopJettySolrRunner(0);
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimClusterStateProvider().simRemoveNode(nodeId);
+    }
+    return nodeId;
+  }
+
+  private void setAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
+    cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getZkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH,
+        Utils.toJSON(cfg), -1, true);
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimDistribStateManager().simSetAutoScalingConfig(cfg);
+    }
+  }
+
+  @Test
+  public void testAddRemoveNode() throws Exception {
+    Set<String> lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    List<String> liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());
+
+    String node = addNode();
+    cloudManager.getTimeSource().sleep(2000);
+    assertFalse(lastNodes.contains(node));
+    lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    assertTrue(lastNodes.contains(node));
+    liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());
+
+    node = deleteNode();
+    cloudManager.getTimeSource().sleep(2000);
+    assertTrue(lastNodes.contains(node));
+    lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    assertFalse(lastNodes.contains(node));
+    liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());  }
+
+  @Test
+  public void testAutoScalingConfig() throws Exception {
+    final CountDownLatch triggered = new CountDownLatch(1);
+    Watcher w = ev -> {
+      if (triggered.getCount() == 0) {
+        fail("already triggered once!");
+      }
+      triggered.countDown();
+    };
+    AutoScalingConfig cfg = cloudManager.getDistribStateManager().getAutoScalingConfig(w);
+    assertEquals(autoScalingConfig, cfg);
+    Preference p = new Preference(Collections.singletonMap("maximize", "freedisk"));
+    cfg = cfg.withPolicy(cfg.getPolicy().withClusterPreferences(Collections.singletonList(p)));
+    setAutoScalingConfig(cfg);
+    if (!triggered.await(10, TimeUnit.SECONDS)) {
+      fail("Watch should be triggered on update!");
+    }
+    AutoScalingConfig cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
+    assertEquals(cfg, cfg1);
+
+    // restore
+    setAutoScalingConfig(autoScalingConfig);
+    cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
+    assertEquals(autoScalingConfig, cfg1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
new file mode 100644
index 0000000..b7053d7d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
@@ -0,0 +1,357 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+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.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.ComputePlanAction;
+import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
+
+/**
+ * Test for {@link ComputePlanAction}
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG;")
+public class TestComputePlanAction extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final AtomicBoolean fired = new AtomicBoolean(false);
+  private static final int NODE_COUNT = 1;
+  private static CountDownLatch triggerFiredLatch = new CountDownLatch(1);
+  private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
+  private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1, TimeSource.get("simTime:50"));
+  }
+
+  @Before
+  public void init() throws Exception {
+
+    fired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    actionContextPropsRef.set(null);
+
+    if (cluster.getClusterStateProvider().getLiveNodes().size() > NODE_COUNT) {
+      // stop some to get to original state
+      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
+      for (int i = 0; i < numJetties - NODE_COUNT; i++) {
+        String node = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+        cluster.getSimClusterStateProvider().simRemoveNode(node);
+      }
+    }
+
+    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);
+    SolrResponse rsp = cluster.request(req);
+    NamedList<Object> response = rsp.getResponse();
+    assertEquals(response.get("result").toString(), "success");
+
+    String setClusterPreferencesCommand = "{" +
+        "'set-cluster-preferences': [" +
+        "{'minimize': 'cores'}," +
+        "{'maximize': 'freedisk','precision': 100}]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
+    rsp = cluster.request(req);
+    response = rsp.getResponse();
+    assertEquals(response.get("result").toString(), "success");
+    cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
+  }
+
+  @After
+  public void printState() throws Exception {
+    log.info("-------------_ FINAL STATE --------------");
+    log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cluster.getClusterStateProvider().getClusterState();
+    for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
+      log.info("* Collection " + coll + " state: " + state.getCollection(coll));
+    }
+
+  }
+
+  @Test
+  public void testNodeLost() throws Exception  {
+    // let's start a node so that we have at least two
+    String node = cluster.simAddNode();
+    AssertingTriggerAction.expectedNode = node;
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '7s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeLost",
+        "conf",1, 2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeLost", clusterShape(1, 2));
+
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    log.debug("-- cluster state: {}", clusterState);
+    DocCollection collection = clusterState.getCollection("testNodeLost");
+    List<Replica> replicas = collection.getReplicas(node);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    // start another node because because when the other node goes away, the cluster policy requires only
+    // 1 replica per node and none on the overseer
+    String node2 = cluster.simAddNode();
+    assertTrue(node2 + "is not live yet", cluster.getClusterStateProvider().getClusterState().liveNodesContain(node2) );
+
+    // stop the original node
+    cluster.simRemoveNode(node, false);
+    log.info("Stopped_node : {}", node);
+
+    assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null , " + eventRef.get(), operations);
+    assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
+    SolrRequest solrRequest = operations.get(0);
+    SolrParams params = solrRequest.getParams();
+    assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+    String replicaToBeMoved = params.get("replica");
+    assertEquals("Unexpected node in computed operation", replicas.get(0).getName(), replicaToBeMoved);
+
+    // shutdown the extra node that we had started
+    cluster.simRemoveNode(node2, false);
+  }
+
+  public void testNodeWithMultipleReplicasLost() throws Exception {
+    AssertingTriggerAction.expectedNode = null;
+
+    // start 3 more nodes
+    cluster.simAddNode();
+    cluster.simAddNode();
+    cluster.simAddNode();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeWithMultipleReplicasLost",
+        "conf",2, 3);
+//    create.setMaxShardsPerNode(2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeWithMultipleReplicasLost", clusterShape(2, 3));
+
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    log.debug("-- cluster state: {}", clusterState);
+    DocCollection docCollection = clusterState.getCollection("testNodeWithMultipleReplicasLost");
+
+    // lets find a node with at least 2 replicas
+    String stoppedNodeName = null;
+    List<Replica> replicasToBeMoved = null;
+    for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
+      List<Replica> replicas = docCollection.getReplicas(node);
+      if (replicas != null && replicas.size() == 2) {
+        stoppedNodeName = node;
+        replicasToBeMoved = replicas;
+        cluster.simRemoveNode(node, false);
+        break;
+      }
+    }
+    assertNotNull(stoppedNodeName);
+
+    assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+
+    TriggerEvent triggerEvent = eventRef.get();
+    assertNotNull(triggerEvent);
+    assertEquals(TriggerEventType.NODELOST, triggerEvent.getEventType());
+    // TODO assertEquals(stoppedNodeName, triggerEvent.getProperty(TriggerEvent.NODE_NAME));
+
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get(), operations);
+    operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
+    assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
+
+    for (SolrRequest solrRequest : operations) {
+      SolrParams params = solrRequest.getParams();
+      assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+      String moved = params.get("replica");
+      assertTrue(replicasToBeMoved.stream().anyMatch(replica -> replica.getName().equals(moved)));
+    }
+  }
+
+  @Test
+  public void testNodeAdded() throws Exception {
+    AssertingTriggerAction.expectedNode = null;
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // the default policy limits 1 replica per node, we need more right now
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<3', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeAdded",
+        "conf",1, 2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeAdded", (liveNodes, collectionState) -> collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
+
+    // reset to the original policy which has only 1 replica per shard per node
+    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");
+
+    // start a node so that the 'violation' created by the previous policy update is fixed
+    String newNode = cluster.simAddNode();
+    assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    log.info("Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("Live nodes: " + cluster.getClusterStateProvider().getLiveNodes() + ", collection state: " + cluster.getClusterStateProvider().getClusterState().getCollection("testNodeAdded"));
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null" + context, operations);
+    assertEquals("ComputePlanAction should have computed exactly 1 operation, but was: " + operations, 1, operations.size());
+    SolrRequest request = operations.get(0);
+    SolrParams params = request.getParams();
+    assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+    String nodeAdded = params.get("targetNode");
+    assertEquals("Unexpected node in computed operation", newNode, nodeAdded);
+  }
+
+  public static class AssertingTriggerAction implements TriggerAction {
+    static String expectedNode;
+
+    @Override
+    public String getName() {
+      return null;
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) {
+      if (expectedNode != null) {
+        Collection nodes = (Collection) event.getProperty(TriggerEvent.NODE_NAMES);
+        if (nodes == null || !nodes.contains(expectedNode)) return;//this is not the event we are looking for
+      }
+      if (fired.compareAndSet(false, true)) {
+        eventRef.set(event);
+        actionContextPropsRef.set(context.getProperties());
+        triggerFiredLatch.countDown();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
new file mode 100644
index 0000000..a9c5140
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
@@ -0,0 +1,284 @@
+/*
+ * 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.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
+import org.apache.solr.cloud.ZkTestServer;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test compares a ZK-based {@link DistribStateManager} to the simulated one.
+ */
+public class TestDistribStateManager extends SolrTestCaseJ4 {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private DistribStateManager stateManager;
+  private ZkTestServer zkTestServer;
+  private SolrZkClient solrZkClient;
+  private boolean simulated;
+  private SimDistribStateManager.Node root;
+
+  @Before
+  public void setup() throws Exception {
+    simulated = random().nextBoolean();
+    if (simulated) {
+      root = SimDistribStateManager.createNewRootNode();
+    } else {
+      zkTestServer = new ZkTestServer(createTempDir("zkDir").toString());
+      zkTestServer.run();
+    }
+    reInit();
+  }
+
+  private void reInit() throws Exception {
+    if (stateManager != null) {
+      stateManager.close();
+    }
+    if (simulated) {
+      stateManager = new SimDistribStateManager(root);
+    } else {
+      if (solrZkClient != null) {
+        solrZkClient.close();
+      }
+      solrZkClient = new SolrZkClient(zkTestServer.getZkHost(), 30000);
+      stateManager = new ZkDistribStateManager(solrZkClient);
+    }
+    LOG.info("Using " + stateManager.getClass().getName());
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (solrZkClient != null) {
+      solrZkClient.close();
+      solrZkClient = null;
+    }
+    if (zkTestServer != null) {
+      zkTestServer.shutdown();
+      zkTestServer = null;
+    }
+    if (stateManager != null) {
+      stateManager.close();
+    }
+    stateManager = null;
+  }
+
+  @Test
+  public void testHasData() throws Exception {
+    assertFalse(stateManager.hasData("/hasData/foo"));
+    assertFalse(stateManager.hasData("/hasData/bar"));
+    try {
+      stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    stateManager.makePath("/hasData");
+    stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
+    stateManager.createData("/hasData/bar", new byte[0], CreateMode.PERSISTENT);
+    assertTrue(stateManager.hasData("/hasData/foo"));
+    assertTrue(stateManager.hasData("/hasData/bar"));
+  }
+
+  @Test
+  public void testListData() throws Exception {
+    assertFalse(stateManager.hasData("/listData/foo"));
+    assertFalse(stateManager.hasData("/listData/foo/bar"));
+    try {
+      stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+      fail("should not succeed");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    try {
+      stateManager.listData("/listData/foo");
+      fail("should not succeed");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    stateManager.makePath("/listData");
+    List<String> kids = stateManager.listData("/listData");
+    assertEquals(0, kids.size());
+    stateManager.makePath("/listData/foo");
+    kids = stateManager.listData("/listData");
+    assertEquals(1, kids.size());
+    assertEquals("foo", kids.get(0));
+    stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+    stateManager.createData("/listData/foo/baz", new byte[0], CreateMode.PERSISTENT);
+    kids = stateManager.listData("/listData/foo");
+    assertEquals(2, kids.size());
+    assertTrue(kids.contains("bar"));
+    assertTrue(kids.contains("baz"));
+    try {
+      stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+      fail("should not succeed");
+    } catch (AlreadyExistsException e) {
+      // expected
+    }
+  }
+
+  static final byte[] firstData = new byte[] {
+      (byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe
+  };
+
+  static final byte[] secondData = new byte[] {
+      (byte)0xbe, (byte)0xba, (byte)0xfe, (byte)0xca
+  };
+
+  @Test
+  public void testCreateMode() throws Exception {
+    stateManager.makePath("/createMode");
+    stateManager.createData("/createMode/persistent", firstData, CreateMode.PERSISTENT);
+    stateManager.createData("/createMode/persistent_seq", firstData, CreateMode.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+    }
+    // check what happens with gaps
+    stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+    stateManager.removeData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10), -1);
+    stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+
+    stateManager.createData("/createMode/ephemeral", firstData, CreateMode.EPHEMERAL);
+    stateManager.createData("/createMode/ephemeral_seq", firstData, CreateMode.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      stateManager.createData("/createMode/ephemeral_seq/data", firstData, CreateMode.EPHEMERAL_SEQUENTIAL);
+    }
+    assertTrue(stateManager.hasData("/createMode"));
+    assertTrue(stateManager.hasData("/createMode/persistent"));
+    assertTrue(stateManager.hasData("/createMode/ephemeral"));
+    List<String> kids = stateManager.listData("/createMode/persistent_seq");
+    assertEquals(11, kids.size());
+    kids = stateManager.listData("/createMode/ephemeral_seq");
+    assertEquals(10, kids.size());
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    assertFalse(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10)));
+    assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
+
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/ephemeral_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    // check that ephemeral nodes disappear on disconnect
+    reInit();
+    assertTrue(stateManager.hasData("/createMode/persistent"));
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
+
+    assertFalse(stateManager.hasData("/createMode/ephemeral"));
+    assertTrue(stateManager.hasData("/createMode/ephemeral_seq"));
+    kids = stateManager.listData("/createMode/ephemeral_seq");
+    assertEquals(0, kids.size());
+  }
+
+  static class OnceWatcher implements Watcher {
+    CountDownLatch triggered = new CountDownLatch(1);
+    WatchedEvent event;
+
+    @Override
+    public void process(WatchedEvent event) {
+      if (triggered.getCount() == 0) {
+        fail("Watch was already triggered once!");
+      }
+      triggered.countDown();
+      this.event = event;
+    }
+  }
+
+  @Test
+  public void testGetSetRemoveData() throws Exception {
+    stateManager.makePath("/getData");
+    stateManager.createData("/getData/persistentData", firstData, CreateMode.PERSISTENT);
+    OnceWatcher nodeWatcher = new OnceWatcher();
+    VersionedData vd = stateManager.getData("/getData/persistentData", nodeWatcher);
+    assertNotNull(vd);
+    assertEquals(0, vd.getVersion());
+    assertTrue(Arrays.equals(firstData, vd.getData()));
+
+    // update data, test versioning
+    try {
+      stateManager.setData("/getData/persistentData", secondData, 1);
+      fail("should have failed");
+    } catch (BadVersionException e) {
+      // expected
+    }
+    // watch should not have fired
+    assertEquals(1, nodeWatcher.triggered.getCount());
+
+    stateManager.setData("/getData/persistentData", secondData, 0);
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+    // watch should not fire now because it needs to be reset
+    stateManager.setData("/getData/persistentData", secondData, -1);
+
+    nodeWatcher = new OnceWatcher();
+    stateManager.createData("/getData/ephemeralData", firstData, CreateMode.EPHEMERAL);
+    vd = stateManager.getData("/getData/ephemeralData", nodeWatcher);
+    reInit();
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+    assertTrue(stateManager.hasData("/getData/persistentData"));
+    assertFalse(stateManager.hasData("/getData/ephemeralData"));
+
+    nodeWatcher = new OnceWatcher();
+    vd = stateManager.getData("/getData/persistentData", nodeWatcher);
+    // try wrong version
+    try {
+      stateManager.removeData("/getData/persistentData", vd.getVersion() - 1);
+      fail("should have failed");
+    } catch (BadVersionException e) {
+      // expected
+    }
+    // watch should not have fired
+    assertEquals(1, nodeWatcher.triggered.getCount());
+
+    stateManager.removeData("/getData/persistentData", vd.getVersion());
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+  }
+
+  @Test
+  public void testMulti() throws Exception {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
new file mode 100644
index 0000000..18d76dc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
@@ -0,0 +1,216 @@
+/*
+ * 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.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
+import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test for {@link ExecutePlanAction}
+ */
+@LogLevel("org.apache.solr.cloud=DEBUG")
+public class TestExecutePlanAction extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final int NODE_COUNT = 2;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
+  }
+
+  @Before
+  public void setUp() throws Exception  {
+    super.setUp();
+
+    if (cluster.getClusterStateProvider().getLiveNodes().size() < NODE_COUNT) {
+      // start some to get to original state
+      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
+      for (int i = 0; i < NODE_COUNT - numJetties; i++) {
+        cluster.simAddNode();
+      }
+    }
+  }
+
+  @After
+  public void printState() throws Exception {
+    log.info("-------------_ FINAL STATE --------------");
+    log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cluster.getClusterStateProvider().getClusterState();
+    for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
+      log.info("* Collection " + coll + " state: " + state.getCollection(coll));
+    }
+
+  }
+
+  @Test
+  public void testExecute() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String collectionName = "testExecute";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 1, 2);
+    create.setMaxShardsPerNode(1);
+    create.process(solrClient);
+
+    log.info("Collection ready after " + waitForState(collectionName, 120, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
+
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    DocCollection docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
+        .filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
+    assertFalse(otherNodes.isEmpty());
+    String survivor = otherNodes.get(0);
+
+    try (ExecutePlanAction action = new ExecutePlanAction()) {
+      action.init(Collections.singletonMap("name", "execute_plan"));
+
+      // used to signal if we found that ExecutePlanAction did in fact create the right znode before executing the operation
+      AtomicBoolean znodeCreated = new AtomicBoolean(false);
+
+      CollectionAdminRequest.AsyncCollectionAdminRequest moveReplica = new CollectionAdminRequest.MoveReplica(collectionName, replicas.get(0).getName(), survivor);
+      CollectionAdminRequest.AsyncCollectionAdminRequest mockRequest = new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.OVERSEERSTATUS) {
+        @Override
+        public void setAsyncId(String asyncId) {
+          super.setAsyncId(asyncId);
+          String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/xyz/execute_plan";
+          try {
+            if (cluster.getDistribStateManager().hasData(parentPath)) {
+              java.util.List<String> children = cluster.getDistribStateManager().listData(parentPath);
+              if (!children.isEmpty()) {
+                String child = children.get(0);
+                VersionedData data = cluster.getDistribStateManager().getData(parentPath + "/" + child);
+                Map m = (Map) Utils.fromJSON(data.getData());
+                if (m.containsKey("requestid")) {
+                  znodeCreated.set(m.get("requestid").equals(asyncId));
+                }
+              }
+            }
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+
+        }
+      };
+      List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
+      NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
+          "mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTime()),
+          Collections.singletonList(sourceNodeName));
+      ActionContext actionContext = new ActionContext(cluster, null,
+          new HashMap<>(Collections.singletonMap("operations", operations)));
+      action.process(nodeLostEvent, actionContext);
+
+//      assertTrue("ExecutePlanAction should have stored the requestid in ZK before executing the request", znodeCreated.get());
+      List<NamedList<Object>> responses = (List<NamedList<Object>>) actionContext.getProperty("responses");
+      assertNotNull(responses);
+      assertEquals(2, responses.size());
+      NamedList<Object> response = responses.get(0);
+      assertNull(response.get("failure"));
+      assertNotNull(response.get("success"));
+    }
+
+    log.info("Collection ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
+  }
+
+  @Test
+  public void testIntegration() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'execute_plan','class':'solr.ExecutePlanAction'}]" +
+        "}}";
+    SolrRequest req = AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String collectionName = "testIntegration";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 1, 2);
+    create.setMaxShardsPerNode(1);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        collectionName, clusterShape(1, 2));
+
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    DocCollection docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
+        .filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
+    assertFalse(otherNodes.isEmpty());
+    String survivor = otherNodes.get(0);
+
+    cluster.simRemoveNode(sourceNodeName, false);
+
+    waitForState("Timed out waiting for replicas of collection to be 2 again",
+        collectionName, clusterShape(1, 2));
+
+    clusterState = cluster.getClusterStateProvider().getClusterState();
+    docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicasOnSurvivor = docCollection.getReplicas(survivor);
+    assertNotNull(replicasOnSurvivor);
+    assertEquals(2, replicasOnSurvivor.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
new file mode 100644
index 0000000..cba700b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sim;
+
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+
+/**
+ *
+ */
+public class TestGenericDistributedQueue extends TestSimDistributedQueue {
+  DistribStateManager stateManager = new SimDistribStateManager();
+
+  @Override
+  protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
+    return new GenericDistributedQueue(stateManager, dqZNode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
new file mode 100644
index 0000000..034a039
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -0,0 +1,266 @@
+/*
+ * 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.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.ComputePlanAction;
+import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
+import org.apache.solr.cloud.autoscaling.TriggerActionBase;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.cloud.autoscaling.CapturedEvent;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.LogLevel;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
+/**
+ *
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+public class TestLargeCluster extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SPEED = 50;
+
+  public static final int NUM_NODES = 100;
+
+  static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
+  static AtomicInteger triggerFiredCount = new AtomicInteger();
+  static CountDownLatch triggerFiredLatch;
+  static int waitForSeconds;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+
+    waitForSeconds = 1 + random().nextInt(3);
+    triggerFiredCount.set(0);
+    triggerFiredLatch = new CountDownLatch(1);
+    listenerEvents.clear();
+    while (cluster.getClusterStateProvider().getLiveNodes().size() < NUM_NODES) {
+      // perhaps a test stopped a node but didn't start it back
+      // lets start a node
+      cluster.simAddNode();
+    }
+  }
+
+  public static class TestTriggerListener extends TriggerListenerBase {
+    @Override
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
+    }
+
+    @Override
+    public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
+                                     ActionContext context, Throwable error, String message) {
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(cluster.getTimeSource().getTime(), context, config, stage, actionName, event, message));
+    }
+  }
+
+  public static class TestTriggerAction extends TriggerActionBase {
+    @Override
+    public void process(TriggerEvent event, ActionContext context) throws Exception {
+      triggerFiredCount.incrementAndGet();
+      triggerFiredLatch.countDown();
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'foo'," +
+        "'trigger' : 'node_lost_trigger'," +
+        "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+        "'beforeAction' : ['compute', 'execute']," +
+        "'afterAction' : ['compute', 'execute']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    cluster.getTimeSource().sleep(5000);
+
+    // pick a few random nodes
+    List<String> nodes = new ArrayList<>();
+    int limit = 75;
+    for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
+      nodes.add(node);
+      if (nodes.size() > limit) {
+        break;
+      }
+    }
+    Collections.shuffle(nodes, random());
+    // create collection on these nodes
+    String collectionName = "testBasic";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 5, 5, 5, 5);
+    create.setMaxShardsPerNode(1);
+    create.setCreateNodeSet(String.join(",", nodes));
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+
+    int KILL_NODES = 8;
+    // kill off a number of nodes
+    for (int i = 0; i < KILL_NODES; i++) {
+      cluster.simRemoveNode(nodes.get(i), false);
+    }
+    // should fully recover
+    log.info("Ready after " + waitForState(collectionName, 90 * KILL_NODES, TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+
+    log.info("OP COUNTS: " + cluster.simGetOpCounts());
+    long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+
+    // simulate a number of flaky nodes
+    int FLAKY_NODES = 10;
+    int flakyReplicas = 0;
+    for (int cnt = 0; cnt < 10; cnt++) {
+      for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+        flakyReplicas += cluster.getSimClusterStateProvider().simGetReplicaInfos(nodes.get(i))
+            .stream().filter(r -> r.getState().equals(Replica.State.ACTIVE)).count();
+        cluster.simRemoveNode(nodes.get(i), false);
+      }
+      cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(waitForSeconds) * 2);
+      for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+        final String nodeId = nodes.get(i);
+        cluster.submit(() -> cluster.getSimClusterStateProvider().simRestoreNode(nodeId));
+      }
+    }
+
+    log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+    log.info("OP COUNTS: " + cluster.simGetOpCounts());
+    long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+    log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
+    // flaky nodes lead to a number of MOVEREPLICA that is non-zero but lower than the number of flaky replicas
+    assertTrue("there should be new MOVERPLICA ops", newMoveReplicaOps - moveReplicaOps > 0);
+    assertTrue("there should be less than flakyReplicas=" + flakyReplicas + " MOVEREPLICA ops",
+        newMoveReplicaOps - moveReplicaOps < flakyReplicas);
+  }
+
+  @Test
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
+  public void testSearchRate() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'search_rate_trigger'," +
+        "'event' : 'searchRate'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'rate' : 1.0," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'srt'," +
+        "'trigger' : 'search_rate_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String collectionName = "testSearchRate";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 2, 10);
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(2, 10)) + " ms");
+
+    // collect the node names
+    Set<String> nodes = new HashSet<>();
+    cluster.getSimClusterStateProvider().getClusterState().getCollection(collectionName)
+        .getReplicas()
+        .forEach(r -> nodes.add(r.getNodeName()));
+
+    String metricName = "QUERY./select.requestTimes:1minRate";
+    // simulate search traffic
+    cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, true);
+
+    Thread.sleep(1000000000);
+//    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+//    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+    assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
+  }
+}


[7/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeOut.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeOut.java b/solr/core/src/java/org/apache/solr/util/TimeOut.java
index fd91045..bcc29961 100644
--- a/solr/core/src/java/org/apache/solr/util/TimeOut.java
+++ b/solr/core/src/java/org/apache/solr/util/TimeOut.java
@@ -18,26 +18,34 @@ package org.apache.solr.util;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
+
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
 public class TimeOut {
 
   private final long timeoutAt, startTime;
+  private final TimeSource timeSource;
 
-  public TimeOut(long interval, TimeUnit unit) {
-    startTime = System.nanoTime();
+  public TimeOut(long interval, TimeUnit unit, TimeSource timeSource) {
+    this.timeSource = timeSource;
+    startTime = timeSource.getTime();
     this.timeoutAt = startTime + NANOSECONDS.convert(interval, unit);
   }
 
   public boolean hasTimedOut() {
-    return System.nanoTime() > timeoutAt;
+    return timeSource.getTime() > timeoutAt;
+  }
+
+  public void sleep(long ms) throws InterruptedException {
+    timeSource.sleep(ms);
   }
 
   public long timeLeft(TimeUnit unit) {
-    return unit.convert(timeoutAt - System.nanoTime(), NANOSECONDS);
+    return unit.convert(timeoutAt - timeSource.getTime(), NANOSECONDS);
   }
 
   public long timeElapsed(TimeUnit unit) {
-    return unit.convert(System.nanoTime() - startTime, NANOSECONDS);
+    return unit.convert(timeSource.getTime() - startTime, NANOSECONDS);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeSource.java b/solr/core/src/java/org/apache/solr/util/TimeSource.java
deleted file mode 100644
index a0c7bc0..0000000
--- a/solr/core/src/java/org/apache/solr/util/TimeSource.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.concurrent.TimeUnit;
-
-import org.apache.solr.common.util.SuppressForbidden;
-
-/**
- * Source of timestamps.
- */
-public abstract class TimeSource {
-
-  /** Implementation that uses {@link System#currentTimeMillis()}. */
-  public static final class CurrentTimeSource extends TimeSource {
-
-    @Override
-    @SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
-    public long getTime() {
-      return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
-    }
-  }
-
-  /** Implementation that uses {@link System#nanoTime()}. */
-  public static final class NanoTimeSource extends TimeSource {
-
-    @Override
-    public long getTime() {
-      return System.nanoTime();
-    }
-  }
-
-  /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
-  public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
-
-  /** This instance uses {@link NanoTimeSource} for generating timestamps. */
-  public static final TimeSource NANO_TIME = new NanoTimeSource();
-
-  /**
-   * Return a timestamp, in nanosecond unit.
-   */
-  public abstract long getTime();
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
index 76b7285..05c6c9f 100644
--- a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
+++ b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -120,7 +121,7 @@ public class TransformerProvider {
     
     lastFilename = filename;
     lastTemplates = result;
-    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
+    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
index 5463f5b..21a53d8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
@@ -21,7 +21,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Test;
 
 public class ActionThrottleTest extends SolrTestCaseJ4 {
@@ -40,6 +40,16 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
       return returnValues.get(index++);
     }
 
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   // use the same time source as ActionThrottle

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
index 99d785a..07b59aa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
@@ -73,7 +73,7 @@ public class AddReplicaTest extends SolrCloudTestCase {
         success = true;
         break;
       }
-      assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
+      assertFalse(rsp.toString(), rsp.getRequestStatus() == RequestStatusState.FAILED);
       Thread.sleep(500);
     }
     assertTrue(success);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 21c001c..cf26de4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -75,11 +75,11 @@ public class AssignTest extends SolrTestCaseJ4 {
         zkClientData.get(invocation.getArgument(0)));
     // TODO: fix this to be independent of ZK
     ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
-    String nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    String nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node2", nodeName);
   }
 
@@ -145,8 +145,8 @@ public class AssignTest extends SolrTestCaseJ4 {
       slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
 
       DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
-      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
-      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
+      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
+      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildSolrCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
     } finally {
       server.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
index c032f6c..67668c9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -34,6 +34,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -264,7 +265,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
         }
       }
       
-      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS));
+      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //      waitForAllWarmingSearchers();
       
       Set<String> addFails = getAddFails(indexTreads);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
index ce77996..cee7a35 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
@@ -32,6 +32,7 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -206,7 +207,7 @@ public class ChaosMonkeySafeLeaderWithPullReplicasTest extends AbstractFullDistr
     
     log.info("collection state: " + printClusterStateInfo(DEFAULT_COLLECTION));
     
-    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS));
+    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //    waitForAllWarmingSearchers();
 
     checkShardConsistency(batchSize == 1, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index 4ae2b70..5615918 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -66,6 +66,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean.Category;
@@ -458,7 +459,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
         .add("id", "7")
         .add("id", "8")
         .commit(cluster.getSolrClient(), collectionName);
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       try {
         long numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -514,7 +515,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
   private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
 
 
-    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     boolean allTimesAreCorrect = false;
     while (! timeout.hasTimedOut()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
index bc3b9c5..57d38cd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -140,7 +141,7 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
     
     @Override
     public void run() {
-      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
+      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       while (! timeout.hasTimedOut() && failure.get() == null) {
         doWork();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
index 31f8418..8847cec 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.Replica;
 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.TimeSource;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory;
 import org.apache.solr.util.TimeOut;
@@ -175,7 +176,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
                                 SolrParams params)
       throws SolrServerException, InterruptedException, IOException {
 
-    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = cluster.getSolrClient().query(COLLECTION, params).getResults().getNumFound();
     while (0L < numFound && ! timeout.hasTimedOut()) {
       Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
index 8136d3e..0423428 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -153,7 +154,7 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
       // shutdown the original leader
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
-      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       waitTillNodesActive();
       log.info("Updating mappings from zk");
       updateMappingsFromZk(jettys, clients, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
index b57d06f..57ff0b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.RoutingRule;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
 
     @Override
     public void run() {
-      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       for (int id = 26*3; id < 500 && ! timeout.hasTimedOut(); id++) {
         String shardKey = "" + (char) ('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
         SolrInputDocument doc = new SolrInputDocument();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index b6e7415..8a46808 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -44,7 +44,9 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -87,6 +89,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   private static ClusterState clusterStateMock;
   private static SolrZkClient solrZkClientMock;
   private static DistribStateManager stateManagerMock;
+  private static ObjectCache objectCache;
   private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
   private final Map zkMap = new HashMap();
   private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
@@ -136,6 +139,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     overseerMock = mock(Overseer.class);
     zkControllerMock = mock(ZkController.class);
     cloudDataProviderMock = mock(SolrCloudManager.class);
+    objectCache = new ObjectCache();
     clusterStateProviderMock = mock(ClusterStateProvider.class);
     stateManagerMock = mock(DistribStateManager.class);
   }
@@ -173,6 +177,9 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     reset(overseerMock);
     reset(zkControllerMock);
     reset(cloudDataProviderMock);
+    objectCache.clear();
+    when(cloudDataProviderMock.getObjectCache()).thenReturn(objectCache);
+    when(cloudDataProviderMock.getTimeSource()).thenReturn(TimeSource.NANO_TIME);
     reset(clusterStateProviderMock);
     reset(stateManagerMock);
 
@@ -537,7 +544,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   }
   
   protected void waitForEmptyQueue(long maxWait) throws Exception {
-    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS);
+    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (queue.peek() != null) {
       if (timeout.hasTimedOut())
         fail("Queue not empty within " + maxWait + " ms");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index 72b5142..edc6695 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -51,7 +52,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
   }
 
   private void waitForNewOverseer(int seconds, Predicate<String> state) throws Exception {
-    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     String current = null;
     while (timeout.hasTimedOut() == false) {
       current = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index 7ff5334..3dac890 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -48,6 +48,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.TimeOut;
@@ -162,7 +163,7 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
       log.info("Updating mappings from zk");
-      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       updateMappingsFromZk(jettys, clients, true);
       assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
index 41c7c40..457af81 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
@@ -54,6 +54,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -389,7 +390,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
   }
 
   private void assertSliceAndReplicaCount(String collection, int numSlices, int numReplicas, int timeOutInMs) throws InterruptedException {
-    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS);
+    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
       Collection<Slice> slices = clusterState.getCollection(collection).getActiveSlices();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
index b6efa53..0fbc0a1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
@@ -24,6 +24,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.util.MockCoreContainer.MockCoreDescriptor;
 import org.apache.solr.util.TimeOut;
@@ -98,7 +99,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // kill the replica
     int children = cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size();
     ChaosMonkey.stop(notLeader.jetty);
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (children > cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size()) {
         break;
@@ -173,7 +174,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // this should have published a down state so assert that cversion has incremented
     assertTrue(getOverseerCversion() > cversion);
 
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica r = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName());
       if (r.getState() == Replica.State.DOWN) {
@@ -217,7 +218,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     thread = new LeaderInitiatedRecoveryThread(zkController, coreContainer,
         DEFAULT_COLLECTION, SHARD1, replicaCoreNodeProps, 1, coreContainer.getCores().iterator().next().getCoreDescriptor());
     thread.publishDownState(replicaCoreNodeProps.getCoreName(), replica.getName(), replica.getNodeName(), replicaCoreNodeProps.getCoreUrl(), false);
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica.State state = zkController.getLeaderInitiatedRecoveryState(DEFAULT_COLLECTION, SHARD1, replica.getName());
       if (state == Replica.State.DOWN) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 04e22f7..7d652b8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -52,6 +52,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
@@ -229,7 +230,7 @@ public class TestPullReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.PULL))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient readOnlyReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -500,7 +501,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
         while (true) {
@@ -521,7 +522,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
index 12b3ef0..dae0b4c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
@@ -43,6 +43,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -86,7 +87,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       proxies.put(proxy.getUrl(), proxy);
       jettys.put(proxy.getUrl(), jetty);
     }
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (true) {
       try {
         CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false");
@@ -159,7 +160,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       assertNumberOfReplicas(numShards, 0, numShards, true, true);// Replica should still be active, since it doesn't disconnect from ZooKeeper
       {
         long numFound = 0;
-        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (numFound < 20 && !t.hasTimedOut()) {
           Thread.sleep(200);
           numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -233,7 +234,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void assertNumDocs(int numDocs, SolrClient client, int timeoutSecs) throws InterruptedException, SolrServerException, IOException {
-    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = -1;
     while (!t.hasTimedOut()) {
       Thread.sleep(200);
@@ -299,7 +300,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
index 798a8a8..b47424f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -118,7 +119,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
   // 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue.
   // 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader.
   void checkConsistency() throws InterruptedException, KeeperException {
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     boolean checkAppearOnce = false;
     boolean checkElectionZero = false;
     boolean checkZkLeadersAgree = false;
@@ -323,7 +324,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
 
   boolean waitForAllPreferreds() throws KeeperException, InterruptedException {
     boolean goAgain = true;
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (! timeout.hasTimedOut()) {
       goAgain = false;
       Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index ff9fbb8..4ac9f02 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -60,6 +60,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrIndexWriter;
@@ -232,7 +233,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.TLOG))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient tlogReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -400,7 +401,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
     docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
     
     // Wait until a new leader is elected
-    TimeOut t = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!t.hasTimedOut()) {
       docCollection = getCollectionState(collectionName);
       Replica leader = docCollection.getSlice("shard1").getLeader();
@@ -488,7 +489,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
 
     waitForNumDocsInAllActiveReplicas(2);
     // There are a small delay between new searcher and copy over old updates operation
-    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (assertCopyOverOldUpdates(1, timeCopyOverPerCores)) {
         break;
@@ -748,7 +749,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query, int timeout) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       if (!r.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
         continue;
@@ -772,7 +773,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       try {
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
index 28a0a4e..16673da 100644
--- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -73,7 +74,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
   private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
       boolean shouldBePresent, int expectedSliceCount) throws Exception {
-    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     Boolean isPresent = null; // null meaning "don't know"
     while (null == isPresent || shouldBePresent != isPresent.booleanValue()) {
       final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
index 600cd65..e3df94b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
@@ -36,6 +36,7 @@ import org.apache.solr.common.params.CollectionParams;
 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.TimeSource;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
index b3147a4..d31bba5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
@@ -37,6 +37,7 @@ 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.SuppressForbidden;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,6 +45,7 @@ import org.junit.Test;
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 
 public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(3)
@@ -141,7 +143,7 @@ public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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
index 985d3aa..cf3bff3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
@@ -69,7 +70,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
   }
 
   private static void testAutoAddReplicas() throws Exception {
-    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
       ZkNodeProps loaded = ZkNodeProps.load(data);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
new file mode 100644
index 0000000..e08d37b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
@@ -0,0 +1,63 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+
+/**
+ *
+ */
+public class CapturedEvent {
+  public final AutoScalingConfig.TriggerListenerConfig config;
+  public final TriggerEventProcessorStage stage;
+  public final String actionName;
+  public final TriggerEvent event;
+  public final String message;
+  public final Map<String, Object> context = new HashMap<>();
+  public final long timestamp;
+
+  public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
+                       TriggerEvent event, String message) {
+    if (context != null) {
+      context.toMap(this.context);
+    }
+    this.config = config;
+    this.stage = stage;
+    this.actionName = actionName;
+    this.event = event;
+    this.message = message;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String toString() {
+    return "CapturedEvent{" +
+        "timestamp=" + timestamp +
+        ", stage=" + stage +
+        ", actionName='" + actionName + '\'' +
+        ", event=" + event +
+        ", context=" + context +
+        ", config=" + config +
+        ", message='" + message + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
index 837c683..2b80ec3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -38,11 +39,13 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -139,6 +142,19 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
   }
 
+  @After
+  public void printState() throws Exception {
+    log.debug("-------------_ FINAL STATE --------------");
+    SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
+      Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
+      log.debug("* Node values: " + node + "\n" + Utils.toJSONString(values));
+    }
+    log.debug("* Live nodes: " + cloudManager.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
+    state.forEachCollection(coll -> log.debug("* Collection " + coll.getName() + " state: " + coll));
+  }
+
   @Test
   public void testNodeLost() throws Exception  {
     // let's start a node so that we have at least two

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
index 74a1a82..fa826b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
@@ -41,7 +41,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index f2f3f74..d477594 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index 8bca296..dad81e0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index cacf39c..c104a99 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -56,7 +56,7 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.data.Stat;
@@ -87,6 +87,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   private static AtomicBoolean triggerFired;
   private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
   private static ZkStateReader zkStateReader;
+  private static SolrCloudManager cloudManager;
 
   // use the same time source as triggers use
   private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
@@ -146,6 +147,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       // lets start a node
       cluster.startJettySolrRunner();
     }
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
   }
 
   private void deleteChildrenRecursively(String path) throws Exception {
@@ -322,7 +324,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -338,7 +340,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     cluster.stopJettySolrRunner(index);
 
     // ensure that the old trigger sees the stopped node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -386,7 +388,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -396,7 +398,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     JettySolrRunner newNode = cluster.startJettySolrRunner();
 
     // ensure that the old trigger sees the new node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -954,52 +956,22 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(TriggerEventType.NODELOST, ev.getEventType());
   }
 
-  private static class TestEvent {
-    final AutoScalingConfig.TriggerListenerConfig config;
-    final TriggerEventProcessorStage stage;
-    final String actionName;
-    final TriggerEvent event;
-    final String message;
-    final long timestamp;
-
-    TestEvent(AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName, TriggerEvent event, String message) {
-      this.config = config;
-      this.stage = stage;
-      this.actionName = actionName;
-      this.event = event;
-      this.message = message;
-      this.timestamp = timeSource.getTime();
-    }
-
-    @Override
-    public String toString() {
-      return "TestEvent{" +
-          "timestamp=" + timestamp +
-          ", config=" + config +
-          ", stage=" + stage +
-          ", actionName='" + actionName + '\'' +
-          ", event=" + event +
-          ", message='" + message + '\'' +
-          '}';
-    }
-  }
-
-  static Map<String, List<TestEvent>> listenerEvents = new HashMap<>();
+  static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
   static CountDownLatch listenerCreated = new CountDownLatch(1);
   static boolean failDummyAction = false;
 
   public static class TestTriggerListener extends TriggerListenerBase {
     @Override
-    public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-      super.init(dataProvider, config);
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
       listenerCreated.countDown();
     }
 
     @Override
     public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
                                      ActionContext context, Throwable error, String message) {
-      List<TestEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
-      lst.add(new TestEvent(config, stage, actionName, event, message));
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(timeSource.getTime(), context, config, stage, actionName, event, message));
     }
   }
 
@@ -1079,38 +1051,38 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    List<TestEvent> testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 5, testEvents.size());
+    List<CapturedEvent> capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 5, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(4).stage);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(3).stage);
 
     // reset
     triggerFired.set(false);
@@ -1125,37 +1097,37 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 4, testEvents.size());
+    capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
   }
 
   @Test
@@ -1200,7 +1172,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(1000);
 
-    List<TestEvent> capturedEvents = listenerEvents.get("bar");
+    List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
     // we may get a few IGNORED events if other tests caused events within cooldown period
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
     long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
@@ -1220,11 +1192,11 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     capturedEvents = listenerEvents.get("bar");
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
     for (int i = 0; i < capturedEvents.size() - 1; i++) {
-      TestEvent ev = capturedEvents.get(i);
+      CapturedEvent ev = capturedEvents.get(i);
       assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
       assertTrue(ev.toString(), ev.message.contains("cooldown"));
     }
-    TestEvent ev = capturedEvents.get(capturedEvents.size() - 1);
+    CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
     assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
     // the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
     // must be larger than cooldown period
@@ -1380,8 +1352,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       Map<String, Object> props = map("waitFor", 0L, "actions", Collections.singletonList(map("name","throttler", "class", ThrottlingTesterAction.class.getName())));
       scheduledTriggers.add(new NodeAddedTrigger("y1", props, resourceLoader, solrCloudManager));
       scheduledTriggers.add(new NodeAddedTrigger("y2", props, resourceLoader, solrCloudManager));
+      scheduledTriggers.resetActionThrottle();
       JettySolrRunner newNode = cluster.startJettySolrRunner();
-      assertTrue(getTriggerFiredLatch().await(10, TimeUnit.SECONDS));
+      assertTrue(getTriggerFiredLatch().await(20, TimeUnit.SECONDS));
       for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
         if (cluster.getJettySolrRunner(i) == newNode) {
           cluster.stopJettySolrRunner(i);
@@ -1439,7 +1412,12 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
   public void testSearchRate() throws Exception {
+    // start a few more jetty-s
+    for (int i = 0; i < 3; i++) {
+      cluster.startJettySolrRunner();
+    }
     CloudSolrClient solrClient = cluster.getSolrClient();
     String COLL1 = "collection1";
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
@@ -1453,6 +1431,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'enabled' : true," +
         "'rate' : 1.0," +
         "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
         "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
         "]" +
         "}}";
@@ -1466,6 +1446,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'name' : 'srt'," +
         "'trigger' : 'search_rate_trigger'," +
         "'stage' : ['FAILED','SUCCEEDED']," +
+        "'afterAction': ['compute', 'execute', 'test']," +
         "'class' : '" + TestTriggerListener.class.getName() + "'" +
         "}" +
         "}";
@@ -1481,7 +1462,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(2000);
     assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
-    TestEvent ev = listenerEvents.get("srt").get(0);
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
     long now = timeSource.getTime();
     // verify waitFor
     assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
new file mode 100644
index 0000000..c1c070d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
@@ -0,0 +1,24 @@
+/*
+ * 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.sim;
+
+/**
+ * Interface that helps simulating action errors.
+ */
+public interface ActionError {
+  boolean shouldFail(String... args);
+}


[3/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
new file mode 100644
index 0000000..9b4e2bc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
@@ -0,0 +1,306 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
+import org.apache.solr.cloud.autoscaling.NodeAddedTrigger;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link NodeAddedTrigger}
+ */
+public class TestNodeAddedTrigger extends SimSolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
+
+  private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
+    fail("Did not expect the listener to fire on first run!");
+    return true;
+  };
+
+  private static int SPEED = 50;
+
+  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(2);
+
+  private static TimeSource timeSource;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
+  @Test
+  public void testTrigger() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String newNode1 = cluster.simAddNode();
+      String newNode2 = cluster.simAddNode();
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Newly added node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeAddedEvent = eventRef.get();
+      assertNotNull(nodeAddedEvent);
+      List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames.contains(newNode1));
+      assertTrue(nodeNames.contains(newNode2));
+    }
+
+    // add a new node but remove it before the waitFor period expires
+    // and assert that the trigger doesn't fire at all
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      final long waitTime = 2;
+      props.put("waitFor", waitTime);
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String newNode = cluster.simAddNode();
+      AtomicBoolean fired = new AtomicBoolean(false);
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      trigger.run(); // first run should detect the new node
+      cluster.simRemoveNode(newNode, true);
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > waitTime + 1) { // run it a little more than the wait time
+          break;
+        }
+      } while (true);
+
+      // ensure the event was not fired
+      assertFalse(fired.get());
+    }
+  }
+
+  public void testActionLifecycle() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", TestNodeAddedTrigger.AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
+  @Test
+  public void testListenerAcceptance() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run(); // starts tracking live nodes
+
+      String newNode = cluster.simAddNode();
+      AtomicInteger callCount = new AtomicInteger(0);
+      AtomicBoolean fired = new AtomicBoolean(false);
+
+      trigger.setProcessor(event -> {
+        if (callCount.incrementAndGet() < 2) {
+          return false;
+        } else  {
+          fired.compareAndSet(false, true);
+          return true;
+        }
+      });
+
+      trigger.run(); // first run should detect the new node and fire immediately but listener isn't ready
+      assertEquals(1, callCount.get());
+      assertFalse(fired.get());
+      trigger.run(); // second run should again fire
+      assertEquals(2, callCount.get());
+      assertTrue(fired.get());
+      trigger.run(); // should not fire
+      assertEquals(2, callCount.get());
+    }
+  }
+
+  @Test
+  public void testRestoreState() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    // add a new node but update the trigger before the waitFor period expires
+    // and assert that the new trigger still fires
+    NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster);
+    trigger.setProcessor(noFirstRunProcessor);
+    trigger.run();
+
+    String newNode = cluster.simAddNode();
+    trigger.run(); // this run should detect the new node
+    trigger.close(); // close the old trigger
+
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, cluster.getLoader(), cluster))  {
+      try {
+        newTrigger.restoreState(trigger);
+        fail("Trigger should only be able to restore state from an old trigger of the same name");
+      } catch (AssertionError e) {
+        // expected
+      }
+    }
+
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster))  {
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      newTrigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      newTrigger.restoreState(trigger); // restore state from the old trigger
+      int counter = 0;
+      do {
+        newTrigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Newly added node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      // ensure the event was fired
+      assertTrue(fired.get());
+      TriggerEvent nodeAddedEvent = eventRef.get();
+      assertNotNull(nodeAddedEvent);
+      //TODO assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
+    }
+  }
+
+  private Map<String, Object> createTriggerProps(long waitForSeconds) {
+    Map<String, Object> props = new HashMap<>();
+    props.put("event", "nodeLost");
+    props.put("waitFor", waitForSeconds);
+    props.put("enabled", true);
+    List<Map<String, String>> actions = new ArrayList<>(3);
+    Map<String, String> map = new HashMap<>(2);
+    map.put("name", "compute_plan");
+    map.put("class", "solr.ComputePlanAction");
+    actions.add(map);
+    map = new HashMap<>(2);
+    map.put("name", "execute_plan");
+    map.put("class", "solr.ExecutePlanAction");
+    actions.add(map);
+    props.put("actions", actions);
+    return props;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
new file mode 100644
index 0000000..109cee3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
@@ -0,0 +1,331 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link NodeLostTrigger}
+ */
+public class TestNodeLostTrigger extends SimSolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
+
+  private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
+    fail("Did not expect the listener to fire on first run!");
+    return true;
+  };
+
+  private static final int SPEED = 50;
+  // use the same time source as the trigger
+  private static TimeSource timeSource;
+  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(5, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
+  @Test
+  public void testTrigger() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+      Iterator<String> it = cluster.getLiveNodesSet().get().iterator();
+      String lostNodeName1 = it.next();
+      String lostNodeName2 = it.next();
+      cluster.simRemoveNode(lostNodeName1, true);
+      cluster.simRemoveNode(lostNodeName2, true);
+      timeSource.sleep(1000);
+
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Lost node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeLostEvent = eventRef.get();
+      assertNotNull(nodeLostEvent);
+      List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames + " doesn't contain " + lostNodeName1, nodeNames.contains(lostNodeName1));
+      assertTrue(nodeNames + " doesn't contain " + lostNodeName2, nodeNames.contains(lostNodeName2));
+
+    }
+
+    // remove a node but add it back before the waitFor period expires
+    // and assert that the trigger doesn't fire at all
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      final long waitTime = 2;
+      props.put("waitFor", waitTime);
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String lostNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+      cluster.simRemoveNode(lostNode, false);
+      AtomicBoolean fired = new AtomicBoolean(false);
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitTime, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      trigger.run(); // first run should detect the lost node
+      int counter = 0;
+      do {
+        if (cluster.getLiveNodesSet().get().size() == 2) {
+          break;
+        }
+        timeSource.sleep(100);
+        if (counter++ > 20) {
+          fail("Live nodes not updated!");
+        }
+      } while (true);
+      counter = 0;
+      cluster.getSimClusterStateProvider().simRestoreNode(lostNode);
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > waitTime + 1) { // run it a little more than the wait time
+          break;
+        }
+      } while (true);
+
+      // ensure the event was not fired
+      assertFalse(fired.get());
+    }
+  }
+
+  public void testActionLifecycle() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
+  @Test
+  public void testListenerAcceptance() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+
+      String newNode = cluster.simAddNode();
+
+      trigger.run(); // starts tracking live nodes
+
+      // stop the newly created node
+      cluster.simRemoveNode(newNode, true);
+
+      AtomicInteger callCount = new AtomicInteger(0);
+      AtomicBoolean fired = new AtomicBoolean(false);
+
+      trigger.setProcessor(event -> {
+        if (callCount.incrementAndGet() < 2) {
+          return false;
+        } else  {
+          fired.compareAndSet(false, true);
+          return true;
+        }
+      });
+
+      trigger.run(); // first run should detect the lost node and fire immediately but listener isn't ready
+      assertEquals(1, callCount.get());
+      assertFalse(fired.get());
+      trigger.run(); // second run should again fire
+      assertEquals(2, callCount.get());
+      assertTrue(fired.get());
+      trigger.run(); // should not fire
+      assertEquals(2, callCount.get());
+    }
+  }
+
+  @Test
+  public void testRestoreState() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    String newNode = cluster.simAddNode();
+
+    // remove a node but update the trigger before the waitFor period expires
+    // and assert that the new trigger still fires
+
+    NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster);
+    trigger.setProcessor(noFirstRunProcessor);
+    trigger.run();
+
+    // stop the newly created node
+    cluster.simRemoveNode(newNode, true);
+
+    trigger.run(); // this run should detect the lost node
+    trigger.close(); // close the old trigger
+
+    try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, cluster.getLoader(), cluster))  {
+      try {
+        newTrigger.restoreState(trigger);
+        fail("Trigger should only be able to restore state from an old trigger of the same name");
+      } catch (AssertionError e) {
+        // expected
+      }
+    }
+
+    try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      newTrigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      newTrigger.restoreState(trigger); // restore state from the old trigger
+      int counter = 0;
+      do {
+        newTrigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Lost node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeLostEvent = eventRef.get();
+      assertNotNull(nodeLostEvent);
+      List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames.contains(newNode));
+    }
+  }
+
+  private Map<String, Object> createTriggerProps(long waitForSeconds) {
+    Map<String, Object> props = new HashMap<>();
+    props.put("event", "nodeLost");
+    props.put("waitFor", waitForSeconds);
+    props.put("enabled", true);
+    List<Map<String, String>> actions = new ArrayList<>(3);
+    Map<String, String> map = new HashMap<>(2);
+    map.put("name", "compute_plan");
+    map.put("class", "solr.ComputePlanAction");
+    actions.add(map);
+    map = new HashMap<>(2);
+    map.put("name", "execute_plan");
+    map.put("class", "solr.ExecutePlanAction");
+    actions.add(map);
+    props.put("actions", actions);
+    return props;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
new file mode 100644
index 0000000..22736db
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
@@ -0,0 +1,357 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.Row;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
+public class TestPolicyCloud extends SimSolrCloudTestCase {
+
+  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, TimeSource.get("simTime:50"));
+  }
+
+  public void testDataProviderPerReplicaDetails() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    CollectionAdminRequest.createCollection("perReplicaDataColl", "conf", 1, 5)
+        .process(solrClient);
+
+    waitForState("Timeout waiting for collection to become active", "perReplicaDataColl", clusterShape(1, 5));
+    DocCollection coll = getCollectionState("perReplicaDataColl");
+    String autoScaleJson = "{" +
+        "  'cluster-preferences': [" +
+        "    { maximize : freedisk , precision: 50}," +
+        "    { minimize : cores, precision: 2}" +
+        "  ]," +
+        "  'cluster-policy': [" +
+        "    { replica : '0' , 'nodeRole': 'overseer'}," +
+        "    { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'" +
+        "    }" +
+        "  ]," +
+        "  'policies': {" +
+        "    'policy1': [" +
+        "      { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
+        "    ]" +
+        "  }" +
+        "}";
+    AutoScalingConfig config = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScaleJson));
+    Policy.Session session = config.getPolicy().createSession(cluster);
+
+    AtomicInteger count = new AtomicInteger(0);
+    for (Row row : session.getSorted()) {
+      row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
+        for (ReplicaInfo replicaInfo : replicaInfos) {
+          if (replicaInfo.getVariables().containsKey(Suggestion.ConditionType.CORE_IDX.tagName)) count.incrementAndGet();
+        }
+      }));
+    }
+    assertTrue(count.get() > 0);
+
+    CollectionAdminRequest.deleteCollection("perReplicaDataColl").process(solrClient);
+
+  }
+
+  public void testCreateCollectionAddReplica() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
+
+    String commands =  "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" + port + "'}]}}";
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+
+    String collectionName = "testCreateCollectionAddReplica";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 1));
+
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(solrClient);
+    waitForState("Timed out waiting to see 2 replicas for collection: " + collectionName,
+        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
+
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+  }
+
+  public void testCreateCollectionSplitShard() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String firstNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    int firstNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(firstNode, ImplicitSnitch.PORT);
+
+    String secondNode;
+    int secondNodePort;
+    while (true)  {
+      secondNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+      secondNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(secondNode, ImplicitSnitch.PORT);
+      if (secondNodePort != firstNodePort)  break;
+    }
+
+    String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + firstNodePort + "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
+    NamedList<Object> response = solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    assertEquals("success", response.get("result"));
+
+    String collectionName = "testCreateCollectionSplitShard";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 2));
+
+    DocCollection docCollection = getCollectionState(collectionName);
+    List<Replica> list = docCollection.getReplicas(firstNode);
+    int replicasOnNode1 = list != null ? list.size() : 0;
+    list = docCollection.getReplicas(secondNode);
+    int replicasOnNode2 = list != null ? list.size() : 0;
+
+    assertEquals("Expected exactly one replica of collection on node with port: " + firstNodePort, 1, replicasOnNode1);
+    assertEquals("Expected exactly one replica of collection on node with port: " + secondNodePort, 1, replicasOnNode2);
+
+    CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(solrClient);
+
+    waitForState("Timed out waiting to see 6 replicas for collection: " + collectionName,
+        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 6);
+
+    docCollection = getCollectionState(collectionName);
+    list = docCollection.getReplicas(firstNode);
+    replicasOnNode1 = list != null ? list.size() : 0;
+    list = docCollection.getReplicas(secondNode);
+    replicasOnNode2 = list != null ? list.size() : 0;
+
+    assertEquals("Expected exactly three replica of collection on node with port: " + firstNodePort, 3, replicasOnNode1);
+    assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort, 3, replicasOnNode2);
+    CollectionAdminRequest.deleteCollection(collectionName).process(solrClient);
+
+  }
+
+  public void testMetricsTag() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'metrics:abc':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    try {
+      solrClient.request(req);
+      fail("expected exception");
+    } catch (Exception e) {
+      // expected
+      assertTrue(e.toString().contains("Invalid metrics: param in"));
+    }
+    setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'metrics:solr.node:ADMIN./admin/authorization.clientErrors:count':'>58768765', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    solrClient.request(req);
+
+    //org.eclipse.jetty.server.handler.DefaultHandler.2xx-responses
+    CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "metricsTest", clusterShape(1, 1));
+
+    DocCollection collection = getCollectionState("metricsTest");
+    List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
+        "metrics:solr.jvm:buffers.direct.Count");
+    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), tags);
+    for (String tag : tags) {
+      assertNotNull( "missing : "+ tag , val.get(tag));
+    }
+
+
+  }
+
+  public void testCreateCollectionAddShardWithReplicaTypeUsingPolicy() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    List<String> nodes = new ArrayList<>(cluster.getClusterStateProvider().getLiveNodes());
+    String nrtNodeName = nodes.get(0);
+    int nrtPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nrtNodeName, ImplicitSnitch.PORT);
+
+
+    String pullNodeName = nodes.get(1);
+    int pullPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(pullNodeName, ImplicitSnitch.PORT);
+
+    String tlogNodeName = nodes.get(1);
+    int tlogPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(tlogNodeName, ImplicitSnitch.PORT);
+    log.info("NRT {} PULL {} , TLOG {} ", nrtNodeName, pullNodeName, tlogNodeName);
+
+    String commands = "{set-cluster-policy :[" +
+        "{replica:0 , shard:'#EACH', type: NRT, port: '!" + nrtPort + "'}" +
+        "{replica:0 , shard:'#EACH', type: PULL, port: '!" + pullPort + "'}" +
+        "{replica:0 , shard:'#EACH', type: TLOG, port: '!" + tlogPort + "'}" +
+        "]}";
+
+
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + nrtPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[0]/port"));
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + pullPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[1]/port"));
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + tlogPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[2]/port"));
+
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1", 1, 1, 1)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 3));
+
+    DocCollection coll = getCollectionState("policiesTest");
+
+
+    BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
+      switch (replica.getType()) {
+        case NRT: {
+          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNodeName().equals(nrtNodeName));
+          break;
+        }
+        case TLOG: {
+          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNodeName().equals(tlogNodeName));
+          break;
+        }
+        case PULL: {
+          assertTrue("PULL replica should be in " + pullNodeName, replica.getNodeName().equals(pullNodeName));
+          break;
+        }
+      }
+
+    };
+    coll.forEachReplica(verifyReplicas);
+
+    CollectionAdminRequest.createShard("policiesTest", "s3").
+        process(solrClient);
+    coll = getCollectionState("policiesTest");
+    assertEquals(3, coll.getSlice("s3").getReplicas().size());
+    coll.forEachReplica(verifyReplicas);
+  }
+
+  public void testCreateCollectionAddShardUsingPolicy() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
+
+    String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + port + "'}]}}";
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    assertEquals("full json:"+ Utils.toJSONString(json) , "#EACH",
+        Utils.getObjectByPath(json, true, "/policies/c1[0]/shard"));
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1,s2", 1)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(2, 1));
+
+    DocCollection coll = getCollectionState("policiesTest");
+    assertEquals("c1", coll.getPolicyName());
+    assertEquals(2,coll.getReplicas().size());
+    coll.forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+    CollectionAdminRequest.createShard("policiesTest", "s3").process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(3, 1));
+
+    coll = getCollectionState("policiesTest");
+    assertEquals(1, coll.getSlice("s3").getReplicas().size());
+    coll.getSlice("s3").forEach(replica -> assertEquals(nodeId, replica.getNodeName()));
+  }
+
+  public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 2));
+    DocCollection rulesCollection = getCollectionState("policiesTest");
+
+    Map<String, Object> val = cluster.getNodeStateProvider().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")).doubleValue(),  Double.compare(((Number) val.get("freedisk")).doubleValue(), 0.0d) > 0);
+    assertTrue("heapUsage value is " + ((Number) val.get("heapUsage")).doubleValue(), Double.compare(((Number) val.get("heapUsage")).doubleValue(), 0.0d) > 0);
+    if (!Constants.WINDOWS)  {
+      // the system load average metrics is not available on windows platform
+      assertTrue("sysLoadAvg value is " + ((Number) val.get("sysLoadAvg")).doubleValue(), Double.compare(((Number) val.get("sysLoadAvg")).doubleValue(), 0.0d) > 0);
+    }
+    // simulator doesn't have Overseer, so just pick a random node
+    String overseerNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    solrClient.request(CollectionAdminRequest.addRole(overseerNode, "overseer"));
+    for (int i = 0; i < 10; i++) {
+      Map<String, Object> data = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.ROLES);
+      if (i >= 9 && data.isEmpty()) {
+        throw new RuntimeException("NO overseer node created");
+      }
+      cluster.getTimeSource().sleep(100);
+    }
+    val = cluster.getNodeStateProvider().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/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
new file mode 100644
index 0000000..1c41795
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
@@ -0,0 +1,220 @@
+/*
+ * 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.sim;
+
+import java.nio.charset.Charset;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Predicate;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class TestSimDistributedQueue extends SolrTestCaseJ4 {
+  private static final Charset UTF8 = Charset.forName("UTF-8");
+  protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("sdqtest-"));
+
+  @Test
+  public void testDistributedQueue() throws Exception {
+    String dqZNode = "/distqueue/test1";
+    byte[] data = "hello world".getBytes(UTF8);
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    // basic ops
+    assertNull(dq.poll());
+    try {
+      dq.remove();
+      fail("NoSuchElementException expected");
+    } catch (NoSuchElementException expected) {
+      // expected
+    }
+
+    dq.offer(data);
+    assertArrayEquals(dq.peek(500), data);
+    assertArrayEquals(dq.remove(), data);
+    assertNull(dq.poll());
+
+    dq.offer(data);
+    assertArrayEquals(dq.take(), data); // waits for data
+    assertNull(dq.poll());
+
+    dq.offer(data);
+    dq.peek(true); // wait until data is definitely there before calling remove
+    assertArrayEquals(dq.remove(), data);
+    assertNull(dq.poll());
+
+    // should block until the background thread makes the offer
+    (new QueueChangerThread(dq, 1000)).start();
+    assertNotNull(dq.peek(true));
+    assertNotNull(dq.remove());
+    assertNull(dq.poll());
+
+    // timeout scenario ... background thread won't offer until long after the peek times out
+    QueueChangerThread qct = new QueueChangerThread(dq, 1000);
+    qct.start();
+    assertNull(dq.peek(500));
+    qct.join();
+  }
+
+  @Test
+  public void testDistributedQueueBlocking() throws Exception {
+    String dqZNode = "/distqueue/test2";
+    String testData = "hello world";
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    assertNull(dq.peek());
+    Future<String> future = executor.submit(() -> new String(dq.peek(true), UTF8));
+    try {
+      future.get(1000, TimeUnit.MILLISECONDS);
+      fail("TimeoutException expected");
+    } catch (TimeoutException expected) {
+      assertFalse(future.isDone());
+    }
+
+    dq.offer(testData.getBytes(UTF8));
+    assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
+    assertNotNull(dq.poll());
+
+    assertNull(dq.peek(100));
+
+    // Rerun the earlier test make sure updates are still seen, post reconnection.
+    future = executor.submit(() -> new String(dq.peek(true), UTF8));
+    try {
+      future.get(1000, TimeUnit.MILLISECONDS);
+      fail("TimeoutException expected");
+    } catch (TimeoutException expected) {
+      assertFalse(future.isDone());
+    }
+
+    dq.offer(testData.getBytes(UTF8));
+    assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
+    assertNotNull(dq.poll());
+    assertNull(dq.poll());
+  }
+
+  @Test
+  public void testLocallyOffer() throws Exception {
+    String dqZNode = "/distqueue/test3";
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+    dq.peekElements(1, 1, s -> true);
+    for (int i = 0; i < 100; i++) {
+      byte[] data = String.valueOf(i).getBytes(UTF8);
+      dq.offer(data);
+      assertNotNull(dq.peek());
+      dq.poll();
+      dq.peekElements(1, 1, s -> true);
+    }
+  }
+
+
+  @Test
+  public void testPeekElements() throws Exception {
+    String dqZNode = "/distqueue/test4";
+    byte[] data = "hello world".getBytes(UTF8);
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    // Populate with data.
+    dq.offer(data);
+    dq.offer(data);
+    dq.offer(data);
+
+    Predicate<String> alwaysTrue = s -> true;
+    Predicate<String> alwaysFalse = s -> false;
+
+    // Should be able to get 0, 1, 2, or 3 instantly
+    for (int i = 0; i <= 3; ++i) {
+      assertEquals(i, dq.peekElements(i, 0, alwaysTrue).size());
+    }
+
+    // Asking for more should return only 3.
+    assertEquals(3, dq.peekElements(4, 0, alwaysTrue).size());
+
+    // If we filter everything out, we should block for the full time.
+    long start = System.nanoTime();
+    assertEquals(0, dq.peekElements(4, 1000, alwaysFalse).size());
+    assertTrue(System.nanoTime() - start >= TimeUnit.MILLISECONDS.toNanos(500));
+
+    // If someone adds a new matching element while we're waiting, we should return immediately.
+    executor.submit(() -> {
+      try {
+        Thread.sleep(500);
+        dq.offer(data);
+      } catch (Exception e) {
+        // ignore
+      }
+    });
+    start = System.nanoTime();
+    assertEquals(1, dq.peekElements(4, 2000, child -> {
+      // The 4th element in the queue will end with a "3".
+      return child.endsWith("3");
+    }).size());
+    long elapsed = System.nanoTime() - start;
+    assertTrue(elapsed < TimeUnit.MILLISECONDS.toNanos(1000));
+    assertTrue(elapsed >= TimeUnit.MILLISECONDS.toNanos(250));
+  }
+
+
+  protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
+    return new SimDistributedQueueFactory.SimDistributedQueue(dqZNode);
+  }
+
+  private static class QueueChangerThread extends Thread {
+
+    DistributedQueue dq;
+    long waitBeforeOfferMs;
+
+    QueueChangerThread(DistributedQueue dq, long waitBeforeOfferMs) {
+      this.dq = dq;
+      this.waitBeforeOfferMs = waitBeforeOfferMs;
+    }
+
+    public void run() {
+      try {
+        Thread.sleep(waitBeforeOfferMs);
+        dq.offer(getName().getBytes(UTF8));
+      } catch (InterruptedException ie) {
+        // do nothing
+      } catch (Exception exc) {
+        throw new RuntimeException(exc);
+      }
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    try {
+      super.tearDown();
+    } catch (Exception exc) {
+    }
+    executor.shutdown();
+  }
+
+}


[5/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
new file mode 100644
index 0000000..1986bac
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -0,0 +1,1275 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.AddReplicaCmd;
+import org.apache.solr.cloud.Assign;
+import org.apache.solr.cloud.CreateCollectionCmd;
+import org.apache.solr.cloud.CreateShardCmd;
+import org.apache.solr.cloud.SplitShardCmd;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
+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.DocRouter;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ReplicaPosition;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.CreateMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
+/**
+ * Simulated {@link ClusterStateProvider}.
+ * <p>The following behaviors are supported:</p>
+ *   <ul>
+ *     <li>using autoscaling policy for replica placements</li>
+ *     <li>maintaining and up-to-date list of /live_nodes and nodeAdded / nodeLost markers</li>
+ *     <li>running a simulated leader election on collection changes (with throttling), when needed</li>
+ *     <li>maintaining an up-to-date /clusterstate.json (single file format), which also tracks replica states,
+ *     leader election changes, replica property changes, etc. Note: this file is only written,
+ *     but never read by the framework!</li>
+ *     <li>maintaining an up-to-date /clusterprops.json. Note: this file is only written, but never read by the
+ *     framework!</li>
+ *   </ul>
+ */
+public class SimClusterStateProvider implements ClusterStateProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, List<ReplicaInfo>> nodeReplicaMap = new ConcurrentHashMap<>();
+  private final LiveNodesSet liveNodes;
+  private final SimDistribStateManager stateManager;
+  private final SimCloudManager cloudManager;
+
+  private final Map<String, Object> clusterProperties = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, Object>> collProperties = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, Map<String, Object>>> sliceProperties = new ConcurrentHashMap<>();
+
+  private final ReentrantLock lock = new ReentrantLock();
+
+  private final ActionThrottle leaderThrottle;
+
+  // default map of: operation -> delay
+  private final Map<String, Long> defaultOpDelays = new HashMap<>();
+  // per-collection map of: collection -> op -> delay
+  private final Map<String, Map<String, Long>> opDelays = new ConcurrentHashMap<>();
+
+
+  private volatile int clusterStateVersion = -1;
+  private Map<String, Object> lastSavedProperties = null;
+
+  private AtomicReference<Map<String, DocCollection>> collectionsStatesRef = new AtomicReference<>();
+  private AtomicBoolean saveClusterState = new AtomicBoolean();
+
+  /**
+   * The instance needs to be initialized using the <code>sim*</code> methods in order
+   * to ensure proper behavior, otherwise it will behave as a cluster with zero replicas.
+   */
+  public SimClusterStateProvider(LiveNodesSet liveNodes, SimCloudManager cloudManager) throws Exception {
+    this.liveNodes = liveNodes;
+    for (String nodeId : liveNodes.get()) {
+      createEphemeralLiveNode(nodeId);
+    }
+    this.cloudManager = cloudManager;
+    this.stateManager = cloudManager.getSimDistribStateManager();
+    this.leaderThrottle = new ActionThrottle("leader", 5000, cloudManager.getTimeSource());
+    // names are CollectionAction operation names, delays are in ms (simulated time)
+    defaultOpDelays.put(CollectionParams.CollectionAction.MOVEREPLICA.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETEREPLICA.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.ADDREPLICA.name(), 500L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.SPLITSHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.CREATESHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETESHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.CREATE.name(), 500L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETE.name(), 5000L);
+  }
+
+  // ============== SIMULATOR SETUP METHODS ====================
+
+  /**
+   * Initialize from an existing cluster state
+   * @param initialState initial cluster state
+   */
+  public void simSetClusterState(ClusterState initialState) throws Exception {
+    lock.lock();
+    try {
+      collProperties.clear();
+      sliceProperties.clear();
+      nodeReplicaMap.clear();
+      liveNodes.clear();
+      for (String nodeId : stateManager.listData(ZkStateReader.LIVE_NODES_ZKNODE)) {
+        if (stateManager.hasData(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId)) {
+          stateManager.removeData(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId, -1);
+        }
+        if (stateManager.hasData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId)) {
+          stateManager.removeData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId, -1);
+        }
+      }
+      liveNodes.addAll(initialState.getLiveNodes());
+      for (String nodeId : liveNodes.get()) {
+        createEphemeralLiveNode(nodeId);
+      }
+      initialState.forEachCollection(dc -> {
+        collProperties.computeIfAbsent(dc.getName(), name -> new ConcurrentHashMap<>()).putAll(dc.getProperties());
+        opDelays.computeIfAbsent(dc.getName(), c -> new HashMap<>()).putAll(defaultOpDelays);
+        dc.getSlices().forEach(s -> {
+          sliceProperties.computeIfAbsent(dc.getName(), name -> new ConcurrentHashMap<>())
+              .computeIfAbsent(s.getName(), name -> new HashMap<>()).putAll(s.getProperties());
+          s.getReplicas().forEach(r -> {
+            ReplicaInfo ri = new ReplicaInfo(r.getName(), r.getCoreName(), dc.getName(), s.getName(), r.getType(), r.getNodeName(), r.getProperties());
+            if (liveNodes.get().contains(r.getNodeName())) {
+              nodeReplicaMap.computeIfAbsent(r.getNodeName(), rn -> new ArrayList<>()).add(ri);
+            }
+          });
+        });
+      });
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Reset the leader election throttle.
+   */
+  public void simResetLeaderThrottle() {
+    leaderThrottle.reset();
+  }
+
+  /**
+   * Get random node id.
+   * @param random instance of random.
+   * @return one of the live nodes
+   */
+  public String simGetRandomNode(Random random) {
+    if (liveNodes.isEmpty()) {
+      return null;
+    }
+    List<String> nodes = new ArrayList<>(liveNodes.get());
+    return nodes.get(random.nextInt(nodes.size()));
+  }
+
+  // todo: maybe hook up DistribStateManager /clusterstate.json watchers?
+
+  /**
+   * Add a new node to the cluster.
+   * @param nodeId unique node id
+   */
+  public void simAddNode(String nodeId) throws Exception {
+    if (liveNodes.contains(nodeId)) {
+      throw new Exception("Node " + nodeId + " already exists");
+    }
+    liveNodes.add(nodeId);
+    createEphemeralLiveNode(nodeId);
+    nodeReplicaMap.putIfAbsent(nodeId, new ArrayList<>());
+  }
+
+  // utility class to run leader election in a separate thread and with throttling
+  // Note: leader election is a no-op if a shard leader already exists for each shard
+  private class LeaderElection implements Callable<Boolean> {
+    Collection<String> collections;
+    boolean saveClusterState;
+
+    LeaderElection(Collection<String> collections, boolean saveClusterState) {
+      this.collections = collections;
+      this.saveClusterState = saveClusterState;
+    }
+
+    @Override
+    public Boolean call() {
+      leaderThrottle.minimumWaitBetweenActions();
+      leaderThrottle.markAttemptingAction();
+      try {
+        simRunLeaderElection(collections, saveClusterState);
+      } catch (Exception e) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Remove node from a cluster. This is equivalent to a situation when a node is lost.
+   * All replicas that were assigned to this node are marked as DOWN.
+   * @param nodeId node id
+   * @return true if a node existed and was removed
+   */
+  public boolean simRemoveNode(String nodeId) throws Exception {
+    lock.lock();
+    try {
+      Set<String> collections = new HashSet<>();
+      // mark every replica on that node as down
+      setReplicaStates(nodeId, Replica.State.DOWN, collections);
+      boolean res = liveNodes.remove(nodeId);
+      if (!collections.isEmpty()) {
+        collectionsStatesRef.set(null);
+      }
+      // remove ephemeral nodes
+      stateManager.getRoot().removeEphemeralChildren(nodeId);
+      // create a nodeLost marker if needed
+      AutoScalingConfig cfg = stateManager.getAutoScalingConfig(null);
+      if (cfg.hasTriggerForEvents(TriggerEventType.NODELOST)) {
+        stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeId);
+      }
+      if (!collections.isEmpty()) {
+        cloudManager.submit(new LeaderElection(collections, true));
+      }
+      return res;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  // this method needs to be called under a lock
+  private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
+    List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
+    if (replicas != null) {
+      replicas.forEach(r -> {
+        r.getVariables().put(ZkStateReader.STATE_PROP, state.toString());
+        changedCollections.add(r.getCollection());
+      });
+    }
+  }
+
+  // this method needs to be called under a lock
+  private void createEphemeralLiveNode(String nodeId) throws Exception {
+    DistribStateManager mgr = stateManager.withEphemeralId(nodeId);
+    mgr.makePath(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId, null, CreateMode.EPHEMERAL, true);
+    AutoScalingConfig cfg = stateManager.getAutoScalingConfig(null);
+    if (cfg.hasTriggerForEvents(TriggerEventType.NODEADDED)) {
+      mgr.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId, null, CreateMode.EPHEMERAL, true);
+    }
+  }
+
+  /**
+   * Restore a previously removed node. This also simulates a short replica recovery state.
+   * @param nodeId node id to restore
+   * @return true when this operation restored any replicas, false otherwise (empty node).
+   */
+  public boolean simRestoreNode(String nodeId) throws Exception {
+    liveNodes.add(nodeId);
+    createEphemeralLiveNode(nodeId);
+    Set<String> collections = new HashSet<>();
+    lock.lock();
+    try {
+      setReplicaStates(nodeId, Replica.State.RECOVERING, collections);
+    } finally {
+      lock.unlock();
+    }
+    cloudManager.getTimeSource().sleep(1000);
+    lock.lock();
+    try {
+      setReplicaStates(nodeId, Replica.State.ACTIVE, collections);
+    } finally {
+      lock.unlock();
+    }
+    if (!collections.isEmpty()) {
+      collectionsStatesRef.set(null);
+      cloudManager.submit(new LeaderElection(collections, true));
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Add a new replica. Note that if any details of a replica (node, coreNodeName, SolrCore name, etc)
+   * are missing they will be filled in using the policy framework.
+   * @param message replica details
+   * @param results result of the operation
+   */
+  public void simAddReplica(ZkNodeProps message, NamedList results) throws Exception {
+    ClusterState clusterState = getClusterState();
+    DocCollection coll = clusterState.getCollection(message.getStr(ZkStateReader.COLLECTION_PROP));
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    message = AddReplicaCmd.assignReplicaDetails(cloudManager, clusterState, message, sessionWrapper);
+    if (sessionWrapper.get() != null) {
+      sessionWrapper.get().release();
+    }
+    if (message.getStr(CoreAdminParams.CORE_NODE_NAME) == null) {
+      message = message.plus(CoreAdminParams.CORE_NODE_NAME, Assign.assignCoreNodeName(stateManager, coll));
+    }
+    ReplicaInfo ri = new ReplicaInfo(
+        message.getStr(CoreAdminParams.CORE_NODE_NAME),
+        message.getStr(CoreAdminParams.NAME),
+        message.getStr(ZkStateReader.COLLECTION_PROP),
+        message.getStr(ZkStateReader.SHARD_ID_PROP),
+        Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT)),
+        message.getStr(CoreAdminParams.NODE),
+        message.getProperties()
+    );
+    simAddReplica(message.getStr(CoreAdminParams.NODE), ri, true);
+    results.add("success", "");
+  }
+
+  /**
+   * Add a replica. Note that all details of the replica must be present here, including
+   * node, coreNodeName and SolrCore name.
+   * @param nodeId node id where the replica will be added
+   * @param replicaInfo replica info
+   * @param runLeaderElection if true then run a leader election after adding the replica.
+   */
+  public void simAddReplica(String nodeId, ReplicaInfo replicaInfo, boolean runLeaderElection) throws Exception {
+    // make sure coreNodeName is unique across cluster
+    for (Map.Entry<String, List<ReplicaInfo>> e : nodeReplicaMap.entrySet()) {
+      for (ReplicaInfo ri : e.getValue()) {
+        if (ri.getCore().equals(replicaInfo.getCore())) {
+          throw new Exception("Duplicate SolrCore name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
+        }
+        if (ri.getName().equals(replicaInfo.getName())) {
+          throw new Exception("Duplicate coreNode name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
+        }
+      }
+    }
+    if (!liveNodes.contains(nodeId)) {
+      throw new Exception("Target node " + nodeId + " is not live: " + liveNodes);
+    }
+    // verify info
+    if (replicaInfo.getCore() == null) {
+      throw new Exception("Missing core: " + replicaInfo);
+    }
+    // XXX replica info is not supposed to have this as a variable
+    replicaInfo.getVariables().remove(ZkStateReader.SHARD_ID_PROP);
+    if (replicaInfo.getName() == null) {
+      throw new Exception("Missing name: " + replicaInfo);
+    }
+    if (replicaInfo.getNode() == null) {
+      throw new Exception("Missing node: " + replicaInfo);
+    }
+    if (!replicaInfo.getNode().equals(nodeId)) {
+      throw new Exception("Wrong node (not " + nodeId + "): " + replicaInfo);
+    }
+
+    lock.lock();
+    try {
+
+      opDelay(replicaInfo.getCollection(), CollectionParams.CollectionAction.ADDREPLICA.name());
+
+      List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, n -> new ArrayList<>());
+      // mark replica as active
+      replicaInfo.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
+      // add a property expected in tests
+      replicaInfo.getVariables().put(Suggestion.coreidxsize, 123450000);
+
+      replicas.add(replicaInfo);
+      // at this point nuke our cached DocCollection state
+      collectionsStatesRef.set(null);
+      LOG.trace("-- simAddReplica {}", replicaInfo);
+
+      Map<String, Object> values = cloudManager.getSimNodeStateProvider().simGetAllNodeValues()
+          .computeIfAbsent(nodeId, id -> new ConcurrentHashMap<>(SimCloudManager.createNodeValues(id)));
+      // update the number of cores and freedisk in node values
+      Integer cores = (Integer)values.get(ImplicitSnitch.CORES);
+      if (cores == null) {
+        cores = 0;
+      }
+      cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.CORES, cores + 1);
+      Integer disk = (Integer)values.get(ImplicitSnitch.DISK);
+      if (disk == null) {
+        disk = 1000;
+      }
+      cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk - 10);
+      if (runLeaderElection) {
+        cloudManager.submit(new LeaderElection(Collections.singleton(replicaInfo.getCollection()), true));
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Remove replica.
+   * @param nodeId node id
+   * @param coreNodeName coreNodeName
+   */
+  public void simRemoveReplica(String nodeId, String coreNodeName) throws Exception {
+    List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, n -> new ArrayList<>());
+    lock.lock();
+    try {
+      for (int i = 0; i < replicas.size(); i++) {
+        if (coreNodeName.equals(replicas.get(i).getName())) {
+          ReplicaInfo ri = replicas.remove(i);
+          collectionsStatesRef.set(null);
+
+          opDelay(ri.getCollection(), CollectionParams.CollectionAction.DELETEREPLICA.name());
+
+          // update the number of cores in node values, if node is live
+          if (liveNodes.contains(nodeId)) {
+            Integer cores = (Integer)cloudManager.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.CORES);
+            if (cores == null || cores == 0) {
+              throw new Exception("Unexpected value of 'cores' (" + cores + ") on node: " + nodeId);
+            }
+            cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.CORES, cores - 1);
+            Integer disk = (Integer)cloudManager.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.DISK);
+            if (disk == null || disk == 0) {
+              throw new Exception("Unexpected value of 'freedisk' (" + disk + ") on node: " + nodeId);
+            }
+            cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk + 10);
+          }
+          LOG.trace("-- simRemoveReplica {}", ri);
+          cloudManager.submit(new LeaderElection(Collections.singleton(ri.getCollection()), true));
+          return;
+        }
+      }
+      throw new Exception("Replica " + coreNodeName + " not found on node " + nodeId);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Save clusterstate.json to {@link DistribStateManager}.
+   * @return saved state
+   */
+  private ClusterState saveClusterState(ClusterState state) throws IOException {
+    byte[] data = Utils.toJSON(state);
+    try {
+      VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
+      int version = oldData != null ? oldData.getVersion() : -1;
+      stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
+      LOG.trace("-- saved cluster state version=" + clusterStateVersion +
+          ", zkVersion=" + (version + 1) + ", {}", state);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return state;
+  }
+
+  /**
+   * Delay an operation by a configured amount.
+   * @param collection collection name
+   * @param op operation name.
+   */
+  private void opDelay(String collection, String op) throws InterruptedException {
+    Map<String, Long> delays = opDelays.get(collection);
+    if (delays == null || delays.isEmpty() || !delays.containsKey(op)) {
+      return;
+    }
+    cloudManager.getTimeSource().sleep(delays.get(op));
+  }
+
+  /**
+   * Simulate running a shard leader election. This operation is a no-op if a leader already exists.
+   * If a new leader is elected the cluster state is saved.
+   * @param collections list of affected collections
+   * @param saveClusterState if true then save cluster state regardless of changes.
+   */
+  private synchronized void simRunLeaderElection(Collection<String> collections, boolean saveClusterState) throws Exception {
+    ClusterState state = getClusterState();
+    AtomicBoolean stateChanged = new AtomicBoolean(Boolean.FALSE);
+
+    state.forEachCollection(dc -> {
+      if (!collections.contains(dc.getName())) {
+        return;
+      }
+      dc.getSlices().forEach(s -> {
+        Replica leader = s.getLeader();
+        if (leader == null || !liveNodes.contains(leader.getNodeName())) {
+          LOG.trace("Running leader election for " + dc.getName() + " / " + s.getName());
+          if (s.getReplicas().isEmpty()) { // no replicas - punt
+            return;
+          }
+          // mark all replicas as non-leader (probably not necessary) and collect all active and live
+          List<ReplicaInfo> active = new ArrayList<>();
+          s.getReplicas().forEach(r -> {
+            AtomicReference<ReplicaInfo> riRef = new AtomicReference<>();
+            // find our ReplicaInfo for this replica
+            nodeReplicaMap.get(r.getNodeName()).forEach(info -> {
+              if (info.getName().equals(r.getName())) {
+                riRef.set(info);
+              }
+            });
+            ReplicaInfo ri = riRef.get();
+            if (ri == null) {
+              throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
+            }
+            synchronized (ri) {
+              if (ri.getVariables().remove(ZkStateReader.LEADER_PROP) != null) {
+                stateChanged.set(true);
+              }
+              if (r.isActive(liveNodes.get())) {
+                active.add(ri);
+              } else { // if it's on a node that is not live mark it down
+                if (!liveNodes.contains(r.getNodeName())) {
+                  ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
+                }
+              }
+            }
+          });
+          if (active.isEmpty()) {
+            LOG.warn("-- can't find any active replicas for " + dc.getName() + " / " + s.getName());
+            return;
+          }
+          // pick first active one
+          ReplicaInfo ri = null;
+          for (ReplicaInfo a : active) {
+            if (!a.getType().equals(Replica.Type.PULL)) {
+              ri = a;
+              break;
+            }
+          }
+          if (ri == null) {
+            LOG.warn("-- can't find any suitable replica type for " + dc.getName() + " / " + s.getName());
+            return;
+          }
+          synchronized (ri) {
+            ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
+          }
+          stateChanged.set(true);
+          LOG.debug("-- elected new leader for " + dc.getName() + " / " + s.getName() + ": " + ri);
+        } else {
+          LOG.trace("-- already has leader for {} / {}", dc.getName(), s.getName());
+        }
+      });
+    });
+    if (saveClusterState || stateChanged.get()) {
+      collectionsStatesRef.set(null);
+    }
+  }
+
+  /**
+   * Create a new collection. This operation uses policy framework for node and replica assignments.
+   * @param props collection details
+   * @param results results of the operation.
+   */
+  public void simCreateCollection(ZkNodeProps props, NamedList results) throws Exception {
+    if (props.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, props.getStr(CommonAdminParams.ASYNC));
+    }
+    boolean waitForFinalState = props.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
+    List<String> nodeList = new ArrayList<>();
+    List<String> shardNames = new ArrayList<>();
+    final String collectionName = props.getStr(NAME);
+    ClusterState clusterState = getClusterState();
+    ZkWriteCommand cmd = new ClusterStateMutator(cloudManager).createCollection(clusterState, props);
+    if (cmd.noop) {
+      LOG.warn("Collection {} already exists. exit", collectionName);
+      results.add("success", "no-op");
+      return;
+    }
+    opDelays.computeIfAbsent(collectionName, c -> new HashMap<>()).putAll(defaultOpDelays);
+
+    opDelay(collectionName, CollectionParams.CollectionAction.CREATE.name());
+
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    List<ReplicaPosition> replicaPositions = CreateCollectionCmd.buildReplicaPositions(cloudManager, getClusterState(), props,
+        nodeList, shardNames, sessionWrapper);
+    if (sessionWrapper.get() != null) {
+      sessionWrapper.get().release();
+    }
+    final CountDownLatch finalStateLatch = new CountDownLatch(replicaPositions.size());
+    AtomicInteger replicaNum = new AtomicInteger(1);
+    replicaPositions.forEach(pos -> {
+      Map<String, Object> replicaProps = new HashMap<>();
+      replicaProps.put(ZkStateReader.NODE_NAME_PROP, pos.node);
+      replicaProps.put(ZkStateReader.REPLICA_TYPE, pos.type.toString());
+      String coreName = String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, pos.shard, pos.type.name().substring(0,1).toLowerCase(Locale.ROOT),
+          replicaNum.getAndIncrement());
+      try {
+        replicaProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+        ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+            coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
+        cloudManager.submit(() -> {
+          simAddReplica(pos.node, ri, false);
+          finalStateLatch.countDown();
+          return true;
+        });
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    });
+    // add collection props
+    DocCollection coll = cmd.collection;
+    collProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()).putAll(coll.getProperties());
+    // add slice props
+    coll.getSlices().forEach(s -> {
+      Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll.getName(), c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(s.getName(), slice -> new ConcurrentHashMap<>());
+      s.getProperties().forEach((k, v) -> {
+        if (k != null && v != null) {
+          sliceProps.put(k, v);
+        }
+      });
+    });
+    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    if (waitForFinalState) {
+      boolean finished = finalStateLatch.await(cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, 60, TimeUnit.MILLISECONDS),
+          TimeUnit.MILLISECONDS);
+      if (!finished) {
+        results.add("failure", "Timeout waiting for all replicas to become active.");
+        return;
+      }
+    }
+    results.add("success", "");
+  }
+
+  /**
+   * Delete a collection
+   * @param collection collection name
+   * @param async async id
+   * @param results results of the operation
+   */
+  public void simDeleteCollection(String collection, String async, NamedList results) throws IOException {
+    if (async != null) {
+      results.add(CoreAdminParams.REQUESTID, async);
+    }
+    lock.lock();
+    try {
+      collProperties.remove(collection);
+      sliceProperties.remove(collection);
+
+      opDelay(collection, CollectionParams.CollectionAction.DELETE.name());
+
+      opDelays.remove(collection);
+      nodeReplicaMap.forEach((n, replicas) -> {
+        for (Iterator<ReplicaInfo> it = replicas.iterator(); it.hasNext(); ) {
+          ReplicaInfo ri = it.next();
+          if (ri.getCollection().equals(collection)) {
+            it.remove();
+            // update the number of cores in node values
+            Integer cores = (Integer) cloudManager.getSimNodeStateProvider().simGetNodeValue(n, "cores");
+            if (cores != null) { // node is still up
+              if (cores == 0) {
+                throw new RuntimeException("Unexpected value of 'cores' (" + cores + ") on node: " + n);
+              }
+              cloudManager.getSimNodeStateProvider().simSetNodeValue(n, "cores", cores - 1);
+            }
+          }
+        }
+      });
+      collectionsStatesRef.set(null);
+      results.add("success", "");
+    } catch (Exception e) {
+      LOG.warn("Exception", e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Remove all collections.
+   */
+  public void simDeleteAllCollections() throws Exception {
+    lock.lock();
+    try {
+      nodeReplicaMap.clear();
+      collProperties.clear();
+      sliceProperties.clear();
+      cloudManager.getSimNodeStateProvider().simGetAllNodeValues().forEach((n, values) -> {
+        values.put("cores", 0);
+      });
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Move replica. This uses a similar algorithm as {@link org.apache.solr.cloud.MoveReplicaCmd#moveNormalReplica(ClusterState, NamedList, String, String, DocCollection, Replica, Slice, int, boolean)}.
+   * @param message operation details
+   * @param results operation results.
+   */
+  public void simMoveReplica(ZkNodeProps message, NamedList results) throws Exception {
+    if (message.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, message.getStr(CommonAdminParams.ASYNC));
+    }
+    String collection = message.getStr(COLLECTION_PROP);
+    String targetNode = message.getStr("targetNode");
+    ClusterState clusterState = getClusterState();
+    DocCollection coll = clusterState.getCollection(collection);
+    if (coll == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
+    }
+    String replicaName = message.getStr(REPLICA_PROP);
+    Replica replica = coll.getReplica(replicaName);
+    if (replica == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Collection: " + collection + " replica: " + replicaName + " does not exist");
+    }
+    Slice slice = null;
+    for (Slice s : coll.getSlices()) {
+      if (s.getReplicas().contains(replica)) {
+        slice = s;
+      }
+    }
+    if (slice == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Replica has no 'slice' property! : " + replica);
+    }
+
+    opDelay(collection, CollectionParams.CollectionAction.MOVEREPLICA.name());
+
+    // TODO: for now simulate moveNormalReplica sequence, where we first add new replica and then delete the old one
+
+    String newSolrCoreName = Assign.buildSolrCoreName(stateManager, coll, slice.getName(), replica.getType());
+    String coreNodeName = Assign.assignCoreNodeName(stateManager, coll);
+    ReplicaInfo newReplica = new ReplicaInfo(coreNodeName, newSolrCoreName, collection, slice.getName(), replica.getType(), targetNode, null);
+    LOG.debug("-- new replica: " + newReplica);
+    // xxx should run leader election here already?
+    simAddReplica(targetNode, newReplica, false);
+    // this will trigger leader election
+    simRemoveReplica(replica.getNodeName(), replica.getName());
+    results.add("success", "");
+  }
+
+  /**
+   * Create a new shard. This uses a similar algorithm as {@link CreateShardCmd}.
+   * @param message operation details
+   * @param results operation results
+   */
+  public void simCreateShard(ZkNodeProps message, NamedList results) throws Exception {
+    if (message.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, message.getStr(CommonAdminParams.ASYNC));
+    }
+    String collectionName = message.getStr(COLLECTION_PROP);
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    ClusterState clusterState = getClusterState();
+    lock.lock();
+    try {
+      ZkWriteCommand cmd = new CollectionMutator(cloudManager).createShard(clusterState, message);
+      if (cmd.noop) {
+        results.add("success", "no-op");
+        return;
+      }
+
+      opDelay(collectionName, CollectionParams.CollectionAction.CREATESHARD.name());
+
+      // copy shard properties -- our equivalent of creating an empty shard in cluster state
+      DocCollection collection = cmd.collection;
+      Slice slice = collection.getSlice(sliceName);
+      Map<String, Object> props = sliceProperties.computeIfAbsent(collection.getName(), c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(sliceName, s -> new ConcurrentHashMap<>());
+      props.clear();
+      slice.getProperties().entrySet().stream()
+          .filter(e -> !e.getKey().equals("range"))
+          .filter(e -> !e.getKey().equals("replicas"))
+          .forEach(e -> props.put(e.getKey(), e.getValue()));
+      // 2. create new replicas
+      AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+      List<ReplicaPosition> positions = CreateShardCmd.buildReplicaPositions(cloudManager, clusterState, collectionName,
+          message, sessionWrapper);
+      if (sessionWrapper.get() != null) {
+        sessionWrapper.get().release();
+      }
+      AtomicInteger replicaNum = new AtomicInteger(1);
+      positions.forEach(pos -> {
+        Map<String, Object> replicaProps = new HashMap<>();
+        replicaProps.put(ZkStateReader.SHARD_ID_PROP, pos.shard);
+        replicaProps.put(ZkStateReader.NODE_NAME_PROP, pos.node);
+        replicaProps.put(ZkStateReader.REPLICA_TYPE, pos.type.toString());
+        replicaProps.put(ZkStateReader.BASE_URL_PROP, Utils.getBaseUrlForNodeName(pos.node, "http"));
+        String coreName = String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, pos.shard, pos.type.name().substring(0,1).toLowerCase(Locale.ROOT),
+            replicaNum.getAndIncrement());
+        try {
+          replicaProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+          ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+              coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
+          simAddReplica(pos.node, ri, false);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      });
+      Map<String, Object> colProps = collProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>());
+
+      cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+      results.add("success", "");
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Split a shard. This uses a similar algorithm as {@link SplitShardCmd}, including simulating its
+   * quirks, and leaving the original parent slice in place.
+   * @param message operation details
+   * @param results operation results.
+   */
+  public void simSplitShard(ZkNodeProps message, NamedList results) throws Exception {
+    String collectionName = message.getStr(COLLECTION_PROP);
+    AtomicReference<String> sliceName = new AtomicReference<>();
+    sliceName.set(message.getStr(SHARD_ID_PROP));
+    String splitKey = message.getStr("split.key");
+    ClusterState clusterState = getClusterState();
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice parentSlice = SplitShardCmd.getParentSlice(clusterState, collectionName, sliceName, splitKey);
+    List<DocRouter.Range> subRanges = new ArrayList<>();
+    List<String> subSlices = new ArrayList<>();
+    List<String> subShardNames = new ArrayList<>();
+
+    opDelay(collectionName, CollectionParams.CollectionAction.SPLITSHARD.name());
+
+    SplitShardCmd.fillRanges(cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames);
+    // mark the old slice as inactive
+    sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
+        .computeIfAbsent(sliceName.get(), s -> new ConcurrentHashMap<>())
+        .put(ZkStateReader.SHARD_STATE_PROP, Slice.State.INACTIVE.toString());
+    // add slice props
+    for (int i = 0; i < subRanges.size(); i++) {
+      String subSlice = subSlices.get(i);
+      DocRouter.Range range = subRanges.get(i);
+      Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(subSlice, ss -> new ConcurrentHashMap<>());
+      sliceProps.put(Slice.RANGE, range);
+      sliceProps.put(Slice.PARENT, sliceName.get());
+      sliceProps.put(ZkStateReader.SHARD_STATE_PROP, Slice.State.ACTIVE.toString());
+    }
+    // add replicas for new subShards
+    int repFactor = parentSlice.getReplicas().size();
+    List<ReplicaPosition> replicaPositions = Assign.identifyNodes(cloudManager,
+        clusterState,
+        new ArrayList<>(clusterState.getLiveNodes()),
+        collectionName,
+        new ZkNodeProps(collection.getProperties()),
+        // reproduce the bug
+        subSlices, repFactor, 0, 0);
+    PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
+    if (sessionWrapper != null) sessionWrapper.release();
+
+    for (ReplicaPosition replicaPosition : replicaPositions) {
+      String subSliceName = replicaPosition.shard;
+      String subShardNodeName = replicaPosition.node;
+      String solrCoreName = collectionName + "_" + subSliceName + "_replica" + (replicaPosition.index);
+      Map<String, Object> replicaProps = new HashMap<>();
+      replicaProps.put(ZkStateReader.SHARD_ID_PROP, replicaPosition.shard);
+      replicaProps.put(ZkStateReader.NODE_NAME_PROP, replicaPosition.node);
+      replicaProps.put(ZkStateReader.REPLICA_TYPE, replicaPosition.type.toString());
+      replicaProps.put(ZkStateReader.BASE_URL_PROP, Utils.getBaseUrlForNodeName(subShardNodeName, "http"));
+
+      ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+          solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps);
+      simAddReplica(replicaPosition.node, ri, false);
+    }
+    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    results.add("success", "");
+
+  }
+
+  /**
+   * Delete a shard. This uses a similar algorithm as {@link org.apache.solr.cloud.DeleteShardCmd}
+   * @param message operation details
+   * @param results operation results
+   */
+  public void simDeleteShard(ZkNodeProps message, NamedList results) throws Exception {
+    String collectionName = message.getStr(COLLECTION_PROP);
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    ClusterState clusterState = getClusterState();
+    DocCollection collection = clusterState.getCollection(collectionName);
+    if (collection == null) {
+      throw new Exception("Collection " + collectionName + " doesn't exist");
+    }
+    Slice slice = collection.getSlice(sliceName);
+    if (slice == null) {
+      throw new Exception(" Collection " + collectionName + " slice " + sliceName + " doesn't exist.");
+    }
+
+    opDelay(collectionName, CollectionParams.CollectionAction.DELETESHARD.name());
+
+    lock.lock();
+    try {
+      sliceProperties.computeIfAbsent(collectionName, coll -> new ConcurrentHashMap<>()).remove(sliceName);
+      nodeReplicaMap.forEach((n, replicas) -> {
+        Iterator<ReplicaInfo> it = replicas.iterator();
+        while (it.hasNext()) {
+          ReplicaInfo ri = it.next();
+          if (ri.getCollection().equals(collectionName) && ri.getShard().equals(sliceName)) {
+            it.remove();
+          }
+        }
+      });
+      collectionsStatesRef.set(null);
+      results.add("success", "");
+    } catch (Exception e) {
+      results.add("failure", e.toString());
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Saves cluster properties to clusterprops.json.
+   * @return current properties
+   */
+  private synchronized Map<String, Object> saveClusterProperties() throws Exception {
+    if (lastSavedProperties != null && lastSavedProperties.equals(clusterProperties)) {
+      return lastSavedProperties;
+    }
+    byte[] data = Utils.toJSON(clusterProperties);
+    VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_PROPS);
+    int version = oldData != null ? oldData.getVersion() : -1;
+    stateManager.setData(ZkStateReader.CLUSTER_PROPS, data, version);
+    lastSavedProperties = (Map)Utils.fromJSON(data);
+    return lastSavedProperties;
+  }
+
+  /**
+   * Set all cluster properties. This also updates the clusterprops.json data in
+   * {@link DistribStateManager}
+   * @param properties properties to set
+   */
+  public void simSetClusterProperties(Map<String, Object> properties) throws Exception {
+    lock.lock();
+    try {
+      clusterProperties.clear();
+      if (properties != null) {
+        this.clusterProperties.putAll(properties);
+      }
+      saveClusterProperties();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set a cluster property. This also updates the clusterprops.json data in
+   * {@link DistribStateManager}
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetClusterProperty(String key, Object value) throws Exception {
+    lock.lock();
+    try {
+      if (value != null) {
+        clusterProperties.put(key, value);
+      } else {
+        clusterProperties.remove(key);
+      }
+      saveClusterProperties();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set collection properties.
+   * @param coll collection name
+   * @param properties properties
+   */
+  public void simSetCollectionProperties(String coll, Map<String, Object> properties) throws Exception {
+    lock.lock();
+    try {
+      if (properties == null) {
+        collProperties.remove(coll);
+      } else {
+        Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+        props.clear();
+        props.putAll(properties);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set collection property.
+   * @param coll collection name
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetCollectionProperty(String coll, String key, String value) throws Exception {
+    Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+    lock.lock();
+    try {
+      if (value == null) {
+        props.remove(key);
+      } else {
+        props.put(key, value);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set slice properties.
+   * @param coll collection name
+   * @param slice slice name
+   * @param properties slice properties
+   */
+  public void simSetSliceProperties(String coll, String slice, Map<String, Object> properties) throws Exception {
+    Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll, c -> new HashMap<>()).computeIfAbsent(slice, s -> new HashMap<>());
+    lock.lock();
+    try {
+      sliceProps.clear();
+      if (properties != null) {
+        sliceProps.putAll(properties);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica.
+   * @param collection collection name
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetCollectionValue(String collection, String key, Object value) throws Exception {
+    simSetCollectionValue(collection, key, value, false);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica.
+   * @param collection collection name
+   * @param key property name
+   * @param value property value
+   * @param divide if the value is a {@link Number} and this param is true, then the value will be evenly
+   *               divided by the number of replicas.
+   */
+  public void simSetCollectionValue(String collection, String key, Object value, boolean divide) throws Exception {
+    simSetShardValue(collection, null, key, value, divide);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica in a selected shard.
+   * @param collection collection name
+   * @param shard shard name. If null then all shards will be affected.
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetShardValue(String collection, String shard, String key, Object value) throws Exception {
+    simSetShardValue(collection, shard, key, value, false);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica in a selected shard.
+   * @param collection collection name
+   * @param shard shard name. If null then all shards will be affected.
+   * @param key property name
+   * @param value property value
+   * @param divide if the value is a {@link Number} and this is true, then the value will be evenly
+   *               divided by the number of replicas.
+   */
+  public void simSetShardValue(String collection, String shard, String key, Object value, boolean divide) throws Exception {
+    List<ReplicaInfo> infos = new ArrayList<>();
+    nodeReplicaMap.forEach((n, replicas) -> {
+      replicas.forEach(r -> {
+        if (r.getCollection().equals(collection)) {
+          if (shard != null && !shard.equals(r.getShard())) {
+            return;
+          }
+          infos.add(r);
+        }
+      });
+    });
+    if (infos.isEmpty()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection " + collection + " doesn't exist.");
+    }
+    if (divide && value != null && (value instanceof Number)) {
+      value = ((Number)value).doubleValue() / infos.size();
+    }
+    for (ReplicaInfo r : infos) {
+      synchronized (r) {
+        if (value == null) {
+          r.getVariables().remove(key);
+        } else {
+          r.getVariables().put(key, value);
+        }
+      }
+    }
+  }
+
+  /**
+   * Return all replica infos for a node.
+   * @param node node id
+   * @return list of replicas on that node, or empty list if none
+   */
+  public List<ReplicaInfo> simGetReplicaInfos(String node) {
+    List<ReplicaInfo> replicas = nodeReplicaMap.get(node);
+    if (replicas == null) {
+      return Collections.emptyList();
+    } else {
+      return replicas;
+    }
+  }
+
+  /**
+   * List collections.
+   * @return list of existing collections.
+   */
+  public List<String> simListCollections() {
+    final Set<String> collections = new HashSet<>();
+    lock.lock();
+    try {
+      nodeReplicaMap.forEach((n, replicas) -> {
+        replicas.forEach(ri -> collections.add(ri.getCollection()));
+      });
+      return new ArrayList<>(collections);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  // interface methods
+
+  @Override
+  public ClusterState.CollectionRef getState(String collection) {
+    try {
+      return getClusterState().getCollectionRef(collection);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public Set<String> getLiveNodes() {
+    return liveNodes.get();
+  }
+
+  @Override
+  public List<String> resolveAlias(String alias) {
+    throw new UnsupportedOperationException("resolveAlias not implemented");
+  }
+
+  @Override
+  public ClusterState getClusterState() throws IOException {
+    Map<String, DocCollection> states = getCollectionStates();
+    ClusterState state = new ClusterState(clusterStateVersion, liveNodes.get(), states);
+    if (saveClusterState.getAndSet(false)) {
+      saveClusterState(state);
+    }
+    return state;
+  }
+
+  private Map<String, DocCollection> getCollectionStates() {
+    Map<String, DocCollection> collectionStates = collectionsStatesRef.get();
+    if (collectionStates != null) {
+      return collectionStates;
+    }
+    lock.lock();
+    collectionsStatesRef.set(null);
+    clusterStateVersion++;
+    saveClusterState.set(true);
+    try {
+      Map<String, Map<String, Map<String, Replica>>> collMap = new HashMap<>();
+      nodeReplicaMap.forEach((n, replicas) -> {
+        replicas.forEach(ri -> {
+          Map<String, Object> props;
+          synchronized (ri) {
+            props = new HashMap<>(ri.getVariables());
+          }
+          props.put(ZkStateReader.NODE_NAME_PROP, n);
+          props.put(ZkStateReader.CORE_NAME_PROP, ri.getCore());
+          props.put(ZkStateReader.REPLICA_TYPE, ri.getType().toString());
+          props.put(ZkStateReader.STATE_PROP, ri.getState().toString());
+          Replica r = new Replica(ri.getName(), props);
+          collMap.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
+              .computeIfAbsent(ri.getShard(), s -> new HashMap<>())
+              .put(ri.getName(), r);
+        });
+      });
+
+      // add empty slices
+      sliceProperties.forEach((c, perSliceProps) -> {
+        perSliceProps.forEach((slice, props) -> {
+          collMap.computeIfAbsent(c, co -> new ConcurrentHashMap<>()).computeIfAbsent(slice, s -> new ConcurrentHashMap<>());
+        });
+      });
+      // add empty collections
+      collProperties.keySet().forEach(c -> {
+        collMap.computeIfAbsent(c, co -> new ConcurrentHashMap<>());
+      });
+
+      Map<String, DocCollection> res = new HashMap<>();
+      collMap.forEach((coll, shards) -> {
+        Map<String, Slice> slices = new HashMap<>();
+        shards.forEach((s, replicas) -> {
+          Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>()).computeIfAbsent(s, sl -> new ConcurrentHashMap<>());
+          Slice slice = new Slice(s, replicas, sliceProps);
+          slices.put(s, slice);
+        });
+        Map<String, Object> collProps = collProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>());
+        DocCollection dc = new DocCollection(coll, slices, collProps, DocRouter.DEFAULT, clusterStateVersion, ZkStateReader.CLUSTER_STATE);
+        res.put(coll, dc);
+      });
+      collectionsStatesRef.set(res);
+      return res;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public Map<String, Object> getClusterProperties() {
+    return clusterProperties;
+  }
+
+  @Override
+  public String getPolicyNameByCollection(String coll) {
+    Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+    return (String)props.get("policy");
+  }
+
+  @Override
+  public void connect() {
+
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
new file mode 100644
index 0000000..f9f17a0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
@@ -0,0 +1,580 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.jute.Record;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.AutoScalingParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.IdUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CheckVersionRequest;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.DeleteRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link DistribStateManager} that keeps all data locally in a static structure. Instances of this
+ * class are identified by their id in order to simulate the deletion of ephemeral nodes when {@link #close()} is
+ * invoked.
+ */
+public class SimDistribStateManager implements DistribStateManager {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final class Node {
+    ReentrantLock dataLock = new ReentrantLock();
+    private int version = -1;
+    private int seq = 0;
+    private final CreateMode mode;
+    private final String clientId;
+    private final String path;
+    private final String name;
+    private final Node parent;
+    private byte[] data = null;
+    private Map<String, Node> children = new ConcurrentHashMap<>();
+    Set<Watcher> dataWatches = ConcurrentHashMap.newKeySet();
+    Set<Watcher> childrenWatches = ConcurrentHashMap.newKeySet();
+
+    Node(Node parent, String name, String path, CreateMode mode, String clientId) {
+      this.parent = parent;
+      this.name = name;
+      this.path = path;
+      this.mode = mode;
+      this.clientId = clientId;
+
+    }
+
+    public void clear() {
+      dataLock.lock();
+      try {
+        children.clear();
+        version = 0;
+        seq = 0;
+        dataWatches.clear();
+        childrenWatches.clear();
+        data = null;
+      } finally {
+        dataLock.unlock();
+      }
+    }
+
+    public void setData(byte[] data, int version) throws BadVersionException, IOException {
+      Set<Watcher> currentWatchers = new HashSet<>(dataWatches);
+      dataLock.lock();
+      try {
+        if (version != -1 && version != this.version) {
+          throw new BadVersionException(version, path);
+        }
+        if (data != null) {
+          this.data = Arrays.copyOf(data, data.length);
+        } else {
+          this.data = null;
+        }
+        this.version++;
+        dataWatches.clear();
+      } finally {
+        dataLock.unlock();
+      }
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeDataChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+    }
+
+    public VersionedData getData(Watcher w) {
+      dataLock.lock();
+      try {
+        VersionedData res = new VersionedData(version, data, clientId);
+        if (w != null && !dataWatches.contains(w)) {
+          dataWatches.add(w);
+        }
+        return res;
+      } finally {
+        dataLock.unlock();
+      }
+    }
+
+    public void setChild(String name, Node child) {
+      assert child.name.equals(name);
+      Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
+      dataLock.lock();
+      try {
+        children.put(name, child);
+        childrenWatches.clear();
+      } finally {
+        dataLock.unlock();
+      }
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+    }
+
+    public void removeChild(String name, int version) throws NoSuchElementException, BadVersionException, IOException {
+      Node n = children.get(name);
+      if (n == null) {
+        throw new NoSuchElementException(path + "/" + name);
+      }
+      if (version != -1 && version != n.version) {
+        throw new BadVersionException(version, path);
+      }
+      children.remove(name);
+      Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
+      childrenWatches.clear();
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+      currentWatchers = new HashSet<>(n.dataWatches);
+      n.dataWatches.clear();
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeDeleted, Watcher.Event.KeeperState.SyncConnected, n.path));
+      }
+      // TODO: not sure if it's correct to recurse and fire watches???
+      Set<String> kids = new HashSet<>(n.children.keySet());
+      for (String kid : kids) {
+        n.removeChild(kid, -1);
+      }
+    }
+
+    public void removeEphemeralChildren(String id) throws NoSuchElementException, BadVersionException, IOException {
+      Set<String> kids = new HashSet<>(children.keySet());
+      for (String kid : kids) {
+        Node n = children.get(kid);
+        if (n == null) {
+          continue;
+        }
+        if ((CreateMode.EPHEMERAL == n.mode || CreateMode.EPHEMERAL_SEQUENTIAL == n.mode) &&
+            id.equals(n.clientId)) {
+          removeChild(n.name, -1);
+        } else {
+          n.removeEphemeralChildren(id);
+        }
+      }
+    }
+
+  }
+
+  private final ReentrantLock multiLock = new ReentrantLock();
+
+  public static Node createNewRootNode() {
+    return new Node(null, "", "/", CreateMode.PERSISTENT, "__root__");
+  }
+
+  private final ExecutorService watchersPool;
+
+  private final AtomicReference<ActionThrottle> throttleRef = new AtomicReference<>();
+  private final AtomicReference<ActionError> errorRef = new AtomicReference<>();
+  private final String id;
+  private final Node root;
+
+  public SimDistribStateManager() {
+    this(null);
+  }
+
+  /**
+   * Construct new state manager that uses provided root node for storing data.
+   * @param root if null then a new root node will be created.
+   */
+  public SimDistribStateManager(Node root) {
+    this.id = IdUtils.timeRandomId();
+    this.root = root != null ? root : createNewRootNode();
+    watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new DefaultSolrThreadFactory("sim-watchers"));
+  }
+
+  public SimDistribStateManager(ActionThrottle actionThrottle, ActionError actionError) {
+    this(null, actionThrottle, actionError);
+  }
+
+  public SimDistribStateManager(Node root, ActionThrottle actionThrottle, ActionError actionError) {
+    this(root);
+    this.throttleRef.set(actionThrottle);
+    this.errorRef.set(actionError);
+  }
+
+  private SimDistribStateManager(String id, ExecutorService watchersPool, Node root, ActionThrottle actionThrottle,
+                                 ActionError actionError) {
+    this.id = id;
+    this.watchersPool = watchersPool;
+    this.root = root;
+    this.throttleRef.set(actionThrottle);
+    this.errorRef.set(actionError);
+  }
+
+  /**
+   * Create a copy of this instance using a specified ephemeral owner id. This is useful when performing
+   * node operations that require using a specific id. Note: this instance should never be closed, it can
+   * be just discarded after use.
+   * @param id ephemeral owner id
+   */
+  public SimDistribStateManager withEphemeralId(String id) {
+    return new SimDistribStateManager(id, watchersPool, root, throttleRef.get(), errorRef.get()) {
+      @Override
+      public void close() {
+        throw new UnsupportedOperationException("this instance should never be closed - instead close the parent instance.");
+      }
+    };
+  }
+
+  /**
+   * Get the root node of the tree used by this instance. It could be a static shared root node.
+   */
+  public Node getRoot() {
+    return root;
+  }
+
+  /**
+   * Clear this instance. All nodes, watchers and data is deleted.
+   */
+  public void clear() {
+    root.clear();
+  }
+
+  private void throttleOrError(String path) throws IOException {
+    ActionError err = errorRef.get();
+    if (err != null && err.shouldFail(path)) {
+      throw new IOException("Simulated error, path=" + path);
+    }
+    ActionThrottle throttle = throttleRef.get();
+    if (throttle != null) {
+      throttle.minimumWaitBetweenActions();
+      throttle.markAttemptingAction();
+    }
+  }
+
+  // this method should always be invoked under lock
+  private Node traverse(String path, boolean create, CreateMode mode) throws IOException {
+    if (path == null || path.isEmpty()) {
+      return null;
+    }
+    throttleOrError(path);
+    if (path.charAt(0) == '/') {
+      path = path.substring(1);
+    }
+    StringBuilder currentPath = new StringBuilder();
+    String[] elements = path.split("/");
+    Node parentNode = root;
+    Node n = null;
+    for (int i = 0; i < elements.length; i++) {
+      String currentName = elements[i];
+      currentPath.append('/');
+      LOG.info(" - parentNode=" + parentNode + ", currentName=" + currentName);
+      n = parentNode.children != null ? parentNode.children.get(currentName) : null;
+      if (n == null) {
+        if (create) {
+          if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
+              (mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
+            throw new IOException("NoChildrenEphemerals for " + parentNode.path);
+          }
+          if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
+            currentName = currentName + String.format(Locale.ROOT, "%010d", parentNode.seq);
+            parentNode.seq++;
+          }
+          currentPath.append(currentName);
+          n = new Node(parentNode, currentName, currentPath.toString(), mode, id);
+          parentNode.setChild(currentName, n);
+        } else {
+          break;
+        }
+      } else {
+        currentPath.append(currentName);
+      }
+      parentNode = n;
+    }
+    return n;
+  }
+
+  @Override
+  public void close() throws IOException {
+    multiLock.lock();
+    try {
+      // remove all my ephemeral nodes
+      root.removeEphemeralChildren(id);
+    } catch (BadVersionException e) {
+      // not happening
+    } finally {
+      multiLock.unlock();
+    }
+
+  }
+
+  @Override
+  public boolean hasData(String path) throws IOException {
+    multiLock.lock();
+    try {
+      return traverse(path, false, CreateMode.PERSISTENT) != null;
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public List<String> listData(String path) throws NoSuchElementException, IOException {
+    multiLock.lock();
+    try {
+      Node n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      List<String> res = new ArrayList<>(n.children.keySet());
+      Collections.sort(res);
+      return res;
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException {
+    Node n;
+    List<String> res;
+    multiLock.lock();
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      res = new ArrayList<>(n.children.keySet());
+      Collections.sort(res);
+    } finally {
+      multiLock.unlock();
+    }
+    if (watcher != null) {
+      n.dataWatches.add(watcher);
+      n.childrenWatches.add(watcher);
+    }
+    return res;
+  }
+
+  @Override
+  public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
+    Node n = null;
+    multiLock.lock();
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    return n.getData(watcher);
+  }
+
+  @Override
+  public void makePath(String path) throws IOException {
+    multiLock.lock();
+    try {
+      traverse(path, true, CreateMode.PERSISTENT);
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    Node n = null;
+    multiLock.lock();
+    try {
+      if (failOnExists && hasData(path)) {
+        throw new AlreadyExistsException(path);
+      }
+      n = traverse(path, true, createMode);
+    } finally {
+      multiLock.unlock();
+    }
+    try {
+      n.setData(data, -1);
+    } catch (BadVersionException e) {
+      throw new IOException("should not happen!", e);
+    }
+  }
+
+  @Override
+  public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, NoSuchElementException, IOException {
+    if ((CreateMode.EPHEMERAL == mode || CreateMode.PERSISTENT == mode) && hasData(path)) {
+      throw new AlreadyExistsException(path);
+    }
+    // check if parent exists
+    String relPath = path.charAt(0) == '/' ? path.substring(1) : path;
+    if (relPath.length() > 0) { // non-root path - check if parent exists
+      String[] elements = relPath.split("/");
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < elements.length - 1; i++) {
+        sb.append('/');
+        sb.append(elements[i]);
+      }
+      if (!hasData(sb.toString())) {
+        throw new NoSuchElementException(sb.toString());
+      }
+    }
+    Node n = null;
+    multiLock.lock();
+    try {
+      n = traverse(path, true, mode);
+    } finally {
+      multiLock.unlock();
+    }
+    try {
+      n.setData(data, -1);
+      return n.path;
+    } catch (BadVersionException e) {
+      // not happening
+      return null;
+    }
+  }
+
+  @Override
+  public void removeData(String path, int version) throws NoSuchElementException, BadVersionException, IOException {
+    multiLock.lock();
+    try {
+      Node n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      Node parent = n.parent;
+      if (parent == null) {
+        throw new IOException("Cannot remove root node");
+      }
+      parent.removeChild(n.name, version);
+    } finally {
+      multiLock.unlock();
+    }
+
+  }
+
+  @Override
+  public void setData(String path, byte[] data, int version) throws NoSuchElementException, BadVersionException, IOException {
+    multiLock.lock();
+    Node n = null;
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    n.setData(data, version);
+  }
+
+  @Override
+  public List<OpResult> multi(Iterable<Op> ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    multiLock.lock();
+    List<OpResult> res = new ArrayList<>();
+    try {
+      for (Op op : ops) {
+        Record r = op.toRequestRecord();
+        try {
+          if (op instanceof Op.Check) {
+            CheckVersionRequest rr = (CheckVersionRequest)r;
+            Node n = traverse(rr.getPath(), false, CreateMode.PERSISTENT);
+            if (n == null) {
+              throw new NoSuchElementException(rr.getPath());
+            }
+            if (rr.getVersion() != -1 && n.version != rr.getVersion()) {
+              throw new Exception("version mismatch");
+            }
+            // everything ok
+            res.add(new OpResult.CheckResult());
+          } else if (op instanceof Op.Create) {
+            CreateRequest rr = (CreateRequest)r;
+            createData(rr.getPath(), rr.getData(), CreateMode.fromFlag(rr.getFlags()));
+            res.add(new OpResult.CreateResult(rr.getPath()));
+          } else if (op instanceof Op.Delete) {
+            DeleteRequest rr = (DeleteRequest)r;
+            removeData(rr.getPath(), rr.getVersion());
+            res.add(new OpResult.DeleteResult());
+          } else if (op instanceof Op.SetData) {
+            SetDataRequest rr = (SetDataRequest)r;
+            setData(rr.getPath(), rr.getData(), rr.getVersion());
+            VersionedData vd = getData(rr.getPath());
+            Stat s = new Stat();
+            s.setVersion(vd.getVersion());
+            res.add(new OpResult.SetDataResult(s));
+          } else {
+            throw new Exception("Unknown Op: " + op);
+          }
+        } catch (Exception e) {
+          res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue()));
+        }
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    return res;
+  }
+
+  @Override
+  public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws InterruptedException, IOException {
+    Map<String, Object> map = new HashMap<>();
+    int version = -1;
+    try {
+      VersionedData data = getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, watcher);
+      if (data != null && data.getData() != null && data.getData().length > 0) {
+        map = (Map<String, Object>) Utils.fromJSON(data.getData());
+        version = data.getVersion();
+      }
+    } catch (NoSuchElementException e) {
+      // ignore
+    }
+    map.put(AutoScalingParams.ZK_VERSION, version);
+    return new AutoScalingConfig(map);
+  }
+
+  // ------------ simulator methods --------------
+
+  public void simSetAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
+    try {
+      makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    } catch (Exception e) {
+      // ignore
+    }
+    setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(cfg), -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
new file mode 100644
index 0000000..e9616f0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
@@ -0,0 +1,284 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Predicate;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Preconditions;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.cloud.Stats;
+import org.apache.solr.common.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link DistributedQueueFactory} that keeps all data in memory. Unlike
+ * the {@link GenericDistributedQueueFactory} this queue implementation data is not
+ * exposed anywhere.
+ */
+public class SimDistributedQueueFactory implements DistributedQueueFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  Map<String, SimDistributedQueue> queues = new ConcurrentHashMap<>();
+
+  public SimDistributedQueueFactory() {
+  }
+
+  @Override
+  public DistributedQueue makeQueue(final String path) throws IOException {
+    return queues.computeIfAbsent(path, p -> new SimDistributedQueue(path));
+  }
+
+  @Override
+  public void removeQueue(String path) throws IOException {
+    queues.remove(path);
+  }
+
+  public static class SimDistributedQueue implements DistributedQueue {
+    private final Queue<Pair<String, byte[]>> queue = new ConcurrentLinkedQueue<>();
+    private final ReentrantLock updateLock = new ReentrantLock();
+    private final Condition changed = updateLock.newCondition();
+    private final Stats stats = new Stats();
+    private final String dir;
+    private int seq = 0;
+
+    public SimDistributedQueue(String dir) {
+      this.dir = dir;
+    }
+
+    @Override
+    public byte[] peek() throws Exception {
+      Timer.Context time = stats.time(dir + "_peek");
+      try {
+        Pair<String, byte[]> pair = queue.peek();
+        return pair != null ? pair.second() : null;
+      } finally {
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] peek(boolean block) throws Exception {
+      return block ? peek(Long.MAX_VALUE) : peek();
+    }
+
+    @Override
+    public byte[] peek(long wait) throws Exception {
+      Timer.Context time;
+      if (wait == Long.MAX_VALUE) {
+        time = stats.time(dir + "_peek_wait_forever");
+      } else {
+        time = stats.time(dir + "_peek_wait" + wait);
+      }
+      try {
+        Pair<String, byte[]> pair = peekInternal(wait);
+        return pair != null ? pair.second() : null;
+      } finally {
+        time.stop();
+      }
+    }
+
+    private Pair<String, byte[]> peekInternal(long wait) throws Exception {
+      Preconditions.checkArgument(wait > 0);
+      long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+      updateLock.lockInterruptibly();
+      try {
+        while (waitNanos > 0) {
+          Pair<String, byte[]> pair = queue.peek();
+          if (pair != null) {
+            return pair;
+          }
+          waitNanos = changed.awaitNanos(waitNanos);
+          if (waitNanos < 0) { // timed out
+            return null;
+          }
+        }
+      } finally {
+        updateLock.unlock();
+      }
+      return null;
+    }
+
+    @Override
+    public byte[] poll() throws Exception {
+      Timer.Context time = stats.time(dir + "_poll");
+      updateLock.lockInterruptibly();
+      try {
+        Pair<String, byte[]>  pair = queue.poll();
+        if (pair != null) {
+          changed.signalAll();
+          return pair.second();
+        } else {
+          return null;
+        }
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] remove() throws Exception {
+      Timer.Context time = stats.time(dir + "_remove");
+      updateLock.lockInterruptibly();
+      try {
+        byte[] res = queue.remove().second();
+        changed.signalAll();
+        return res;
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] take() throws Exception {
+      Timer.Context timer = stats.time(dir + "_take");
+      updateLock.lockInterruptibly();
+      try {
+        while (true) {
+          byte[] result = poll();
+          if (result != null) {
+            return result;
+          }
+          changed.await();
+        }
+      } finally {
+        updateLock.unlock();
+        timer.stop();
+      }
+    }
+
+    @Override
+    public void offer(byte[] data) throws Exception {
+      Timer.Context time = stats.time(dir + "_offer");
+      updateLock.lockInterruptibly();
+      try {
+        queue.offer(new Pair(String.format(Locale.ROOT, "qn-%010d", seq), data));
+        seq++;
+        LOG.trace("=== offer " + System.nanoTime());
+        changed.signalAll();
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
+      updateLock.lockInterruptibly();
+      try {
+        List<Pair<String, byte[]>> res = new LinkedList<>();
+        final int maximum = max < 0 ? Integer.MAX_VALUE : max;
+        final AtomicReference<Pair<String, byte[]>> pairRef = new AtomicReference<>();
+        queue.forEach(pair -> {
+          if (acceptFilter != null && !acceptFilter.test(pair.first())) {
+            return;
+          }
+          if (res.size() < maximum) {
+            pairRef.set(pair);
+            res.add(pair);
+          }
+        });
+        if (res.size() < maximum && waitMillis > 0) {
+          long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
+          waitNanos = changed.awaitNanos(waitNanos);
+          if (waitNanos < 0) {
+            return res;
+          }
+          AtomicBoolean seen = new AtomicBoolean(false);
+          queue.forEach(pair -> {
+            if (!seen.get()) {
+              if (pairRef.get() == null) {
+                seen.set(true);
+              } else {
+                if (pairRef.get().first().equals(pair.first())) {
+                  seen.set(true);
+                  return;
+                }
+              }
+            }
+            if (!seen.get()) {
+              return;
+            }
+            if (!acceptFilter.test(pair.first())) {
+              return;
+            }
+            if (res.size() < maximum) {
+              res.add(pair);
+              pairRef.set(pair);
+            } else {
+              return;
+            }
+          });
+        }
+        return res;
+      } finally {
+        updateLock.unlock();
+      }
+    }
+
+    public Stats getZkStats() {
+      return stats;
+    }
+
+    @Override
+    public Map<String, Object> getStats() {
+      if (stats == null) {
+        return Collections.emptyMap();
+      }
+      Map<String, Object> res = new HashMap<>();
+      res.put("queueLength", stats.getQueueLength());
+      final Map<String, Object> statsMap = new HashMap<>();
+      res.put("stats", statsMap);
+      stats.getStats().forEach((op, stat) -> {
+        final Map<String, Object> statMap = new HashMap<>();
+        statMap.put("success", stat.success.get());
+        statMap.put("errors", stat.errors.get());
+        final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
+        statMap.put("failureDetails", failed);
+        stat.failureDetails.forEach(failedOp -> {
+          Map<String, Object> fo = new HashMap<>();
+          fo.put("req", failedOp.req);
+          fo.put("resp", failedOp.resp);
+        });
+        statsMap.put(op, statMap);
+      });
+      return res;
+    }
+  }
+}


[8/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
index 8f65255..9732616 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
@@ -26,9 +26,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.cloud.overseer.OverseerAction;
@@ -79,56 +81,26 @@ public class SplitShardCmd implements Cmd {
   }
 
   public boolean split(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
-    String collectionName = message.getStr("collection");
-    String slice = message.getStr(ZkStateReader.SHARD_ID_PROP);
     boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
+    String collectionName = message.getStr(CoreAdminParams.COLLECTION);
 
     log.info("Split shard invoked");
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     zkStateReader.forceUpdateCollection(collectionName);
+    AtomicReference<String> slice = new AtomicReference<>();
+    slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
 
     String splitKey = message.getStr("split.key");
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
-
     DocCollection collection = clusterState.getCollection(collectionName);
-    DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
-    PolicyHelper.SessionWrapper sessionWrapper = null;
-
-
-    Slice parentSlice;
 
-    if (slice == null) {
-      if (router instanceof CompositeIdRouter) {
-        Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
-        if (searchSlices.isEmpty()) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
-        }
-        if (searchSlices.size() > 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
-        }
-        parentSlice = searchSlices.iterator().next();
-        slice = parentSlice.getName();
-        log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
-      } else {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
-                + router.getClass().getName());
-      }
-    } else {
-      parentSlice = collection.getSlice(slice);
-    }
+    PolicyHelper.SessionWrapper sessionWrapper = null;
 
-    if (parentSlice == null) {
-      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
-      // an exception already
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
-    }
+    Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
 
     // find the leader for the shard
     Replica parentShardLeader = null;
     try {
-      parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice, 10000);
+      parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice.get(), 10000);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
     }
@@ -140,80 +112,13 @@ public class SplitShardCmd implements Cmd {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
     }
 
-    DocRouter.Range range = parentSlice.getRange();
-    if (range == null) {
-      range = new PlainIdRouter().fullRange();
-    }
+    List<DocRouter.Range> subRanges = new ArrayList<>();
+    List<String> subSlices = new ArrayList<>();
+    List<String> subShardNames = new ArrayList<>();
 
-    List<DocRouter.Range> subRanges = null;
-    String rangesStr = message.getStr(CoreAdminParams.RANGES);
-    if (rangesStr != null) {
-      String[] ranges = rangesStr.split(",");
-      if (ranges.length == 0 || ranges.length == 1) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
-      } else {
-        subRanges = new ArrayList<>(ranges.length);
-        for (int i = 0; i < ranges.length; i++) {
-          String r = ranges[i];
-          try {
-            subRanges.add(DocRouter.DEFAULT.fromString(r));
-          } catch (Exception e) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
-          }
-          if (!subRanges.get(i).isSubsetOf(range)) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
-          }
-        }
-        List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
-        Collections.sort(temp);
-        if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
-        }
-        for (int i = 1; i < temp.size(); i++) {
-          if (temp.get(i - 1).max + 1 != temp.get(i).min) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
-                + " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
-          }
-        }
-      }
-    } else if (splitKey != null) {
-      if (router instanceof CompositeIdRouter) {
-        CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
-        subRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
-        if (subRanges.size() == 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
-              + " has a hash range that is exactly equal to hash range of shard: " + slice);
-        }
-        for (DocRouter.Range subRange : subRanges) {
-          if (subRange.min == subRange.max) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
-          }
-        }
-        log.info("Partitioning parent shard " + slice + " range: " + parentSlice.getRange() + " yields: " + subRanges);
-        rangesStr = "";
-        for (int i = 0; i < subRanges.size(); i++) {
-          DocRouter.Range subRange = subRanges.get(i);
-          rangesStr += subRange.toString();
-          if (i < subRanges.size() - 1) rangesStr += ',';
-        }
-      }
-    } else {
-      // todo: fixed to two partitions?
-      subRanges = router.partitionRange(2, range);
-    }
+    String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames);
 
     try {
-      List<String> subSlices = new ArrayList<>(subRanges.size());
-      List<String> subShardNames = new ArrayList<>(subRanges.size());
-      String nodeName = parentShardLeader.getNodeName();
-      for (int i = 0; i < subRanges.size(); i++) {
-        String subSlice = slice + "_" + i;
-        subSlices.add(subSlice);
-        String subShardName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
-        subShardNames.add(subShardName);
-      }
 
       boolean oldShardsDeleted = false;
       for (String subSlice : subSlices) {
@@ -252,6 +157,7 @@ public class SplitShardCmd implements Cmd {
 
       final String asyncId = message.getStr(ASYNC);
       Map<String, String> requestMap = new HashMap<>();
+      String nodeName = parentShardLeader.getNodeName();
 
       for (int i = 0; i < subRanges.size(); i++) {
         String subSlice = subSlices.get(i);
@@ -300,6 +206,8 @@ public class SplitShardCmd implements Cmd {
         ocmh.addReplica(clusterState, new ZkNodeProps(propMap), results, null);
       }
 
+      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
+
       ocmh.processResponses(results, shardHandler, true, "SPLITSHARD failed to create subshard leaders", asyncId, requestMap);
 
       for (String subShardName : subShardNames) {
@@ -388,7 +296,7 @@ public class SplitShardCmd implements Cmd {
 
       // TODO: change this to handle sharding a slice into > 2 sub-shards.
 
-      List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh,
+      List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh.cloudManager,
           clusterState,
           new ArrayList<>(clusterState.getLiveNodes()),
           collectionName,
@@ -401,15 +309,15 @@ public class SplitShardCmd implements Cmd {
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String sliceName = replicaPosition.shard;
         String subShardNodeName = replicaPosition.node;
-        String shardName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
+        String solrCoreName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
 
-        log.info("Creating replica shard " + shardName + " as part of slice " + sliceName + " of collection "
+        log.info("Creating replica shard " + solrCoreName + " as part of slice " + sliceName + " of collection "
             + collectionName + " on " + subShardNodeName);
 
         ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
             ZkStateReader.COLLECTION_PROP, collectionName,
             ZkStateReader.SHARD_ID_PROP, sliceName,
-            ZkStateReader.CORE_NAME_PROP, shardName,
+            ZkStateReader.CORE_NAME_PROP, solrCoreName,
             ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
             ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
             ZkStateReader.NODE_NAME_PROP, subShardNodeName,
@@ -421,7 +329,7 @@ public class SplitShardCmd implements Cmd {
         propMap.put(COLLECTION_PROP, collectionName);
         propMap.put(SHARD_ID_PROP, sliceName);
         propMap.put("node", subShardNodeName);
-        propMap.put(CoreAdminParams.NAME, shardName);
+        propMap.put(CoreAdminParams.NAME, solrCoreName);
         // copy over property params:
         for (String key : message.keySet()) {
           if (key.startsWith(COLL_PROP_PREFIX)) {
@@ -478,7 +386,7 @@ public class SplitShardCmd implements Cmd {
         DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkStateReader.getZkClient());
         Map<String, Object> propMap = new HashMap<>();
         propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
-        propMap.put(slice, Slice.State.INACTIVE.toString());
+        propMap.put(slice.get(), Slice.State.INACTIVE.toString());
         for (String subSlice : subSlices) {
           propMap.put(subSlice, Slice.State.ACTIVE.toString());
         }
@@ -507,7 +415,7 @@ public class SplitShardCmd implements Cmd {
 
       log.info("Successfully created all replica shards for all sub-slices " + subSlices);
 
-      ocmh.commit(results, slice, parentShardLeader);
+      ocmh.commit(results, slice.get(), parentShardLeader);
 
       return true;
     } catch (SolrException e) {
@@ -519,4 +427,116 @@ public class SplitShardCmd implements Cmd {
       if (sessionWrapper != null) sessionWrapper.release();
     }
   }
+
+  public static Slice getParentSlice(ClusterState clusterState, String collectionName, AtomicReference<String> slice, String splitKey) {
+    DocCollection collection = clusterState.getCollection(collectionName);
+    DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+
+    Slice parentSlice;
+
+    if (slice.get() == null) {
+      if (router instanceof CompositeIdRouter) {
+        Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
+        if (searchSlices.isEmpty()) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
+        }
+        if (searchSlices.size() > 1) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
+        }
+        parentSlice = searchSlices.iterator().next();
+        slice.set(parentSlice.getName());
+        log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
+                + router.getClass().getName());
+      }
+    } else {
+      parentSlice = collection.getSlice(slice.get());
+    }
+
+    if (parentSlice == null) {
+      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
+      // an exception already
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
+    }
+    return parentSlice;
+  }
+
+  public static String fillRanges(SolrCloudManager cloudManager, ZkNodeProps message, DocCollection collection, Slice parentSlice,
+                                List<DocRouter.Range> subRanges, List<String> subSlices, List<String> subShardNames) {
+    String splitKey = message.getStr("split.key");
+    DocRouter.Range range = parentSlice.getRange();
+    if (range == null) {
+      range = new PlainIdRouter().fullRange();
+    }
+    DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+
+    String rangesStr = message.getStr(CoreAdminParams.RANGES);
+    if (rangesStr != null) {
+      String[] ranges = rangesStr.split(",");
+      if (ranges.length == 0 || ranges.length == 1) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
+      } else {
+        for (int i = 0; i < ranges.length; i++) {
+          String r = ranges[i];
+          try {
+            subRanges.add(DocRouter.DEFAULT.fromString(r));
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
+          }
+          if (!subRanges.get(i).isSubsetOf(range)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                "Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
+          }
+        }
+        List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
+        Collections.sort(temp);
+        if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
+        }
+        for (int i = 1; i < temp.size(); i++) {
+          if (temp.get(i - 1).max + 1 != temp.get(i).min) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
+                + " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
+          }
+        }
+      }
+    } else if (splitKey != null) {
+      if (router instanceof CompositeIdRouter) {
+        CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
+        List<DocRouter.Range> tmpSubRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
+        if (tmpSubRanges.size() == 1) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
+              + " has a hash range that is exactly equal to hash range of shard: " + parentSlice.getName());
+        }
+        for (DocRouter.Range subRange : tmpSubRanges) {
+          if (subRange.min == subRange.max) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
+          }
+        }
+        subRanges.addAll(tmpSubRanges);
+        log.info("Partitioning parent shard " + parentSlice.getName() + " range: " + parentSlice.getRange() + " yields: " + subRanges);
+        rangesStr = "";
+        for (int i = 0; i < subRanges.size(); i++) {
+          DocRouter.Range subRange = subRanges.get(i);
+          rangesStr += subRange.toString();
+          if (i < subRanges.size() - 1) rangesStr += ',';
+        }
+      }
+    } else {
+      // todo: fixed to two partitions?
+      subRanges.addAll(router.partitionRange(2, range));
+    }
+
+    for (int i = 0; i < subRanges.size(); i++) {
+      String subSlice = parentSlice.getName() + "_" + i;
+      subSlices.add(subSlice);
+      String subShardName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
+      subShardNames.add(subShardName);
+    }
+    return rangesStr;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
index e61536b..039067c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
@@ -133,13 +133,13 @@ public class AutoScaling {
    */
   public static class TriggerFactoryImpl extends TriggerFactory {
 
-    private final SolrCloudManager dataProvider;
+    private final SolrCloudManager cloudManager;
     private final SolrResourceLoader loader;
 
-    public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager dataProvider) {
-      Objects.requireNonNull(dataProvider);
+    public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager cloudManager) {
+      Objects.requireNonNull(cloudManager);
       Objects.requireNonNull(loader);
-      this.dataProvider = dataProvider;
+      this.cloudManager = cloudManager;
       this.loader = loader;
     }
 
@@ -150,11 +150,11 @@ public class AutoScaling {
       }
       switch (type) {
         case NODEADDED:
-          return new NodeAddedTrigger(name, props, loader, dataProvider);
+          return new NodeAddedTrigger(name, props, loader, cloudManager);
         case NODELOST:
-          return new NodeLostTrigger(name, props, loader, dataProvider);
+          return new NodeLostTrigger(name, props, loader, cloudManager);
         case SEARCHRATE:
-          return new SearchRateTrigger(name, props, loader, dataProvider);
+          return new SearchRateTrigger(name, props, loader, cloudManager);
         default:
           throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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
index 7ca0d36..86fd04a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
@@ -36,26 +36,23 @@ import java.util.stream.Stream;
 
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
-import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
 import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
-import org.apache.solr.cloud.ZkDistributedQueueFactory;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionAdminParams;
 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.core.SolrResourceLoader;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.RequestHandlerUtils;
 import org.apache.solr.request.SolrQueryRequest;
@@ -63,7 +60,7 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -80,15 +77,18 @@ import static org.apache.solr.common.params.CommonParams.JSON;
 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());
-  protected final CoreContainer container;
+  protected final SolrCloudManager cloudManager;
+  protected final SolrResourceLoader loader;
   private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
   private static Set<String> singletonCommands = Stream.of("set-cluster-preferences", "set-cluster-policy")
       .collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
-  private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
 
+  private final TimeSource timeSource;
 
-  public AutoScalingHandler(CoreContainer container) {
-    this.container = container;
+  public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
+    this.cloudManager = cloudManager;
+    this.loader = loader;
+    this.timeSource = cloudManager.getTimeSource();
     Map<String, String> map = new HashMap<>(2);
     map.put(NAME, "compute_plan");
     map.put(CLASS, "solr.ComputePlanAction");
@@ -116,7 +116,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
         }
 
-        AutoScalingConfig autoScalingConf = container.getZkController().zkStateReader.getAutoScalingConfig();
+        AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
         if (parts.size() == 2)  {
           autoScalingConf.writeMap(new MapWriter.EntryWriter() {
 
@@ -154,21 +154,14 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
 
   private void handleSuggestions(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      rsp.getValues().add("suggestions",
-          PolicyHelper.getSuggestions(autoScalingConf, new SolrClientCloudManager(queueFactory, build)));
-    }
-
-
+    rsp.getValues().add("suggestions",
+        PolicyHelper.getSuggestions(autoScalingConf, cloudManager));
   }
 
   public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List<CommandOperation> ops)
       throws KeeperException, InterruptedException, IOException {
     while (true) {
-      AutoScalingConfig initialConfig = container.getZkController().zkStateReader.getAutoScalingConfig();
+      AutoScalingConfig initialConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
       AutoScalingConfig currentConfig = initialConfig;
       for (CommandOperation op : ops) {
         switch (op.name) {
@@ -216,7 +209,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
       if (!currentConfig.equals(initialConfig)) {
         // update in ZK
-        if (zkSetAutoScalingConfig(container.getZkController().getZkStateReader(), currentConfig)) {
+        if (setAutoScalingConfig(currentConfig)) {
           break;
         } else {
           // someone else updated the config, get the latest one and re-apply our ops
@@ -244,12 +237,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
   private void handleDiagnostics(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
     Policy policy = autoScalingConf.getPolicy();
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, new SolrClientCloudManager(queueFactory, build)));
-    }
+    rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, cloudManager));
   }
 
   private AutoScalingConfig handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
@@ -302,7 +290,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       return currentConfig;
     }
 
-    container.getZkController().getZkStateReader().getClusterState().forEachCollection(coll -> {
+    cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
       if (policyName.equals(coll.getPolicyName()))
         op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
     });
@@ -470,7 +458,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
     // validate that we can load the listener class
     // todo allow creation from blobstore
     try {
-      container.getResourceLoader().findClass(listenerClass, TriggerListener.class);
+      loader.findClass(listenerClass, TriggerListener.class);
     } catch (Exception e) {
       log.warn("error loading listener class ", e);
       op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
@@ -535,7 +523,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       }
       String klass = action.get(CLASS);
       try {
-        container.getResourceLoader().findClass(klass, TriggerAction.class);
+        loader.findClass(klass, TriggerAction.class);
       } catch (Exception e) {
         log.warn("Could not load class : ", e);
         op.addError("Action not found: " + klass + " " + e.getMessage());
@@ -632,26 +620,22 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
   }
 
 
-  private boolean zkSetAutoScalingConfig(ZkStateReader reader, AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
+  private boolean setAutoScalingConfig(AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
     verifyAutoScalingConf(currentConfig);
     try {
-      reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion(), true);
-    } catch (KeeperException.BadVersionException bve) {
+      cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion());
+    } catch (BadVersionException bve) {
       // somebody else has changed the configuration so we must retry
       return false;
     }
+    //log.debug("-- saved version " + currentConfig.getZkVersion() + ": " + currentConfig);
     return true;
   }
 
   private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      Policy.Session session = autoScalingConf.getPolicy()
-          .createSession(new SolrClientCloudManager(queueFactory, build));
-      log.debug("Verified autoscaling configuration");
-    }
+    Policy.Session session = autoScalingConf.getPolicy()
+        .createSession(cloudManager);
+    log.debug("Verified autoscaling configuration");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
index ccffea7..8cce976 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
@@ -33,8 +33,10 @@ import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -57,12 +59,13 @@ public class ComputePlanAction extends TriggerActionBase {
       if (autoScalingConf.isEmpty()) {
         throw new Exception("Action: " + getName() + " executed but no policy is configured");
       }
-
-      //    Policy.Session session = cloudManager.getDistribStateManager().getAutoScalingConfig().getPolicy().createSession(cloudManager);
-//    return new PolicyHelper.SessionWrapper(session, null);
       PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(cloudManager);
       Policy.Session session = sessionWrapper.get();
-//      Policy policy = autoScalingConf.getPolicy();
+      if (log.isTraceEnabled()) {
+        ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
+        log.trace("-- session: {}", session);
+        log.trace("-- state: {}", state);
+      }
       try {
         Suggester suggester = getSuggester(session, event, cloudManager);
         while (true) {
@@ -99,12 +102,10 @@ public class ComputePlanAction extends TriggerActionBase {
       case NODEADDED:
         suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
             .hint(Suggester.Hint.TARGET_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
-        log.debug("NODEADDED Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
         break;
       case NODELOST:
         suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
             .hint(Suggester.Hint.SRC_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
-        log.debug("NODELOST Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
         break;
       case SEARCHRATE:
         Map<String, Map<String, Double>> hotShards = (Map<String, Map<String, Double>>)event.getProperty(AutoScalingParams.SHARD);
@@ -125,13 +126,10 @@ public class ComputePlanAction extends TriggerActionBase {
         } else {
           // collection || shard || replica -> ADDREPLICA
           suggester = session.getSuggester(CollectionParams.CollectionAction.ADDREPLICA);
-          Set<String> collections = new HashSet<>();
-          // XXX improve this when AddReplicaSuggester supports coll_shard hint
-          hotReplicas.forEach(r -> collections.add(r.getCollection()));
-          hotShards.forEach((coll, shards) -> collections.add(coll));
-          hotCollections.forEach((coll, rate) -> collections.add(coll));
-          for (String coll : collections) {
-            suggester = suggester.hint(Suggester.Hint.COLL, coll);
+          Set<Pair> collectionShards = new HashSet<>();
+          hotShards.forEach((coll, shards) -> shards.forEach((s, r) -> collectionShards.add(new Pair(coll, s))));
+          for (Pair<String, String> colShard : collectionShards) {
+            suggester = suggester.hint(Suggester.Hint.COLL_SHARD, colShard);
           }
         }
         break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
index ebe0660..47b3440 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
@@ -56,7 +56,7 @@ public class ExecutePlanAction extends TriggerActionBase {
   @Override
   public void process(TriggerEvent event, ActionContext context) throws Exception {
     log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
-    SolrCloudManager dataProvider = context.getCloudManager();
+    SolrCloudManager cloudManager = context.getCloudManager();
     List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
     if (operations == null || operations.isEmpty()) {
       log.info("No operations to execute for event: {}", event);
@@ -64,7 +64,7 @@ public class ExecutePlanAction extends TriggerActionBase {
     }
     try {
       for (SolrRequest operation : operations) {
-        log.info("Executing operation: {}", operation.getParams());
+        log.debug("Executing operation: {}", operation.getParams());
         try {
           SolrResponse response = null;
           int counter = 0;
@@ -73,22 +73,22 @@ public class ExecutePlanAction extends TriggerActionBase {
             // waitForFinalState so that the end effects of operations are visible
             req.setWaitForFinalState(true);
             String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
-            String znode = saveAsyncId(dataProvider.getDistribStateManager(), event, asyncId);
+            String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
             log.debug("Saved requestId: {} in znode: {}", asyncId, znode);
             // TODO: find a better way of using async calls using dataProvider API !!!
             req.setAsyncId(asyncId);
-            SolrResponse asyncResponse = dataProvider.request(req);
+            SolrResponse asyncResponse = cloudManager.request(req);
             if (asyncResponse.getResponse().get("error") != null) {
               throw new IOException("" + asyncResponse.getResponse().get("error"));
             }
             asyncId = (String)asyncResponse.getResponse().get("requestid");
-            CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(dataProvider, asyncId,
+            CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(cloudManager, asyncId,
                 DEFAULT_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
             if (statusResponse != null) {
               RequestStatusState state = statusResponse.getRequestStatus();
               if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
                 try {
-                  dataProvider.getDistribStateManager().removeData(znode, -1);
+                  cloudManager.getDistribStateManager().removeData(znode, -1);
                 } catch (Exception e) {
                   log.warn("Unexpected exception while trying to delete znode: " + znode, e);
                 }
@@ -96,7 +96,7 @@ public class ExecutePlanAction extends TriggerActionBase {
               response = statusResponse;
             }
           } else {
-            response = dataProvider.request(operation);
+            response = cloudManager.request(operation);
           }
           NamedList<Object> result = response.getResponse();
           context.getProperties().compute("responses", (s, o) -> {
@@ -106,16 +106,15 @@ public class ExecutePlanAction extends TriggerActionBase {
             return responses;
           });
         } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to talk to ZooKeeper", e);
-//        } catch (InterruptedException e) {
-//          Thread.currentThread().interrupt();
-//          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Unexpected exception executing operation: " + operation.getParams(), e);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
         } catch (Exception e) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Unexpected exception executing operation: " + operation.getParams(), e);
         }
-
-//        counter++;
       }
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -124,22 +123,22 @@ public class ExecutePlanAction extends TriggerActionBase {
   }
 
 
-  static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager dataProvider, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
+  static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager cloudManager, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
     long timeoutSeconds = timeUnit.toSeconds(duration);
     RequestStatusState state = RequestStatusState.NOT_FOUND;
     CollectionAdminRequest.RequestStatusResponse statusResponse = null;
     for (int i = 0; i < timeoutSeconds; i++) {
       try {
-        statusResponse = (CollectionAdminRequest.RequestStatusResponse)dataProvider.request(CollectionAdminRequest.requestStatus(requestId));
+        statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
         state = statusResponse.getRequestStatus();
         if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
-          log.info("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
-          dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
+          log.debug("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
+          cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
           return statusResponse;
         } else if (state == RequestStatusState.NOT_FOUND) {
           // the request for this id was never actually submitted! no harm done, just bail out
           log.warn("Task with requestId={} was not found on overseer", requestId);
-          dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
+          cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
           return statusResponse;
         }
       } catch (Exception e) {
@@ -154,11 +153,12 @@ public class ExecutePlanAction extends TriggerActionBase {
           throw e;
         }
         log.error("Unexpected Exception while querying status of requestId=" + requestId, e);
+        throw e;
       }
       if (i > 0 && i % 5 == 0) {
         log.debug("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
       }
-      TimeUnit.SECONDS.sleep(5);
+      cloudManager.getTimeSource().sleep(5000);
     }
     log.debug("Task with requestId={} did not complete within 5 minutes. Last state={}", requestId, state);
     return statusResponse;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
index 2003cb8..0388472 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
@@ -66,8 +66,8 @@ public class HttpTriggerListener extends TriggerListenerBase {
   private boolean followRedirects;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(dataProvider, config);
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    super.init(cloudManager, config);
     urlTemplate = (String)config.properties.get("url");
     payloadTemplate = (String)config.properties.get("payload");
     contentType = (String)config.properties.get("contentType");
@@ -148,7 +148,7 @@ public class HttpTriggerListener extends TriggerListenerBase {
     });
     headers.put("Content-Type", type);
     try {
-      dataProvider.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
+      cloudManager.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
     } catch (IOException e) {
       LOG.warn("Exception sending request for event " + event, e);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
index 2ef1b1d..a1e19e9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
@@ -35,7 +35,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,8 +44,6 @@ import org.slf4j.LoggerFactory;
 public class NodeAddedTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
-
   private Set<String> lastLiveNodes;
 
   private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
@@ -55,7 +52,6 @@ public class NodeAddedTrigger extends TriggerBase {
                           SolrResourceLoader loader,
                           SolrCloudManager cloudManager) {
     super(TriggerEventType.NODEADDED, name, properties, loader, cloudManager);
-    this.timeSource = TimeSource.CURRENT_TIME;
     lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
     log.debug("Initial livenodes: {}", lastLiveNodes);
     log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
@@ -71,7 +67,7 @@ public class NodeAddedTrigger extends TriggerBase {
         // don't add nodes that have since gone away
         if (lastLiveNodes.contains(n)) {
           log.debug("Adding node from marker path: {}", n);
-          nodeNameVsTimeAdded.put(n, timeSource.getTime());
+          nodeNameVsTimeAdded.put(n, cloudManager.getTimeSource().getTime());
         }
         removeMarker(n);
       });
@@ -131,7 +127,7 @@ public class NodeAddedTrigger extends TriggerBase {
       log.debug("Running NodeAddedTrigger {}", name);
 
       Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-      log.debug("Found livenodes: {}", newLiveNodes);
+      log.debug("Found livenodes: {}", newLiveNodes.size());
 
       // have any nodes that we were tracking been removed from the cluster?
       // if so, remove them from the tracking map
@@ -142,7 +138,7 @@ public class NodeAddedTrigger extends TriggerBase {
       Set<String> copyOfNew = new HashSet<>(newLiveNodes);
       copyOfNew.removeAll(lastLiveNodes);
       copyOfNew.forEach(n -> {
-        long eventTime = timeSource.getTime();
+        long eventTime = cloudManager.getTimeSource().getTime();
         log.debug("Tracking new node: {} at time {}", n, eventTime);
         nodeNameVsTimeAdded.put(n, eventTime);
       });
@@ -154,7 +150,7 @@ public class NodeAddedTrigger extends TriggerBase {
         Map.Entry<String, Long> entry = it.next();
         String nodeName = entry.getKey();
         Long timeAdded = entry.getValue();
-        long now = timeSource.getTime();
+        long now = cloudManager.getTimeSource().getTime();
         if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
           nodeNames.add(nodeName);
           times.add(timeAdded);
@@ -163,7 +159,8 @@ public class NodeAddedTrigger extends TriggerBase {
       AutoScaling.TriggerEventProcessor processor = processorRef.get();
       if (!nodeNames.isEmpty()) {
         if (processor != null) {
-          log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name, nodeNames, times, timeSource.getTime());
+          log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name,
+              nodeNames, times, cloudManager.getTimeSource().getTime());
           if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames))) {
             // remove from tracking set only if the fire was accepted
             nodeNames.forEach(n -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
index d53a354..57c76c0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
@@ -34,7 +34,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,8 +43,6 @@ import org.slf4j.LoggerFactory;
 public class NodeLostTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
-
   private Set<String> lastLiveNodes;
 
   private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
@@ -54,7 +51,6 @@ public class NodeLostTrigger extends TriggerBase {
                          SolrResourceLoader loader,
                          SolrCloudManager dataProvider) {
     super(TriggerEventType.NODELOST, name, properties, loader, dataProvider);
-    this.timeSource = TimeSource.CURRENT_TIME;
     lastLiveNodes = new HashSet<>(dataProvider.getClusterStateProvider().getLiveNodes());
     log.debug("Initial livenodes: {}", lastLiveNodes);
   }
@@ -69,7 +65,7 @@ public class NodeLostTrigger extends TriggerBase {
         // don't add nodes that have since came back
         if (!lastLiveNodes.contains(n)) {
           log.debug("Adding lost node from marker path: {}", n);
-          nodeNameVsTimeRemoved.put(n, timeSource.getTime());
+          nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
         }
         removeMarker(n);
       });
@@ -122,12 +118,12 @@ public class NodeLostTrigger extends TriggerBase {
       synchronized (this) {
         if (isClosed) {
           log.warn("NodeLostTrigger ran but was already closed");
-          throw new RuntimeException("Trigger has been closed");
+          return;
         }
       }
 
       Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-      log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes);
+      log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes.size());
 
       // have any nodes that we were tracking been added to the cluster?
       // if so, remove them from the tracking map
@@ -139,7 +135,7 @@ public class NodeLostTrigger extends TriggerBase {
       copyOfLastLiveNodes.removeAll(newLiveNodes);
       copyOfLastLiveNodes.forEach(n -> {
         log.debug("Tracking lost node: {}", n);
-        nodeNameVsTimeRemoved.put(n, timeSource.getTime());
+        nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
       });
 
       // has enough time expired to trigger events for a node?
@@ -149,7 +145,7 @@ public class NodeLostTrigger extends TriggerBase {
         Map.Entry<String, Long> entry = it.next();
         String nodeName = entry.getKey();
         Long timeRemoved = entry.getValue();
-        long now = timeSource.getTime();
+        long now = cloudManager.getTimeSource().getTime();
         if (TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
           nodeNames.add(nodeName);
           times.add(timeRemoved);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
index 3171404..7a9390b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
@@ -57,7 +57,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final SolrCloudManager dataProvider;
+  private final SolrCloudManager cloudManager;
 
   private final CloudConfig cloudConfig;
 
@@ -80,11 +80,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
 
   private AutoScalingConfig autoScalingConfig;
 
-  public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager dataProvider, CloudConfig cloudConfig) {
-    this.dataProvider = dataProvider;
+  public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager, CloudConfig cloudConfig) {
+    this.cloudManager = cloudManager;
     this.cloudConfig = cloudConfig;
-    scheduledTriggers = new ScheduledTriggers(loader, dataProvider);
-    triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, dataProvider);
+    scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
+    triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
   }
 
   @Override
@@ -114,11 +114,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
     // we automatically add a trigger for auto add replicas if it does not exists already
     while (!isClosed)  {
       try {
-        AutoScalingConfig autoScalingConfig = dataProvider.getDistribStateManager().getAutoScalingConfig();
+        AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
         AutoScalingConfig withAutoAddReplicasTrigger = withAutoAddReplicasTrigger(autoScalingConfig);
         if (withAutoAddReplicasTrigger.equals(autoScalingConfig)) break;
         log.debug("Adding .autoAddReplicas trigger");
-        dataProvider.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
+        cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
         break;
       } catch (BadVersionException bve) {
         // somebody else has changed the configuration so we must retry
@@ -225,7 +225,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
           throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
         }
       }
-      DistribStateManager stateManager = dataProvider.getDistribStateManager();
+      DistribStateManager stateManager = cloudManager.getDistribStateManager();
       if (cleanOldNodeLostMarkers) {
         log.debug("-- clean old nodeLost markers");
         try {
@@ -259,7 +259,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
   private void removeNodeMarker(String path, String nodeName) {
     path = path + "/" + nodeName;
     try {
-      dataProvider.getDistribStateManager().removeData(path, -1);
+      cloudManager.getDistribStateManager().removeData(path, -1);
       log.debug("  -- deleted " + path);
     } catch (NoSuchElementException e) {
       // ignore
@@ -297,7 +297,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
       if (isClosed) {
         return;
       }
-      AutoScalingConfig currentConfig = dataProvider.getDistribStateManager().getAutoScalingConfig(watcher);
+      AutoScalingConfig currentConfig = cloudManager.getDistribStateManager().getAutoScalingConfig(watcher);
       log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion());
       if (znodeVersion >= currentConfig.getZkVersion()) {
         // protect against reordered watcher fires by ensuring that we only move forward

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 8278977..8ebdf1a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -38,12 +38,12 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
@@ -78,13 +78,14 @@ import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_SCHEDULE_D
  */
 public class ScheduledTriggers implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
-  static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
-  static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
-  static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
+  public static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
+  public static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
+  public static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
+  public static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
 
   static final Map<String, Object> DEFAULT_PROPERTIES = new HashMap<>();
 
+  // Note: values must be all in milliseconds!
   static {
     DEFAULT_PROPERTIES.put(TRIGGER_SCHEDULE_DELAY_SECONDS, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
     DEFAULT_PROPERTIES.put(TRIGGER_COOLDOWN_PERIOD_SECONDS, DEFAULT_COOLDOWN_PERIOD_SECONDS);
@@ -114,7 +115,7 @@ public class ScheduledTriggers implements Closeable {
 
   private final AtomicLong cooldownPeriod = new AtomicLong(TimeUnit.SECONDS.toNanos(DEFAULT_COOLDOWN_PERIOD_SECONDS));
 
-  private final AtomicInteger triggerDelay = new AtomicInteger(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
+  private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
 
   private final AtomicReference<ActionThrottle> actionThrottle;
 
@@ -136,14 +137,14 @@ public class ScheduledTriggers implements Closeable {
     scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
     scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
     actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
-    actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS)));
+    actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS), cloudManager.getTimeSource()));
     this.cloudManager = cloudManager;
     this.stateManager = cloudManager.getDistribStateManager();
     this.loader = loader;
     queueStats = new Stats();
     listeners = new TriggerListeners();
     // initialize cooldown timer
-    cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
+    cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
   }
 
   /**
@@ -168,7 +169,9 @@ public class ScheduledTriggers implements Closeable {
               scheduledTriggers.forEach((s, scheduledTrigger) -> {
                 if (scheduledTrigger.scheduledFuture.cancel(false)) {
                   scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(
-                      scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
+                      scheduledTrigger, 0,
+                      cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
+                      TimeUnit.MILLISECONDS);
                 } else  {
                   log.debug("Failed to cancel scheduled task: {}", s);
                 }
@@ -188,9 +191,10 @@ public class ScheduledTriggers implements Closeable {
             if (oldThrottle.getLastActionStartedAt() != null) {
               newThrottle = new ActionThrottle("action",
                   minMsBetweenActions,
-                  oldThrottle.getLastActionStartedAt());
+                  oldThrottle.getLastActionStartedAt(),
+                  cloudManager.getTimeSource());
             } else  {
-              newThrottle = new ActionThrottle("action", minMsBetweenActions);
+              newThrottle = new ActionThrottle("action", minMsBetweenActions, cloudManager.getTimeSource());
             }
             this.actionThrottle.set(newThrottle);
             break;
@@ -200,12 +204,17 @@ public class ScheduledTriggers implements Closeable {
     this.autoScalingConfig = autoScalingConfig;
 
     // reset cooldown and actionThrottle
-    cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
+    cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
     actionThrottle.get().reset();
 
     listeners.setAutoScalingConfig(autoScalingConfig);
   }
 
+  @VisibleForTesting
+  void resetActionThrottle() {
+    actionThrottle.get().reset();
+  }
+
   /**
    * Adds a new trigger or replaces an existing one. The replaced trigger, if any, is closed
    * <b>before</b> the new trigger is run. If a trigger is replaced with itself then this
@@ -261,7 +270,7 @@ public class ScheduledTriggers implements Closeable {
       }
       boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
       AutoScaling.Trigger source = scheduledSource.trigger;
-      if (source.isClosed()) {
+      if (scheduledSource.isClosed || source.isClosed()) {
         String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s has already been closed", event.toString(), source);
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
         log.warn(msg);
@@ -269,7 +278,7 @@ public class ScheduledTriggers implements Closeable {
         return false;
       }
       // reject events during cooldown period
-      if (cooldownStart.get() + cooldownPeriod.get() > System.nanoTime()) {
+      if (cooldownStart.get() + cooldownPeriod.get() > cloudManager.getTimeSource().getTime()) {
         log.debug("-------- Cooldown period - rejecting event: " + event);
         event.getProperties().put(TriggerEvent.COOLDOWN, true);
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "In cooldown period.");
@@ -288,8 +297,16 @@ public class ScheduledTriggers implements Closeable {
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
         List<TriggerAction> actions = source.getActions();
         if (actions != null) {
+          if (actionExecutor.isShutdown()) {
+            String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the executor has already been closed", event.toString(), source);
+            listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
+            log.warn(msg);
+            // we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
+            return false;
+          }
           actionExecutor.submit(() -> {
             assert hasPendingActions.get();
+            long eventProcessingStart = cloudManager.getTimeSource().getTime();
             log.debug("-- processing actions for " + event);
             try {
               // let the action executor thread wait instead of the trigger thread so we use the throttle here
@@ -324,9 +341,11 @@ public class ScheduledTriggers implements Closeable {
             } catch (Exception e) {
               log.warn("Exception executing actions", e);
             } finally {
-              cooldownStart.set(System.nanoTime());
+              cooldownStart.set(cloudManager.getTimeSource().getTime());
               hasPendingActions.set(false);
             }
+            log.debug("-- processing took {} ms for event id={}",
+                TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getTime() - eventProcessingStart), event.id);
           });
         } else {
           if (enqueued) {
@@ -347,7 +366,9 @@ public class ScheduledTriggers implements Closeable {
       }
     });
     newTrigger.init(); // mark as ready for scheduling
-    scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
+    scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0,
+        cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
+        TimeUnit.MILLISECONDS);
   }
 
   private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
index 0c6ffd4..ec3110e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
@@ -37,7 +37,6 @@ import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.metrics.SolrCoreMetricManager;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +46,6 @@ import org.slf4j.LoggerFactory;
 public class SearchRateTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
   private final String handler;
   private final String collection;
   private final String shard;
@@ -63,7 +61,6 @@ public class SearchRateTrigger extends TriggerBase {
                            SolrResourceLoader loader,
                            SolrCloudManager cloudManager) {
     super(TriggerEventType.SEARCHRATE, name, properties, loader, cloudManager);
-    this.timeSource = TimeSource.CURRENT_TIME;
     this.state.put("lastCollectionEvent", lastCollectionEvent);
     this.state.put("lastNodeEvent", lastNodeEvent);
     this.state.put("lastShardEvent", lastShardEvent);
@@ -168,6 +165,9 @@ public class SearchRateTrigger extends TriggerBase {
           });
         });
       });
+      if (metricTags.isEmpty()) {
+        continue;
+      }
       Map<String, Object> rates = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
       rates.forEach((tag, rate) -> {
         ReplicaInfo info = metricTags.get(tag);
@@ -184,7 +184,7 @@ public class SearchRateTrigger extends TriggerBase {
       });
     }
 
-    long now = timeSource.getTime();
+    long now = cloudManager.getTimeSource().getTime();
     // check for exceeded rates and filter out those with less than waitFor from previous events
     Map<String, Double> hotNodes = nodeRates.entrySet().stream()
         .filter(entry -> node.equals(Policy.ANY) || node.equals(entry.getKey()))
@@ -274,7 +274,7 @@ public class SearchRateTrigger extends TriggerBase {
   private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
     Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
     long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
-    log.debug("name=" + name + ", lastTime=" + lastTime + ", elapsed=" + elapsed);
+    log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
     if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
index 34761f2..3282075 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
@@ -72,8 +72,8 @@ public class SystemLogListener extends TriggerListenerBase {
   private boolean enabled = true;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(dataProvider, config);
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    super.init(cloudManager, config);
     collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
     enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
   }
@@ -119,7 +119,7 @@ public class SystemLogListener extends TriggerListenerBase {
       UpdateRequest req = new UpdateRequest();
       req.add(doc);
       req.setParam(CollectionAdminParams.COLLECTION, collection);
-      dataProvider.request(req);
+      cloudManager.request(req);
     } catch (Exception e) {
       if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
         // relatively benign

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
index f40a49f..311c735 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
@@ -27,7 +27,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.cloud.Stats;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,7 +48,7 @@ public class TriggerEventQueue {
     // TODO: collect stats
     this.delegate = cloudManager.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
     this.triggerName = triggerName;
-    this.timeSource = TimeSource.CURRENT_TIME;
+    this.timeSource = cloudManager.getTimeSource();
   }
 
   public boolean offerEvent(TriggerEvent event) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
index 507c77d..61a95db 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
@@ -27,11 +27,11 @@ import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 public abstract class TriggerListenerBase implements TriggerListener {
 
   protected AutoScalingConfig.TriggerListenerConfig config;
-  protected SolrCloudManager dataProvider;
+  protected SolrCloudManager cloudManager;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    this.dataProvider = dataProvider;
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    this.cloudManager = cloudManager;
     this.config = config;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index c14f180..951c752 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -43,12 +43,12 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class CollectionMutator {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  protected final SolrCloudManager dataProvider;
+  protected final SolrCloudManager cloudManager;
   protected final DistribStateManager stateManager;
 
-  public CollectionMutator(SolrCloudManager dataProvider) {
-    this.dataProvider = dataProvider;
-    this.stateManager = dataProvider.getDistribStateManager();
+  public CollectionMutator(SolrCloudManager cloudManager) {
+    this.cloudManager = cloudManager;
+    this.stateManager = cloudManager.getDistribStateManager();
   }
 
   public ZkWriteCommand createShard(final ClusterState clusterState, ZkNodeProps message) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index 035de68..dbcdd3d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -245,7 +245,7 @@ public class ReplicaMutator {
         log.debug("node=" + coreNodeName + " is already registered");
       } else {
         // if coreNodeName is null, auto assign one
-        coreNodeName = Assign.assignNode(stateManager, collection);
+        coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
       }
       message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
           coreNodeName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 6e820b0..6718a80 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -73,7 +73,7 @@ public class SliceMutator {
     if (message.getStr(ZkStateReader.CORE_NODE_NAME_PROP) != null) {
       coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
     } else {
-      coreNodeName = Assign.assignNode(stateManager, collection);
+      coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
     }
     Replica replica = new Replica(coreNodeName,
         makeMap(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 5ed72d7..eb13775 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -540,8 +540,6 @@ public class CoreContainer {
     // 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);
@@ -690,6 +688,10 @@ public class CoreContainer {
 
     if (isZooKeeperAware()) {
       zkSys.getZkController().checkOverseerDesignate();
+      // initialize this handler here when SolrCloudManager is ready
+      autoScalingHandler = new AutoScalingHandler(getZkController().getSolrCloudManager(), loader);
+      containerHandlers.put(AutoScalingHandler.HANDLER_PATH, autoScalingHandler);
+      autoScalingHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AutoScalingHandler.HANDLER_PATH);
     }
     // This is a bit redundant but these are two distinct concepts for all they're accomplished at the same time.
     status |= LOAD_COMPLETE | INITIAL_CORE_LOAD_COMPLETE;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
index 6ef5ebd..d8f3d52 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
@@ -45,6 +45,7 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.CdcrUpdateLog;
 import org.apache.solr.util.TimeOut;
@@ -273,7 +274,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
         while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED)  {
           Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
         }
-        TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+        TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (!timeOut.hasTimedOut()) {
           if (closed) {
             log.warn("Cancelling waiting for bootstrap on target: {} shard: {} to complete", targetCollection, shard);
@@ -285,7 +286,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
             try {
               log.info("CDCR bootstrap running for {} seconds, sleeping for {} ms",
                   BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS), BOOTSTRAP_RETRY_DELAY_MS);
-              Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
+              timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
             } catch (InterruptedException e) {
               Thread.currentThread().interrupt();
             }
@@ -309,7 +310,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
               while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED)  {
                 Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
               }
-              timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS); // reset the timer
+              timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
               retries++;
             }
           } else if (status == BootstrapStatus.NOTFOUND || status == BootstrapStatus.CANCELLED) {
@@ -321,13 +322,13 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
               Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
             }
             retries = 1;
-            timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS); // reset the timer
+            timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
           } else if (status == BootstrapStatus.UNKNOWN || status == BootstrapStatus.SUBMITTED) {
             log.info("CDCR bootstrap is " + (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
                 BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
             // we were not able to query the status on the remote end
             // so just sleep for a bit and try again
-            Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
+            timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
           }
         }
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 de066d5..d339f27 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
@@ -391,7 +391,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     results.add("status", status);
   }
 
-  enum CollectionOperation implements CollectionOp {
+  public enum CollectionOperation implements CollectionOp {
     /**
      * very simple currently, you can pass a template collection, and the new collection is created on
      * every node the template collection is on

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 d75fccd..217c726 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -93,7 +94,7 @@ public class SchemaManager {
     if (timeout < 1) {
       timeout = 600;
     }
-    TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     SolrCore core = req.getCore();
     String errorMsg = "Unable to persist managed schema. ";
     List errors = Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 0e575d6..ba4aa13 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -78,6 +78,7 @@ import org.apache.solr.common.util.JsonSchemaValidator;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.core.CoreContainer;
@@ -347,7 +348,7 @@ public class HttpSolrCall {
       if (rsp.getValues().get("success") == null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create " + SYSTEM_COLL + " collection: "+ Utils.toJSONString(rsp.getValues()));
       }
-      TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       for (; ; ) {
         if (cores.getZkController().getClusterState().getCollectionOrNull(SYSTEM_COLL) != null) {
           break;
@@ -355,7 +356,7 @@ public class HttpSolrCall {
           if (timeOut.hasTimedOut()) {
             throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find " + SYSTEM_COLL + " collection even after 3 seconds");
           }
-          Thread.sleep(50);
+          timeOut.sleep(50);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 9f9b742..de031a2 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -68,6 +68,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.Hash;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.component.RealTimeGetComponent;
@@ -1142,7 +1143,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   private long waitForDependentUpdates(AddUpdateCommand cmd, long versionOnUpdate,
                                boolean isReplayOrPeersync, VersionBucket bucket) throws IOException {
     long lastFoundVersion = 0;
-    TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS); 
+    TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     vinfo.lockForUpdate();
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/IdUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/IdUtils.java b/solr/core/src/java/org/apache/solr/util/IdUtils.java
index a6ea7d2..4f841f3 100644
--- a/solr/core/src/java/org/apache/solr/util/IdUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/IdUtils.java
@@ -19,6 +19,7 @@ package org.apache.solr.util;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.util.StringHelper;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * Helper class for generating unique ID-s.


[9/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
SOLR-11285: Simulation framework for autoscaling.


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

Branch: refs/heads/master
Commit: d6d2e3b2e30b600b9f550abf9414743d628cbcf0
Parents: d66d954
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Dec 14 12:13:05 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Dec 14 12:56:23 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |    4 +
 .../org/apache/solr/cloud/ActionThrottle.java   |   14 +-
 .../org/apache/solr/cloud/AddReplicaCmd.java    |  149 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |   24 +-
 .../solr/cloud/CloudConfigSetService.java       |    2 +-
 .../java/org/apache/solr/cloud/CloudUtil.java   |    7 +
 .../apache/solr/cloud/CreateCollectionCmd.java  |  244 ++--
 .../org/apache/solr/cloud/CreateShardCmd.java   |  102 +-
 .../apache/solr/cloud/DeleteCollectionCmd.java  |    9 +-
 .../org/apache/solr/cloud/DeleteShardCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/MigrateCmd.java  |    9 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/Overseer.java    |   16 +-
 .../cloud/OverseerCollectionMessageHandler.java |   43 +-
 .../java/org/apache/solr/cloud/RestoreCmd.java  |    2 +-
 .../org/apache/solr/cloud/SplitShardCmd.java    |  248 ++--
 .../solr/cloud/autoscaling/AutoScaling.java     |   14 +-
 .../cloud/autoscaling/AutoScalingHandler.java   |   70 +-
 .../cloud/autoscaling/ComputePlanAction.java    |   24 +-
 .../cloud/autoscaling/ExecutePlanAction.java    |   38 +-
 .../cloud/autoscaling/HttpTriggerListener.java  |    6 +-
 .../cloud/autoscaling/NodeAddedTrigger.java     |   15 +-
 .../solr/cloud/autoscaling/NodeLostTrigger.java |   14 +-
 .../autoscaling/OverseerTriggerThread.java      |   20 +-
 .../cloud/autoscaling/ScheduledTriggers.java    |   53 +-
 .../cloud/autoscaling/SearchRateTrigger.java    |   10 +-
 .../cloud/autoscaling/SystemLogListener.java    |    6 +-
 .../cloud/autoscaling/TriggerEventQueue.java    |    4 +-
 .../cloud/autoscaling/TriggerListenerBase.java  |    6 +-
 .../solr/cloud/overseer/CollectionMutator.java  |    8 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |    2 +-
 .../solr/cloud/overseer/SliceMutator.java       |    2 +-
 .../org/apache/solr/core/CoreContainer.java     |    6 +-
 .../solr/handler/CdcrReplicatorManager.java     |   11 +-
 .../solr/handler/admin/CollectionsHandler.java  |    2 +-
 .../org/apache/solr/schema/SchemaManager.java   |    3 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |    5 +-
 .../processor/DistributedUpdateProcessor.java   |    3 +-
 .../src/java/org/apache/solr/util/IdUtils.java  |    1 +
 .../src/java/org/apache/solr/util/TimeOut.java  |   18 +-
 .../java/org/apache/solr/util/TimeSource.java   |   57 -
 .../solr/util/xslt/TransformerProvider.java     |    3 +-
 .../apache/solr/cloud/ActionThrottleTest.java   |   12 +-
 .../org/apache/solr/cloud/AddReplicaTest.java   |    2 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   10 +-
 ...MonkeyNothingIsSafeWithPullReplicasTest.java |    3 +-
 ...aosMonkeySafeLeaderWithPullReplicasTest.java |    3 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |    5 +-
 ...ConcurrentDeleteAndCreateCollectionTest.java |    3 +-
 ...DistribDocExpirationUpdateProcessorTest.java |    3 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java |    3 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |    3 +-
 ...verseerCollectionConfigSetProcessorTest.java |    9 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |    3 +-
 .../solr/cloud/PeerSyncReplicationTest.java     |    3 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java  |    3 +-
 .../TestLeaderInitiatedRecoveryThread.java      |    7 +-
 .../org/apache/solr/cloud/TestPullReplica.java  |    7 +-
 .../cloud/TestPullReplicaErrorHandling.java     |    9 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java |    5 +-
 .../org/apache/solr/cloud/TestTlogReplica.java  |   11 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |    3 +-
 .../AutoAddReplicasIntegrationTest.java         |    3 +-
 .../AutoAddReplicasPlanActionTest.java          |    4 +-
 .../autoscaling/AutoScalingHandlerTest.java     |    3 +-
 .../solr/cloud/autoscaling/CapturedEvent.java   |   63 +
 .../autoscaling/ComputePlanActionTest.java      |   16 +
 .../autoscaling/ExecutePlanActionTest.java      |    2 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java |    2 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |    2 +-
 .../autoscaling/TriggerIntegrationTest.java     |  155 +--
 .../solr/cloud/autoscaling/sim/ActionError.java |   24 +
 .../sim/GenericDistributedQueue.java            |  599 ++++++++
 .../sim/GenericDistributedQueueFactory.java     |   45 +
 .../cloud/autoscaling/sim/LiveNodesSet.java     |   99 ++
 .../cloud/autoscaling/sim/SimCloudManager.java  |  607 +++++++++
 .../sim/SimClusterStateProvider.java            | 1275 ++++++++++++++++++
 .../autoscaling/sim/SimDistribStateManager.java |  580 ++++++++
 .../sim/SimDistributedQueueFactory.java         |  284 ++++
 .../autoscaling/sim/SimNodeStateProvider.java   |  267 ++++
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  251 ++++
 .../sim/TestClusterStateProvider.java           |  221 +++
 .../autoscaling/sim/TestComputePlanAction.java  |  357 +++++
 .../sim/TestDistribStateManager.java            |  284 ++++
 .../autoscaling/sim/TestExecutePlanAction.java  |  216 +++
 .../sim/TestGenericDistributedQueue.java        |   32 +
 .../cloud/autoscaling/sim/TestLargeCluster.java |  266 ++++
 .../autoscaling/sim/TestNodeAddedTrigger.java   |  306 +++++
 .../autoscaling/sim/TestNodeLostTrigger.java    |  331 +++++
 .../cloud/autoscaling/sim/TestPolicyCloud.java  |  357 +++++
 .../sim/TestSimDistributedQueue.java            |  220 +++
 .../autoscaling/sim/TestTriggerIntegration.java | 1217 +++++++++++++++++
 .../cloud/autoscaling/sim/package-info.java     |   21 +
 .../cloud/cdcr/BaseCdcrDistributedZkTest.java   |    3 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |    3 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |    3 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |    3 +
 .../solr/core/OpenCloseCoreStressTest.java      |    7 +-
 .../cloud/autoscaling/AutoScalingConfig.java    |    9 +
 .../autoscaling/DelegatingCloudManager.java     |   17 +
 .../DelegatingDistribStateManager.java          |   17 +-
 .../DelegatingNodeStateProvider.java            |   11 +
 .../cloud/autoscaling/DistribStateManager.java  |   23 +-
 .../cloud/autoscaling/NodeStateProvider.java    |    4 +-
 .../client/solrj/cloud/autoscaling/Policy.java  |   21 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java   |   26 +-
 .../solrj/cloud/autoscaling/ReplicaInfo.java    |   33 +-
 .../cloud/autoscaling/SolrCloudManager.java     |   10 +-
 .../solrj/cloud/autoscaling/Suggestion.java     |    2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |    5 +-
 .../client/solrj/impl/ClusterStateProvider.java |    4 +-
 .../solrj/impl/HttpClusterStateProvider.java    |   21 +-
 .../solrj/impl/SolrClientCloudManager.java      |    6 +
 .../solrj/impl/SolrClientNodeStateProvider.java |    6 +
 .../solrj/impl/ZkDistribStateManager.java       |   32 +-
 .../apache/solr/common/cloud/ClusterState.java  |   10 +
 .../org/apache/solr/common/cloud/Replica.java   |   11 +
 .../apache/solr/common/cloud/ZkStateReader.java |   19 +-
 .../org/apache/solr/common/util/TimeSource.java |  161 +++
 .../java/org/apache/solr/common/util/Utils.java |   23 +-
 .../solr/client/solrj/SolrExampleTestsBase.java |    3 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |    3 +-
 .../solrj/cloud/autoscaling/TestPolicy.java     |   12 +
 .../cloud/AbstractFullDistribZkTestBase.java    |    7 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |    3 +-
 .../org/apache/solr/cloud/ZkTestServer.java     |    3 +-
 126 files changed, 9186 insertions(+), 843 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b6e729f..a3f6f75 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -49,6 +49,10 @@ Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.10
 Jetty 9.3.20.v20170531
 
+New Features
+----------------------
+* SOLR-11285: Simulation framework for autoscaling. (ab)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
index f60332c..520a269 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
@@ -20,7 +20,7 @@ import java.lang.invoke.MethodHandles;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,9 +35,7 @@ public class ActionThrottle {
   private final TimeSource timeSource;
 
   public ActionThrottle(String name, long minMsBetweenActions) {
-    this.name = name;
-    this.minMsBetweenActions = minMsBetweenActions;
-    this.timeSource = TimeSource.NANO_TIME;
+    this(name, minMsBetweenActions, TimeSource.NANO_TIME);
   }
   
   public ActionThrottle(String name, long minMsBetweenActions, TimeSource timeSource) {
@@ -47,16 +45,20 @@ public class ActionThrottle {
   }
 
   public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt)  {
+    this(name, minMsBetweenActions, lastActionStartedAt, TimeSource.NANO_TIME);
+  }
+
+  public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt, TimeSource timeSource)  {
     this.name = name;
     this.minMsBetweenActions = minMsBetweenActions;
     this.lastActionStartedAt = lastActionStartedAt;
-    this.timeSource = TimeSource.NANO_TIME;
+    this.timeSource = timeSource;
   }
 
   public void reset() {
     lastActionStartedAt = null;
   }
-  
+
   public void markAttemptingAction() {
     lastActionStartedAt = timeSource.getTime();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
index c785f9f..71a54c14 100644
--- a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -48,7 +49,6 @@ import org.apache.solr.handler.component.ShardHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.Assign.getNodesForNewReplicas;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@@ -56,6 +56,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonAdminParams.TIMEOUT;
 import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
 
 public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@@ -69,81 +70,30 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
-    addReplica(ocmh.zkStateReader.getClusterState(), message, results, null);
+    addReplica(state, message, results, null);
   }
 
   ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
       throws IOException, InterruptedException {
     log.debug("addReplica() : {}", Utils.toJSONString(message));
+    boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
+    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
+    final String asyncId = message.getStr(ASYNC);
+
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    message = assignReplicaDetails(ocmh.cloudManager, clusterState, message, sessionWrapper);
+
     String collection = message.getStr(COLLECTION_PROP);
+    DocCollection coll = clusterState.getCollection(collection);
+
     String node = message.getStr(CoreAdminParams.NODE);
     String shard = message.getStr(SHARD_ID_PROP);
     String coreName = message.getStr(CoreAdminParams.NAME);
     String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
-    int timeout = message.getInt("timeout", 10 * 60); // 10 minutes
+    int timeout = message.getInt(TIMEOUT, 10 * 60); // 10 minutes
     Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
     boolean parallel = message.getBool("parallel", false);
-    boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
-    if (StringUtils.isBlank(coreName)) {
-      coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
-    }
-
-    final String asyncId = message.getStr(ASYNC);
-
-    DocCollection coll = clusterState.getCollection(collection);
-    if (coll == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
-    }
-    if (coll.getSlice(shard) == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Collection: " + collection + " shard: " + shard + " does not exist");
-    }
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
-    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
-
 
-    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
-    // Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
-    if (!skipCreateReplicaInClusterState) {
-      if (CreateShardCmd.usePolicyFramework(coll, ocmh)) {
-        if (node == null) {
-          if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
-          node = Assign.identifyNodes(ocmh,
-              clusterState,
-              Collections.emptyList(),
-              collection,
-              message,
-              Collections.singletonList(shard),
-              replicaType == Replica.Type.NRT ? 0 : 1,
-              replicaType == Replica.Type.TLOG ? 0 : 1,
-              replicaType == Replica.Type.PULL ? 0 : 1
-          ).get(0).node;
-          sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
-        }
-      } else {
-        node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
-            ocmh.overseer.getSolrCloudManager()).get(0).nodeName;// TODO: use replica type in this logic too
-      }
-    }
-    log.info("Node Identified {} for creating new replica", node);
-
-    if (!clusterState.liveNodesContain(node)) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
-    }
-    if (coreName == null) {
-      coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, shard, replicaType);
-    } else if (!skipCreateReplicaInClusterState) {
-      //Validate that the core name is unique in that collection
-      for (Slice slice : coll.getSlices()) {
-        for (Replica replica : slice.getReplicas()) {
-          String replicaCoreName = replica.getStr(CORE_NAME_PROP);
-          if (coreName.equals(replicaCoreName)) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
-                " for this collection");
-          }
-        }
-      }
-    }
     ModifiableSolrParams params = new ModifiableSolrParams();
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
@@ -210,6 +160,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
     // For tracking async calls.
     Map<String,String> requestMap = new HashMap<>();
+    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
+
     ocmh.sendShardRequest(node, params, shardHandler, asyncId, requestMap);
 
     final String fnode = node;
@@ -253,4 +205,75 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     );
   }
 
+  public static ZkNodeProps assignReplicaDetails(SolrCloudManager cloudManager, ClusterState clusterState,
+                                                 ZkNodeProps message, AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
+
+    String collection = message.getStr(COLLECTION_PROP);
+    String node = message.getStr(CoreAdminParams.NODE);
+    String shard = message.getStr(SHARD_ID_PROP);
+    String coreName = message.getStr(CoreAdminParams.NAME);
+    String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
+    Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
+    if (StringUtils.isBlank(coreName)) {
+      coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
+    }
+
+    DocCollection coll = clusterState.getCollection(collection);
+    if (coll == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
+    }
+    if (coll.getSlice(shard) == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Collection: " + collection + " shard: " + shard + " does not exist");
+    }
+
+    // Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
+    if (!skipCreateReplicaInClusterState) {
+      if (CloudUtil.usePolicyFramework(coll, cloudManager)) {
+        if (node == null) {
+          if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
+          node = Assign.identifyNodes(cloudManager,
+              clusterState,
+              Collections.emptyList(),
+              collection,
+              message,
+              Collections.singletonList(shard),
+              replicaType == Replica.Type.NRT ? 0 : 1,
+              replicaType == Replica.Type.TLOG ? 0 : 1,
+              replicaType == Replica.Type.PULL ? 0 : 1
+          ).get(0).node;
+          sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+        }
+      } else {
+        node = Assign.getNodesForNewReplicas(clusterState, collection, shard, 1, node,
+            cloudManager).get(0).nodeName;// TODO: use replica type in this logic too
+      }
+    }
+    log.info("Node Identified {} for creating new replica", node);
+
+    if (!clusterState.liveNodesContain(node)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
+    }
+    if (coreName == null) {
+      coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
+    } else if (!skipCreateReplicaInClusterState) {
+      //Validate that the core name is unique in that collection
+      for (Slice slice : coll.getSlices()) {
+        for (Replica replica : slice.getReplicas()) {
+          String replicaCoreName = replica.getStr(CORE_NAME_PROP);
+          if (coreName.equals(replicaCoreName)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
+                " for this collection");
+          }
+        }
+      }
+    }
+    if (coreNodeName != null) {
+      message = message.plus(CoreAdminParams.CORE_NODE_NAME, coreNodeName);
+    }
+    message = message.plus(CoreAdminParams.NAME, coreName);
+    message = message.plus(CoreAdminParams.NODE, node);
+    return message;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index fd0738f..c746c94 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -114,7 +114,7 @@ public class Assign {
     }
   }
 
-  public static String assignNode(DistribStateManager stateManager, DocCollection collection) {
+  public static String assignCoreNodeName(DistribStateManager stateManager, DocCollection collection) {
     // for backward compatibility;
     int defaultValue = defaultCounterValue(collection, false);
     String coreNodeName = "core_node" + incAndGetId(stateManager, collection.getName(), defaultValue);
@@ -170,7 +170,7 @@ public class Assign {
     return returnShardId;
   }
 
-  private static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
+  private static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
     // TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
     return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
   }
@@ -187,20 +187,20 @@ public class Assign {
     return defaultValue * 20;
   }
 
-  public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
+  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
     Slice slice = collection.getSlice(shard);
     int defaultValue = defaultCounterValue(collection, newCollection);
     int replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
-    String coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
+    String coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
     while (existCoreName(coreName, slice)) {
       replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
-      coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
+      coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
     }
     return coreName;
   }
 
-  public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
-    return buildCoreName(stateManager, collection, shard, type, false);
+  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
+    return buildSolrCoreName(stateManager, collection, shard, type, false);
   }
 
   private static boolean existCoreName(String coreName, Slice slice) {
@@ -237,7 +237,7 @@ public class Assign {
     return nodeList;
   }
 
-  public static List<ReplicaPosition> identifyNodes(OverseerCollectionMessageHandler ocmh,
+  public static List<ReplicaPosition> identifyNodes(SolrCloudManager cloudManager,
                                                     ClusterState clusterState,
                                                     List<String> nodeList,
                                                     String collectionName,
@@ -248,7 +248,7 @@ public class Assign {
                                                     int numPullReplicas) throws IOException, InterruptedException {
     List<Map> rulesMap = (List) message.get("rule");
     String policyName = message.getStr(POLICY);
-    AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
 
     if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
       log.debug("Identify nodes using default");
@@ -283,7 +283,7 @@ public class Assign {
           (List<Map>) message.get(SNITCH),
           new HashMap<>(),//this is a new collection. So, there are no nodes in any shard
           nodeList,
-          ocmh.overseer.getSolrCloudManager(),
+          cloudManager,
           clusterState);
 
       Map<ReplicaPosition, String> nodeMappings = replicaAssigner.getNodeMappings();
@@ -294,7 +294,7 @@ public class Assign {
       if (message.getStr(CREATE_NODE_SET) == null)
         nodeList = Collections.emptyList();// unless explicitly specified do not pass node list to Policy
       return getPositionsUsingPolicy(collectionName,
-          shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.overseer.getSolrCloudManager(), nodeList);
+          shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, cloudManager, nodeList);
     }
   }
 
@@ -397,7 +397,7 @@ public class Assign {
           nodesList);
       return replicaPositions;
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error getting replica locations", e);
     } finally {
       if (log.isTraceEnabled()) {
         if (replicaPositions != null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
index 6e0583f..3cdc903 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
@@ -42,7 +42,7 @@ public class CloudConfigSetService extends ConfigSetService {
     try {
       // for back compat with cores that can create collections without the collections API
       if (!zkController.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cd.getCollectionName(), true)) {
-        CreateCollectionCmd.createCollectionZkNode(zkController.getZkClient(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
+        CreateCollectionCmd.createCollectionZkNode(zkController.getSolrCloudManager().getDistribStateManager(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
       }
     } catch (KeeperException e) {
       SolrException.log(log, null, e);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
index 62cde7c..30de3d4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -25,6 +25,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
@@ -130,4 +132,9 @@ public class CloudUtil {
 
   }
 
+  static boolean usePolicyFramework(DocCollection collection, SolrCloudManager cloudManager)
+      throws IOException, InterruptedException {
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
+    return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 2c4f01e..2171c60 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud;
 
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -26,11 +27,18 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.cloud.overseer.ClusterStateMutator;
 import org.apache.solr.common.SolrException;
@@ -40,7 +48,6 @@ import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
-import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -51,6 +58,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.admin.ConfigSetsHandlerApi;
 import org.apache.solr.handler.component.ShardHandler;
@@ -70,7 +78,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -81,11 +88,13 @@ import static org.apache.solr.common.util.StrUtils.formatString;
 public class CreateCollectionCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
-  private SolrZkClient zkClient;
+  private final TimeSource timeSource;
+  private final DistribStateManager stateManager;
 
   public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
-    this.zkClient = ocmh.zkStateReader.getZkClient();
+    this.stateManager = ocmh.cloudManager.getDistribStateManager();
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -103,95 +112,20 @@ public class CreateCollectionCmd implements Cmd {
     }
 
     ocmh.validateConfigOrThrowSolrException(configName);
-    PolicyHelper.SessionWrapper sessionWrapper = null;
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
 
     try {
-      // look at the replication factor and see if it matches reality
-      // if it does not, find best nodes to create more cores
 
-      int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
-      int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
-      int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
-      Map autoScalingJson = Utils.getJson(ocmh.zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
-      String policy = message.getStr(Policy.POLICY);
-      boolean usePolicyFramework = autoScalingJson.get(Policy.CLUSTER_POLICY) != null || policy != null;
-
-      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
       final String async = message.getStr(ASYNC);
 
-      Integer numSlices = message.getInt(NUM_SLICES, null);
-      String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
+      List<String> nodeList = new ArrayList<>();
       List<String> shardNames = new ArrayList<>();
-      if(ImplicitDocRouter.NAME.equals(router)){
-        ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
-        numSlices = shardNames.size();
-      } else {
-        if (numSlices == null ) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
-        }
-        ClusterStateMutator.getShardNames(numSlices, shardNames);
-      }
-
-      int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
-      if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
-      }
-      if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
-      if (numNrtReplicas + numTlogReplicas <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
-      }
-
-      if (numSlices <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
-      }
-
-      // we need to look at every node and see how many cores it serves
-      // add our new cores to existing nodes serving the least number of cores
-      // but (for now) require that each core goes on a distinct node.
-
-      final List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM);
-      List<ReplicaPosition> replicaPositions;
-      if (nodeList.isEmpty()) {
-        log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
-
-        replicaPositions = new ArrayList<>();
-      } else {
-        int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
-        if (totalNumReplicas > nodeList.size()) {
-          log.warn("Specified number of replicas of "
-              + totalNumReplicas
-              + " on collection "
-              + collectionName
-              + " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
-              + nodeList.size()
-              + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
-        }
-
-        int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
-            Integer.MAX_VALUE :
-            maxShardsPerNode * nodeList.size();
-        int requestedShardsToCreate = numSlices * totalNumReplicas;
-        if (maxShardsAllowedToCreate < requestedShardsToCreate) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
-              + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
-              + ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
-              + ". This allows a maximum of " + maxShardsAllowedToCreate
-              + " to be created. Value of " + NUM_SLICES + " is " + numSlices
-              + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
-              + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
-              + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
-              + ". This requires " + requestedShardsToCreate
-              + " shards to be created (higher than the allowed number)");
-        }
-        replicaPositions = Assign.identifyNodes(ocmh
-            , clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
-        sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
-      }
-
+      List<ReplicaPosition> replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, message,
+          nodeList, shardNames, sessionWrapper);
       ZkStateReader zkStateReader = ocmh.zkStateReader;
       boolean isLegacyCloud = Overseer.isLegacy(zkStateReader);
 
-      ocmh.createConfNode(configName, collectionName, isLegacyCloud);
+      ocmh.createConfNode(stateManager, configName, collectionName, isLegacyCloud);
 
       Map<String,String> collectionParams = new HashMap<>();
       Map<String,Object> collectionProps = message.getProperties();
@@ -201,16 +135,16 @@ public class CreateCollectionCmd implements Cmd {
         }
       }
       
-      createCollectionZkNode(zkClient, collectionName, collectionParams);
+      createCollectionZkNode(stateManager, collectionName, collectionParams);
       
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
 
       // wait for a while until we don't see the collection
-      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean created = false;
       while (! waitUntil.hasTimedOut()) {
-        Thread.sleep(100);
-        created = zkStateReader.getClusterState().hasCollection(collectionName);
+        waitUntil.sleep(100);
+        created = ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
         if(created) break;
       }
       if (!created)
@@ -225,12 +159,14 @@ public class CreateCollectionCmd implements Cmd {
       Map<String, String> requestMap = new HashMap<>();
 
 
-      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , nrtReplicas : {2}, tlogReplicas: {3}, pullReplicas: {4}",
-          collectionName, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas));
+      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}",
+          collectionName, shardNames, message));
       Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
+      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String nodeName = replicaPosition.node;
-        String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), zkStateReader.getClusterState().getCollection(collectionName),
+        String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(),
+            ocmh.cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName),
             replicaPosition.shard, replicaPosition.type, true);
         log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
             , coreName, replicaPosition.shard, collectionName, nodeName));
@@ -260,7 +196,7 @@ public class CreateCollectionCmd implements Cmd {
         params.set(COLL_CONF, configName);
         params.set(CoreAdminParams.COLLECTION, collectionName);
         params.set(CoreAdminParams.SHARD, replicaPosition.shard);
-        params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
+        params.set(ZkStateReader.NUM_SHARDS_PROP, shardNames.size());
         params.set(CoreAdminParams.NEW_COLLECTION, "true");
         params.set(CoreAdminParams.REPLICA_TYPE, replicaPosition.type.name());
 
@@ -320,10 +256,93 @@ public class CreateCollectionCmd implements Cmd {
     } catch (Exception ex) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
     } finally {
-      if(sessionWrapper != null) sessionWrapper.release();
+      if (sessionWrapper.get() != null) sessionWrapper.get().release();
     }
   }
 
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
+                                                            ZkNodeProps message,
+                                                            List<String> nodeList, List<String> shardNames,
+                                                            AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    final String collectionName = message.getStr(NAME);
+    // look at the replication factor and see if it matches reality
+    // if it does not, find best nodes to create more cores
+    int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
+    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
+    int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
+    String policy = message.getStr(Policy.POLICY);
+    boolean usePolicyFramework = !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || policy != null;
+
+    Integer numSlices = message.getInt(NUM_SLICES, null);
+    String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
+    if(ImplicitDocRouter.NAME.equals(router)){
+      ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
+      numSlices = shardNames.size();
+    } else {
+      if (numSlices == null ) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
+      }
+      if (numSlices <= 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
+      }
+      ClusterStateMutator.getShardNames(numSlices, shardNames);
+    }
+
+    int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
+    if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
+    }
+    if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
+    if (numNrtReplicas + numTlogReplicas <= 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
+    }
+
+    // we need to look at every node and see how many cores it serves
+    // add our new cores to existing nodes serving the least number of cores
+    // but (for now) require that each core goes on a distinct node.
+
+    List<ReplicaPosition> replicaPositions;
+    nodeList.addAll(Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM));
+    if (nodeList.isEmpty()) {
+      log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
+
+      replicaPositions = new ArrayList<>();
+    } else {
+      int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
+      if (totalNumReplicas > nodeList.size()) {
+        log.warn("Specified number of replicas of "
+            + totalNumReplicas
+            + " on collection "
+            + collectionName
+            + " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
+            + nodeList.size()
+            + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
+      }
+
+      int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
+          Integer.MAX_VALUE :
+          maxShardsPerNode * nodeList.size();
+      int requestedShardsToCreate = numSlices * totalNumReplicas;
+      if (maxShardsAllowedToCreate < requestedShardsToCreate) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
+            + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
+            + ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
+            + ". This allows a maximum of " + maxShardsAllowedToCreate
+            + " to be created. Value of " + NUM_SLICES + " is " + numSlices
+            + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
+            + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
+            + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
+            + ". This requires " + requestedShardsToCreate
+            + " shards to be created (higher than the allowed number)");
+      }
+      replicaPositions = Assign.identifyNodes(cloudManager
+          , clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
+      sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+    }
+    return replicaPositions;
+  }
+
   String getConfigName(String coll, ZkNodeProps message) throws KeeperException, InterruptedException {
     String configName = message.getStr(COLL_CONF);
 
@@ -375,12 +394,12 @@ public class CreateCollectionCmd implements Cmd {
     }
   }
 
-  public static void createCollectionZkNode(SolrZkClient zkClient, String collection, Map<String,String> params) {
+  public static void createCollectionZkNode(DistribStateManager stateManager, String collection, Map<String,String> params) {
     log.debug("Check for collection zkNode:" + collection);
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
 
     try {
-      if (!zkClient.exists(collectionPath, true)) {
+      if (!stateManager.hasData(collectionPath)) {
         log.debug("Creating collection in ZooKeeper:" + collection);
 
         try {
@@ -394,7 +413,7 @@ public class CreateCollectionCmd implements Cmd {
             // if the config name wasn't passed in, use the default
             if (!collectionProps.containsKey(ZkController.CONFIGNAME_PROP)) {
               // users can create the collection node and conf link ahead of time, or this may return another option
-              getConfName(zkClient, collection, collectionPath, collectionProps);
+              getConfName(stateManager, collection, collectionPath, collectionProps);
             }
 
           } else if (System.getProperty("bootstrap_confdir") != null) {
@@ -417,19 +436,21 @@ public class CreateCollectionCmd implements Cmd {
             // the conf name should should be the collection name of this core
             collectionProps.put(ZkController.CONFIGNAME_PROP, collection);
           } else {
-            getConfName(zkClient, collection, collectionPath, collectionProps);
+            getConfName(stateManager, collection, collectionPath, collectionProps);
           }
 
           collectionProps.remove(ZkStateReader.NUM_SHARDS_PROP);  // we don't put numShards in the collections properties
 
           ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
-          zkClient.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, null, true);
+          stateManager.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, false);
 
         } catch (KeeperException e) {
           // it's okay if the node already exists
           if (e.code() != KeeperException.Code.NODEEXISTS) {
             throw e;
           }
+        } catch (AlreadyExistsException e) {
+          // it's okay if the node already exists
         }
       } else {
         log.debug("Collection zkNode exists");
@@ -441,6 +462,8 @@ public class CreateCollectionCmd implements Cmd {
         return;
       }
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
     } catch (InterruptedException e) {
       Thread.interrupted();
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
@@ -448,8 +471,8 @@ public class CreateCollectionCmd implements Cmd {
 
   }
   
-  private static void getConfName(SolrZkClient zkClient, String collection, String collectionPath, Map<String,Object> collectionProps) throws KeeperException,
-  InterruptedException {
+  private static void getConfName(DistribStateManager stateManager, String collection, String collectionPath, Map<String,Object> collectionProps) throws IOException,
+      KeeperException, InterruptedException {
     // check for configName
     log.debug("Looking for collection configName");
     if (collectionProps.containsKey("configName")) {
@@ -461,17 +484,17 @@ public class CreateCollectionCmd implements Cmd {
     int retry = 1;
     int retryLimt = 6;
     for (; retry < retryLimt; retry++) {
-      if (zkClient.exists(collectionPath, true)) {
-        ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
+      if (stateManager.hasData(collectionPath)) {
+        VersionedData data = stateManager.getData(collectionPath);
+        ZkNodeProps cProps = ZkNodeProps.load(data.getData());
         if (cProps.containsKey(ZkController.CONFIGNAME_PROP)) {
           break;
         }
       }
 
       try {
-        configNames = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null,
-            true);
-      } catch (NoNodeException e) {
+        configNames = stateManager.listData(ZkConfigManager.CONFIGS_ZKNODE);
+      } catch (NoSuchElementException | NoNodeException e) {
         // just keep trying
       }
 
@@ -507,15 +530,4 @@ public class CreateCollectionCmd implements Cmd {
           "Could not find configName for collection " + collection + " found:" + configNames);
     }
   }
-
-  public static boolean usePolicyFramework(ZkStateReader zkStateReader, ZkNodeProps message) {
-    Map autoScalingJson = Collections.emptyMap();
-    try {
-      autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
-    } catch (Exception e) {
-      return false;
-    }
-    return autoScalingJson.get(Policy.CLUSTER_POLICY) != null || message.getStr(Policy.POLICY) != null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
index 18b0b63..c6afdcc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
@@ -25,11 +25,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
@@ -76,59 +77,21 @@ public class CreateShardCmd implements Cmd {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'collection' and 'shard' are required parameters");
 
     DocCollection collection = clusterState.getCollection(collectionName);
-    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
-    int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
-    int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
-    int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
-    
-    if (numNrtReplicas + numTlogReplicas <= 0) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
-    }
-    
-    Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
-    PolicyHelper.SessionWrapper sessionWrapper = null;
-    boolean usePolicyFramework = usePolicyFramework(collection,ocmh);
-    List<ReplicaPosition> positions = null;
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
     SolrCloseableLatch countDownLatch;
     try {
-      if (usePolicyFramework) {
-        if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
-        positions = Assign.identifyNodes(ocmh,
-            clusterState,
-            Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
-            collectionName,
-            message,
-            Collections.singletonList(sliceName),
-            numNrtReplicas,
-            numTlogReplicas,
-            numPullReplicas);
-        sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
-      } else {
-        List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
-            createNodeSetStr, ocmh.overseer.getSolrCloudManager());
-        int i = 0;
-        positions = new ArrayList<>();
-        for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
-            Replica.Type.TLOG, numTlogReplicas,
-            Replica.Type.PULL, numPullReplicas
-        ).entrySet()) {
-          for (int j = 0; j < e.getValue(); j++) {
-            positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
-            i++;
-          }
-        }
-      }
+      List<ReplicaPosition> positions = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, sessionWrapper);
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
       // wait for a while until we see the shard
       ocmh.waitForNewShard(collectionName, sliceName);
 
       String async = message.getStr(ASYNC);
-      countDownLatch = new SolrCloseableLatch(totalReplicas, ocmh);
+      countDownLatch = new SolrCloseableLatch(positions.size(), ocmh);
       for (ReplicaPosition position : positions) {
         String nodeName = position.node;
-        String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, sliceName, position.type);
+        String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(), collection, sliceName, position.type);
         log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
             + " on " + nodeName);
 
@@ -166,7 +129,7 @@ public class CreateShardCmd implements Cmd {
         });
       }
     } finally {
-      if(sessionWrapper != null) sessionWrapper.release();
+      if (sessionWrapper.get() != null) sessionWrapper.get().release();
     }
 
     log.debug("Waiting for create shard action to complete");
@@ -177,9 +140,52 @@ public class CreateShardCmd implements Cmd {
 
   }
 
-  static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh)
-      throws IOException, InterruptedException {
-    AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
-    return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
+         String collectionName, ZkNodeProps message, AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    DocCollection collection = clusterState.getCollection(collectionName);
+
+    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
+    int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
+    int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
+    int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
+
+    if (numNrtReplicas + numTlogReplicas <= 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
+    }
+
+    Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
+
+    boolean usePolicyFramework = CloudUtil.usePolicyFramework(collection, cloudManager);
+    List<ReplicaPosition> positions;
+    if (usePolicyFramework) {
+      if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
+      positions = Assign.identifyNodes(cloudManager,
+          clusterState,
+          Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
+          collection.getName(),
+          message,
+          Collections.singletonList(sliceName),
+          numNrtReplicas,
+          numTlogReplicas,
+          numPullReplicas);
+      sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+    } else {
+      List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collection.getName(), sliceName, totalReplicas,
+          createNodeSetStr, cloudManager);
+      int i = 0;
+      positions = new ArrayList<>();
+      for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
+          Replica.Type.TLOG, numTlogReplicas,
+          Replica.Type.PULL, numPullReplicas
+      ).entrySet()) {
+        for (int j = 0; j < e.getValue(); j++) {
+          positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
+          i++;
+        }
+      }
+    }
+    return positions;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
index d2e40f7..dc91905 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.util.TimeOut;
@@ -49,9 +50,11 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public DeleteCollectionCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -94,13 +97,13 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
 
       // wait for a while until we don't see the collection
-      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean removed = false;
       while (! timeout.hasTimedOut()) {
-        Thread.sleep(100);
+        timeout.sleep(100);
         removed = !zkStateReader.getClusterState().hasCollection(collection);
         if (removed) {
-          Thread.sleep(500); // just a bit of time so it's more likely other
+          timeout.sleep(500); // just a bit of time so it's more likely other
           // readers see on return
           break;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
index f13fed5..58c4e63 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -55,9 +56,11 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 public class DeleteShardCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public DeleteShardCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -134,14 +137,14 @@ public class DeleteShardCmd implements Cmd {
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
 
       // wait for a while until we don't see the shard
-      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean removed = false;
       while (!timeout.hasTimedOut()) {
-        Thread.sleep(100);
+        timeout.sleep(100);
         DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
         removed = collection.getSlice(sliceId) == null;
         if (removed) {
-          Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
+          timeout.sleep(100); // just a bit of time so it's more likely other readers see on return
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
index cacccb0..02fdb5c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
@@ -38,6 +38,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -63,9 +64,11 @@ import static org.apache.solr.common.util.Utils.makeMap;
 public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public MigrateCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
 
@@ -179,10 +182,10 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     // wait for a while until we see the new rule
     log.info("Waiting to see routing rule updated in clusterstate");
-    TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS, timeSource);
     boolean added = false;
     while (!waitUntil.hasTimedOut()) {
-      Thread.sleep(100);
+      waitUntil.sleep(100);
       sourceCollection = zkStateReader.getClusterState().getCollection(sourceCollection.getName());
       sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
       Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
@@ -257,7 +260,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
         tempSourceCollectionName, targetLeader.getNodeName());
-    String tempCollectionReplica2 = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
+    String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
         zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
     props = new HashMap<>();
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
index 71d5c82..44493ec 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.util.TimeOut;
@@ -54,9 +55,11 @@ public class MoveReplicaCmd implements Cmd{
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public MoveReplicaCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -158,11 +161,11 @@ public class MoveReplicaCmd implements Cmd{
         return;
       }
 
-      TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS, timeSource);
       while (!timeOut.hasTimedOut()) {
         coll = ocmh.zkStateReader.getClusterState().getCollection(coll.getName());
         if (coll.getReplica(replica.getName()) != null) {
-          Thread.sleep(100);
+          timeOut.sleep(100);
         } else {
           break;
         }
@@ -233,7 +236,7 @@ public class MoveReplicaCmd implements Cmd{
 
   private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
                                  DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
-    String newCoreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
+    String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
     ZkNodeProps addReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
         SHARD_ID_PROP, slice.getName(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 3b65d6f..d1bb13a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -80,7 +80,7 @@ public class Overseer implements SolrCloseable {
   enum LeaderStatus {DONT_KNOW, NO, YES}
 
   private class ClusterStateUpdater implements Runnable, Closeable {
-    
+
     private final ZkStateReader reader;
     private final SolrZkClient zkClient;
     private final String myId;
@@ -88,7 +88,7 @@ public class Overseer implements SolrCloseable {
     private final ZkDistributedQueue stateUpdateQueue;
     //TODO remove in 9.0, we do not push message into this queue anymore
     //Internal queue where overseer stores events that have not yet been published into cloudstate
-    //If Overseer dies while extracting the main queue a new overseer will start from this queue 
+    //If Overseer dies while extracting the main queue a new overseer will start from this queue
     private final ZkDistributedQueue workQueue;
     // Internal map which holds the information about running tasks.
     private final DistributedMap runningMap;
@@ -120,7 +120,7 @@ public class Overseer implements SolrCloseable {
     public Stats getWorkQueueStats()  {
       return workQueue.getZkStats();
     }
-    
+
     @Override
     public void run() {
 
@@ -442,7 +442,7 @@ public class Overseer implements SolrCloseable {
 
   }
 
-  static class OverseerThread extends Thread implements Closeable {
+  public static class OverseerThread extends Thread implements Closeable {
 
     protected volatile boolean isClosed;
     private Closeable thread;
@@ -466,9 +466,9 @@ public class Overseer implements SolrCloseable {
     public boolean isClosed() {
       return this.isClosed;
     }
-    
+
   }
-  
+
   private OverseerThread ccThread;
 
   private OverseerThread updaterThread;
@@ -478,7 +478,7 @@ public class Overseer implements SolrCloseable {
   private final ZkStateReader reader;
 
   private final ShardHandler shardHandler;
-  
+
   private final UpdateShardHandler updateShardHandler;
 
   private final String adminPath;
@@ -505,7 +505,7 @@ public class Overseer implements SolrCloseable {
     this.stats = new Stats();
     this.config = config;
   }
-  
+
   public synchronized void start(String id) {
     this.id = id;
     closed = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 26f1709..abfecab 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -36,6 +36,10 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@@ -65,6 +69,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -73,6 +78,7 @@ import org.apache.solr.handler.component.ShardResponse;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.TimeOut;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -143,8 +149,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   ShardHandlerFactory shardHandlerFactory;
   String adminPath;
   ZkStateReader zkStateReader;
+  SolrCloudManager cloudManager;
   String myId;
   Stats stats;
+  TimeSource timeSource;
 
   // Set that tracks collections that are currently being processed by a running task.
   // This is used for handling mutual exclusion of the tasks.
@@ -182,6 +190,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     this.myId = myId;
     this.stats = stats;
     this.overseer = overseer;
+    this.cloudManager = overseer.getSolrCloudManager();
+    this.timeSource = cloudManager.getTimeSource();
     this.isClosed = false;
     commandMap = new ImmutableMap.Builder<CollectionAction, Cmd>()
         .put(REPLACENODE, new ReplaceNodeCmd(this))
@@ -229,7 +239,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       CollectionAction action = getCollectionAction(operation);
       Cmd command = commandMap.get(action);
       if (command != null) {
-        command.call(zkStateReader.getClusterState(), message, results);
+        command.call(cloudManager.getClusterStateProvider().getClusterState(), message, results);
       } else {
         throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
             + operation);
@@ -423,9 +433,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
-    TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS, timeSource);
     while (! timeout.hasTimedOut()) {
-      Thread.sleep(100);
+      timeout.sleep(100);
       DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
       if (docCollection == null) { // someone already deleted the collection
         return true;
@@ -465,7 +475,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
     boolean firstLoop = true;
     // wait for a while until the state format changes
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     while (! timeout.hasTimedOut()) {
       DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
       if (collection == null) {
@@ -483,7 +493,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, MIGRATESTATEFORMAT.toLower(), COLLECTION_PROP, collectionName);
         Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
       }
-      Thread.sleep(100);
+      timeout.sleep(100);
     }
     throw new SolrException(ErrorCode.SERVER_ERROR, "Could not migrate state format for collection: " + collectionName);
   }
@@ -642,16 +652,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       validateConfigOrThrowSolrException(configName);
       
       boolean isLegacyCloud =  Overseer.isLegacy(zkStateReader);
-      createConfNode(configName, collectionName, isLegacyCloud);
+      createConfNode(cloudManager.getDistribStateManager(), configName, collectionName, isLegacyCloud);
       reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
     }
     
     overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
 
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     boolean areChangesVisible = true;
     while (!timeout.hasTimedOut()) {
-      DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+      DocCollection collection = cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName);
       areChangesVisible = true;
       for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
         String updateKey = updateEntry.getKey();
@@ -663,7 +673,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         }
       }
       if (areChangesVisible) break;
-      Thread.sleep(100);
+      timeout.sleep(100);
     }
 
     if (!areChangesVisible)
@@ -680,7 +690,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
   Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
     Map<String, Replica> result = new HashMap<>();
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     while (true) {
       DocCollection coll = zkStateReader.getClusterState().getCollection(collectionName);
       for (String coreName : coreNames) {
@@ -746,8 +756,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
 
-  void validateConfigOrThrowSolrException(String configName) throws KeeperException, InterruptedException {
-    boolean isValid = zkStateReader.getZkClient().exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
+  void validateConfigOrThrowSolrException(String configName) throws IOException, KeeperException, InterruptedException {
+    boolean isValid = cloudManager.getDistribStateManager().hasData(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
     if(!isValid) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Can not find the specified config set: " + configName);
     }
@@ -757,16 +767,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
    * This doesn't validate the config (path) itself and is just responsible for creating the confNode.
    * That check should be done before the config node is created.
    */
-  void createConfNode(String configName, String coll, boolean isLegacyCloud) throws KeeperException, InterruptedException {
+  public static void createConfNode(DistribStateManager stateManager, String configName, String coll, boolean isLegacyCloud) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException {
     
     if (configName != null) {
       String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
       log.debug("creating collections conf node {} ", collDir);
       byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
-      if (zkStateReader.getZkClient().exists(collDir, true)) {
-        zkStateReader.getZkClient().setData(collDir, data, true);
+      if (stateManager.hasData(collDir)) {
+        stateManager.setData(collDir, data, -1);
       } else {
-        zkStateReader.getZkClient().makePath(collDir, data, true);
+        stateManager.makePath(collDir, data, CreateMode.PERSISTENT, false);
       }
     } else {
       if(isLegacyCloud){
@@ -775,7 +785,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         throw new SolrException(ErrorCode.BAD_REQUEST,"Unable to get config name");
       }
     }
-
   }
   
   private void collectionCmd(ZkNodeProps message, ModifiableSolrParams params,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
index 039ab5c..9c9a5c9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
@@ -223,7 +223,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
 
     try {
       List<ReplicaPosition> replicaPositions = Assign.identifyNodes(
-          ocmh, clusterState,
+          ocmh.cloudManager, clusterState,
           nodeList, restoreCollectionName,
           message, sliceNames,
           numNrtReplicas, numTlogReplicas, numPullReplicas);


[2/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
new file mode 100644
index 0000000..a05eb78
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
@@ -0,0 +1,1217 @@
+/*
+ * 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.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
+import org.apache.solr.cloud.autoscaling.TriggerActionBase;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.cloud.autoscaling.TriggerEventQueue;
+import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.cloud.autoscaling.CapturedEvent;
+import org.apache.solr.common.cloud.LiveNodesListener;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.LogLevel;
+import org.apache.solr.util.TimeOut;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+import static org.apache.solr.cloud.autoscaling.ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS;
+
+/**
+ * An end-to-end integration test for triggers
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+public class TestTriggerIntegration extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SPEED = 50;
+
+  private static CountDownLatch actionConstructorCalled;
+  private static CountDownLatch actionInitCalled;
+  private static CountDownLatch triggerFiredLatch;
+  private static int waitForSeconds = 1;
+  private static CountDownLatch actionStarted;
+  private static CountDownLatch actionInterrupted;
+  private static CountDownLatch actionCompleted;
+  private static AtomicBoolean triggerFired;
+  private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
+
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2, TimeSource.get("simTime:" + SPEED));
+  }
+
+  private static CountDownLatch getTriggerFiredLatch() {
+    return triggerFiredLatch;
+  }
+
+  private static CountDownLatch getActionStarted() {
+    return actionStarted;
+  }
+
+  private static CountDownLatch getActionInterrupted() {
+    return actionInterrupted;
+  }
+
+  private static CountDownLatch getActionCompleted() {
+    return actionCompleted;
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+
+    waitForSeconds = 1 + random().nextInt(3);
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+    triggerFiredLatch = new CountDownLatch(1);
+    triggerFired = new AtomicBoolean(false);
+    actionStarted = new CountDownLatch(1);
+    actionInterrupted = new CountDownLatch(1);
+    actionCompleted = new CountDownLatch(1);
+    events.clear();
+    listenerEvents.clear();
+    while (cluster.getClusterStateProvider().getLiveNodes().size() < 2) {
+      // perhaps a test stopped a node but didn't start it back
+      // lets start a node
+      cluster.simAddNode();
+    }
+  }
+
+  @Test
+  public void testTriggerThrottling() throws Exception  {
+    // for this test we want to create two triggers so we must assert that the actions were created twice
+    actionInitCalled = new CountDownLatch(2);
+    // similarly we want both triggers to fire
+    triggerFiredLatch = new CountDownLatch(2);
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    // first trigger
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger1'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // second trigger
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger2'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the two instances of action are created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    String newNode = cluster.simAddNode();
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Both triggers should have fired by now");
+    }
+
+    // reset shared state
+    lastActionExecutedAt.set(0);
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+    triggerFiredLatch = new CountDownLatch(2);
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger1'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger2'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the two instances of action are created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    // stop the node we had started earlier
+    cluster.simRemoveNode(newNode, false);
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Both triggers should have fired by now");
+    }
+  }
+
+  static AtomicLong lastActionExecutedAt = new AtomicLong(0);
+  static ReentrantLock lock = new ReentrantLock();
+  public static class ThrottlingTesterAction extends TestTriggerAction {
+    // nanos are very precise so we need a delta for comparison with ms
+    private static final long DELTA_MS = 2;
+
+    // sanity check that an action instance is only invoked once
+    private final AtomicBoolean onlyOnce = new AtomicBoolean(false);
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      boolean locked = lock.tryLock();
+      if (!locked)  {
+        log.info("We should never have a tryLock fail because actions are never supposed to be executed concurrently");
+        return;
+      }
+      try {
+        if (lastActionExecutedAt.get() != 0)  {
+          log.info("last action at " + lastActionExecutedAt.get() + " time = " + cluster.getTimeSource().getTime());
+          if (TimeUnit.NANOSECONDS.toMillis(cluster.getTimeSource().getTime() - lastActionExecutedAt.get()) <
+              TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS) - DELTA_MS) {
+            log.info("action executed again before minimum wait time from {}", event.getSource());
+            fail("TriggerListener was fired before the throttling period");
+          }
+        }
+        if (onlyOnce.compareAndSet(false, true)) {
+          log.info("action executed from {}", event.getSource());
+          lastActionExecutedAt.set(cluster.getTimeSource().getTime());
+          getTriggerFiredLatch().countDown();
+        } else  {
+          log.info("action executed more than once from {}", event.getSource());
+          fail("Trigger should not have fired more than once!");
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+  }
+
+  @Test
+  public void testNodeLostTriggerRestoreState() throws Exception {
+    // for this test we want to update the trigger so we must assert that the actions were created twice
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+
+    // start a new node
+    String nodeName = cluster.simAddNode();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    waitForSeconds = 5;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_restore_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '5s'," + // should be enough for us to update the trigger
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
+      timeOut.sleep(200);
+    }
+    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+
+    cluster.simRemoveNode(nodeName, false);
+
+    // ensure that the old trigger sees the stopped node, todo find a better way to do this
+    timeOut.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
+
+    waitForSeconds = 0;
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_restore_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the second instance of action is created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    boolean await = triggerFiredLatch.await(5000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    NodeLostTrigger.NodeLostEvent nodeLostEvent = (NodeLostTrigger.NodeLostEvent) events.iterator().next();
+    assertNotNull(nodeLostEvent);
+    List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(nodeName));
+  }
+
+  @Test
+  public void testNodeAddedTriggerRestoreState() throws Exception {
+    // for this test we want to update the trigger so we must assert that the actions were created twice
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    waitForSeconds = 5;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_restore_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '5s'," + // should be enough for us to update the trigger
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
+      timeOut.sleep(200);
+    }
+    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+
+    // start a new node
+    String newNode = cluster.simAddNode();
+
+    // ensure that the old trigger sees the new node, todo find a better way to do this
+    cluster.getTimeSource().sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
+
+    waitForSeconds = 0;
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_restore_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the second instance of action is created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    boolean await = triggerFiredLatch.await(5000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.toString(), nodeNames.contains(newNode));
+  }
+
+  @Test
+  public void testNodeAddedTrigger() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeAddedEvent.toString(), nodeNames.contains(newNode));
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2000 / SPEED, TimeUnit.MILLISECONDS));
+  }
+
+  @Test
+  public void testNodeLostTrigger() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String lostNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    cluster.simRemoveNode(lostNodeName, false);
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeLostEvent = events.iterator().next();
+    assertNotNull(nodeLostEvent);
+    List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(lostNodeName));
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2000 / SPEED, TimeUnit.MILLISECONDS));
+  }
+
+  // simulator doesn't support overseer functionality yet
+  /*
+  @Test
+  public void testContinueTriggersOnOverseerRestart() throws Exception  {
+    CollectionAdminRequest.OverseerStatus status = new CollectionAdminRequest.OverseerStatus();
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    CollectionAdminResponse adminResponse = status.process(solrClient);
+    NamedList<Object> response = adminResponse.getResponse();
+    String leader = (String) response.get("leader");
+    JettySolrRunner overseerNode = null;
+    int index = -1;
+    List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
+    for (int i = 0; i < jettySolrRunners.size(); i++) {
+      JettySolrRunner runner = jettySolrRunners.get(i);
+      if (runner.getNodeName().equals(leader)) {
+        overseerNode = runner;
+        index = i;
+        break;
+      }
+    }
+    assertNotNull(overseerNode);
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    // stop the overseer, somebody else will take over as the overseer
+    cluster.stopJettySolrRunner(index);
+    Thread.sleep(10000);
+    JettySolrRunner newNode = cluster.startJettySolrRunner();
+    boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(newNode.getNodeName()));
+  }
+
+*/
+
+  public static class TestTriggerAction extends TriggerActionBase {
+
+    public TestTriggerAction() {
+      actionConstructorCalled.countDown();
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      try {
+        if (triggerFired.compareAndSet(false, true))  {
+          events.add(event);
+          long currentTimeNanos = cluster.getTimeSource().getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail(event.getSource() + " was fired before the configured waitFor period");
+          }
+          getTriggerFiredLatch().countDown();
+        } else  {
+          fail(event.getSource() + " was fired more than once!");
+        }
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.info("TestTriggerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  public static class TestEventQueueAction extends TriggerActionBase {
+
+    public TestEventQueueAction() {
+      log.info("TestEventQueueAction instantiated");
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      log.info("-- event: " + event);
+      events.add(event);
+      getActionStarted().countDown();
+      try {
+        Thread.sleep(eventQueueActionWait);
+        triggerFired.compareAndSet(false, true);
+        getActionCompleted().countDown();
+      } catch (InterruptedException e) {
+        getActionInterrupted().countDown();
+        return;
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.debug("TestTriggerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  public static long eventQueueActionWait = 5000;
+
+  @Test
+  public void testEventQueue() throws Exception {
+    waitForSeconds = 1;
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger1'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
+        "}}";
+
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    // add node to generate the event
+    String newNode = cluster.simAddNode();
+    boolean await = actionStarted.await(60000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action did not start", await);
+    // event should be there
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    // but action did not complete yet so the event is still enqueued
+    assertFalse(triggerFired.get());
+    events.clear();
+    actionStarted = new CountDownLatch(1);
+    eventQueueActionWait = 1;
+    // kill overseer
+    cluster.simRestartOverseer(overseerLeader);
+    cluster.getTimeSource().sleep(5000);
+    // new overseer leader should be elected and run triggers
+    await = actionInterrupted.await(3000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't interrupted", await);
+    // it should fire again from enqueued event
+    await = actionStarted.await(60000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't started", await);
+    TriggerEvent replayedEvent = events.iterator().next();
+    assertTrue(replayedEvent.getProperty(TriggerEventQueue.ENQUEUE_TIME) != null);
+    assertTrue(events + "\n" + replayedEvent.toString(), replayedEvent.getProperty(TriggerEventQueue.DEQUEUE_TIME) != null);
+    await = actionCompleted.await(10000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't completed", await);
+    assertTrue(triggerFired.get());
+  }
+
+  @Test
+  public void testEventFromRestoredState() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '10s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    events.clear();
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    // reset
+    triggerFired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(newNode));
+    // add a second node - state of the trigger will change but it won't fire for waitFor sec.
+    String newNode2 = cluster.simAddNode();
+    cluster.getTimeSource().sleep(10000);
+    // kill overseer
+    cluster.simRestartOverseer(null);
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+  }
+
+  private static class TestLiveNodesListener implements LiveNodesListener {
+    Set<String> lostNodes = new HashSet<>();
+    Set<String> addedNodes = new HashSet<>();
+    CountDownLatch onChangeLatch = new CountDownLatch(1);
+
+    public void reset() {
+      lostNodes.clear();
+      addedNodes.clear();
+      onChangeLatch = new CountDownLatch(1);
+    }
+
+    @Override
+    public void onChange(SortedSet<String> oldLiveNodes, SortedSet<String> newLiveNodes) {
+      onChangeLatch.countDown();
+      Set<String> old = new HashSet<>(oldLiveNodes);
+      old.removeAll(newLiveNodes);
+      if (!old.isEmpty()) {
+        lostNodes.addAll(old);
+      }
+      newLiveNodes.removeAll(oldLiveNodes);
+      if (!newLiveNodes.isEmpty()) {
+        addedNodes.addAll(newLiveNodes);
+      }
+    }
+  }
+
+  private TestLiveNodesListener registerLiveNodesListener() {
+    TestLiveNodesListener listener = new TestLiveNodesListener();
+    cluster.getLiveNodesSet().registerLiveNodesListener(listener);
+    return listener;
+  }
+
+  public static class TestEventMarkerAction extends TriggerActionBase {
+
+    public TestEventMarkerAction() {
+      actionConstructorCalled.countDown();
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      boolean locked = lock.tryLock();
+      if (!locked)  {
+        log.info("We should never have a tryLock fail because actions are never supposed to be executed concurrently");
+        return;
+      }
+      try {
+        events.add(event);
+        getTriggerFiredLatch().countDown();
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.info("TestEventMarkerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  @Test
+  public void testNodeMarkersRegistration() throws Exception {
+    // for this test we want to create two triggers so we must assert that the actions were created twice
+    actionInitCalled = new CountDownLatch(2);
+    // similarly we want both triggers to fire
+    triggerFiredLatch = new CountDownLatch(2);
+    TestLiveNodesListener listener = registerLiveNodesListener();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    // pick overseer node
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    // add a node
+    String node = cluster.simAddNode();
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.addedNodes.size());
+    assertEquals(node, listener.addedNodes.iterator().next());
+    // verify that a znode doesn't exist (no trigger)
+    String pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node;
+    assertFalse("Path " + pathAdded + " was created but there are no nodeAdded triggers",
+        cluster.getDistribStateManager().hasData(pathAdded));
+    listener.reset();
+    // stop overseer
+    log.info("====== KILL OVERSEER 1");
+    cluster.simRestartOverseer(overseerLeader);
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.lostNodes.size());
+    assertEquals(overseerLeader, listener.lostNodes.iterator().next());
+    assertEquals(0, listener.addedNodes.size());
+    // wait until the new overseer is up
+    cluster.getTimeSource().sleep(5000);
+    // verify that a znode does NOT exist - there's no nodeLost trigger,
+    // so the new overseer cleaned up existing nodeLost markers
+    String pathLost = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + overseerLeader;
+    assertFalse("Path " + pathLost + " exists", cluster.getDistribStateManager().hasData(pathLost));
+
+    listener.reset();
+
+    // set up triggers
+
+    log.info("====== ADD TRIGGERS");
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    // create another node
+    log.info("====== ADD NODE 1");
+    String node1 = cluster.simAddNode();
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.addedNodes.size());
+    assertEquals(node1, listener.addedNodes.iterator().next());
+    // verify that a znode exists
+    pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node1;
+    assertTrue("Path " + pathAdded + " wasn't created", cluster.getDistribStateManager().hasData(pathAdded));
+
+    cluster.getTimeSource().sleep(5000);
+    // nodeAdded marker should be consumed now by nodeAdded trigger
+    assertFalse("Path " + pathAdded + " should have been deleted",
+        cluster.getDistribStateManager().hasData(pathAdded));
+
+    listener.reset();
+    events.clear();
+    triggerFiredLatch = new CountDownLatch(1);
+    // kill overseer again
+    log.info("====== KILL OVERSEER 2");
+    cluster.simRestartOverseer(overseerLeader);
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Trigger should have fired by now");
+    }
+    assertEquals(1, events.size());
+    TriggerEvent ev = events.iterator().next();
+    List<String> nodeNames = (List<String>)ev.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(overseerLeader));
+    assertEquals(TriggerEventType.NODELOST, ev.getEventType());
+  }
+
+  static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
+  static CountDownLatch listenerCreated = new CountDownLatch(1);
+  static boolean failDummyAction = false;
+
+  public static class TestTriggerListener extends TriggerListenerBase {
+    @Override
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
+      listenerCreated.countDown();
+    }
+
+    @Override
+    public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
+                                     ActionContext context, Throwable error, String message) {
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(cluster.getTimeSource().getTime(), context, config, stage, actionName, event, message));
+    }
+  }
+
+  public static class TestDummyAction extends TriggerActionBase {
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) {
+      if (failDummyAction) {
+        throw new RuntimeException("failure");
+      }
+
+    }
+  }
+
+  @Test
+  public void testListeners() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}," +
+        "{'name':'test1','class':'" + TestDummyAction.class.getName() + "'}," +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String setListenerCommand = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'foo'," +
+        "'trigger' : 'node_added_trigger'," +
+        "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+        "'beforeAction' : 'test'," +
+        "'afterAction' : ['test', 'test1']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'bar'," +
+        "'trigger' : 'node_added_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'beforeAction' : ['test', 'test1']," +
+        "'afterAction' : 'test'," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    listenerEvents.clear();
+    failDummyAction = false;
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+
+    assertEquals("both listeners should have fired", 2, listenerEvents.size());
+
+    cluster.getTimeSource().sleep(2000);
+
+    // check foo events
+    List<CapturedEvent> testEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + testEvents, testEvents);
+    assertEquals("foo events: " + testEvents, 5, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
+    assertEquals("test", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
+
+    // check bar events
+    testEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", testEvents);
+    assertEquals("bar events", 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
+    assertEquals("test", testEvents.get(0).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
+    assertEquals("test1", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
+
+    // reset
+    triggerFired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    listenerEvents.clear();
+    failDummyAction = true;
+
+    newNode = cluster.simAddNode();
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+
+    cluster.getTimeSource().sleep(2000);
+
+    // check foo events
+    testEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + testEvents, testEvents);
+    assertEquals("foo events: " + testEvents, 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
+    assertEquals("test", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+
+    // check bar events
+    testEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", testEvents);
+    assertEquals("bar events", 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
+    assertEquals("test", testEvents.get(0).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
+    assertEquals("test1", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+  }
+
+  @Test
+  public void testCooldown() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    failDummyAction = false;
+    waitForSeconds = 1;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_cooldown_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'bar'," +
+        "'trigger' : 'node_added_cooldown_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED', 'IGNORED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    listenerCreated = new CountDownLatch(1);
+    listenerEvents.clear();
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(1000);
+
+    List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
+    // we may get a few IGNORED events if other tests caused events within cooldown period
+    assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
+    long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
+
+    // reset the trigger and captured events
+    listenerEvents.clear();
+    triggerFiredLatch = new CountDownLatch(1);
+    triggerFired.compareAndSet(true, false);
+
+    String newNode2 = cluster.simAddNode();
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+
+    // there must be at least one IGNORED event due to cooldown, and one SUCCEEDED event
+    capturedEvents = listenerEvents.get("bar");
+    assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
+    for (int i = 0; i < capturedEvents.size() - 1; i++) {
+      CapturedEvent ev = capturedEvents.get(i);
+      assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
+      assertTrue(ev.toString(), ev.message.contains("cooldown"));
+    }
+    CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
+    assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
+    // the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
+    // must be larger than cooldown period
+    assertTrue("timestamp delta is less than default cooldown period", ev.timestamp - prevTimestamp > TimeUnit.SECONDS.toNanos(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
+  }
+
+  public static class TestSearchRateAction extends TriggerActionBase {
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) throws Exception {
+      try {
+        events.add(event);
+        long currentTimeNanos = cluster.getTimeSource().getTime();
+        long eventTimeNanos = event.getEventTime();
+        long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+        if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+          fail(event.getSource() + " was fired before the configured waitFor period");
+        }
+        getTriggerFiredLatch().countDown();
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      }
+    }
+  }
+
+  @Test
+  public void testSearchRate() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String COLL1 = "collection1";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
+        "conf", 1, 2);
+    create.process(solrClient);
+    waitForState(COLL1, 10, TimeUnit.SECONDS, clusterShape(1, 2));
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'search_rate_trigger'," +
+        "'event' : 'searchRate'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'rate' : 1.0," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'srt'," +
+        "'trigger' : 'search_rate_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+//    SolrParams query = params(CommonParams.Q, "*:*");
+//    for (int i = 0; i < 500; i++) {
+//      solrClient.query(COLL1, query);
+//    }
+
+    cluster.getSimClusterStateProvider().simSetCollectionValue(COLL1, "QUERY./select.requestTimes:1minRate", 500, true);
+
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+    assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
+    long now = cluster.getTimeSource().getTime();
+    // verify waitFor
+    assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());
+    Map<String, Double> nodeRates = (Map<String, Double>)ev.event.getProperties().get("node");
+    assertNotNull("nodeRates", nodeRates);
+    assertTrue(nodeRates.toString(), nodeRates.size() > 0);
+    AtomicDouble totalNodeRate = new AtomicDouble();
+    nodeRates.forEach((n, r) -> totalNodeRate.addAndGet(r));
+    List<ReplicaInfo> replicaRates = (List<ReplicaInfo>)ev.event.getProperties().get("replica");
+    assertNotNull("replicaRates", replicaRates);
+    assertTrue(replicaRates.toString(), replicaRates.size() > 0);
+    AtomicDouble totalReplicaRate = new AtomicDouble();
+    replicaRates.forEach(r -> {
+      assertTrue(r.toString(), r.getVariable("rate") != null);
+      totalReplicaRate.addAndGet((Double)r.getVariable("rate"));
+    });
+    Map<String, Object> shardRates = (Map<String, Object>)ev.event.getProperties().get("shard");
+    assertNotNull("shardRates", shardRates);
+    assertEquals(shardRates.toString(), 1, shardRates.size());
+    shardRates = (Map<String, Object>)shardRates.get(COLL1);
+    assertNotNull("shardRates", shardRates);
+    assertEquals(shardRates.toString(), 1, shardRates.size());
+    AtomicDouble totalShardRate = new AtomicDouble();
+    shardRates.forEach((s, r) -> totalShardRate.addAndGet((Double)r));
+    Map<String, Double> collectionRates = (Map<String, Double>)ev.event.getProperties().get("collection");
+    assertNotNull("collectionRates", collectionRates);
+    assertEquals(collectionRates.toString(), 1, collectionRates.size());
+    Double collectionRate = collectionRates.get(COLL1);
+    assertNotNull(collectionRate);
+    assertTrue(collectionRate > 5.0);
+    assertEquals(collectionRate, totalNodeRate.get(), 5.0);
+    assertEquals(collectionRate, totalShardRate.get(), 5.0);
+    assertEquals(collectionRate, totalReplicaRate.get(), 5.0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
new file mode 100644
index 0000000..0b412cb
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/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.
+ */
+/**
+ * Simulated environment for autoscaling tests.
+ */
+package org.apache.solr.cloud.autoscaling.sim;
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
index 42af083..c242809 100644
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
@@ -59,6 +59,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
@@ -794,7 +795,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
 
   protected void waitForBootstrapToComplete(String collectionName, String shardId) throws Exception {
     NamedList rsp;// we need to wait until bootstrap is complete otherwise the replicator thread will never start
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut())  {
       rsp = invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.BOOTSTRAP_STATUS);
       if (rsp.get(RESPONSE_STATUS).toString().equals(COMPLETED))  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
index 329de79..500655d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.BadHdfsThreadsFilter;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -221,7 +222,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
     request.setPath("/admin/collections");
     cloudClient.request(request);
 
-    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cloudClient.getZkStateReader().getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) {
       if (timeout.hasTimedOut()) {
         throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
index 906e27b..0639479 100644
--- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 
@@ -213,7 +214,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
       writer.writePendingUpdates();
 
       boolean found = false;
-      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       while (!timeOut.hasTimedOut())  {
         DocCollection c1 = reader.getClusterState().getCollection("c1");
         if ("y".equals(c1.getStr("x"))) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
index 76c5c0f..626374c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
@@ -201,6 +201,9 @@ public class RuleEngineTest extends SolrTestCaseJ4{
       public NodeStateProvider getNodeStateProvider() {
         return new NodeStateProvider() {
           @Override
+          public void close() throws IOException { }
+
+          @Override
           public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
             return (Map<String, Object>) MockSnitch.nodeVsTags.get(node);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
index 0bdf90c..f85b293 100644
--- a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
+++ b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
@@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.Before;
@@ -318,8 +319,8 @@ class Indexer {
   ArrayList<OneIndexer> _threads = new ArrayList<>();
 
   public Indexer(OpenCloseCoreStressTest OCCST, String url, List<HttpSolrClient> clients, int numThreads, int secondsToRun, Random random) {
-    stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS);
-    nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
+    stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     docsThisCycle.set(0);
     qTimesAccum.set(0);
     updateCounts.set(0);
@@ -353,7 +354,7 @@ class Indexer {
       log.info(String.format(Locale.ROOT, " s indexed: [run %,8d] [cycle %,8d] [last minute %,8d] Last core updated: %s. Seconds left in cycle %,4d",
           myId, docsThisCycle.get(), myId - lastCount, core, stopTimeout.timeLeft(TimeUnit.SECONDS)));
       lastCount = myId;
-      nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
+      nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
index e439d03..01f9199 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
@@ -520,6 +520,15 @@ public class AutoScalingConfig implements MapWriter {
     return withTriggerListenerConfigs(configs);
   }
 
+  @Override
+  public Object clone() {
+    if (jsonMap != null) {
+      return new AutoScalingConfig(jsonMap);
+    } else {
+      return new AutoScalingConfig(getPolicy(), getTriggerConfigs(), getTriggerListenerConfigs(), getProperties(), zkVersion);
+    }
+  }
+
   /**
    * Return the znode version that was used to create this configuration.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
index 09b6193..17c48d5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
@@ -24,6 +24,7 @@ import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * Base class for overriding some behavior of {@link SolrCloudManager}.
@@ -31,6 +32,7 @@ import org.apache.solr.common.util.ObjectCache;
 public class DelegatingCloudManager implements SolrCloudManager {
   private final SolrCloudManager delegate;
   private ObjectCache objectCache = new ObjectCache();
+  private TimeSource timeSource = TimeSource.NANO_TIME;
 
   public DelegatingCloudManager(SolrCloudManager delegate) {
     this.delegate = delegate;
@@ -62,6 +64,16 @@ public class DelegatingCloudManager implements SolrCloudManager {
   }
 
   @Override
+  public boolean isClosed() {
+    return delegate.isClosed();
+  }
+
+  @Override
+  public TimeSource getTimeSource() {
+    return delegate == null ? timeSource : delegate.getTimeSource();
+  }
+
+  @Override
   public SolrResponse request(SolrRequest req) throws IOException {
     return delegate.request(req);
   }
@@ -70,4 +82,9 @@ public class DelegatingCloudManager implements SolrCloudManager {
   public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
     return delegate.httpRequest(url, method, headers, payload, timeout, followRedirects);
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
index b47d1c8..2fea23b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
@@ -45,6 +45,11 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+    return delegate.listData(path, watcher);
+  }
+
+  @Override
   public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     return delegate.getData(path, watcher);
   }
@@ -60,12 +65,17 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    delegate.makePath(path, data, createMode, failOnExists);
+  }
+
+  @Override
   public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
     return delegate.createData(path, data, mode);
   }
 
   @Override
-  public void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public void removeData(String path, int version) throws NoSuchElementException, IOException, BadVersionException, KeeperException, InterruptedException {
     delegate.removeData(path, version);
   }
 
@@ -88,4 +98,9 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   public AutoScalingConfig getAutoScalingConfig() throws InterruptedException, IOException {
     return delegate.getAutoScalingConfig();
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
index 8b717f8..9ffde0f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -40,4 +41,14 @@ public class DelegatingNodeStateProvider implements NodeStateProvider {
   public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
     return delegate.getReplicaInfo(node, keys);
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
+
+  @Override
+  public boolean isClosed() {
+    return delegate.isClosed();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
index 4318418..26aaead 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
@@ -16,11 +16,11 @@
  */
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.NoSuchElementException;
 
+import org.apache.solr.common.SolrCloseable;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
@@ -30,7 +30,7 @@ import org.apache.zookeeper.Watcher;
 /**
  * This interface represents a distributed state repository.
  */
-public interface DistribStateManager extends Closeable {
+public interface DistribStateManager extends SolrCloseable {
 
   // state accessors
 
@@ -38,6 +38,8 @@ public interface DistribStateManager extends Closeable {
 
   List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
 
+  List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
+
   VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
 
   default VersionedData getData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
@@ -48,9 +50,19 @@ public interface DistribStateManager extends Closeable {
 
   void makePath(String path) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
 
+  void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
+
+  /**
+   * Create data (leaf) node at specified path.
+   * @param path base path name of the node.
+   * @param data data to be stored.
+   * @param mode creation mode.
+   * @return actual path of the node - in case of sequential nodes this will differ from the base path because
+   * of the appended sequence number.
+   */
   String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
 
-  void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
+  void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException, BadVersionException;
 
   void setData(String path, byte[] data, int version) throws BadVersionException, NoSuchElementException, IOException, KeeperException, InterruptedException;
 
@@ -61,9 +73,4 @@ public interface DistribStateManager extends Closeable {
   default AutoScalingConfig getAutoScalingConfig() throws InterruptedException, IOException {
     return getAutoScalingConfig(null);
   }
-
-  @Override
-  default void close() throws IOException {
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
index dbf6836..68dfa39 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
@@ -20,10 +20,12 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.solr.common.SolrCloseable;
+
 /**
  * This interface models the access to node and replica information.
  */
-public interface NodeStateProvider {
+public interface NodeStateProvider extends SolrCloseable {
   /**
    * Get the value of each tag for a given node
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
index d73ae6c..f11121d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
@@ -18,6 +18,7 @@
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -37,9 +38,12 @@ import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.emptyList;
 import static java.util.Collections.emptyMap;
@@ -56,6 +60,8 @@ import static java.util.stream.Collectors.toList;
  *
  */
 public class Policy implements MapWriter {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   public static final String POLICY = "policy";
   public static final String EACH = "#EACH";
   public static final String ANY = "#ANY";
@@ -211,16 +217,26 @@ public class Policy implements MapWriter {
     Set<String> collections = new HashSet<>();
     List<Clause> expandedClauses;
     List<Violation> violations = new ArrayList<>();
+    final int znodeVersion;
 
     private Session(List<String> nodes, SolrCloudManager cloudManager,
-                    List<Row> matrix, List<Clause> expandedClauses) {
+                    List<Row> matrix, List<Clause> expandedClauses, int znodeVersion) {
       this.nodes = nodes;
       this.cloudManager = cloudManager;
       this.matrix = matrix;
       this.expandedClauses = expandedClauses;
+      this.znodeVersion = znodeVersion;
     }
 
     Session(SolrCloudManager cloudManager) {
+      ClusterState state = null;
+      try {
+        state = cloudManager.getClusterStateProvider().getClusterState();
+        LOG.trace("-- session created with cluster state: {}", state);
+      } catch (Exception e) {
+        LOG.trace("-- session created, can't obtain cluster state", e);
+      }
+      this.znodeVersion = state != null ? state.getZNodeVersion() : -1;
       this.nodes = new ArrayList<>(cloudManager.getClusterStateProvider().getLiveNodes());
       this.cloudManager = cloudManager;
       for (String node : nodes) {
@@ -256,7 +272,7 @@ public class Policy implements MapWriter {
     }
 
     Session copy() {
-      return new Session(nodes, cloudManager, getMatrixCopy(), expandedClauses);
+      return new Session(nodes, cloudManager, getMatrixCopy(), expandedClauses, znodeVersion);
     }
 
     List<Row> getMatrixCopy() {
@@ -297,6 +313,7 @@ public class Policy implements MapWriter {
 
     @Override
     public void writeMap(EntryWriter ew) throws IOException {
+      ew.put("znodeVersion", znodeVersion);
       for (int i = 0; i < matrix.size(); i++) {
         Row row = matrix.get(i);
         ew.put(row.node, row);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
index 024c6c3..40ca619 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
@@ -30,12 +30,12 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester.Hint;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
-import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
 import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -138,7 +138,7 @@ public class PolicyHelper {
 
   public static final int SESSION_EXPIRY = 180;//3 seconds
 
-  public static MapWriter getDiagnostics(Policy policy, SolrClientCloudManager cloudManager) {
+  public static MapWriter getDiagnostics(Policy policy, SolrCloudManager cloudManager) {
     Policy.Session session = policy.createSession(cloudManager);
     List<Row> sorted = session.getSorted();
     List<Violation> violations = session.getViolations();
@@ -233,9 +233,10 @@ public class PolicyHelper {
      *
      */
     private void returnSession(SessionWrapper sessionWrapper) {
+      TimeSource timeSource = sessionWrapper.session.cloudManager.getTimeSource();
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        log.info("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} ", time(MILLISECONDS),
+        log.info("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} ", time(timeSource, MILLISECONDS),
             sessionWrapper.createTime,
             this.sessionWrapper.createTime);
         if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
@@ -255,13 +256,14 @@ public class PolicyHelper {
 
 
     public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+      TimeSource timeSource = cloudManager.getTimeSource();
       synchronized (lockObj) {
         if (sessionWrapper.status == Status.NULL ||
-            TimeUnit.SECONDS.convert(System.nanoTime() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
+            TimeUnit.SECONDS.convert(timeSource.getTime() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
           //no session available or the session is expired
           return createSession(cloudManager);
         } else {
-          long waitStart = time(MILLISECONDS);
+          long waitStart = time(timeSource, MILLISECONDS);
           //the session is not expired
           log.debug("reusing a session {}", this.sessionWrapper.createTime);
           if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
@@ -269,13 +271,13 @@ public class PolicyHelper {
             return sessionWrapper;
           } else {
             //status= COMPUTING it's being used for computing. computing is
-            log.debug("session being used. waiting... current time {} ", time(MILLISECONDS));
+            log.debug("session being used. waiting... current time {} ", time(timeSource, MILLISECONDS));
             try {
               lockObj.wait(10 * 1000);//wait for a max of 10 seconds
             } catch (InterruptedException e) {
               log.info("interrupted... ");
             }
-            log.debug("out of waiting curr-time:{} time-elapsed {}", time(MILLISECONDS), timeElapsed(waitStart, MILLISECONDS));
+            log.debug("out of waiting curr-time:{} time-elapsed {}", time(timeSource, MILLISECONDS), timeElapsed(timeSource, waitStart, MILLISECONDS));
             // now this thread has woken up because it got timed out after 10 seconds or it is notified after
             //the session was returned from another COMPUTING operation
             if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
@@ -289,8 +291,6 @@ public class PolicyHelper {
           }
         }
       }
-
-
     }
 
     private SessionWrapper createSession(SolrCloudManager cloudManager) throws InterruptedException, IOException {
@@ -361,7 +361,9 @@ public class PolicyHelper {
     }
 
     public SessionWrapper(Policy.Session session, SessionRef ref) {
-      lastUpdateTime = createTime = System.nanoTime();
+      lastUpdateTime = createTime = session != null ?
+          session.cloudManager.getTimeSource().getTime() :
+          TimeSource.NANO_TIME.getTime();
       this.session = session;
       this.status = Status.UNUSED;
       this.ref = ref;
@@ -372,7 +374,9 @@ public class PolicyHelper {
     }
 
     public SessionWrapper update(Policy.Session session) {
-      this.lastUpdateTime = System.nanoTime();
+      this.lastUpdateTime = session != null ?
+          session.cloudManager.getTimeSource().getTime() :
+          TimeSource.NANO_TIME.getTime();
       this.session = session;
       return this;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
index 5f7281f..930ede8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
@@ -18,6 +18,7 @@
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 
@@ -33,21 +34,25 @@ public class ReplicaInfo implements MapWriter {
   private String core, collection, shard;
   private Replica.Type type;
   private String node;
-  private Map<String, Object> variables;
+  private final Map<String, Object> variables = new HashMap<>();
 
-  public ReplicaInfo(String coll,String shard, Replica r, Map<String, Object> vals){
+  public ReplicaInfo(String coll, String shard, Replica r, Map<String, Object> vals) {
     this.name = r.getName();
     this.core = r.getCoreName();
     this.collection = coll;
     this.shard = shard;
     this.type = r.getType();
-    this.variables = vals;
+    if (vals != null) {
+      this.variables.putAll(vals);
+    }
     this.node = r.getNodeName();
   }
 
   public ReplicaInfo(String name, String core, String coll, String shard, Replica.Type type, String node, Map<String, Object> vals) {
     this.name = name;
-    this.variables = vals;
+    if (vals != null) {
+      this.variables.putAll(vals);
+    }
     this.collection = coll;
     this.shard = shard;
     this.type = type;
@@ -58,12 +63,22 @@ public class ReplicaInfo implements MapWriter {
   @Override
   public void writeMap(EntryWriter ew) throws IOException {
     ew.put(name, (MapWriter) ew1 -> {
-      if (variables != null) {
-        for (Map.Entry<String, Object> e : variables.entrySet()) {
-          ew1.put(e.getKey(), e.getValue());
-        }
+      for (Map.Entry<String, Object> e : variables.entrySet()) {
+        ew1.put(e.getKey(), e.getValue());
+      }
+      if (core != null && !variables.containsKey(ZkStateReader.CORE_NAME_PROP)) {
+        ew1.put(ZkStateReader.CORE_NAME_PROP, core);
+      }
+      if (shard != null && !variables.containsKey(ZkStateReader.SHARD_ID_PROP)) {
+        ew1.put(ZkStateReader.SHARD_ID_PROP, shard);
+      }
+      if (collection != null && !variables.containsKey(ZkStateReader.COLLECTION_PROP)) {
+        ew1.put(ZkStateReader.COLLECTION_PROP, collection);
+      }
+      if (node != null && !variables.containsKey(ZkStateReader.NODE_NAME_PROP)) {
+        ew1.put(ZkStateReader.NODE_NAME_PROP, node);
       }
-      if (type != null) ew1.put("type", type.toString());
+      if (type != null) ew1.put(ZkStateReader.REPLICA_TYPE, type.toString());
     });
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
index 8a1f8f0..55cdcee 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
@@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * This interface abstracts the access to a SolrCloud cluster, including interactions with Zookeeper, Solr
@@ -44,16 +45,11 @@ public interface SolrCloudManager extends SolrCloseable {
 
   ObjectCache getObjectCache();
 
+  TimeSource getTimeSource();
+
   // Solr-like methods
 
   SolrResponse request(SolrRequest req) throws IOException;
 
   byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException;
-
-  // distributed queue implementation
-
-  @Override
-  default void close() {
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
index aec5f15..070869a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
@@ -38,7 +38,7 @@ import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.ANY;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
 
 public class Suggestion {
-  static final String coreidxsize = "INDEX.sizeInBytes";
+  public static final String coreidxsize = "INDEX.sizeInBytes";
   static final Map<String, ConditionType> validatetypes = new HashMap<>();
 
   public static ConditionType getTagType(String name) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 2432fb2..dfe15df 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
@@ -81,6 +81,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -1030,13 +1031,13 @@ public class CloudSolrClient extends SolrClient {
       if (!liveNodes.isEmpty()) {
         List<String> liveNodesList = new ArrayList<>(liveNodes);
         Collections.shuffle(liveNodesList, rand);
-        theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNodesList.get(0),
+        theUrlList.add(Utils.getBaseUrlForNodeName(liveNodesList.get(0),
             (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
       }
 
     } else if (ADMIN_PATHS.contains(request.getPath())) {
       for (String liveNode : liveNodes) {
-        theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNode,
+        theUrlList.add(Utils.getBaseUrlForNodeName(liveNode,
             (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
       }
 


[6/9] lucene-solr:master: SOLR-11285: Simulation framework for autoscaling.

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
new file mode 100644
index 0000000..5d7aa4d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
@@ -0,0 +1,599 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Predicate;
+
+import com.codahale.metrics.Timer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.cloud.OverseerTaskQueue;
+import org.apache.solr.cloud.Stats;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.Pair;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A distributed queue that uses {@link DistribStateManager} as the underlying distributed store.
+ * Implementation based on {@link org.apache.solr.cloud.ZkDistributedQueue}
+ */
+public class GenericDistributedQueue implements DistributedQueue {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  static final String PREFIX = "qn-";
+
+  /**
+   * Theory of operation:
+   * <p>
+   * Under ordinary circumstances we neither watch nor poll for children in ZK.
+   * Instead we keep an in-memory list of known child names.  When the in-memory
+   * list is exhausted, we then fetch from ZK.
+   * <p>
+   * We only bother setting a child watcher when the queue has no children in ZK.
+   */
+  private static final Object _IMPLEMENTATION_NOTES = null;
+
+  final String dir;
+
+  final DistribStateManager stateManager;
+
+  final Stats stats;
+
+  /**
+   * A lock that guards all of the mutable state that follows.
+   */
+  private final ReentrantLock updateLock = new ReentrantLock();
+
+  /**
+   * Contains the last set of children fetched from ZK. Elements are removed from the head of
+   * this in-memory set as they are consumed from the queue.  Due to the distributed nature
+   * of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
+   * Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
+   * like {@link #poll()} must resolve any races by attempting to delete the underlying node.
+   */
+  private TreeSet<String> knownChildren = new TreeSet<>();
+
+  /**
+   * Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
+   */
+  private final Condition changed = updateLock.newCondition();
+
+  private boolean isDirty = true;
+
+  private int watcherCount = 0;
+
+  private final int maxQueueSize;
+
+  /**
+   * If {@link #maxQueueSize} is set, the number of items we can queue without rechecking the server.
+   */
+  private final AtomicInteger offerPermits = new AtomicInteger(0);
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir) {
+    this(stateManager, dir, new Stats());
+  }
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats) {
+    this(stateManager, dir, stats, 0);
+  }
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats, int maxQueueSize) {
+    this.dir = dir;
+
+    try {
+      if (!stateManager.hasData(dir)) {
+        try {
+          stateManager.makePath(dir);
+        } catch (AlreadyExistsException e) {
+          // ignore
+        }
+      }
+    } catch (IOException | KeeperException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    }
+
+    this.stateManager = stateManager;
+    this.stats = stats;
+    this.maxQueueSize = maxQueueSize;
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty.
+   *
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek() throws Exception {
+    Timer.Context time = stats.time(dir + "_peek");
+    try {
+      return firstElement();
+    } finally {
+      time.stop();
+    }
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty and block is false.
+   *
+   * @param block if true, blocks until an element enters the queue
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek(boolean block) throws Exception {
+    return block ? peek(Long.MAX_VALUE) : peek();
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty after wait ms.
+   *
+   * @param wait max wait time in ms.
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek(long wait) throws Exception {
+    Preconditions.checkArgument(wait > 0);
+    Timer.Context time;
+    if (wait == Long.MAX_VALUE) {
+      time = stats.time(dir + "_peek_wait_forever");
+    } else {
+      time = stats.time(dir + "_peek_wait" + wait);
+    }
+    updateLock.lockInterruptibly();
+    try {
+      long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+      while (waitNanos > 0) {
+        byte[] result = firstElement();
+        if (result != null) {
+          return result;
+        }
+        waitNanos = changed.awaitNanos(waitNanos);
+      }
+      return null;
+    } finally {
+      updateLock.unlock();
+      time.stop();
+    }
+  }
+
+  /**
+   * Attempts to remove the head of the queue and return it. Returns null if the
+   * queue is empty.
+   *
+   * @return Head of the queue or null.
+   */
+  @Override
+  public byte[] poll() throws Exception {
+    Timer.Context time = stats.time(dir + "_poll");
+    try {
+      return removeFirst();
+    } finally {
+      time.stop();
+    }
+  }
+
+  /**
+   * Attempts to remove the head of the queue and return it.
+   *
+   * @return The former head of the queue
+   */
+  @Override
+  public byte[] remove() throws Exception {
+    Timer.Context time = stats.time(dir + "_remove");
+    try {
+      byte[] result = removeFirst();
+      if (result == null) {
+        throw new NoSuchElementException();
+      }
+      return result;
+    } finally {
+      time.stop();
+    }
+  }
+
+  public void remove(Collection<String> paths) throws Exception {
+    if (paths.isEmpty()) return;
+    List<Op> ops = new ArrayList<>();
+    for (String path : paths) {
+      ops.add(Op.delete(dir + "/" + path, -1));
+    }
+    for (int from = 0; from < ops.size(); from += 1000) {
+      int to = Math.min(from + 1000, ops.size());
+      if (from < to) {
+        try {
+          stateManager.multi(ops.subList(from, to));
+        } catch (NoSuchElementException e) {
+          // don't know which nodes are not exist, so try to delete one by one node
+          for (int j = from; j < to; j++) {
+            try {
+              stateManager.removeData(ops.get(j).getPath(), -1);
+            } catch (NoSuchElementException e2) {
+              LOG.debug("Can not remove node which is not exist : " + ops.get(j).getPath());
+            }
+          }
+        }
+      }
+    }
+
+    int cacheSizeBefore = knownChildren.size();
+    knownChildren.removeAll(paths);
+    if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+      stats.setQueueLength(knownChildren.size());
+    } else {
+      // There are elements get deleted but not present in the cache,
+      // the cache seems not valid anymore
+      knownChildren.clear();
+      isDirty = true;
+    }
+  }
+
+  /**
+   * Removes the head of the queue and returns it, blocks until it succeeds.
+   *
+   * @return The former head of the queue
+   */
+  @Override
+  public byte[] take() throws Exception {
+    // Same as for element. Should refactor this.
+    Timer.Context timer = stats.time(dir + "_take");
+    updateLock.lockInterruptibly();
+    try {
+      while (true) {
+        byte[] result = removeFirst();
+        if (result != null) {
+          return result;
+        }
+        changed.await();
+      }
+    } finally {
+      updateLock.unlock();
+      timer.stop();
+    }
+  }
+
+  /**
+   * Inserts data into queue.  If there are no other queue consumers, the offered element
+   * will be immediately visible when this method returns.
+   */
+  @Override
+  public void offer(byte[] data) throws Exception {
+    Timer.Context time = stats.time(dir + "_offer");
+    try {
+      while (true) {
+        try {
+          if (maxQueueSize > 0) {
+            if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
+              // If a max queue size is set, check it before creating a new queue item.
+              if (!stateManager.hasData(dir)) {
+                // jump to the code below, which tries to create dir if it doesn't exist
+                throw new NoSuchElementException();
+              }
+              List<String> children = stateManager.listData(dir);
+              int remainingCapacity = maxQueueSize - children.size();
+              if (remainingCapacity <= 0) {
+                throw new IllegalStateException("queue is full");
+              }
+
+              // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
+              offerPermits.set(remainingCapacity / 100);
+            }
+          }
+
+          // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
+          // This will get set again when the watcher actually fires, but that's ok.
+          stateManager.createData(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL);
+          isDirty = true;
+          return;
+        } catch (NoSuchElementException e) {
+          try {
+            stateManager.createData(dir, new byte[0], CreateMode.PERSISTENT);
+          } catch (NoSuchElementException ne) {
+            // someone created it
+          }
+        }
+      }
+    } finally {
+      time.stop();
+    }
+  }
+
+  public Stats getZkStats() {
+    return stats;
+  }
+
+  @Override
+  public Map<String, Object> getStats() {
+    if (stats == null) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> res = new HashMap<>();
+    res.put("queueLength", stats.getQueueLength());
+    final Map<String, Object> statsMap = new HashMap<>();
+    res.put("stats", statsMap);
+    stats.getStats().forEach((op, stat) -> {
+      final Map<String, Object> statMap = new HashMap<>();
+      statMap.put("success", stat.success.get());
+      statMap.put("errors", stat.errors.get());
+      final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
+      statMap.put("failureDetails", failed);
+      stat.failureDetails.forEach(failedOp -> {
+        Map<String, Object> fo = new HashMap<>();
+        fo.put("req", failedOp.req);
+        fo.put("resp", failedOp.resp);
+      });
+      statsMap.put(op, statMap);
+    });
+    return res;
+  }
+
+  /**
+   * Returns the name if the first known child node, or {@code null} if the queue is empty.
+   * This is the only place {@link #knownChildren} is ever updated!
+   * The caller must double check that the actual node still exists, since the in-memory
+   * list is inherently stale.
+   */
+  private String firstChild(boolean remove, boolean refetchIfDirty) throws Exception {
+    updateLock.lockInterruptibly();
+    try {
+      // We always return from cache first, the cache will be cleared if the node is not exist
+      if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
+        return remove ? knownChildren.pollFirst() : knownChildren.first();
+      }
+
+      if (!isDirty && knownChildren.isEmpty()) {
+        return null;
+      }
+
+      // Dirty, try to fetch an updated list of children from ZK.
+      // Only set a new watcher if there isn't already a watcher.
+      ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
+      knownChildren = fetchZkChildren(newWatcher);
+      if (newWatcher != null) {
+        watcherCount++; // watcher was successfully set
+      }
+      isDirty = false;
+      if (knownChildren.isEmpty()) {
+        return null;
+      }
+      changed.signalAll();
+      return remove ? knownChildren.pollFirst() : knownChildren.first();
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  /**
+   * Return the current set of children from ZK; does not change internal state.
+   */
+  TreeSet<String> fetchZkChildren(Watcher watcher) throws Exception {
+    while (true) {
+      try {
+        TreeSet<String> orderedChildren = new TreeSet<>();
+
+        List<String> childNames = stateManager.listData(dir, watcher);
+        stats.setQueueLength(childNames.size());
+        for (String childName : childNames) {
+          // Check format
+          if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
+            LOG.debug("Found child node with improper name: " + childName);
+            continue;
+          }
+          orderedChildren.add(childName);
+        }
+        return orderedChildren;
+      } catch (NoSuchElementException e) {
+        try {
+          stateManager.makePath(dir);
+        } catch (AlreadyExistsException e2) {
+          // ignore
+        }
+        // go back to the loop and try again
+      }
+    }
+  }
+
+  /**
+   * Return the currently-known set of elements, using child names from memory. If no children are found, or no
+   * children pass {@code acceptFilter}, waits up to {@code waitMillis} for at least one child to become available.
+   * <p>
+   * Package-private to support {@link OverseerTaskQueue} specifically.</p>
+   */
+  @Override
+  public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
+    List<String> foundChildren = new ArrayList<>();
+    long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
+    boolean first = true;
+    while (true) {
+      // Trigger a refresh, but only force it if this is not the first iteration.
+      firstChild(false, !first);
+
+      updateLock.lockInterruptibly();
+      try {
+        for (String child : knownChildren) {
+          if (acceptFilter.test(child)) {
+            foundChildren.add(child);
+          }
+        }
+        if (!foundChildren.isEmpty()) {
+          break;
+        }
+        if (waitNanos <= 0) {
+          break;
+        }
+
+        // If this is our first time through, force a refresh before waiting.
+        if (first) {
+          first = false;
+          continue;
+        }
+
+        waitNanos = changed.awaitNanos(waitNanos);
+      } finally {
+        updateLock.unlock();
+      }
+
+      if (!foundChildren.isEmpty()) {
+        break;
+      }
+    }
+
+    // Technically we could restart the method if we fail to actually obtain any valid children
+    // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
+    // much more sophisticated waitNanos tracking.
+    List<Pair<String, byte[]>> result = new ArrayList<>();
+    for (String child : foundChildren) {
+      if (result.size() >= max) {
+        break;
+      }
+      try {
+        VersionedData data = stateManager.getData(dir + "/" + child);
+        result.add(new Pair<>(child, data.getData()));
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and continue.
+        updateLock.lockInterruptibly();
+        try {
+          knownChildren.remove(child);
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Return the head of the queue without modifying the queue.
+   *
+   * @return the data at the head of the queue.
+   */
+  private byte[] firstElement() throws Exception {
+    while (true) {
+      String firstChild = firstChild(false, false);
+      if (firstChild == null) {
+        return null;
+      }
+      try {
+        VersionedData data = stateManager.getData(dir + "/" + firstChild);
+        return data != null ? data.getData() : null;
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and retry.
+        updateLock.lockInterruptibly();
+        try {
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+  }
+
+  private byte[] removeFirst() throws Exception {
+    while (true) {
+      String firstChild = firstChild(true, false);
+      if (firstChild == null) {
+        return null;
+      }
+      try {
+        String path = dir + "/" + firstChild;
+        VersionedData result = stateManager.getData(path);
+        stateManager.removeData(path, -1);
+        stats.setQueueLength(knownChildren.size());
+        return result.getData();
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and retry.
+        updateLock.lockInterruptibly();
+        try {
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting int watcherCount() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
+      return watcherCount;
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  @VisibleForTesting boolean isDirty() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
+      return isDirty;
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  @VisibleForTesting class ChildWatcher implements Watcher {
+
+    @Override
+    public void process(WatchedEvent event) {
+      // session events are not change events, and do not remove the watcher; except for Expired
+      if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
+        return;
+      }
+      updateLock.lock();
+      try {
+        isDirty = true;
+        watcherCount--;
+        // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
+        changed.signalAll();
+      } finally {
+        updateLock.unlock();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
new file mode 100644
index 0000000..12b4af8
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.sim;
+
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+
+/**
+ * Factory for {@link GenericDistributedQueue}.
+ */
+public class GenericDistributedQueueFactory implements DistributedQueueFactory {
+
+  private final DistribStateManager stateManager;
+
+  public GenericDistributedQueueFactory(DistribStateManager stateManager) {
+    this.stateManager = stateManager;
+  }
+
+  @Override
+  public DistributedQueue makeQueue(String path) throws IOException {
+    return new GenericDistributedQueue(stateManager, path);
+  }
+
+  @Override
+  public void removeQueue(String path) throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
new file mode 100644
index 0000000..45cd66b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
@@ -0,0 +1,99 @@
+/*
+ * 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.sim;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.common.cloud.LiveNodesListener;
+
+/**
+ * This class represents a set of live nodes and allows adding listeners to track their state.
+ */
+public class LiveNodesSet {
+
+  private final Set<String> set = ConcurrentHashMap.newKeySet();
+  private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
+
+  public Set<String> get() {
+    return Collections.unmodifiableSet(set);
+  }
+
+  public void registerLiveNodesListener(LiveNodesListener listener) {
+    listeners.add(listener);
+  }
+
+  public void removeLiveNodesListener(LiveNodesListener listener) {
+    listeners.remove(listener);
+  }
+
+  private void fireListeners(SortedSet<String> oldNodes, SortedSet<String> newNodes) {
+    for (LiveNodesListener listener : listeners) {
+      listener.onChange(oldNodes, newNodes);
+    }
+  }
+
+  public boolean isEmpty() {
+    return set.isEmpty();
+  }
+
+  public boolean contains(String id) {
+    return set.contains(id);
+  }
+
+  public synchronized boolean add(String id) {
+    if (set.contains(id)) {
+      return false;
+    }
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.add(id);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    fireListeners(oldNodes, newNodes);
+    return true;
+  }
+
+  public synchronized boolean addAll(Collection<String> nodes) {
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    boolean changed = set.addAll(nodes);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    if (changed) {
+      fireListeners(oldNodes, newNodes);
+    }
+    return changed;
+  }
+
+  public synchronized boolean remove(String id) {
+    if (!set.contains(id)) {
+      return false;
+    }
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.remove(id);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    fireListeners(oldNodes, newNodes);
+    return true;
+  }
+
+  public synchronized void clear() {
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.clear();
+    fireListeners(oldNodes, Collections.emptySortedSet());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
new file mode 100644
index 0000000..bde4b41
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -0,0 +1,607 @@
+/*
+ * 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.sim;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+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.SolrServerException;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.client.solrj.response.SolrResponseBase;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
+import org.apache.solr.cloud.autoscaling.OverseerTriggerThread;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CloudConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.OverseerCollectionMessageHandler.REQUESTID;
+
+/**
+ * Simulated {@link SolrCloudManager}.
+ */
+public class SimCloudManager implements SolrCloudManager {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final SimDistribStateManager stateManager;
+  private final SimClusterStateProvider clusterStateProvider;
+  private final SimNodeStateProvider nodeStateProvider;
+  private final AutoScalingHandler autoScalingHandler;
+  private final LiveNodesSet liveNodesSet = new LiveNodesSet();
+  private final DistributedQueueFactory queueFactory;
+  private final ObjectCache objectCache = new ObjectCache();
+  private TimeSource timeSource;
+
+  private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
+  private final ExecutorService simCloudManagerPool;
+  private final Map<String, AtomicLong> opCounts = new ConcurrentHashMap<>();
+
+
+  private Overseer.OverseerThread triggerThread;
+  private ThreadGroup triggerThreadGroup;
+  private SolrResourceLoader loader;
+
+  private static int nodeIdPort = 10000;
+
+  /**
+   * Create a simulated cluster. This cluster uses the following components:
+   * <ul>
+   *   <li>{@link SimDistribStateManager} with non-shared root node.</li>
+   *   <li>{@link SimClusterStateProvider}</li>
+   *   <li>{@link SimNodeStateProvider}, where node values are automatically initialized when using
+   *   {@link #simAddNode()} method.</li>
+   *   <li>{@link GenericDistributedQueueFactory} that uses {@link SimDistribStateManager} as its storage.</li>
+   *   <li>an instance of {@link AutoScalingHandler} for managing AutoScalingConfig.</li>
+   *   <li>an instance of {@link OverseerTriggerThread} for managing triggers and processing events.</li>
+   * </ul>
+   * @param timeSource time source to use.
+   */
+  public SimCloudManager(TimeSource timeSource) throws Exception {
+    this.stateManager = new SimDistribStateManager(SimDistribStateManager.createNewRootNode());
+    this.loader = new SolrResourceLoader();
+    // init common paths
+    stateManager.makePath(ZkStateReader.CLUSTER_STATE);
+    stateManager.makePath(ZkStateReader.CLUSTER_PROPS);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    stateManager.makePath(ZkStateReader.LIVE_NODES_ZKNODE);
+    stateManager.makePath(ZkStateReader.ROLES);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+
+    this.timeSource = timeSource != null ? timeSource : TimeSource.NANO_TIME;
+    this.clusterStateProvider = new SimClusterStateProvider(liveNodesSet, this);
+    this.nodeStateProvider = new SimNodeStateProvider(liveNodesSet, this.stateManager, this.clusterStateProvider, null);
+    this.queueFactory = new GenericDistributedQueueFactory(stateManager);
+    this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
+    this.autoScalingHandler = new AutoScalingHandler(this, loader);
+    triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers");
+    OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
+        new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
+    triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
+    triggerThread.start();
+  }
+
+  // ---------- simulator setup methods -----------
+
+  /**
+   * Create a cluster with the specified number of nodes. Node metrics are pre-populated.
+   * @param numNodes number of nodes to create
+   * @param timeSource time source
+   * @return instance of simulated cluster
+   */
+  public static SimCloudManager createCluster(int numNodes, TimeSource timeSource) throws Exception {
+    SimCloudManager cloudManager = new SimCloudManager(timeSource);
+    for (int i = 1; i <= numNodes; i++) {
+      Map<String, Object> values = createNodeValues(null);
+//      if (i == 1) { // designated Overseer ?
+        //values.put(ImplicitSnitch.NODEROLE, "overseer");
+//      }
+      String nodeId = (String)values.get(ImplicitSnitch.NODE);
+      cloudManager.getSimClusterStateProvider().simAddNode(nodeId);
+      cloudManager.getSimNodeStateProvider().simSetNodeValues(nodeId, values);
+    }
+    return cloudManager;
+  }
+
+  /**
+   * Create a cluster initialized from the provided cluster state.
+   * @param initialState existing cluster state
+   * @param timeSource time source
+   * @return instance of simulated cluster with the same layout as the provided cluster state.
+   */
+  public static SimCloudManager createCluster(ClusterState initialState, TimeSource timeSource) throws Exception {
+    SimCloudManager cloudManager = new SimCloudManager(timeSource);
+    cloudManager.getSimClusterStateProvider().simSetClusterState(initialState);
+    for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
+      cloudManager.getSimNodeStateProvider().simSetNodeValues(node, createNodeValues(node));
+    }
+    return cloudManager;
+  }
+
+  /**
+   * Create simulated node values (metrics) for a node.
+   * @param nodeName node name (eg. '127.0.0.1:10000_solr'). If null then a new node name will be
+   *                 created using sequentially increasing port number.
+   * @return node values
+   */
+  public static Map<String, Object> createNodeValues(String nodeName) {
+    Map<String, Object> values = new HashMap<>();
+    String host, nodeId;
+    int port;
+    if (nodeName == null) {
+      host = "127.0.0.1";
+      port = nodeIdPort++;
+      nodeId = host + ":" + port + "_solr";
+      values.put("ip_1", "127");
+      values.put("ip_2", "0");
+      values.put("ip_3", "0");
+      values.put("ip_4", "1");
+    } else {
+      String[] hostPortCtx = nodeName.split(":");
+      if (hostPortCtx.length != 2) {
+        throw new RuntimeException("Invalid nodeName " + nodeName);
+      }
+      host = hostPortCtx[0];
+      String[] portCtx = hostPortCtx[1].split("_");
+      if (portCtx.length != 2) {
+        throw new RuntimeException("Invalid port_context in nodeName " + nodeName);
+      }
+      port = Integer.parseInt(portCtx[0]);
+      nodeId = host + ":" + port + "_" + portCtx[1];
+      String[] ip = host.split("\\.");
+      if (ip.length == 4) {
+        values.put("ip_1", ip[0]);
+        values.put("ip_2", ip[1]);
+        values.put("ip_3", ip[2]);
+        values.put("ip_4", ip[3]);
+      }
+    }
+    values.put(ImplicitSnitch.HOST, host);
+    values.put(ImplicitSnitch.PORT, port);
+    values.put(ImplicitSnitch.NODE, nodeId);
+    values.put(ImplicitSnitch.CORES, 0);
+    values.put(ImplicitSnitch.DISK, 1000);
+    values.put(ImplicitSnitch.SYSLOADAVG, 1.0);
+    values.put(ImplicitSnitch.HEAPUSAGE, 123450000);
+    values.put("sysprop.java.version", System.getProperty("java.version"));
+    values.put("sysprop.java.vendor", System.getProperty("java.vendor"));
+    // fake some metrics expected in tests
+    values.put("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count", 0);
+    values.put("metrics:solr.jvm:buffers.direct.Count", 0);
+    return values;
+  }
+
+  /**
+   * Get the instance of {@link SolrResourceLoader} that is used by the cluster components.
+   */
+  public SolrResourceLoader getLoader() {
+    return loader;
+  }
+
+  /**
+   * Add a new node and initialize its node values (metrics). The
+   * /live_nodes list is updated with the new node id.
+   * @return new node id
+   */
+  public String simAddNode() throws Exception {
+    Map<String, Object> values = createNodeValues(null);
+    String nodeId = (String)values.get(ImplicitSnitch.NODE);
+    clusterStateProvider.simAddNode(nodeId);
+    nodeStateProvider.simSetNodeValues(nodeId, values);
+    LOG.trace("-- added node " + nodeId);
+    return nodeId;
+  }
+
+  /**
+   * Remove a node from the cluster. This simulates a node lost scenario.
+   * Node id is removed from the /live_nodes list.
+   * @param nodeId node id
+   * @param withValues when true, remove also simulated node values. If false
+   *                   then node values are retained to later simulate
+   *                   a node that comes back up
+   */
+  public void simRemoveNode(String nodeId, boolean withValues) throws Exception {
+    clusterStateProvider.simRemoveNode(nodeId);
+    if (withValues) {
+      nodeStateProvider.simRemoveNodeValues(nodeId);
+    }
+    LOG.trace("-- removed node " + nodeId);
+  }
+
+  /**
+   * Remove a number of randomly selected nodes
+   * @param number number of nodes to remove
+   * @param withValues when true, remove also simulated node values. If false
+   *                   then node values are retained to later simulate
+   *                   a node that comes back up
+   * @param random random
+   */
+  public void simRemoveRandomNodes(int number, boolean withValues, Random random) throws Exception {
+    List<String> nodes = new ArrayList<>(liveNodesSet.get());
+    Collections.shuffle(nodes, random);
+    int count = Math.min(number, nodes.size());
+    for (int i = 0; i < count; i++) {
+      simRemoveNode(nodes.get(i), withValues);
+    }
+  }
+
+  /**
+   * Clear the (simulated) .system collection.
+   */
+  public void simClearSystemCollection() {
+    systemColl.clear();
+  }
+
+  /**
+   * Get the content of (simulated) .system collection.
+   * @return documents in the collection, in chronological order starting from the oldest.
+   */
+  public List<SolrInputDocument> simGetSystemCollection() {
+    return systemColl;
+  }
+
+  /**
+   * Get a {@link SolrClient} implementation where calls are forwarded to this
+   * instance of the cluster.
+   * @return simulated SolrClient.
+   */
+  public SolrClient simGetSolrClient() {
+    return new SolrClient() {
+      @Override
+      public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
+        SolrResponse rsp = SimCloudManager.this.request(request);
+        return rsp.getResponse();
+      }
+
+      @Override
+      public void close() throws IOException {
+
+      }
+    };
+  }
+
+  /**
+   * Simulate the effect of restarting Overseer leader - in this case this means restarting the
+   * OverseerTriggerThread and optionally killing a node.
+   * @param killNodeId optional nodeId to kill. If null then don't kill any node, just restart the thread
+   */
+  public void simRestartOverseer(String killNodeId) throws Exception {
+    LOG.info("=== Restarting OverseerTriggerThread and clearing object cache...");
+    triggerThread.interrupt();
+    IOUtils.closeQuietly(triggerThread);
+    if (killNodeId != null) {
+      simRemoveNode(killNodeId, true);
+    }
+    objectCache.clear();
+    OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
+        new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
+    triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
+    triggerThread.start();
+
+  }
+
+  /**
+   * Submit a task to execute in a thread pool.
+   * @param callable task to execute
+   * @return future to obtain results
+   */
+  public <T> Future<T> submit(Callable<T> callable) {
+    return simCloudManagerPool.submit(callable);
+  }
+
+  // ---------- type-safe methods to obtain simulator components ----------
+  public SimClusterStateProvider getSimClusterStateProvider() {
+    return clusterStateProvider;
+  }
+
+  public SimNodeStateProvider getSimNodeStateProvider() {
+    return nodeStateProvider;
+  }
+
+  public SimDistribStateManager getSimDistribStateManager() {
+    return stateManager;
+  }
+
+  public LiveNodesSet getLiveNodesSet() {
+    return liveNodesSet;
+  }
+
+  /**
+   * Get the number and type of operations processed by this cluster.
+   */
+  public Map<String, AtomicLong> simGetOpCounts() {
+    return opCounts;
+  }
+
+  /**
+   * Get the number of processed operations of a specified type.
+   * @param op operation name, eg. MOVEREPLICA
+   * @return number of operations
+   */
+  public long simGetOpCount(String op) {
+    AtomicLong count = opCounts.get(op);
+    return count != null ? count.get() : 0L;
+  }
+
+  // --------- interface methods -----------
+
+
+  @Override
+  public ObjectCache getObjectCache() {
+    return objectCache;
+  }
+
+  @Override
+  public TimeSource getTimeSource() {
+    return timeSource;
+  }
+
+  @Override
+  public ClusterStateProvider getClusterStateProvider() {
+    return clusterStateProvider;
+  }
+
+  @Override
+  public NodeStateProvider getNodeStateProvider() {
+    return nodeStateProvider;
+  }
+
+  @Override
+  public DistribStateManager getDistribStateManager() {
+    return stateManager;
+  }
+
+  @Override
+  public DistributedQueueFactory getDistributedQueueFactory() {
+    return queueFactory;
+  }
+
+  @Override
+  public SolrResponse request(SolrRequest req) throws IOException {
+    try {
+      Future<SolrResponse> rsp = submit(() -> simHandleSolrRequest(req));
+      return rsp.get();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  private void incrementCount(String op) {
+    AtomicLong count = opCounts.computeIfAbsent(op, o -> new AtomicLong());
+    count.incrementAndGet();
+  }
+
+  /**
+   * Handler method for autoscaling requests. NOTE: only a specific subset of autoscaling requests is
+   * supported!
+   * @param req autoscaling request
+   * @return results
+   */
+  public SolrResponse simHandleSolrRequest(SolrRequest req) throws IOException, InterruptedException {
+    // pay the penalty for remote request, at least 5 ms
+    timeSource.sleep(5);
+
+    LOG.trace("--- got SolrRequest: " + req.getMethod() + " " + req.getPath() +
+        (req.getParams() != null ? " " + req.getParams().toQueryString() : ""));
+    if (req.getPath() != null && req.getPath().startsWith("/admin/autoscaling") ||
+        req.getPath().startsWith("/cluster/autoscaling")) {
+      incrementCount("autoscaling");
+      ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
+      params.set(CommonParams.PATH, req.getPath());
+      LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
+      RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cw.write(baos);
+      String payload = baos.toString("UTF-8");
+      LOG.trace("-- payload: {}", payload);
+      queryRequest.setContentStreams(Collections.singletonList(new ContentStreamBase.StringStream(payload)));
+      queryRequest.getContext().put("httpMethod", req.getMethod().toString());
+      SolrQueryResponse queryResponse = new SolrQueryResponse();
+      autoScalingHandler.handleRequest(queryRequest, queryResponse);
+      if (queryResponse.getException() != null) {
+        throw new IOException(queryResponse.getException());
+      }
+      SolrResponse rsp = new SolrResponseBase();
+      rsp.setResponse(queryResponse.getValues());
+      LOG.trace("-- response: {}", rsp);
+      return rsp;
+    }
+    if (req instanceof UpdateRequest) {
+      incrementCount("update");
+      // support only updates to the system collection
+      UpdateRequest ureq = (UpdateRequest)req;
+      if (ureq.getCollection() == null || !ureq.getCollection().equals(CollectionAdminParams.SYSTEM_COLL)) {
+        throw new UnsupportedOperationException("Only .system updates are supported but got: " + req);
+      }
+      List<SolrInputDocument> docs = ureq.getDocuments();
+      if (docs != null) {
+        systemColl.addAll(docs);
+      }
+      return new UpdateResponse();
+    }
+    // support only a specific subset of collection admin ops
+    if (!(req instanceof CollectionAdminRequest)) {
+      throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName());
+    }
+    SolrParams params = req.getParams();
+    String a = params.get(CoreAdminParams.ACTION);
+    SolrResponse rsp = new SolrResponseBase();
+    rsp.setResponse(new NamedList<>());
+    if (a != null) {
+      CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(a);
+      if (action == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
+      }
+      LOG.debug("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
+      NamedList results = new NamedList();
+      rsp.setResponse(results);
+      incrementCount(action.name());
+      switch (action) {
+        case REQUESTSTATUS:
+          // we complete all async ops immediately
+          String requestId = req.getParams().get(REQUESTID);
+          SimpleOrderedMap<String> status = new SimpleOrderedMap<>();
+          status.add("state", RequestStatusState.COMPLETED.getKey());
+          status.add("msg", "found [" + requestId + "] in completed tasks");
+          results.add("status", status);
+          results.add("success", "");
+          // ExecutePlanAction expects a specific response class
+          rsp = new CollectionAdminRequest.RequestStatusResponse();
+          rsp.setResponse(results);
+          break;
+        case DELETESTATUS:
+          requestId = req.getParams().get(REQUESTID);
+          results.add("status", "successfully removed stored response for [" + requestId + "]");
+          results.add("success", "");
+          break;
+        case CREATE:
+          try {
+            clusterStateProvider.simCreateCollection(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case DELETE:
+          clusterStateProvider.simDeleteCollection(req.getParams().get(CommonParams.NAME),
+              req.getParams().get(CommonAdminParams.ASYNC), results);
+          break;
+        case LIST:
+          results.add("collections", clusterStateProvider.simListCollections());
+          break;
+        case ADDREPLICA:
+          try {
+            clusterStateProvider.simAddReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case MOVEREPLICA:
+          try {
+            clusterStateProvider.simMoveReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case OVERSEERSTATUS:
+          if (req.getParams().get(CommonAdminParams.ASYNC) != null) {
+            results.add(REQUESTID, req.getParams().get(CommonAdminParams.ASYNC));
+          }
+          if (!liveNodesSet.get().isEmpty()) {
+            results.add("leader", liveNodesSet.get().iterator().next());
+          }
+          results.add("overseer_queue_size", 0);
+          results.add("overseer_work_queue_size", 0);
+          results.add("overseer_collection_queue_size", 0);
+          results.add("success", "");
+          break;
+        case ADDROLE:
+          nodeStateProvider.simAddNodeValue(req.getParams().get("node"), "nodeRole", req.getParams().get("role"));
+          break;
+        case CREATESHARD:
+          try {
+            clusterStateProvider.simCreateShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case SPLITSHARD:
+          try {
+            clusterStateProvider.simSplitShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        default:
+          throw new UnsupportedOperationException("Unsupported collection admin action=" + action + " in request: " + req.getParams());
+      }
+    } else {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "action is a required param in request: " + req.getParams());
+    }
+    return rsp;
+
+  }
+
+  /**
+   * HTTP requests are not supported by this implementation.
+   */
+  @Override
+  public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
+    throw new UnsupportedOperationException("general HTTP requests are not supported yet");
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.closeQuietly(clusterStateProvider);
+    IOUtils.closeQuietly(nodeStateProvider);
+    IOUtils.closeQuietly(stateManager);
+    triggerThread.interrupt();
+    IOUtils.closeQuietly(triggerThread);
+    IOUtils.closeQuietly(objectCache);
+    simCloudManagerPool.shutdownNow();
+  }
+}