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 2019/01/25 07:52:19 UTC

[lucene-solr] 01/02: Move the simualtion framework to core.

This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch jira/solr-13155-2
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 6337aacdf70c79aa93ad9ede7962f822e502a95e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Jan 24 15:59:09 2019 +0100

    Move the simualtion framework to core.
---
 solr/bin/solr                                      |  21 +-
 .../org/apache/solr/cloud/CloudUtils.java}         | 141 +-----------
 .../solr/cloud/autoscaling/sim/ActionError.java    |   0
 .../autoscaling/sim/GenericDistributedQueue.java   |   0
 .../sim/GenericDistributedQueueFactory.java        |   0
 .../solr/cloud/autoscaling/sim/LiveNodesSet.java   |   0
 .../autoscaling/sim}/MockSearchableSolrClient.java |   5 +-
 .../cloud/autoscaling/sim/SimCloudManager.java     |  14 +-
 .../autoscaling/sim/SimClusterStateProvider.java   |  23 +-
 .../autoscaling/sim/SimDistribStateManager.java    |   0
 .../sim/SimDistributedQueueFactory.java            |   0
 .../autoscaling/sim/SimNodeStateProvider.java      |   0
 .../java/org/apache/solr/util/RedactionUtils.java  |  28 +++
 .../src/java/org/apache/solr/util/SolrCLI.java     | 250 +++++++++++++++++++++
 .../test/org/apache/solr/cloud/CloudTestUtils.java | 129 -----------
 .../solr/cloud/MetricsHistoryIntegrationTest.java  |   4 +-
 .../cloud/autoscaling/IndexSizeTriggerTest.java    |  32 +--
 .../ScheduledMaintenanceTriggerTest.java           |  12 +-
 .../SearchRateTriggerIntegrationTest.java          |  33 +--
 .../cloud/autoscaling/SearchRateTriggerTest.java   |   8 +-
 .../autoscaling/sim/TestSimComputePlanAction.java  |  11 +-
 .../autoscaling/sim/TestSimExecutePlanAction.java  |  21 +-
 .../autoscaling/sim/TestSimExtremeIndexing.java    |   8 +-
 .../cloud/autoscaling/sim/TestSimLargeCluster.java |  33 +--
 .../autoscaling/sim/TestSimNodeLostTrigger.java    |   2 +-
 .../cloud/autoscaling/sim/TestSimPolicyCloud.java  |  48 ++--
 .../autoscaling/sim/TestSimTriggerIntegration.java |   7 +-
 .../handler/admin/MetricsHistoryHandlerTest.java   |   6 +-
 .../metrics/rrd/SolrRrdBackendFactoryTest.java     |   2 +-
 29 files changed, 427 insertions(+), 411 deletions(-)

diff --git a/solr/bin/solr b/solr/bin/solr
index e3d0d4e..3eea38e 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -312,7 +312,7 @@ function print_usage() {
   if [ -z "$CMD" ]; then
     echo ""
     echo "Usage: solr COMMAND OPTIONS"
-    echo "       where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config"
+    echo "       where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, autoscaling"
     echo ""
     echo "  Standalone server example (start Solr running in the background on port 8984):"
     echo ""
@@ -632,6 +632,17 @@ function print_usage() {
     echo ""
     echo "  -V                                     Enable more verbose output."
     echo ""
+  elif [ "$CMD" == "autoscaling" ]; then
+    echo ""
+    echo "Usage: solr autoscaling [-z zkHost] [-a <autoscaling.json.file>] [-s] [-d] [-n] [-r]"
+    echo ""
+    echo "  Calculate autoscaling policy suggestions and diagnostic information, using either the deployed"
+    echo "  autoscaling configuration or the one supplied on the command line. This calculation takes place"
+    echo "  on the client-side without affecting the running cluster except for fetching the node and replica"
+    echo "  metrics from the cluster. For detailed usage instructions, do:"
+    echo ""
+    echo "    bin/solr autoscaling -help"
+    echo ""
   fi
 } # end print_usage
 
@@ -706,7 +717,7 @@ function run_tool() {
 
   "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS $SOLR_ZK_CREDS_AND_ACLS -Dsolr.install.dir="$SOLR_TIP" \
     -Dlog4j.configurationFile="file:$DEFAULT_SERVER_DIR/resources/log4j2-console.xml" \
-    -classpath "$DEFAULT_SERVER_DIR/solr-webapp/webapp/WEB-INF/lib/*:$DEFAULT_SERVER_DIR/lib/ext/*" \
+    -classpath "$DEFAULT_SERVER_DIR/solr-webapp/webapp/WEB-INF/lib/*:$DEFAULT_SERVER_DIR/lib/*:$DEFAULT_SERVER_DIR/lib/ext/*" \
     org.apache.solr.util.SolrCLI "$@"
 
   return $?
@@ -1343,6 +1354,12 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
   exit $?
 fi
 
+
+if [[ "$SCRIPT_CMD" == "autoscaling" ]]; then
+  run_tool autoscaling $@
+  exit $?
+fi
+
 if [[ "$SCRIPT_CMD" == "auth" ]]; then
 
   VERBOSE=""
diff --git a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtils.java
similarity index 59%
copy from solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
copy to solr/core/src/java/org/apache/solr/cloud/CloudUtils.java
index 07a9219..59a1af8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtils.java
@@ -27,8 +27,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.lucene.util.LuceneTestCase;
-
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
@@ -37,7 +35,6 @@ import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.RequestWriter.StringPayloadContentWriter;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
-
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.DocCollection;
@@ -45,18 +42,16 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.util.TimeOut;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.Assert;
 
 import static org.apache.solr.common.params.CommonParams.JSON_MIME;
 
 
 /**
- * Some useful methods for SolrCloud tests.
+ * Some useful methods for SolrCloud.
  */
-public class CloudTestUtils {
+public class CloudUtils {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final int DEFAULT_TIMEOUT = 90;
@@ -189,136 +184,4 @@ public class CloudTestUtils {
       }
     };
   }
-  
-  /**
-   * Wait for a particular named trigger to be scheduled.
-   * <p>
-   * This is a convenience method that polls the autoscaling API looking for a trigger with the 
-   * specified name using the {@link #DEFAULT_TIMEOUT}.  It is particularly useful for tests 
-   * that want to know when the Overseer has finished scheduling the automatic triggers on startup.
-   * </p>
-   *
-   * @param cloudManager current instance of {@link SolrCloudManager}
-   * @param triggerName the name of the trigger we need to see sheduled in order to return successfully
-   * @see #suspendTrigger
-   */
-  public static long waitForTriggerToBeScheduled(final SolrCloudManager cloudManager,
-                                                 final String triggerName)
-    throws InterruptedException, TimeoutException, IOException {
-
-    TimeOut timeout = new TimeOut(DEFAULT_TIMEOUT, TimeUnit.SECONDS, cloudManager.getTimeSource());
-    while (!timeout.hasTimedOut()) {
-      final SolrResponse response = cloudManager.request(AutoScalingRequest.create(SolrRequest.METHOD.GET, null));
-      final Map<String,?> triggers = (Map<String,?>) response.getResponse().get("triggers");
-      Assert.assertNotNull("null triggers in response from autoscaling request", triggers);
-      
-      if ( triggers.containsKey(triggerName) ) {
-        return timeout.timeElapsed(TimeUnit.MILLISECONDS);
-      }
-      timeout.sleep(100);
-    }
-    throw new TimeoutException("Never saw trigger with name: " + triggerName);
-  }
-
-  /**
-   * Suspends the trigger with the specified name
-   * <p>
-   * This is a convenience method that sends a <code>suspend-trigger</code> command to the autoscaling
-   * API for the specified trigger.  It is particularly useful for tests that may need to disable automatic
-   * triggers such as <code>.scheduled_maintenance</code> in order to test their own
-   * triggers.
-   * </p>
-   *
-   * @param cloudManager current instance of {@link SolrCloudManager}
-   * @param triggerName the name of the trigger to suspend.  This must already be scheduled.
-   * @see #assertAutoScalingRequest
-   * @see #waitForTriggerToBeScheduled
-   */
-  public static void suspendTrigger(final SolrCloudManager cloudManager,
-                                    final String triggerName) throws IOException {
-    assertAutoScalingRequest(cloudManager, "{'suspend-trigger' : {'name' : '"+triggerName+"'} }");
-  }
-
-  /**
-   * Creates &amp; executes an autoscaling request against the current cluster, asserting that 
-   * the result is a success.
-   * 
-   * @param cloudManager current instance of {@link SolrCloudManager}
-   * @param json The request to POST to the AutoScaling Handler
-   * @see AutoScalingRequest#create
-   */
-  public static void assertAutoScalingRequest(final SolrCloudManager cloudManager,
-                                              final String json) throws IOException {
-    // TODO: a lot of code that directly uses AutoScalingRequest.create should use this method
-    
-    final SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, json);
-    final SolrResponse rsp = cloudManager.request(req);
-    final String result = rsp.getResponse().get("result").toString();
-    Assert.assertEquals("Unexpected result from auto-scaling command: " + json + " -> " + rsp,
-                        "success", result);
-  }
-
-  
-  /**
-   * Helper class for sending (JSON) autoscaling requests that can randomize between V1 and V2 requests
-   */
-  public static class AutoScalingRequest extends SolrRequest {
-
-    /**
-     * Creates a request using a randomized root path (V1 vs V2)
-     *
-     * @param m HTTP Method to use
-     * @aram message JSON payload, may be null
-     */
-    public static SolrRequest create(SolrRequest.METHOD m, String message) {
-      return create(m, null, message);
-    }
-    /**
-     * Creates a request using a randomized root path (V1 vs V2)
-     *
-     * @param m HTTP Method to use
-     * @param subPath optional sub-path under <code>"$ROOT/autoscaling"</code>. may be null, 
-     *        otherwise must start with "/"
-     * @param message JSON payload, may be null
-     */
-    public static SolrRequest create(SolrRequest.METHOD m, String subPath, String message) {
-      final boolean useV1 = LuceneTestCase.random().nextBoolean();
-      String path = useV1 ? "/admin/autoscaling" : "/cluster/autoscaling";
-      if (null != subPath) {
-        assert subPath.startsWith("/");
-        path += subPath;
-      }
-      return useV1
-        ? new AutoScalingRequest(m, path, message)
-        : new V2Request.Builder(path).withMethod(m).withPayload(message).build();
-    }
-    
-    protected final String message;
-
-    /**
-     * Simple request
-     * @param m HTTP Method to use
-     * @param path path to send request to
-     * @param message JSON payload, may be null
-     */
-    private AutoScalingRequest(METHOD m, String path, String message) {
-      super(m, path);
-      this.message = message;
-    }
-
-    @Override
-    public SolrParams getParams() {
-      return null;
-    }
-
-    @Override
-    public RequestWriter.ContentWriter getContentWriter(String expectedType) {
-      return message == null ? null : new StringPayloadContentWriter(message, JSON_MIME);
-    }
-
-    @Override
-    protected SolrResponse createResponse(SolrClient client) {
-      return new SolrResponseBase();
-    }
-  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
diff --git a/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/MockSearchableSolrClient.java
similarity index 97%
rename from solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/MockSearchableSolrClient.java
index 6fce498..2c39c79 100644
--- a/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/MockSearchableSolrClient.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.solr.util;
+package org.apache.solr.cloud.autoscaling.sim;
 
 import java.io.IOException;
 import java.util.LinkedHashMap;
@@ -32,7 +32,6 @@ import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.junit.Assert;
 
 /**
  * Simple mock client that collects added documents and supports simple search by id
@@ -62,7 +61,7 @@ public class MockSearchableSolrClient extends SolrClient {
       if (docList != null) {
         docList.forEach(doc -> {
           String id = (String) doc.getFieldValue("id");
-          Assert.assertNotNull(doc.toString(), id);
+          assert id != null : doc.toString();
           docs.computeIfAbsent(collection, c -> new LinkedHashMap<>()).put(id, doc);
         });
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
similarity index 98%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index ead23fc..fffbd37 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -38,7 +38,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
 import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
 import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
@@ -98,7 +97,6 @@ import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.util.MockSearchableSolrClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -406,13 +404,6 @@ public class SimCloudManager implements SolrCloudManager {
   }
 
   /**
-   * Get the source of randomness (usually initialized by the test suite).
-   */
-  public Random getRandom() {
-    return RandomizedContext.current().getRandom();
-  }
-
-  /**
    * 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
@@ -693,6 +684,7 @@ public class SimCloudManager implements SolrCloudManager {
         ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
         params.set(CommonParams.PATH, req.getPath());
         LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
+        queryRequest.getContext().put("path", req.getPath());
         if (autoscaling) {
           RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
           if (null != cw) {
@@ -762,7 +754,7 @@ public class SimCloudManager implements SolrCloudManager {
       }
     }
     // support only a specific subset of collection admin ops
-    if (!(req instanceof CollectionAdminRequest)) {
+    if (!(req instanceof CollectionAdminRequest) && !req.getPath().startsWith("/admin")) {
       throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName());
     }
     metricManager.registry("solr.node").counter("ADMIN." + req.getPath() + ".requests").inc();
@@ -897,7 +889,7 @@ public class SimCloudManager implements SolrCloudManager {
     try {
       triggerThread.join();
     } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+      // ignore
     }
     IOUtils.closeQuietly(objectCache);
     simCloudManagerPool.shutdownNow();
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
similarity index 99%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 784c0cb..29bd7cb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -61,7 +61,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.ActionThrottle;
-import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.api.collections.AddReplicaCmd;
 import org.apache.solr.cloud.api.collections.Assign;
@@ -94,7 +94,6 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.zookeeper.CreateMode;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -242,14 +241,6 @@ public class SimClusterStateProvider implements ClusterStateProvider {
 
   /**
    * Get random node id.
-   * @return one of the live nodes
-   */
-  public String simGetRandomNode() {
-    return simGetRandomNode(cloudManager.getRandom());
-  }
-
-  /**
-   * Get random node id.
    * @param random instance of random.
    * @return one of the live nodes
    */
@@ -636,7 +627,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     try {
       VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
       int version = oldData != null ? oldData.getVersion() : 0;
-      Assert.assertEquals(clusterStateVersion, version);
+      assert clusterStateVersion == version;
       stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
       log.debug("** saved cluster state version " + (version));
       clusterStateVersion++;
@@ -893,7 +884,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     int numNrtReplicas = props.getInt(NRT_REPLICAS, props.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
     int numPullReplicas = props.getInt(PULL_REPLICAS, 0);
     int totalReplicas = shardNames.size() * (numNrtReplicas + numPullReplicas + numTlogReplicas);
-    Assert.assertEquals("unexpected number of replica positions", totalReplicas, replicaPositions.size());
+    assert totalReplicas == replicaPositions.size() : "unexpected number of replica positions" ;
     final CountDownLatch finalStateLatch = new CountDownLatch(replicaPositions.size());
     AtomicInteger replicaNum = new AtomicInteger(1);
     replicaPositions.forEach(pos -> {
@@ -1046,7 +1037,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   }
 
   /**
-   * Move replica. This uses a similar algorithm as {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd#moveNormalReplica(ClusterState, NamedList, String, String, DocCollection, Replica, Slice, int, boolean)}.
+   * Move replica. This uses a similar algorithm as <code>MoveReplicaCmd.moveNormalReplica(...)</code>
    * @param message operation details
    * @param results operation results.
    */
@@ -1276,7 +1267,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
 
     boolean success = false;
     try {
-      CloudTestUtils.waitForState(cloudManager, collectionName, 30, TimeUnit.SECONDS, (liveNodes, state) -> {
+      CloudUtils.waitForState(cloudManager, collectionName, 30, TimeUnit.SECONDS, (liveNodes, state) -> {
         for (String subSlice : subSlices) {
           Slice s = state.getSlice(subSlice);
           if (s.getLeader() == null) {
@@ -1405,8 +1396,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           OverseerCollectionMessageHandler.NUM_SLICES, "1",
           CommonAdminParams.WAIT_FOR_FINAL_STATE, "true");
       simCreateCollection(props, new NamedList());
-      CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL, 120, TimeUnit.SECONDS,
-          CloudTestUtils.clusterShape(1, Integer.parseInt(repFactor), false, true));
+      CloudUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL, 120, TimeUnit.SECONDS,
+          CloudUtils.clusterShape(1, Integer.parseInt(repFactor), false, true));
     } catch (Exception e) {
       throw new IOException(e);
     }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
diff --git a/solr/core/src/java/org/apache/solr/util/RedactionUtils.java b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
index afa2abf..9fefadd 100644
--- a/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
@@ -17,6 +17,10 @@
 
 package org.apache.solr.util;
 
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.regex.Pattern;
 
 public class RedactionUtils {
@@ -47,5 +51,29 @@ public class RedactionUtils {
     RedactionUtils.redactSystemProperty = redactSystemProperty;
   }
 
+  /**
+   * Replace actual names found in a string with meaningless randomized names.
+   * @param names actual names
+   * @param data string to redact
+   * @return redacted string where all actual names have been replaced.
+   */
+  public static String redactNames(Collection<String> names, String redactionPrefix, String data) {
+    Set<String> uniqueNames = new TreeSet<>(names);
+    Set<Integer> uniqueCode = new HashSet<>();
+    // minimal(ish) hash
+    int codeShift = 0;
+    int codeSpace = names.size();
+    for (String name : uniqueNames) {
+      int code = Math.abs(name.hashCode() % codeSpace);
+      while (uniqueCode.contains(code)) {
+        codeShift++;
+        codeSpace = names.size() << codeShift;
+        code = Math.abs(name.hashCode() % codeSpace);
+      }
+      uniqueCode.add(code);
+      data = data.replaceAll("\\Q" + name + "\\E", redactionPrefix + Integer.toString(code, Character.MAX_RADIX));
+    }
+    return data;
+  }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index 03aa5f8..259b0d0 100755
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -19,12 +19,14 @@ package org.apache.solr.util;
 import javax.net.ssl.SSLPeerUnverifiedException;
 import java.io.Console;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
+import java.net.MalformedURLException;
 import java.net.Socket;
 import java.net.SocketException;
 import java.net.URL;
@@ -43,6 +45,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
@@ -72,6 +75,7 @@ import org.apache.commons.exec.Executor;
 import org.apache.commons.exec.OS;
 import org.apache.commons.exec.environment.EnvironmentUtils;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.BooleanUtils;
 import org.apache.commons.lang.SystemUtils;
 import org.apache.http.HttpEntity;
@@ -92,16 +96,28 @@ import org.apache.lucene.util.Version;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 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.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
+import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
 import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -114,10 +130,15 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStreamBase;
 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.request.SolrQueryRequest;
 import org.apache.solr.security.Sha256AuthenticationProvider;
+import org.apache.solr.servlet.SolrRequestParsers;
 import org.apache.solr.util.configuration.SSLConfigurationsFactory;
 import org.noggit.CharArr;
 import org.noggit.JSONParser;
@@ -392,6 +413,8 @@ public class SolrCLI {
       return new UtilsTool();
     else if ("auth".equals(toolType))
       return new AuthTool();
+    else if ("autoscaling".equals(toolType))
+      return new AutoscalingTool();
 
     // If you add a built-in tool to this class, add it here to avoid
     // classpath scanning
@@ -410,6 +433,7 @@ public class SolrCLI {
     formatter.printHelp("healthcheck", getToolOptions(new HealthcheckTool()));
     formatter.printHelp("status", getToolOptions(new StatusTool()));
     formatter.printHelp("api", getToolOptions(new ApiTool()));
+    formatter.printHelp("autoscaling", getToolOptions(new AutoscalingTool()));
     formatter.printHelp("create_collection", getToolOptions(new CreateCollectionTool()));
     formatter.printHelp("create_core", getToolOptions(new CreateCoreTool()));
     formatter.printHelp("create", getToolOptions(new CreateTool()));
@@ -832,6 +856,232 @@ public class SolrCLI {
   }
   
 
+  public static class AutoscalingTool extends ToolBase {
+    static final String NODE_REDACTION_PREFIX = "N_";
+    static final String COLL_REDACTION_PREFIX = "COLL_";
+
+    public AutoscalingTool() {
+      this(System.out);
+    }
+
+    public AutoscalingTool(PrintStream stdout) {
+      super(stdout);
+    }
+
+    @Override
+    public Option[] getOptions() {
+      return new Option[] {
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Address of the Zookeeper ensemble")
+              .create("zkHost"),
+          OptionBuilder
+              .withArgName("CONFIG")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Autoscaling config file, defaults to the one deployed in the cluster.")
+              .withLongOpt("config")
+              .create("a"),
+          OptionBuilder
+              .withDescription("Show calculated suggestions")
+              .withLongOpt("suggestions")
+              .create("s"),
+          OptionBuilder
+              .withArgName("NUM")
+              .withDescription("Use a simulated cluster with NUM nodes")
+              .hasArg()
+              .isRequired(false)
+              .create("simulate"),
+          OptionBuilder
+              .withArgName("FILE")
+              .withDescription("A FILE with a list of requests to execute (only '/path?params' part), one per line.")
+              .hasArg()
+              .isRequired(false)
+              .create("script"),
+          OptionBuilder
+              .withDescription("Show calculated diagnostics")
+              .withLongOpt("diagnostics")
+              .create("d"),
+          OptionBuilder
+              .withDescription("Show sorted nodes with diagnostics")
+              .withLongOpt("sortedNodes")
+              .create("n"),
+          OptionBuilder
+              .withDescription("Redact node and collection names (original names will be consistently randomized)")
+              .withLongOpt("redact")
+              .create("r")
+      };
+    }
+
+    @Override
+    public String getName() {
+      return "autoscaling";
+    }
+
+    @Override
+    protected void runImpl(CommandLine cli) throws Exception {
+      raiseLogLevelUnlessVerbose(cli);
+      DistributedQueueFactory dummmyFactory = new DistributedQueueFactory() {
+        @Override
+        public DistributedQueue makeQueue(String path) throws IOException {
+          throw new UnsupportedOperationException("makeQueue");
+        }
+
+        @Override
+        public void removeQueue(String path) throws IOException {
+          throw new UnsupportedOperationException("removeQueue");
+        }
+      };
+      SolrCloudManager cloudManager;
+      SolrClient solrClient;
+      if (cli.getOptionValue("simulate") == null) {
+        String zkHost = cli.getOptionValue("zkHost", ZK_HOST);
+
+        log.debug("Connecting to Solr cluster: " + zkHost);
+        CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zkHost), Optional.empty()).build();
+        String collection = cli.getOptionValue("collection");
+        if (collection != null)
+          cloudSolrClient.setDefaultCollection(collection);
+
+        cloudSolrClient.connect();
+        cloudManager = new SolrClientCloudManager(dummmyFactory, cloudSolrClient);
+        solrClient = cloudSolrClient;
+      } else {
+        String numNodes = cli.getOptionValue("simulate");
+        int n = Integer.parseInt(numNodes);
+        cloudManager = SimCloudManager.createCluster(n, TimeSource.get("simTime:10"));
+        solrClient = ((SimCloudManager)cloudManager).simGetSolrClient();
+      }
+      // if there's a script execute it now
+      String script = cli.getOptionValue("script");
+      if (script != null) {
+        stdout.println("- reading script from " + script);
+        List<String> lines = IOUtils.readLines(new FileInputStream(script), "UTF-8");
+        stdout.println("- executing " + lines.size() + " commands...");
+        for (int i = 0; i < lines.size(); i++) {
+          String line = lines.get(i);
+          if (line.charAt(0) == '#' || line.trim().isEmpty()) { // comment or blank
+            continue;
+          }
+          String[] parts = line.trim().split("!");
+          SolrRequest req;
+          SolrParams params;
+          String path;
+          if (parts.length > 0) {
+            String[] pathQuery = parts[0].split("\\?");
+            if (pathQuery.length == 0 || pathQuery.length > 2) {
+              throw new IllegalArgumentException("Invalid request string: " + parts[0]);
+            }
+            path = pathQuery[0];
+            if (pathQuery.length > 1) {
+              params = SolrRequestParsers.parseQueryString(pathQuery[1]);
+            } else {
+              params = new ModifiableSolrParams();
+            }
+          } else {
+            throw new IllegalArgumentException("Empty script command");
+          }
+          if (parts.length == 1) {
+            // GET request
+            req = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
+          } else if (parts.length == 2) {
+            // POST request with JSON payload
+            req = new GenericSolrRequest(SolrRequest.METHOD.POST, path, params);
+            ((GenericSolrRequest) req).setContentWriter(
+                new RequestWriter.StringPayloadContentWriter(parts[1], "application/json"));
+          } else {
+            throw new IllegalArgumentException("Script commands should have 1 part (GET) or 2 parts (POST):\n'" + line + "'");
+          }
+          SolrResponse rsp = cloudManager.request(req);
+          stdout.println((i + 1) + ".\tREQ: " + req.getPath() + "?" + req.getParams());
+          stdout.println("   \tRSP: " + Utils.toJSONString(rsp.getResponse()));
+        }
+      }
+      try {
+        AutoScalingConfig config = null;
+        HashSet<String> liveNodes = new HashSet<>();
+        String configFile = cli.getOptionValue("a");
+        if (configFile != null) {
+          stdout.println("- reading autoscaling config from " + configFile);
+          config = new AutoScalingConfig(IOUtils.toByteArray(new FileInputStream(configFile)));
+        } else {
+          stdout.println("- reading autoscaling config from the cluster.");
+          config = cloudManager.getDistribStateManager().getAutoScalingConfig();
+        }
+        stdout.println("- calculating suggestions...");
+        long start = TimeSource.NANO_TIME.getTimeNs();
+        // collect live node names for optional redaction
+        liveNodes.addAll(cloudManager.getClusterStateProvider().getLiveNodes());
+        List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(config, cloudManager);
+        long end = TimeSource.NANO_TIME.getTimeNs();
+        stdout.println("  (took " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms)");
+        stdout.println("- calculating diagnostics...");
+        start = TimeSource.NANO_TIME.getTimeNs();
+        // update the live nodes
+        liveNodes.addAll(cloudManager.getClusterStateProvider().getLiveNodes());
+        MapWriter mw = PolicyHelper.getDiagnostics(config.getPolicy(), cloudManager);
+        Map<String, Object> diagnostics = new LinkedHashMap<>();
+        mw.toMap(diagnostics);
+        end = TimeSource.NANO_TIME.getTimeNs();
+        stdout.println("  (took " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms)");
+        boolean withSuggestions = cli.hasOption("s");
+        boolean withDiagnostics = cli.hasOption("d") || cli.hasOption("n");
+        boolean withSortedNodes = cli.hasOption("n");
+        // prepare to redact also host names / IPs in base_url and other properties
+        Set<String> redactNames = new HashSet<>();
+        for (String nodeName : liveNodes) {
+          String urlString = Utils.getBaseUrlForNodeName(nodeName, "http");
+          try {
+            URL u = new URL(urlString);
+            // protocol format
+            redactNames.add(u.getHost() + ":" + u.getPort());
+            // node name format
+            redactNames.add(u.getHost() + "_" + u.getPort() + "_");
+          } catch (MalformedURLException e) {
+            log.warn("Invalid URL for node name " + nodeName + ", replacing including protocol and path", e);
+            redactNames.add(urlString);
+            redactNames.add(Utils.getBaseUrlForNodeName(nodeName, "https"));
+          }
+        }
+        // redact collection names too
+        Set<String> redactCollections = new HashSet<>();
+        ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
+        clusterState.forEachCollection(coll -> redactCollections.add(coll.getName()));
+        if (!withSuggestions && !withDiagnostics) {
+          withSuggestions = true;
+        }
+        if (withSuggestions) {
+          stdout.println("\n============= SUGGESTIONS ===========");
+          String suggestionsString = Utils.toJSONString(suggestions);
+          if (cli.hasOption("r")) {
+            // replace collections first with a different prefix
+            suggestionsString = RedactionUtils.redactNames(redactCollections, COLL_REDACTION_PREFIX, suggestionsString);
+            // then replace node names
+            suggestionsString = RedactionUtils.redactNames(redactNames, NODE_REDACTION_PREFIX, suggestionsString);
+          }
+          stdout.println(suggestionsString);
+        }
+        if (!withSortedNodes) {
+          diagnostics.remove("sortedNodes");
+        }
+        if (withDiagnostics) {
+          stdout.println("\n============= DIAGNOSTICS ===========");
+          String diagnosticsString = Utils.toJSONString(diagnostics);
+          if (cli.hasOption("r")) {
+            diagnosticsString = RedactionUtils.redactNames(redactCollections, COLL_REDACTION_PREFIX, diagnosticsString);
+            diagnosticsString = RedactionUtils.redactNames(redactNames, NODE_REDACTION_PREFIX, diagnosticsString);
+          }
+          stdout.println(diagnosticsString);
+        }
+      } finally {
+        solrClient.close();
+        cloudManager.close();
+      }
+    }
+  }
+
   /**
    * Get the status of a Solr server.
    */
diff --git a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
index 07a9219..404d074 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
@@ -62,135 +62,6 @@ public class CloudTestUtils {
   public static final int DEFAULT_TIMEOUT = 90;
 
   /**
-   * Wait for a particular collection state to appear.
-   *
-   * This is a convenience method using the {@link #DEFAULT_TIMEOUT}
-   *
-   * @param cloudManager current instance of {@link SolrCloudManager}
-   * @param message     a message to report on failure
-   * @param collection  the collection to watch
-   * @param predicate   a predicate to match against the collection state
-   */
-  public static long waitForState(final SolrCloudManager cloudManager,
-                                  final String message,
-                                  final String collection,
-                                  final CollectionStatePredicate predicate) {
-    AtomicReference<DocCollection> state = new AtomicReference<>();
-    AtomicReference<Set<String>> liveNodesLastSeen = new AtomicReference<>();
-    try {
-      return waitForState(cloudManager, 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);
-    }
-  }
-
-  /**
-   * Wait for a particular collection state to appear.
-   *
-   * This is a convenience method using the {@link #DEFAULT_TIMEOUT}
-   *
-   * @param cloudManager current instance of {@link SolrCloudManager}
-   * @param collection  the collection to watch
-   * @param wait timeout value
-   * @param unit timeout unit
-   * @param predicate   a predicate to match against the collection state
-   */
-  public static long waitForState(final SolrCloudManager cloudManager,
-                                  final String collection,
-                                  long wait,
-                                  final TimeUnit unit,
-                                  final CollectionStatePredicate predicate) throws InterruptedException, TimeoutException, IOException {
-    TimeOut timeout = new TimeOut(wait, unit, cloudManager.getTimeSource());
-    long timeWarn = timeout.timeLeft(TimeUnit.MILLISECONDS) / 4;
-    ClusterState state = null;
-    DocCollection coll = null;
-    while (!timeout.hasTimedOut()) {
-      state = cloudManager.getClusterStateProvider().getClusterState();
-      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(100);
-        continue;
-      }
-      if (predicate.matches(state.getLiveNodes(), coll)) {
-        log.trace("-- predicate matched with state {}", state);
-        return timeout.timeElapsed(TimeUnit.MILLISECONDS);
-      }
-      timeout.sleep(100);
-      if (timeout.timeLeft(TimeUnit.MILLISECONDS) < timeWarn) {
-        log.trace("-- still not matching predicate: {}", state);
-      }
-    }
-    throw new TimeoutException("last ClusterState: " + state + ", last coll state: " + coll);
-  }
-
-  /**
-   * Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
-   * number of active shards and replicas
-   * @param expectedShards expected number of active shards
-   * @param expectedReplicas expected number of active replicas
-   */
-  public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) {
-    return clusterShape(expectedShards, expectedReplicas, false, false);
-  }
-
-  /**
-   * Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
-   * number of shards and replicas.
-   * <p>Note: for shards marked as inactive the current Solr behavior is that replicas remain active.
-   * {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager} follows this behavior.</p>
-   * @param expectedShards expected number of shards
-   * @param expectedReplicas expected number of active replicas
-   * @param withInactive if true then count also inactive shards
-   * @param requireLeaders if true then require that each shard has a leader
-   */
-  public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas, boolean withInactive,
-                                                      boolean requireLeaders) {
-    return (liveNodes, collectionState) -> {
-      if (collectionState == null) {
-        log.info("-- null collection");
-        return false;
-      }
-      Collection<Slice> slices = withInactive ? collectionState.getSlices() : collectionState.getActiveSlices();
-      if (slices.size() != expectedShards) {
-        log.info("-- wrong number of slices for collection {}, expected={}, found={}: {}", collectionState.getName(), expectedShards, collectionState.getSlices().size(), collectionState.getSlices());
-        return false;
-      }
-      Set<String> leaderless = new HashSet<>();
-      for (Slice slice : slices) {
-        int activeReplicas = 0;
-        if (requireLeaders && slice.getState() != Slice.State.INACTIVE && slice.getLeader() == null) {
-          leaderless.add(slice.getName());
-          continue;
-        }
-        // skip other checks, we're going to fail anyway
-        if (!leaderless.isEmpty()) {
-          continue;
-        }
-        for (Replica replica : slice) {
-          if (replica.isActive(liveNodes))
-            activeReplicas++;
-        }
-        if (activeReplicas != expectedReplicas) {
-          log.info("-- wrong number of active replicas for collection {} in slice {}, expected={}, found={}", collectionState.getName(), slice.getName(), expectedReplicas, activeReplicas);
-          return false;
-        }
-      }
-      if (leaderless.isEmpty()) {
-        return true;
-      } else {
-        log.info("-- shards without leaders: {}", leaderless);
-        return false;
-      }
-    };
-  }
-  
-  /**
    * Wait for a particular named trigger to be scheduled.
    * <p>
    * This is a convenience method that polls the autoscaling API looking for a trigger with the 
diff --git a/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
index d30fe29..dd2a7dd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
@@ -75,8 +75,8 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase {
     // create .system
     CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 1)
         .process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL,
-        30, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL,
+        30, TimeUnit.SECONDS, CloudUtils.clusterShape(1, 1));
     solrClient.query(CollectionAdminParams.SYSTEM_COLL, params(CommonParams.Q, "*:*"));
     // sleep a little to allow the handler to collect some metrics
     if (simulated) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java
index d8975ab..e742996 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java
@@ -40,8 +40,8 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
 import org.apache.solr.common.SolrInputDocument;
@@ -152,8 +152,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     if (SPEED == 1) {
       cluster.waitForActiveCollection(collectionName, 2, 4);
     } else {
-      CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-          CloudTestUtils.clusterShape(2, 2, false, true));
+      CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+          CloudUtils.clusterShape(2, 2, false, true));
     }
 
     long waitForSeconds = 3 + random().nextInt(5);
@@ -258,8 +258,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     if (SPEED == 1) {
       cluster.waitForActiveCollection(collectionName, 2, 4);
     } else {
-      CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-          CloudTestUtils.clusterShape(2, 2, false, true));
+      CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+          CloudUtils.clusterShape(2, 2, false, true));
     }
 
     long waitForSeconds = 6 + random().nextInt(5);
@@ -328,7 +328,7 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
 
     boolean await = finished.await(60000, TimeUnit.MILLISECONDS);
     assertTrue("did not finish processing in time", await);
-    CloudTestUtils.waitForState(cloudManager, collectionName, 20, TimeUnit.SECONDS, CloudTestUtils.clusterShape(6, 2, true, true));
+    CloudUtils.waitForState(cloudManager, collectionName, 20, TimeUnit.SECONDS, CloudUtils.clusterShape(6, 2, true, true));
     assertEquals(1, listenerEvents.size());
     List<CapturedEvent> events = listenerEvents.get("capturing2");
     assertNotNull("'capturing2' events not found", events);
@@ -382,8 +382,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     if (SPEED == 1) {
       cluster.waitForActiveCollection(collectionName, 2, 4);
     } else {
-      CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-          CloudTestUtils.clusterShape(2, 2, false, true));
+      CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+          CloudUtils.clusterShape(2, 2, false, true));
     }
 
     for (int i = 0; i < 20; i++) {
@@ -495,8 +495,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
         "conf", 2, 2).setMaxShardsPerNode(2);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-        CloudTestUtils.clusterShape(2, 2, false, true));
+    CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+        CloudUtils.clusterShape(2, 2, false, true));
 
     for (int j = 0; j < 10; j++) {
       UpdateRequest ureq = new UpdateRequest();
@@ -621,8 +621,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     assertEquals(TriggerEventProcessorStage.SUCCEEDED, events.get(5).stage);
 
     // collection should have 2 inactive and 4 active shards
-    CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-        CloudTestUtils.clusterShape(6, 2, true, true));
+    CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+        CloudUtils.clusterShape(6, 2, true, true));
 
     // check ops
     List<TriggerEvent.Op> ops = (List<TriggerEvent.Op>) events.get(4).event.getProperty(TriggerEvent.REQUESTED_OPS);
@@ -759,8 +759,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     if (SPEED == 1) {
       cluster.waitForActiveCollection(collectionName, 5, 10);
     } else {
-      CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-          CloudTestUtils.clusterShape(5, 2, false, true));
+      CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+          CloudUtils.clusterShape(5, 2, false, true));
     }
     
     long waitForSeconds = 3 + random().nextInt(5);
@@ -901,8 +901,8 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
         "conf", 2, 2).setMaxShardsPerNode(2);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
-        CloudTestUtils.clusterShape(2, 2, false, true));
+    CloudUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName,
+        CloudUtils.clusterShape(2, 2, false, true));
 
     long waitForSeconds = 3 + random().nextInt(5);
     Map<String, Object> props = createTriggerProps(waitForSeconds);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java
index 87b32d7..a7eaaff 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java
@@ -34,8 +34,8 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
 import org.apache.solr.common.cloud.ClusterState;
@@ -206,8 +206,8 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
         "conf", 1, 1);
 
     create1.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, "failed to create " + collection1, collection1,
-        CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cloudManager, "failed to create " + collection1, collection1,
+        CloudUtils.clusterShape(1, 1));
 
     // also create a very stale lock
     Map<String, Object> lockData = new HashMap<>();
@@ -257,8 +257,8 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
     CollectionAdminRequest.SplitShard split1 = CollectionAdminRequest.splitShard(collection1)
         .setShardName("shard1");
     split1.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, "failed to split " + collection1, collection1,
-        CloudTestUtils.clusterShape(3, 1, true, true));
+    CloudUtils.waitForState(cloudManager, "failed to split " + collection1, collection1,
+        CloudUtils.clusterShape(3, 1, true, true));
 
 
     await = triggerFired.await(90, TimeUnit.SECONDS);
@@ -309,7 +309,7 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
 
     ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
 
-    CloudTestUtils.clusterShape(2, 1).matches(state.getLiveNodes(), state.getCollection(collection1));
+    CloudUtils.clusterShape(2, 1).matches(state.getLiveNodes(), state.getCollection(collection1));
   }
 
   public static CountDownLatch getTriggerFired() {
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
index a53a389..ebb1432 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
@@ -36,6 +36,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.Replica;
@@ -124,8 +125,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
         "conf", 1, 2);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2));
 
     CloudTestUtils.assertAutoScalingRequest
       (cloudManager, 
@@ -276,8 +277,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
         "conf", 1, 2);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2));
 
     // add a couple of spare replicas above RF. Use different types.
     // these additional replicas will be placed on other nodes in the cluster
@@ -285,8 +286,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     solrClient.request(CollectionAdminRequest.addReplicaToShard(COLL1, "shard1", Replica.Type.TLOG));
     solrClient.request(CollectionAdminRequest.addReplicaToShard(COLL1, "shard1", Replica.Type.PULL));
 
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 5));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 5));
 
     CloudTestUtils.assertAutoScalingRequest
       (cloudManager, 
@@ -393,8 +394,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals("cold replicas", 3, coldReplicas.get());
 
     // now the collection should be down to RF = 2
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2));
 
     listenerEvents.clear();
     listenerEventLatch = new CountDownLatch(3);
@@ -494,8 +495,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals("coldNodes: " +ops.toString(), 2, coldNodes2.get());
 
     // now the collection should be at RF == 1, with one additional PULL replica
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 1));
   }
 
   @Test
@@ -507,8 +508,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
         "conf", 1, 2);
 
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2));
 
     // add a couple of spare replicas above RF. Use different types to verify that only
     // searchable replicas are considered
@@ -517,8 +518,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     solrClient.request(CollectionAdminRequest.addReplicaToShard(COLL1, "shard1", Replica.Type.TLOG));
     solrClient.request(CollectionAdminRequest.addReplicaToShard(COLL1, "shard1", Replica.Type.PULL));
 
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 5));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 5));
 
     CloudTestUtils.assertAutoScalingRequest
       (cloudManager, 
@@ -687,8 +688,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(responses.toString(), 4, nodes.get());
 
     // we are left with one searchable replica
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 1));
   }
 
   public static class CapturingTriggerListener extends TriggerListenerBase {
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
index f750a5e..3e544d9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
@@ -39,7 +39,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
 import org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.cloud.ZkDistributedQueueFactory;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -106,8 +106,8 @@ public class SearchRateTriggerTest extends SolrCloudTestCase {
     create.setMaxShardsPerNode(1);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS, clusterShape(2, 2));
-    CloudTestUtils.waitForState(cloudManager, COLL2, 60, TimeUnit.SECONDS, clusterShape(2, 2));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS, clusterShape(2, 2));
+    CloudUtils.waitForState(cloudManager, COLL2, 60, TimeUnit.SECONDS, clusterShape(2, 2));
 
     double rate = 1.0;
     URL baseUrl = targetNode.getBaseUrl();
@@ -238,7 +238,7 @@ public class SearchRateTriggerTest extends SolrCloudTestCase {
         "conf", 2, 2);
     create.setMaxShardsPerNode(1);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS, clusterShape(2, 4));
+    CloudUtils.waitForState(cloudManager, COLL1, 60, TimeUnit.SECONDS, clusterShape(2, 4));
 
     long waitForSeconds = 5 + random().nextInt(5);
     Map<String, Object> props = createTriggerProps(Arrays.asList(COLL1, COLL2), waitForSeconds, 1.0, 0.1);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
index 19f9b8d..3304c04 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
@@ -35,6 +35,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.autoscaling.ActionContext;
 import org.apache.solr.cloud.autoscaling.ComputePlanAction;
 import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
@@ -145,8 +146,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
         "conf",1, 2);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
-        "testNodeLost", CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
+        "testNodeLost", CloudUtils.clusterShape(1, 2, false, true));
 
     ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
     log.debug("-- cluster state: {}", clusterState);
@@ -211,8 +212,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     create.setMaxShardsPerNode(2);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
-        "testNodeWithMultipleReplicasLost", CloudTestUtils.clusterShape(2, 3, false, true));
+    CloudUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
+        "testNodeWithMultipleReplicasLost", CloudUtils.clusterShape(2, 3, false, true));
 
     ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
     log.debug("-- cluster state: {}", clusterState);
@@ -296,7 +297,7 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
         "conf",1, 4).setMaxShardsPerNode(-1);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
+    CloudUtils.waitForState(cluster, "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
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
index a0c18a9..fa5974c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
@@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.autoscaling.ActionContext;
 import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
 import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
@@ -90,10 +91,10 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
     create.setMaxShardsPerNode(1);
     create.process(solrClient);
 
-    log.info("Collection ready after " + CloudTestUtils.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2, false, true)) + "ms");
+    log.info("Collection ready after " + CloudUtils.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2, false, true)) + "ms");
 
-    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode();
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
     ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
     DocCollection docCollection = clusterState.getCollection(collectionName);
     List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
@@ -152,8 +153,8 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
       assertNotNull(response.get("success"));
     }
 
-    log.info("Collection ready after " + CloudTestUtils.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 2, false, true)) + "ms");
+    log.info("Collection ready after " + CloudUtils.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 2, false, true)) + "ms");
   }
 
   @Test
@@ -181,10 +182,10 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
     create.setMaxShardsPerNode(1);
     create.process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
-        collectionName, CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
+        collectionName, CloudUtils.clusterShape(1, 2, false, true));
 
-    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode();
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
     ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
     DocCollection docCollection = clusterState.getCollection(collectionName);
     List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
@@ -200,8 +201,8 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
 
     cluster.getTimeSource().sleep(3000);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of collection to be 2 again",
-        collectionName, CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, "Timed out waiting for replicas of collection to be 2 again",
+        collectionName, CloudUtils.clusterShape(1, 2, false, true));
 
     clusterState = cluster.getClusterStateProvider().getClusterState();
     docCollection = clusterState.getCollection(collectionName);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
index 56cfdf6..9f5b0a8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
@@ -26,8 +26,8 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrInputDocument;
@@ -99,9 +99,9 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
         "conf", 2, 2).setMaxShardsPerNode(10);
     create.process(solrClient);
-    
-    CloudTestUtils.waitForState(cluster, collectionName, 90, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(2, 2, false, true));
+
+    CloudUtils.waitForState(cluster, collectionName, 90, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(2, 2, false, true));
 
     //long waitForSeconds = 3 + random().nextInt(5);
     long waitForSeconds = 1;
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
index fed7b14..2f55f14 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
@@ -41,6 +41,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
 import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.autoscaling.ActionContext;
 import org.apache.solr.cloud.autoscaling.CapturedEvent;
 import org.apache.solr.cloud.autoscaling.ComputePlanAction;
@@ -197,8 +198,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     create.setCreateNodeSet(String.join(",", nodes));
     create.process(solrClient);
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(5, 15, false, true)) + "ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
+        CloudUtils.clusterShape(5, 15, false, true)) + "ms");
 
     int KILL_NODES = 8;
     // kill off a number of nodes
@@ -206,8 +207,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
       cluster.simRemoveNode(nodes.get(i), false);
     }
     // should fully recover
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 90 * KILL_NODES, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(5, 15, false, true)) + "ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 90 * KILL_NODES, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(5, 15, false, true)) + "ms");
 
     log.info("OP COUNTS: " + cluster.simGetOpCounts());
     long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
@@ -241,8 +242,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     }
 
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(5, 15, false, true)) + "ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
+        CloudUtils.clusterShape(5, 15, false, true)) + "ms");
     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
@@ -281,8 +282,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     create.setAutoAddReplicas(false);
     create.process(solrClient);
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 90 * NUM_NODES, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 90 * NUM_NODES, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
 
     // start adding nodes
     int numAddNode = NUM_NODES / 5;
@@ -326,8 +327,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     // make sure some replicas have been moved
     assertTrue("no MOVEREPLICA ops?", cluster.simGetOpCount("MOVEREPLICA") > 0);
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
 
     int count = 1000;
     SolrInputDocument finishedEvent = null;
@@ -486,8 +487,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     create.setAutoAddReplicas(false);
     create.process(solrClient);
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 60 * NUM_NODES, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 60 * NUM_NODES, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
 
     // start killing nodes
     int numNodes = NUM_NODES / 5;
@@ -571,8 +572,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
 
     if (listenerEvents.isEmpty()) {
       // no failed movements - verify collection shape
-      log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
-          CloudTestUtils.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
+      log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
+          CloudUtils.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
     } else {
       cluster.getTimeSource().sleep(NUM_NODES * 100);
     }
@@ -630,8 +631,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
         "conf", 2, 10);
     create.process(solrClient);
 
-    log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(2, 10, false, true)) + " ms");
+    log.info("Ready after " + CloudUtils.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(2, 10, false, true)) + " ms");
 
     // collect the node names for shard1
     Set<String> nodes = new HashSet<>();
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
index 8eb6156..9060a4c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
@@ -132,7 +132,7 @@ public class TestSimNodeLostTrigger extends SimSolrCloudTestCase {
       trigger.setProcessor(noFirstRunProcessor);
       trigger.run();
 
-      String lostNode = cluster.getSimClusterStateProvider().simGetRandomNode();
+      String lostNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
       cluster.simRemoveNode(lostNode, false);
       AtomicBoolean fired = new AtomicBoolean(false);
       trigger.setProcessor(event -> {
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
index 9b68d36..0ddfe1e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
@@ -36,8 +36,8 @@ 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.Variable.Type;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -75,8 +75,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CollectionAdminRequest.createCollection("perReplicaDataColl", "conf", 1, 5)
         .process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "perReplicaDataColl",
-        CloudTestUtils.clusterShape(1, 5, false, true));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "perReplicaDataColl",
+        CloudUtils.clusterShape(1, 5, false, true));
     DocCollection coll = getCollectionState("perReplicaDataColl");
     String autoScaleJson = "{" +
         "  'cluster-preferences': [" +
@@ -115,7 +115,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
   public void testCreateCollectionAddReplica() throws Exception  {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode();
+    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
 
     int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
 
@@ -126,13 +126,13 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
         .setPolicy("c1")
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
-        CloudTestUtils.clusterShape(1, 1, false, true));
+    CloudUtils.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
+        CloudUtils.clusterShape(1, 1, false, true));
 
     getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
 
     CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(solrClient);
-    CloudTestUtils.waitForState(cluster,
+    CloudUtils.waitForState(cluster,
         collectionName, 120l, TimeUnit.SECONDS,
         (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
 
@@ -141,13 +141,13 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
 
   public void testCreateCollectionSplitShard() throws Exception  {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String firstNode = cluster.getSimClusterStateProvider().simGetRandomNode();
+    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();
+      secondNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
       secondNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(secondNode, ImplicitSnitch.PORT);
       if (secondNodePort != firstNodePort)  break;
     }
@@ -160,8 +160,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
         .setPolicy("c1")
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", collectionName,
-        CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", collectionName,
+        CloudUtils.clusterShape(1, 2, false, true));
 
     DocCollection docCollection = getCollectionState(collectionName);
     List<Replica> list = docCollection.getReplicas(firstNode);
@@ -174,7 +174,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
 
     CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(solrClient);
 
-    CloudTestUtils.waitForState(cluster, "Timed out waiting to see 6 replicas for collection: " + collectionName,
+    CloudUtils.waitForState(cluster, "Timed out waiting to see 6 replicas for collection: " + collectionName,
         collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 6);
 
     docCollection = getCollectionState(collectionName);
@@ -219,8 +219,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     //org.eclipse.jetty.server.handler.DefaultHandler.2xx-responses
     CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "metricsTest",
-        CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "metricsTest",
+        CloudUtils.clusterShape(1, 1));
 
     DocCollection collection = getCollectionState("metricsTest");
     List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
@@ -266,8 +266,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1", 1, 1, 1)
         .setMaxShardsPerNode(-1)
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
-        CloudTestUtils.clusterShape(1, 3, false, true));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
+        CloudUtils.clusterShape(1, 3, false, true));
 
     DocCollection coll = getCollectionState("policiesTest");
 
@@ -300,7 +300,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
   
   public void testCreateCollectionAddShardUsingPolicy() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode();
+    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 + "'}]}}";
@@ -311,16 +311,16 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1,s2", 1)
         .setPolicy("c1")
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
-        CloudTestUtils.clusterShape(2, 1));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
+        CloudUtils.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);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
-        CloudTestUtils.clusterShape(3, 1));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
+        CloudUtils.clusterShape(3, 1));
 
     coll = getCollectionState("policiesTest");
     assertEquals(1, coll.getSlice("s3").getReplicas().size());
@@ -331,8 +331,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     SolrClient solrClient = cluster.simGetSolrClient();
     CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
         .process(solrClient);
-    CloudTestUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
-        CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
+        CloudUtils.clusterShape(1, 2, false, true));
     DocCollection rulesCollection = getCollectionState("policiesTest");
 
     Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
@@ -351,7 +351,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
       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();
+    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);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
index 8b3ecd0..d2f1737 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
@@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.autoscaling.ActionContext;
 import org.apache.solr.cloud.autoscaling.AutoScaling;
 import org.apache.solr.cloud.autoscaling.CapturedEvent;
@@ -554,7 +555,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
       fail("The TriggerAction should have been created by now");
     }
 
-    String lostNodeName = cluster.getSimClusterStateProvider().simGetRandomNode();
+    String lostNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
     cluster.simRemoveNode(lostNodeName, false);
     boolean await = triggerFiredLatch.await(45000 / SPEED, TimeUnit.MILLISECONDS);
     assertTrue("The trigger did not fire at all", await);
@@ -713,7 +714,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   public void testEventQueue() throws Exception {
     waitForSeconds = 1;
     SolrClient solrClient = cluster.simGetSolrClient();
-    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
 
     assertAutoScalingRequest
       ("{" +
@@ -1326,7 +1327,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
         "conf", 1, 2);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cluster, COLL1, 10, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 2, false, true));
+    CloudUtils.waitForState(cluster, COLL1, 10, TimeUnit.SECONDS, CloudUtils.clusterShape(1, 2, false, true));
 
     listenerEventLatch = new CountDownLatch(4);
     
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
index 2f55c7b..5408f11 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
@@ -25,7 +25,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.CloudUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -96,8 +96,8 @@ public class MetricsHistoryHandlerTest extends SolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL,
         "conf", 1, 1);
     create.process(solrClient);
-    CloudTestUtils.waitForState(cloudManager, "failed to create " + CollectionAdminParams.SYSTEM_COLL,
-        CollectionAdminParams.SYSTEM_COLL, CloudTestUtils.clusterShape(1, 1));
+    CloudUtils.waitForState(cloudManager, "failed to create " + CollectionAdminParams.SYSTEM_COLL,
+        CollectionAdminParams.SYSTEM_COLL, CloudUtils.clusterShape(1, 1));
   }
 
   @AfterClass
diff --git a/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
index 936c41a..c5b816d 100644
--- a/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
@@ -27,7 +27,7 @@ import org.apache.solr.common.params.CollectionAdminParams;
 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.util.MockSearchableSolrClient;
+import org.apache.solr.cloud.autoscaling.sim.MockSearchableSolrClient;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;