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/10/23 20:02:15 UTC

[lucene-solr] branch branch_8x updated: SOLR-13831: Support defining arbitrary autoscaling simulation scenarios.

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 6e6b36c  SOLR-13831: Support defining arbitrary autoscaling simulation scenarios.
6e6b36c is described below

commit 6e6b36cbcd7cff4b3daa53715248a93bd4e6f1f4
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Oct 23 21:14:05 2019 +0200

    SOLR-13831: Support defining arbitrary autoscaling simulation scenarios.
---
 solr/CHANGES.txt                                   |    2 +
 .../src/java/org/apache/solr/cloud/CloudUtil.java  |    6 +-
 .../solr/cloud/autoscaling/AutoScalingHandler.java |    2 +-
 .../solr/cloud/autoscaling/ScheduledTriggers.java  |   61 ++
 .../cloud/autoscaling/sim/FakeDocIterator.java     |    0
 .../cloud/autoscaling/sim/SimCloudManager.java     |    3 +
 .../autoscaling/sim/SimClusterStateProvider.java   |   13 +
 .../solr/cloud/autoscaling/sim/SimScenario.java    | 1090 ++++++++++++++++++++
 .../solr/cloud/autoscaling/sim/SimUtils.java       |   16 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |   17 +-
 .../cloud/autoscaling/sim/TestSimScenario.java     |  137 +++
 .../solrcloud-autoscaling-policy-preferences.adoc  |  140 +++
 .../solrj/impl/SolrClientNodeStateProvider.java    |    2 +-
 13 files changed, 1478 insertions(+), 11 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ecda632..18ca2fa 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -41,6 +41,8 @@ Improvements
 
 *SOLR-13731: javabin must support a 1:1 mapping of the JSON update format (noble)
 
+* SOLR-13831: Support defining arbitrary autoscaling simulation scenarios. (ab)
+
 
 Optimizations
 ---------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
index 407d548..1558389 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -244,12 +244,12 @@ public class CloudUtil {
                                                       boolean requireLeaders) {
     return (liveNodes, collectionState) -> {
       if (collectionState == null) {
-        log.info("-- null collection");
+        log.debug("-- 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());
+        log.debug("-- wrong number of slices for collection {}, expected={}, found={}: {}", collectionState.getName(), expectedShards, collectionState.getSlices().size(), collectionState.getSlices());
         return false;
       }
       Set<String> leaderless = new HashSet<>();
@@ -268,7 +268,7 @@ public class CloudUtil {
             activeReplicas++;
         }
         if (activeReplicas != expectedReplicas) {
-          log.info("-- wrong number of active replicas for collection {} in slice {}, expected={}, found={}", collectionState.getName(), slice.getName(), expectedReplicas, activeReplicas);
+          log.debug("-- wrong number of active replicas for collection {} in slice {}, expected={}, found={}", collectionState.getName(), slice.getName(), expectedReplicas, activeReplicas);
           return false;
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
index b6755b1..1341a24 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
@@ -595,7 +595,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
   private static String fullName = SystemLogListener.class.getName();
   private static String solrName = "solr." + SystemLogListener.class.getSimpleName();
 
-  static AutoScalingConfig withSystemLogListener(AutoScalingConfig autoScalingConfig, String triggerName) {
+  public static AutoScalingConfig withSystemLogListener(AutoScalingConfig autoScalingConfig, String triggerName) {
     Map<String, AutoScalingConfig.TriggerListenerConfig> configs = autoScalingConfig.getTriggerListenerConfigs();
     for (AutoScalingConfig.TriggerListenerConfig cfg : configs.values()) {
       if (triggerName.equals(cfg.trigger)) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 768d8a7..522bf01 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -135,6 +135,8 @@ public class ScheduledTriggers implements Closeable {
 
   private final TriggerListeners listeners;
 
+  private final List<TriggerListener> additionalListeners = new ArrayList<>();
+
   private AutoScalingConfig autoScalingConfig;
 
   public ScheduledTriggers(SolrResourceLoader loader, SolrCloudManager cloudManager) {
@@ -553,6 +555,22 @@ public class ScheduledTriggers implements Closeable {
     log.debug("ScheduledTriggers closed completely");
   }
 
+  /**
+   * Add a temporary listener for internal use (tests, simulation).
+   * @param listener listener instance
+   */
+  public void addAdditionalListener(TriggerListener listener) {
+    listeners.addAdditionalListener(listener);
+  }
+
+  /**
+   * Remove a temporary listener for internal use (tests, simulation).
+   * @param listener listener instance
+   */
+  public void removeAdditionalListener(TriggerListener listener) {
+    listeners.removeAdditionalListener(listener);
+  }
+
   private class TriggerWrapper implements Runnable, Closeable {
     AutoScaling.Trigger trigger;
     ScheduledFuture<?> scheduledFuture;
@@ -658,6 +676,7 @@ public class ScheduledTriggers implements Closeable {
   private class TriggerListeners {
     Map<String, Map<TriggerEventProcessorStage, List<TriggerListener>>> listenersPerStage = new HashMap<>();
     Map<String, TriggerListener> listenersPerName = new HashMap<>();
+    List<TriggerListener> additionalListeners = new ArrayList<>();
     ReentrantLock updateLock = new ReentrantLock();
 
     public TriggerListeners() {
@@ -681,6 +700,41 @@ public class ScheduledTriggers implements Closeable {
       return new TriggerListeners(listenersPerStage, listenersPerName);
     }
 
+    public void addAdditionalListener(TriggerListener listener) {
+      updateLock.lock();
+      try {
+        AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
+        for (TriggerEventProcessorStage stage : config.stages) {
+          addPerStage(config.trigger, stage, listener);
+        }
+        // add also for beforeAction / afterAction TriggerStage
+        if (!config.beforeActions.isEmpty()) {
+          addPerStage(config.trigger, TriggerEventProcessorStage.BEFORE_ACTION, listener);
+        }
+        if (!config.afterActions.isEmpty()) {
+          addPerStage(config.trigger, TriggerEventProcessorStage.AFTER_ACTION, listener);
+        }
+        additionalListeners.add(listener);
+      } finally {
+        updateLock.unlock();
+      }
+    }
+
+    public void removeAdditionalListener(TriggerListener listener) {
+      updateLock.lock();
+      try {
+        listenersPerName.remove(listener.getConfig().name);
+        listenersPerStage.forEach((trigger, perStage) -> {
+          perStage.forEach((stage, listeners) -> {
+            listeners.remove(listener);
+          });
+        });
+        additionalListeners.remove(listener);
+      } finally {
+        updateLock.unlock();
+      }
+    }
+
     void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
       updateLock.lock();
       // we will recreate this from scratch
@@ -757,6 +811,13 @@ public class ScheduledTriggers implements Closeable {
             addPerStage(config.trigger, TriggerEventProcessorStage.AFTER_ACTION, listener);
           }
         }
+        // re-add additional listeners
+        List<TriggerListener> additional = new ArrayList<>(additionalListeners);
+        additionalListeners.clear();
+        for (TriggerListener listener : additional) {
+          addAdditionalListener(listener);
+        }
+
       } finally {
         updateLock.unlock();
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
rename to solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index a029ac3..71d4f29 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -842,11 +842,14 @@ public class SimCloudManager implements SolrCloudManager {
     String a = params != null ? params.get(CoreAdminParams.ACTION) : null;
     SolrResponse rsp = new SolrResponseBase();
     rsp.setResponse(new NamedList<>());
+    String path = params != null ? params.get("path") : null;
     if (!(req instanceof CollectionAdminRequest)) {
       // maybe a V2Request?
       if (req instanceof V2Request) {
         params = SimUtils.v2AdminRequestToV1Params((V2Request)req);
         a = params.get(CoreAdminParams.ACTION);
+      } else if (path != null && (path.startsWith("/admin/") || path.startsWith("/cluster/"))) {
+        // pass it through, it's likely a generic request containing admin params
       } else {
         throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName() + ": " + req.getPath() + " " + req.getParams());
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 2e5be68..861ef07 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -700,6 +700,19 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     cloudManager.getTimeSource().sleep(delays.get(op));
   }
 
+  public void simSetOpDelays(String collection, Map<String, Long> delays) {
+    Map<String, Long> currentDelays = opDelays.getOrDefault(collection, Collections.emptyMap());
+    Map<String, Long> newDelays = new HashMap<>(currentDelays);
+    delays.forEach((k, v) -> {
+      if (v == null) {
+        newDelays.remove(k);
+      } else {
+        newDelays.put(k, v);
+      }
+    });
+    opDelays.put(collection, newDelays);
+  }
+
   /**
    * Simulate running a shard leader election. This operation is a no-op if a leader already exists.
    * If a new leader is elected the cluster state is saved.
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
new file mode 100644
index 0000000..755d236
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
@@ -0,0 +1,1090 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud.autoscaling.sim;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.net.URLDecoder;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
+import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
+import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.cloud.CloudUtil;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.cloud.autoscaling.TriggerListener;
+import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.common.params.AutoScalingParams;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.PropertiesUtil;
+import org.apache.solr.util.RedactionUtils;
+import org.apache.solr.util.TimeOut;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class represents an autoscaling scenario consisting of a series of autoscaling
+ * operations on a simulated cluster.
+ */
+public class SimScenario implements AutoCloseable {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /** Context variable: Random live node name. */
+  public static final String RANDOM_NODE_CTX_PROP = "_random_node_";
+  /** Context variable: Node name of the current Overseer leader. */
+  public static final String OVERSEER_LEADER_CTX_PROP = "_overseer_leader_";
+  /** Context variable: List of live nodes. */
+  public static final String LIVE_NODES_CTX_PROP = "_live_nodes_";
+  /** Context variable: List of collections. */
+  public static final String COLLECTIONS_CTX_PROP = "_collections_";
+  /** Context variable: List of calculated suggestions. */
+  public static final String SUGGESTIONS_CTX_PROP = "_suggestions_";
+  /** Context variable: List of SolrResponses of SOLR_REQUEST operations. */
+  public static final String RESPONSES_CTX_PROP = "_responses_";
+  /** Context variable: Current loop iteration or none if outside of loop. */
+  public static final String LOOP_ITER_PROP = "_loop_iter_";
+  /** Last trigger event captured by WAIT_EVENT. */
+  public static final String TRIGGER_EVENT_PREFIX = "_trigger_event_";
+
+  public SimCloudManager cluster;
+  public AutoScalingConfig config;
+  public List<SimOp> ops = new ArrayList<>();
+  public Map<String, Object> context = new HashMap<>();
+  public PrintStream console = System.err;
+  public boolean verbose;
+  public boolean abortLoop;
+  public boolean abortScenario;
+
+  /** Base class for implementation of scenario DSL actions. */
+  public static abstract class SimOp {
+    ModifiableSolrParams initParams;
+    ModifiableSolrParams params;
+
+    public void init(SolrParams params) {
+      this.initParams = new ModifiableSolrParams(params);
+    }
+
+    /**
+     * This method prepares a copy of initial params (and sets the value of {@link #params}
+     * with all property references resolved against the current {@link SimScenario#context}
+     * and system properties. This method should always be called before invoking
+     * {@link #execute(SimScenario)}.
+     * @param scenario current scenario
+     */
+    public void prepareCurrentParams(SimScenario scenario) {
+      Properties props = new Properties();
+      scenario.context.forEach((k, v) -> {
+        if (v instanceof String[]) {
+          v = String.join(",", (String[]) v);
+        } else if (v instanceof Collection) {
+          StringBuilder sb = new StringBuilder();
+          for (Object o : (Collection<Object>)v) {
+            if (sb.length() > 0) {
+              sb.append(',');
+            }
+            if ((o instanceof String) || (o instanceof Number)) {
+              sb.append(o);
+            } else {
+              // skip all values
+              return;
+            }
+          }
+          v = sb.toString();
+        } else if ((v instanceof String) || (v instanceof Number)) {
+          // don't convert, put as is
+        } else {
+          // skip
+          return;
+        }
+        props.put(k, v);
+      });
+      ModifiableSolrParams currentParams = new ModifiableSolrParams();
+      initParams.forEach(e -> {
+        String newKey = PropertiesUtil.substituteProperty(e.getKey(), props);
+        if (newKey == null) {
+          newKey = e.getKey();
+        }
+        String[] newValues;
+        if (e.getValue() != null && e.getValue().length > 0) {
+          String[] values = e.getValue();
+          newValues = new String[values.length];
+          for (int k = 0; k < values.length; k++) {
+            String newVal = PropertiesUtil.substituteProperty(values[k], props);
+            if (newVal == null) {
+              newVal = values[k];
+            }
+            newValues[k] = newVal;
+          }
+        } else {
+          newValues = e.getValue();
+        }
+        currentParams.add(newKey, newValues);
+      });
+      params = currentParams;
+    }
+
+    /**
+     * Execute the operation.
+     * @param scenario current scenario.
+     */
+    public abstract void execute (SimScenario scenario) throws Exception;
+  }
+
+
+  /**
+   * Actions supported by the scenario.
+   */
+  public enum SimAction {
+    /** Create a new simulated cluster. */
+    CREATE_CLUSTER,
+    /** Create a simulated cluster from autoscaling snapshot. */
+    LOAD_SNAPSHOT,
+    /** Save autoscaling snapshot of the current simulated cluster. */
+    SAVE_SNAPSHOT,
+    /** Calculate autoscaling suggestions and put them in the scenario's context. */
+    CALCULATE_SUGGESTIONS,
+    /** Apply previously calculated autoscaling suggestions. */
+    APPLY_SUGGESTIONS,
+    /** Kill specific nodes, or a number of randomly selected nodes. */
+    KILL_NODES,
+    /** Add new nodes. */
+    ADD_NODES,
+    /** Load autoscaling.json configuration from a file. */
+    LOAD_AUTOSCALING,
+    /** Start a loop. */
+    LOOP_START,
+    /** End a loop. */
+    LOOP_END,
+    /** Set operation delays to simulate long-running actions. */
+    SET_OP_DELAYS,
+    /** Execute a SolrRequest (must be supported by {@link SimCloudManager}). */
+    SOLR_REQUEST,
+    /** Wait for a collection to reach the indicated number of shards and replicas. */
+    WAIT_COLLECTION,
+    /** Prepare a listener to listen for an autoscaling event. */
+    EVENT_LISTENER,
+    /** Wait for an autoscaling event using previously prepared listener. */
+    WAIT_EVENT,
+    /** Run the simulation for a while, allowing background tasks to execute. */
+    RUN,
+    /** Dump the internal state of the simulator to console. */
+    DUMP,
+    /** Set a variable in context. */
+    CTX_SET,
+    /** Remove a variable from context. */
+    CTX_REMOVE,
+    /** Set metrics for a node. */
+    SET_NODE_METRICS,
+    /** Set metrics for each replica of a collection's shard(s). */
+    SET_SHARD_METRICS,
+    /** Bulk index a number of simulated documents. */
+    INDEX_DOCS,
+    /** Assert a condition. */
+    ASSERT;
+
+    public static SimAction get(String str) {
+      if (str != null) {
+        try {
+          return SimAction.valueOf(str.toUpperCase(Locale.ROOT));
+        } catch (Exception e) {
+          return null;
+        }
+      } else {
+        return null;
+      }
+    }
+
+    public String toLower() {
+      return toString().toLowerCase(Locale.ROOT);
+    }
+  }
+
+  public static Map<SimAction, Class<? extends SimOp>> simOps = new HashMap<>();
+  static {
+    simOps.put(SimAction.CREATE_CLUSTER, CreateCluster.class);
+    simOps.put(SimAction.LOAD_SNAPSHOT, LoadSnapshot.class);
+    simOps.put(SimAction.SAVE_SNAPSHOT, SaveSnapshot.class);
+    simOps.put(SimAction.LOAD_AUTOSCALING, LoadAutoscaling.class);
+    simOps.put(SimAction.CALCULATE_SUGGESTIONS, CalculateSuggestions.class);
+    simOps.put(SimAction.APPLY_SUGGESTIONS, ApplySuggestions.class);
+    simOps.put(SimAction.KILL_NODES, KillNodes.class);
+    simOps.put(SimAction.ADD_NODES, AddNodes.class);
+    simOps.put(SimAction.LOOP_START, LoopOp.class);
+    simOps.put(SimAction.LOOP_END, null);
+    simOps.put(SimAction.SET_OP_DELAYS, SetOpDelays.class);
+    simOps.put(SimAction.SOLR_REQUEST, RunSolrRequest.class);
+    simOps.put(SimAction.RUN, RunSimulator.class);
+    simOps.put(SimAction.WAIT_COLLECTION, WaitCollection.class);
+    simOps.put(SimAction.EVENT_LISTENER, SetEventListener.class);
+    simOps.put(SimAction.WAIT_EVENT, WaitEvent.class);
+    simOps.put(SimAction.CTX_SET, CtxSet.class);
+    simOps.put(SimAction.CTX_REMOVE, CtxRemove.class);
+    simOps.put(SimAction.DUMP, Dump.class);
+    simOps.put(SimAction.SET_NODE_METRICS, SetNodeMetrics.class);
+    simOps.put(SimAction.SET_SHARD_METRICS, SetShardMetrics.class);
+    simOps.put(SimAction.INDEX_DOCS, IndexDocs.class);
+    simOps.put(SimAction.ASSERT, Assert.class);
+  }
+
+  /**
+   * Loop action.
+   */
+  public static class LoopOp extends SimOp {
+    // populated by the DSL parser
+    List<SimOp> ops = new ArrayList<>();
+    int iterations;
+
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      iterations = Integer.parseInt(params.get("iterations", "10"));
+      for (int i = 0; i < iterations; i++) {
+        if (scenario.abortLoop) {
+          log.info("        -- abortLoop requested, aborting after " + i + " iterations.");
+          return;
+        }
+        scenario.context.put(LOOP_ITER_PROP, i);
+        log.info("   * iter " + (i + 1) + ":");
+        for (SimOp op : ops) {
+          op.prepareCurrentParams(scenario);
+          log.info("     - " + op.getClass().getSimpleName() + "\t" + op.params.toString());
+          op.execute(scenario);
+          if (scenario.abortLoop) {
+            log.info("        -- abortLoop requested, aborting after " + i + " iterations.");
+            return;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Set a context property.
+   */
+  public static class CtxSet extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String key = params.required().get("key");
+      String[] values = params.required().getParams("value");
+      if (values != null) {
+        scenario.context.put(key, Arrays.asList(values));
+      } else {
+        scenario.context.remove(key);
+      }
+    }
+  }
+
+  /**
+   * Remove a context property.
+   */
+  public static class CtxRemove extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String key = params.required().get("key");
+      scenario.context.remove(key);
+    }
+  }
+
+  /**
+   * Create a simulated cluster.
+   */
+  public static class CreateCluster extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      int numNodes = Integer.parseInt(params.get("numNodes", "5"));
+      boolean disableMetricsHistory = Boolean.parseBoolean(params.get("disableMetricsHistory", "false"));
+      String timeSourceStr = params.get("timeSource", "simTime:50");
+      if (scenario.cluster != null) { // close & reset
+        IOUtils.closeQuietly(scenario.cluster);
+        scenario.context.clear();
+      }
+      scenario.cluster = SimCloudManager.createCluster(numNodes, TimeSource.get(timeSourceStr));
+      if (disableMetricsHistory) {
+        scenario.cluster.disableMetricsHistory();
+      }
+      scenario.config = scenario.cluster.getDistribStateManager().getAutoScalingConfig();
+    }
+  }
+
+  /**
+   * Create a simulated cluster from an autoscaling snapshot.
+   */
+  public static class LoadSnapshot extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String path = params.get("path");
+      SnapshotCloudManager snapshotCloudManager;
+      if (path == null) {
+        String zkHost = params.get("zkHost");
+        if (zkHost == null) {
+          throw new IOException(SimAction.LOAD_SNAPSHOT + " must specify 'path' or 'zkHost'");
+        } else {
+          try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zkHost), Optional.empty()).build()) {
+            cloudSolrClient.connect();
+            try (SolrClientCloudManager realCloudManager = new SolrClientCloudManager(NoopDistributedQueueFactory.INSTANCE, cloudSolrClient)) {
+              snapshotCloudManager = new SnapshotCloudManager(realCloudManager, null);
+            }
+          }
+        }
+      } else {
+        snapshotCloudManager = SnapshotCloudManager.readSnapshot(new File(path));
+      }
+      scenario.cluster = SimCloudManager.createCluster(snapshotCloudManager, null, snapshotCloudManager.getTimeSource());
+      scenario.config = scenario.cluster.getDistribStateManager().getAutoScalingConfig();
+    }
+  }
+
+  /**
+   * Save an autoscaling snapshot.
+   */
+  public static class SaveSnapshot extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String path = params.get("path");
+      if (path == null) {
+        throw new IOException(SimAction.SAVE_SNAPSHOT + " must specify 'path'");
+      }
+      boolean redact = Boolean.parseBoolean(params.get("redact", "false"));
+      SnapshotCloudManager snapshotCloudManager = new SnapshotCloudManager(scenario.cluster, null);
+      snapshotCloudManager.saveSnapshot(new File(path), true, redact);
+    }
+  }
+
+  /**
+   * Load autoscaling.json configuration.
+   */
+  public static class LoadAutoscaling extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      Map<String, Object> map;
+      boolean addDefaults = Boolean.parseBoolean(params.get("withDefaultTriggers", "true"));
+      int defaultWaitFor = Integer.parseInt(params.get("defaultWaitFor", "120"));
+      String path = params.get("path");
+      if (path == null) {
+        String json = params.get("json");
+        if (json == null) {
+          throw new IOException(SimAction.LOAD_AUTOSCALING + " must specify either 'path' or 'json'");
+        } else {
+          map = (Map<String, Object>) Utils.fromJSONString(json);
+        }
+      } else {
+        File f = new File(path);
+        Reader r;
+        if (f.exists()) {
+          r = new InputStreamReader(new FileInputStream(f), Charset.forName("UTF-8"));
+        } else {
+          InputStream is = getClass().getResourceAsStream(path);
+          if (is == null) {
+            throw new IOException("path " + path + " does not exist and it's not a resource");
+          }
+          r = new InputStreamReader(is, Charset.forName("UTF-8"));
+        }
+        map = (Map<String, Object>) Utils.fromJSON(r);
+      }
+      AutoScalingConfig config = new AutoScalingConfig(map);
+      if (addDefaults) {
+        // add default triggers
+        if (!config.getTriggerConfigs().containsKey(AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_NAME)) {
+          Map<String, Object> props = new HashMap<>(AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_PROPS);
+          props.put("waitFor", defaultWaitFor);
+          AutoScalingConfig.TriggerConfig trigger = new AutoScalingConfig.TriggerConfig(AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_NAME, props);
+          config = config.withTriggerConfig(trigger);
+          config = AutoScalingHandler.withSystemLogListener(config, AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_NAME);
+        }
+        if (!config.getTriggerConfigs().containsKey(AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME)) {
+          AutoScalingConfig.TriggerConfig trigger = new AutoScalingConfig.TriggerConfig(AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME, AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_PROPS);
+          config = config.withTriggerConfig(trigger);
+          config = AutoScalingHandler.withSystemLogListener(config, AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME);
+        }
+      }
+      scenario.config = config;
+      // set this config on the simulator
+      scenario.cluster.getSimDistribStateManager().simSetAutoScalingConfig(config);
+      // wait until it finished processing the config
+      (new TimeOut(30, TimeUnit.SECONDS, scenario.cluster.getTimeSource()))
+          .waitFor("OverseerTriggerThread never caught up to the latest znodeVersion", () -> {
+            try {
+              AutoScalingConfig autoscalingConfig = scenario.cluster.getDistribStateManager().getAutoScalingConfig();
+              return autoscalingConfig.getZkVersion() == scenario.cluster.getOverseerTriggerThread().getProcessedZnodeVersion();
+            } catch (Exception e) {
+              throw new RuntimeException("FAILED", e);
+            }
+          });
+
+    }
+  }
+
+  /**
+   * Kill one or more nodes.
+   */
+  public static class KillNodes extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      if (params.get("numNodes") != null) {
+        int numNodes = Integer.parseInt(params.get("numNodes"));
+        scenario.cluster.simRemoveRandomNodes(numNodes, false, scenario.cluster.getRandom());
+      } else if (params.get("nodes") != null || params.get("node") != null) {
+        Set<String> nodes = new HashSet<>();
+        String[] nodesValues = params.getParams("nodes");
+        if (nodesValues != null) {
+          for (String nodesValue : nodesValues) {
+            String[] vals = nodesValue.split(",");
+            nodes.addAll(Arrays.asList(vals));
+          }
+        }
+        nodesValues = params.getParams("node");
+        if (nodesValues != null) {
+          nodes.addAll(Arrays.asList(nodesValues));
+        }
+        for (String node : nodes) {
+          scenario.cluster.simRemoveNode(node, false);
+        }
+      }
+    }
+  }
+
+  /**
+   * Add one or more nodes.
+   */
+  public static class AddNodes extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      int numNodes = Integer.parseInt(params.get("numNodes"));
+      for (int i = 0; i < numNodes; i++) {
+        scenario.cluster.simAddNode();
+      }
+    }
+  }
+
+  /**
+   * Calculate autoscaling suggestions.
+   */
+  public static class CalculateSuggestions extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(scenario.config, scenario.cluster);
+      scenario.context.put(SUGGESTIONS_CTX_PROP, suggestions);
+      log.info("        - " + suggestions.size() + " suggestions");
+      if (suggestions.isEmpty()) {
+        scenario.abortLoop = true;
+      }
+    }
+  }
+
+  /**
+   * Apply autoscaling suggestions.
+   */
+  public static class ApplySuggestions extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      List<Suggester.SuggestionInfo> suggestions = (List<Suggester.SuggestionInfo>) scenario.context.getOrDefault(SUGGESTIONS_CTX_PROP, Collections.emptyList());
+      int unresolvedCount = 0;
+      for (Suggester.SuggestionInfo suggestion : suggestions) {
+        SolrRequest operation = suggestion.getOperation();
+        if (operation == null) {
+          unresolvedCount++;
+          if (suggestion.getViolation() == null) {
+            log.error("       -- ignoring suggestion without violation and without operation: " + suggestion);
+          }
+          continue;
+        }
+        SolrParams params = operation.getParams();
+        if (operation instanceof V2Request) {
+          params = SimUtils.v2AdminRequestToV1Params((V2Request)operation);
+        }
+        Map<String, Object> paramsMap = new LinkedHashMap<>();
+        params.toMap(paramsMap);
+        ReplicaInfo info = scenario.cluster.getSimClusterStateProvider().simGetReplicaInfo(
+            params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
+        if (info == null) {
+          log.error("Could not find ReplicaInfo for params: " + params);
+        } else if (scenario.verbose) {
+          paramsMap.put("replicaInfo", info);
+        } else if (info.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
+          paramsMap.put(Variable.Type.CORE_IDX.tagName, info.getVariable(Variable.Type.CORE_IDX.tagName));
+        }
+        try {
+          scenario.cluster.request(operation);
+        } catch (Exception e) {
+          log.error("Aborting - error executing suggestion " + suggestion, e);
+          break;
+        }
+      }
+      if (suggestions.size() > 0 && unresolvedCount == suggestions.size()) {
+        log.info("        -- aborting simulation, only " + unresolvedCount + " unresolved violations remain");
+        scenario.abortLoop = true;
+      }
+    }
+  }
+
+  /**
+   * Execute a SolrRequest supported by {@link SimCloudManager}.
+   */
+  public static class RunSolrRequest extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String path = params.get("path", "/");
+      SolrRequest.METHOD m = SolrRequest.METHOD.valueOf(params.get("httpMethod", "GET"));
+      params.remove("httpMethod");
+      String streamBody = params.get("stream.body");
+      params.remove("stream.body");
+      GenericSolrRequest req = new GenericSolrRequest(m, path, params);
+      if (streamBody != null) {
+        req.setContentWriter(new RequestWriter.StringPayloadContentWriter(streamBody, "application/json"));
+      }
+      SolrResponse rsp = scenario.cluster.request(req);
+      List<SolrResponse> responses = (List<SolrResponse>) scenario.context.computeIfAbsent(RESPONSES_CTX_PROP, Utils.NEW_ARRAYLIST_FUN);
+      responses.add(rsp);
+    }
+  }
+
+  /**
+   * Set delays for specified collection operations in order to simulate slow execution.
+   */
+  public static class SetOpDelays extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String[] collections = params.remove("collection");
+      if (collections == null || collections.length == 0) {
+        throw new IOException("'collection' param is required but missing: " + params);
+      }
+      Map<String, Long> delays = new HashMap<>();
+      params.forEach(e -> {
+        String key = e.getKey();
+        CollectionParams.CollectionAction a = CollectionParams.CollectionAction.get(key);
+        if (a == null) {
+          log.warn("Invalid collection action " + key + ", skipping...");
+          return;
+        }
+        String[] values = e.getValue();
+        if (values == null || values[0].trim().isEmpty()) {
+          delays.put(a.name(), null);
+        } else {
+          Long value = Long.parseLong(values[0]);
+          delays.put(a.name(), value);
+        }
+      });
+      for (String collection : collections) {
+        scenario.cluster.getSimClusterStateProvider().simSetOpDelays(collection, delays);
+      }
+    }
+  }
+
+  /**
+   * Run the simulator for a while.
+   */
+  public static class RunSimulator extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      int timeMs = Integer.parseInt(params.get("time", "60000"));
+      scenario.cluster.getTimeSource().sleep(timeMs);
+    }
+  }
+
+  /**
+   * Wait for a specific collection shape.
+   */
+  public static class WaitCollection extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String collection = params.required().get("collection");
+      int shards = Integer.parseInt(params.required().get("shards"));
+      int replicas = Integer.parseInt(params.required().get("replicas"));
+      boolean withInactive = Boolean.parseBoolean(params.get("withInactive", "false"));
+      boolean requireLeaders = Boolean.parseBoolean(params.get("requireLeaders", "true"));
+      int waitSec = Integer.parseInt(params.required().get("wait", "" + CloudUtil.DEFAULT_TIMEOUT));
+      CloudUtil.waitForState(scenario.cluster, collection, waitSec, TimeUnit.SECONDS,
+          CloudUtil.clusterShape(shards, replicas, withInactive, requireLeaders));
+    }
+  }
+
+  private static class SimWaitListener extends TriggerListenerBase {
+    private final TimeSource timeSource;
+    private final AutoScalingConfig.TriggerListenerConfig config;
+    private CountDownLatch triggerFired = new CountDownLatch(1);
+    private TriggerEvent event;
+
+    SimWaitListener(TimeSource timeSource, AutoScalingConfig.TriggerListenerConfig config) {
+      this.timeSource = timeSource;
+      this.config = config;
+    }
+
+    @Override
+    public AutoScalingConfig.TriggerListenerConfig getConfig() {
+      return config;
+    }
+
+    @Override
+    public boolean isEnabled() {
+      return true;
+    }
+
+    @Override
+    public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) throws Exception {
+      triggerFired.countDown();
+      this.event = event;
+    }
+
+    public TriggerEvent getEvent() {
+      return event;
+    }
+
+    public void wait(int waitSec) throws Exception {
+      long waitTime = timeSource.convertDelay(TimeUnit.SECONDS, waitSec, TimeUnit.MILLISECONDS);
+      boolean await =  triggerFired.await(waitTime, TimeUnit.MILLISECONDS);
+      if (!await) {
+        throw new IOException("Timed out waiting for trigger " + config.trigger + " to fire after simulated " +
+            waitSec + "s (real " + waitTime + "ms).");
+      }
+    }
+  }
+
+  /**
+   * Set a temporary listener to wait for a specific trigger event processing.
+   */
+  public static class SetEventListener extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String trigger = params.required().get(AutoScalingParams.TRIGGER);
+      Map<String, Object> cfgMap = new HashMap<>();
+      String name = ".sim_wait_event_" + trigger;
+      cfgMap.put(AutoScalingParams.NAME, name);
+      cfgMap.put(AutoScalingParams.TRIGGER, trigger);
+
+      String[] beforeActions = params.getParams(AutoScalingParams.BEFORE_ACTION);
+      String[] afterActions = params.getParams(AutoScalingParams.AFTER_ACTION);
+      if (beforeActions != null) {
+        for (String beforeAction : beforeActions) {
+          ((List<String>)cfgMap.computeIfAbsent(AutoScalingParams.BEFORE_ACTION, Utils.NEW_ARRAYLIST_FUN)).add(beforeAction);
+        }
+      }
+      if (afterActions != null) {
+        for (String afterAction : afterActions) {
+          ((List<String>)cfgMap.computeIfAbsent(AutoScalingParams.AFTER_ACTION, Utils.NEW_ARRAYLIST_FUN)).add(afterAction);
+        }
+      }
+      String[] stages = params.required().getParams(AutoScalingParams.STAGE);
+      for (String stage : stages) {
+        String[] lst = stage.split("[,\\s]+");
+        for (String val : lst) {
+          try {
+            TriggerEventProcessorStage.valueOf(val);
+            ((List<String>)cfgMap.computeIfAbsent(AutoScalingParams.STAGE, Utils.NEW_ARRAYLIST_FUN)).add(val);
+          } catch (IllegalArgumentException e) {
+            throw new IOException("Invalid stage name '" + val + "'");
+          }
+        }
+      }
+      final AutoScalingConfig.TriggerListenerConfig listenerConfig = new AutoScalingConfig.TriggerListenerConfig(name, cfgMap);
+      TriggerListener listener = new SimWaitListener(scenario.cluster.getTimeSource(), listenerConfig);
+      if (scenario.context.containsKey("_sim_waitListener_" + trigger)) {
+        throw new IOException("currently only one listener can be set per trigger. Trigger name: " + trigger);
+      }
+      scenario.context.put("_sim_waitListener_" + trigger, listener);
+      scenario.cluster.getOverseerTriggerThread().getScheduledTriggers().addAdditionalListener(listener);
+    }
+  }
+
+  /**
+   * Wait for the previously set listener to capture an event.
+   */
+  public static class WaitEvent extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String trigger = params.required().get(AutoScalingParams.TRIGGER);
+      int waitSec = Integer.parseInt(params.get("wait", "" + CloudUtil.DEFAULT_TIMEOUT));
+      SimWaitListener listener = (SimWaitListener)scenario.context.remove("_sim_waitListener_" + trigger);
+      if (listener == null) {
+        throw new IOException(SimAction.WAIT_EVENT + " must be preceded by " + SimAction.EVENT_LISTENER + " for trigger " + trigger);
+      }
+      try {
+        listener.wait(waitSec);
+        scenario.context.remove(TRIGGER_EVENT_PREFIX + trigger);
+        if (listener.getEvent() != null) {
+          Map<String, Object> ev = listener.getEvent().toMap(new LinkedHashMap<>());
+          scenario.context.put(TRIGGER_EVENT_PREFIX + trigger, ev);
+        }
+      } finally {
+        scenario.cluster.getOverseerTriggerThread().getScheduledTriggers().removeAdditionalListener(listener);
+      }
+    }
+  }
+
+  public static class SetNodeMetrics extends SimOp {
+
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String nodeset = params.required().get(Clause.NODESET);
+      Set<String> nodes = new HashSet<>();
+      if (nodeset.equals(Policy.ANY)) {
+        nodes.addAll(scenario.cluster.getLiveNodesSet().get());
+      } else {
+        String[] list = nodeset.split("[,\\s]+");
+        for (String node : list) {
+          if (node.trim().isEmpty()) {
+            continue;
+          }
+          nodes.add(node);
+        }
+      }
+      Map<String, Object> values = new HashMap<>();
+      params.remove(Clause.NODESET);
+      for (String key : params.getParameterNames()) {
+        values.put(key, params.get(key));
+      }
+      for (String node : nodes) {
+        scenario.cluster.getSimNodeStateProvider().simSetNodeValues(node, values);
+      }
+    }
+  }
+
+  public static class SetShardMetrics extends SimOp {
+
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String collection = params.required().get("collection");
+      String shard = params.get("shard");
+      boolean delta = params.getBool("delta", false);
+      boolean divide = params.getBool("divide", false);
+      params.remove("collection");
+      params.remove("shard");
+      params.remove("delta");
+      params.remove("divide");
+      Map<String, Object> values = new HashMap<>();
+      for (String key : params.getParameterNames()) {
+        // try guessing if it's a number
+        try {
+          Double d = Double.valueOf(params.get(key));
+          values.put(key, d);
+        } catch (NumberFormatException nfe) {
+          // not a number
+          values.put(key, params.get(key));
+        }
+      }
+      values.forEach((k, v) -> {
+        try {
+          scenario.cluster.getSimClusterStateProvider().simSetShardValue(collection, shard, k, v, delta, divide);
+        } catch (Exception e) {
+          throw new RuntimeException("Error setting shard value", e);
+        }
+      });
+    }
+  }
+
+  public static class IndexDocs extends SimOp {
+
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String collection = params.required().get("collection");
+      long numDocs = params.required().getLong("numDocs");
+      long start = params.getLong("start", 0L);
+
+      UpdateRequest ureq = new UpdateRequest();
+      ureq.setParam("collection", collection);
+      ureq.setDocIterator(new FakeDocIterator(start, numDocs));
+      scenario.cluster.simGetSolrClient().request(ureq);
+    }
+  }
+
+  public enum Condition {
+    EQUALS,
+    NOT_EQUALS,
+    NULL,
+    NOT_NULL;
+
+    public static Condition get(String p) {
+      if (p == null) {
+        return null;
+      } else {
+        try {
+          return Condition.valueOf(p.toUpperCase(Locale.ROOT));
+        } catch (Exception e) {
+          return null;
+        }
+      }
+    }
+  }
+
+  public static class Assert extends SimOp {
+
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      String key = params.get("key");
+      Condition condition = Condition.get(params.required().get("condition"));
+      if (condition == null) {
+        throw new IOException("Invalid 'condition' in params: " + params);
+      }
+      String expected = params.get("expected");
+      if (condition != Condition.NOT_NULL && condition != Condition.NULL && expected == null) {
+        throw new IOException("'expected' param is required when condition is " + condition);
+      }
+      Object value;
+      if (key != null) {
+        if (key.contains("/")) {
+          value = Utils.getObjectByPath(scenario.context, true, key);
+        } else {
+          value = scenario.context.get(key);
+        }
+      } else {
+        value = params.required().get("value");
+      }
+      switch (condition) {
+        case NULL:
+          if (value != null) {
+            throw new IOException("expected value should be null but was '" + value + "'");
+          }
+          break;
+        case NOT_NULL:
+          if (value == null) {
+            throw new IOException("expected value should not be null");
+          }
+          break;
+        case EQUALS:
+          if (!expected.equals(String.valueOf(value))) {
+            throw new IOException("expected value is '" + expected + "' but actual value is '" + value + "'");
+          }
+          break;
+        case NOT_EQUALS:
+          if (expected.equals(String.valueOf(value))) {
+            throw new IOException("expected value is '" + expected + "' and actual value is the same while it should be different");
+          }
+          break;
+      }
+    }
+  }
+
+
+  /**
+   * Dump the simulator state to the console.
+   */
+  public static class Dump extends SimOp {
+    @Override
+    public void execute(SimScenario scenario) throws Exception {
+      boolean redact = Boolean.parseBoolean(params.get("redact", "false"));
+      boolean withData = Boolean.parseBoolean(params.get("withData", "false"));
+      boolean withStats = Boolean.parseBoolean(params.get("withStats", "false"));
+      boolean withSuggestions = Boolean.parseBoolean(params.get("withSuggestions", "true"));
+      boolean withDiagnostics = Boolean.parseBoolean(params.get("withDiagnostics", "false"));
+      boolean withNodeState = Boolean.parseBoolean(params.get("withNodeState", "false"));
+      boolean withClusterState = Boolean.parseBoolean(params.get("withClusterState", "false"));
+      boolean withManagerState = Boolean.parseBoolean(params.get("withManagerState", "false"));
+      SnapshotCloudManager snapshotCloudManager = new SnapshotCloudManager(scenario.cluster, null);
+      Map<String, Object> snapshot = snapshotCloudManager.getSnapshot(true, redact);
+      if (!withData) {
+        snapshot.remove(SnapshotCloudManager.DISTRIB_STATE_KEY);
+      }
+      if (!withNodeState) {
+        snapshot.remove(SnapshotCloudManager.NODE_STATE_KEY);
+      }
+      if (!withClusterState) {
+        snapshot.remove(SnapshotCloudManager.CLUSTER_STATE_KEY);
+      }
+      if (!withStats) {
+        snapshot.remove(SnapshotCloudManager.STATISTICS_STATE_KEY);
+      }
+      if (!withManagerState) {
+        snapshot.remove(SnapshotCloudManager.MANAGER_STATE_KEY);
+      }
+      if (!withDiagnostics) {
+        ((Map<String, Object>)snapshot.get(SnapshotCloudManager.AUTOSCALING_STATE_KEY)).remove("diagnostics");
+      }
+      if (!withSuggestions) {
+        ((Map<String, Object>)snapshot.get(SnapshotCloudManager.AUTOSCALING_STATE_KEY)).remove("suggestions");
+      }
+      String data = Utils.toJSONString(snapshot);
+      if (redact) {
+        RedactionUtils.RedactionContext ctx = SimUtils.getRedactionContext(snapshotCloudManager.getClusterStateProvider().getClusterState());
+        data = RedactionUtils.redactNames(ctx.getRedactions(), data);
+      }
+      scenario.console.println(data);
+    }
+  }
+
+  /**
+   * Parse a DSL string and create a scenario ready to run.
+   * @param data DSL string with commands and parameters
+   * @return configured scenario
+   * @throws Exception on syntax errors
+   */
+  public static SimScenario load(String data) throws Exception {
+    SimScenario scenario = new SimScenario();
+    String[] lines = data.split("\\r?\\n");
+    for (int i = 0; i < lines.length; i++) {
+      String line = lines[i];
+      line = line.trim();
+      if (line.trim().isEmpty() || line.startsWith("#") || line.startsWith("//")) {
+        continue;
+      }
+      // remove trailing / / comments
+      String[] comments = line.split("//");
+      String expr = comments[0];
+      // split on blank
+      String[] parts = expr.split("\\s+");
+      if (parts.length > 2) {
+        log.warn("Invalid line - wrong number of parts " + parts.length + ", skipping: " + line);
+        continue;
+      }
+      SimAction action = SimAction.get(parts[0]);
+      if (action == null) {
+        log.warn("Invalid scenario action " + parts[0] + ", skipping...");
+        continue;
+      }
+      if (action == SimAction.LOOP_END) {
+        if (!scenario.context.containsKey("loop")) {
+          throw new IOException("LOOP_END without start!");
+        }
+        scenario.context.remove("loop");
+        continue;
+      }
+      Class<? extends SimOp> opClass = simOps.get(action);
+      SimOp op = opClass.getConstructor().newInstance();
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      if (parts.length > 1) {
+        String paramsString = parts[1];
+        if (parts[1].contains("?")) { // url-like with path?params...
+          String[] urlParts = parts[1].split("\\?");
+          params.set("path", urlParts[0]);
+          paramsString = urlParts.length > 1 ? urlParts[1] : "";
+        }
+        String[] paramsParts = paramsString.split("&");
+        for (String paramPair : paramsParts) {
+          String[] paramKV = paramPair.split("=");
+          String k = URLDecoder.decode(paramKV[0], "UTF-8");
+          String v = paramKV.length > 1 ? URLDecoder.decode(paramKV[1], "UTF-8") : null;
+          params.add(k, v);
+        }
+      }
+      op.init(params);
+      // loop handling
+      if (action == SimAction.LOOP_START) {
+        if (scenario.context.containsKey("loop")) {
+          throw new IOException("only one loop level is allowed");
+        }
+        scenario.context.put("loop", op);
+        scenario.ops.add(op);
+        continue;
+      }
+      LoopOp currentLoop = (LoopOp) scenario.context.get("loop");
+      if (currentLoop != null) {
+        currentLoop.ops.add(op);
+      } else {
+        scenario.ops.add(op);
+      }
+    }
+    if (scenario.context.containsKey("loop")) {
+      throw new IOException("Unterminated loop statement");
+    }
+    // sanity check set_listener / wait_listener
+    int numSets = 0, numWaits = 0;
+    for (SimOp op : scenario.ops) {
+      if (op instanceof SetEventListener) {
+        numSets++;
+      } else if (op instanceof WaitEvent) {
+        numWaits++;
+      }
+      if (numWaits > numSets) {
+        throw new Exception("Unexpected " + SimAction.WAIT_EVENT + " without previous " + SimAction.EVENT_LISTENER);
+      }
+    }
+    if (numSets > numWaits) {
+      throw new Exception(SimAction.EVENT_LISTENER + " count should be equal to " + SimAction.WAIT_EVENT + " count but was " +
+          numSets + " > " + numWaits);
+    }
+    return scenario;
+  }
+
+  /**
+   * Run the scenario.
+   */
+  public void run() throws Exception {
+    for (int i = 0; i < ops.size(); i++) {
+      if (abortScenario) {
+        log.info("-- abortScenario requested, aborting after " + i + " ops.");
+        return;
+      }
+      SimOp op = ops.get(i);
+      log.info((i + 1) + ".\t" + op.getClass().getSimpleName() + "\t" + op.initParams.toString());
+      // substitute parameters based on the current context
+      if (cluster != null && cluster.getLiveNodesSet().size() > 0) {
+        context.put(LIVE_NODES_CTX_PROP, new ArrayList<>(cluster.getLiveNodesSet().get()));
+        context.put(RANDOM_NODE_CTX_PROP, cluster.getSimClusterStateProvider().simGetRandomNode());
+        context.put(COLLECTIONS_CTX_PROP, cluster.getSimClusterStateProvider().simListCollections());
+        context.put(OVERSEER_LEADER_CTX_PROP, cluster.getSimClusterStateProvider().simGetOverseerLeader());
+      } else {
+        context.remove(LIVE_NODES_CTX_PROP);
+        context.remove(COLLECTIONS_CTX_PROP);
+        context.remove(RANDOM_NODE_CTX_PROP);
+        context.remove(SUGGESTIONS_CTX_PROP);
+        context.remove(OVERSEER_LEADER_CTX_PROP);
+      }
+      op.prepareCurrentParams(this);
+      log.info("\t\t" + op.getClass().getSimpleName() + "\t" + op.params.toString());
+      op.execute(this);
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (cluster != null) {
+      cluster.close();
+      cluster = null;
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
index 1c5d606..b7d59de 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
@@ -134,17 +134,20 @@ public class SimUtils {
     }
     allReplicaInfos.keySet().forEach(collection -> {
       Set<String> infosCores = allReplicaInfos.getOrDefault(collection, Collections.emptyMap()).keySet();
-      Set<String> csCores = allReplicas.getOrDefault(collection, Collections.emptyMap()).keySet();
+      Map<String, Replica> replicas = allReplicas.getOrDefault(collection, Collections.emptyMap());
+      Set<String> csCores = replicas.keySet();
       if (!infosCores.equals(csCores)) {
         Set<String> notInClusterState = infosCores.stream()
             .filter(k -> !csCores.contains(k))
             .collect(Collectors.toSet());
         Set<String> notInNodeProvider = csCores.stream()
-            .filter(k -> !infosCores.contains(k))
+            .filter(k -> !infosCores.contains(k) && replicas.get(k).isActive(solrCloudManager.getClusterStateProvider().getLiveNodes()))
             .collect(Collectors.toSet());
-        throw new RuntimeException("Mismatched replica data between ClusterState and NodeStateProvider:\n\t" +
-            "replica not in ClusterState: " + notInClusterState + "\n\t" +
-            "replica not in NodeStateProvider: " + notInNodeProvider);
+        if (!notInClusterState.isEmpty() || !notInNodeProvider.isEmpty()) {
+          throw new RuntimeException("Mismatched replica data for collection " + collection + " between ClusterState and NodeStateProvider:\n\t" +
+              "replica in NodeStateProvider but not in ClusterState: " + notInClusterState + "\n\t" +
+              "replica in ClusterState but not in NodeStateProvider: " + notInNodeProvider);
+        }
       }
     });
     // verify all replicas have size info
@@ -349,6 +352,9 @@ public class SimUtils {
     }
     String a = cmd.keySet().iterator().next();
     ModifiableSolrParams params = new ModifiableSolrParams();
+    if (req.getParams() != null) {
+      params.add(req.getParams());
+    }
     params.add(CollectionAdminParams.COLLECTION, path.substring(3));
     if (req.getParams() != null) {
       params.add(req.getParams());
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 1e7c691..3f97b82 100755
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -116,6 +116,7 @@ import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.autoscaling.sim.NoopDistributedQueueFactory;
 import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
+import org.apache.solr.cloud.autoscaling.sim.SimScenario;
 import org.apache.solr.cloud.autoscaling.sim.SimUtils;
 import org.apache.solr.cloud.autoscaling.sim.SnapshotCloudManager;
 import org.apache.solr.common.MapWriter;
@@ -927,12 +928,17 @@ public class SolrCLI {
               .withLongOpt("iterations")
               .create("i"),
           OptionBuilder
-              .withDescription("Save autoscaling shapshots at each step of simulated execution.")
+              .withDescription("Save autoscaling snapshots at each step of simulated execution.")
               .withArgName("DIR")
               .withLongOpt("saveSimulated")
               .hasArg()
               .create("ss"),
           OptionBuilder
+              .withDescription("Execute a scenario from a file (and ignore all other options).")
+              .withArgName("FILE")
+              .hasArg()
+              .create("scenario"),
+          OptionBuilder
               .withDescription("Turn on all options to get all available information.")
               .create("all")
 
@@ -946,6 +952,15 @@ public class SolrCLI {
 
     protected void runImpl(CommandLine cli) throws Exception {
       raiseLogLevelUnlessVerbose(cli);
+      if (cli.hasOption("scenario")) {
+        String data = IOUtils.toString(new FileInputStream(cli.getOptionValue("scenario")), "UTF-8");
+        try (SimScenario scenario = SimScenario.load(data)) {
+          scenario.verbose = verbose;
+          scenario.console = System.err;
+          scenario.run();
+        }
+        return;
+      }
       SnapshotCloudManager cloudManager;
       AutoScalingConfig config = null;
       String configFile = cli.getOptionValue("a");
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimScenario.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimScenario.java
new file mode 100644
index 0000000..51c1216
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimScenario.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.autoscaling.sim;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
+import org.apache.solr.cloud.CloudUtil;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.junit.Test;
+
+/**
+ *
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+public class TestSimScenario extends SimSolrCloudTestCase {
+
+  // simple scenario to test .autoAddReplicas trigger
+  String autoAddReplicasScenario =
+      "# standard comment\n" +
+      "// java comment\n" +
+      "create_cluster numNodes=2 // inline comment\n" +
+      "load_autoscaling json={'cluster-policy'+:+[{'replica'+:+'<3',+'shard'+:+'#EACH',+'collection'+:+'testCollection','node':'#ANY'}]}&defaultWaitFor=10\n" +
+      "solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=2\n" +
+      "wait_collection collection=testCollection&shards=2&replicas=2\n" +
+      "event_listener trigger=.auto_add_replicas&stage=SUCCEEDED\n" +
+      "kill_nodes node=${_random_node_}\n" +
+      "wait_event trigger=.auto_add_replicas&wait=60\n" +
+      "wait_collection collection=testCollection&shards=2&replicas=2\n" +
+      "save_snapshot path=${snapshotPath}\n";
+
+  @Test
+  public void testAutoAddReplicas() throws Exception {
+    String snapshotPath = createTempDir() + "/snapshot";
+    try (SimScenario scenario = SimScenario.load(autoAddReplicasScenario)) {
+      scenario.context.put("snapshotPath", snapshotPath);
+      scenario.run();
+    }
+    SnapshotCloudManager snapshotCloudManager = SnapshotCloudManager.readSnapshot(new File(snapshotPath));
+    CloudUtil.waitForState(snapshotCloudManager, "testCollection", 1, TimeUnit.SECONDS,
+        CloudUtil.clusterShape(2, 2));
+  }
+
+  String testSuggestionsScenario =
+      "create_cluster numNodes=2\n" +
+      "solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=2\n" +
+      "wait_collection collection=testCollection&shards=2&replicas=2\n" +
+      "ctx_set key=myNode&value=${_random_node_}\n" +
+      "solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}\n" +
+      "solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}\n" +
+      "loop_start iterations=${iterative}\n" +
+      "  calculate_suggestions\n" +
+      "  apply_suggestions\n" +
+      "  solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}\n" +
+      "  solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}\n" +
+      "loop_end\n" +
+      "loop_start iterations=${justCalc}\n" +
+      "  calculate_suggestions\n" +
+      "loop_end\n" +
+      "dump redact=true";
+
+  @Test
+  public void testSuggestions() throws Exception {
+    try (SimScenario scenario = SimScenario.load(testSuggestionsScenario)) {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      PrintStream ps = new PrintStream(baos, true, "UTF-8");
+      scenario.console = ps;
+      scenario.context.put("iterative", "0");
+      scenario.context.put("justCalc", "1");
+      scenario.run();
+      List<Suggester.SuggestionInfo> suggestions = (List<Suggester.SuggestionInfo>)scenario.context.get(SimScenario.SUGGESTIONS_CTX_PROP);
+      assertNotNull(suggestions);
+      assertEquals(suggestions.toString(), 1, suggestions.size());
+      // reconstruct the snapshot from the dump
+      Map<String, Object> snapshot = (Map<String, Object>)Utils.fromJSON(baos.toByteArray());
+      Map<String, Object> autoscalingState = (Map<String, Object>)snapshot.get(SnapshotCloudManager.AUTOSCALING_STATE_KEY);
+      assertNotNull(autoscalingState);
+      assertEquals(autoscalingState.toString(), 1, autoscalingState.size());
+      assertTrue(autoscalingState.toString(), autoscalingState.containsKey("suggestions"));
+      List<Map<String, Object>> snapSuggestions = (List<Map<String, Object>>)autoscalingState.get("suggestions");
+      assertEquals(snapSuggestions.toString(), 1, snapSuggestions.size());
+      // _loop_iter_ should be present and 0 (first iteration)
+      assertEquals(0, scenario.context.get(SimScenario.LOOP_ITER_PROP));
+    }
+    // try looping more times
+    try (SimScenario scenario = SimScenario.load(testSuggestionsScenario)) {
+      scenario.context.put("iterative", "10");
+      scenario.context.put("justCalc", "0");
+      scenario.run();
+      assertEquals(9, scenario.context.get(SimScenario.LOOP_ITER_PROP));
+    }
+
+  }
+
+  String indexingScenario =
+      "create_cluster numNodes=100\n" +
+      "solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=2\n" +
+      "wait_collection collection=testCollection&shards=2&replicas=2\n" +
+      "solr_request /admin/autoscaling?httpMethod=POST&stream.body=" +
+          "{'set-trigger':{'name':'indexSizeTrigger','event':'indexSize','waitFor':'10s','aboveDocs':1000,'enabled':true,"+
+          "'actions':[{'name':'compute_plan','class':'solr.ComputePlanAction'},{'name':'execute_plan','class':'solr.ExecutePlanAction'}]}}\n" +
+      "event_listener trigger=indexSizeTrigger&stage=SUCCEEDED\n" +
+      "index_docs collection=testCollection&numDocs=3000\n" +
+      "run\n" +
+      "wait_event trigger=indexSizeTrigger&wait=60\n" +
+      "assert condition=not_null&key=_trigger_event_indexSizeTrigger\n" +
+      "assert condition=equals&key=_trigger_event_indexSizeTrigger/eventType&expected=INDEXSIZE\n" +
+      "assert condition=equals&key=_trigger_event_indexSizeTrigger/properties/requestedOps[0]/action&expected=SPLITSHARD\n" +
+      "wait_collection collection=testCollection&shards=6&withInactive=true&requireLeaders=false&replicas=2";
+
+  @Test
+  public void testIndexing() throws Exception {
+    try (SimScenario scenario = SimScenario.load(indexingScenario)) {
+      scenario.run();
+    }
+  }
+}
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
index 5721e01..cc3373c 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
@@ -499,3 +499,143 @@ Number of iterations of the simulation loop. Default is 10.
 
 Results of the simulation contain the initial suggestions, suggestions at each step of the
 simulation and the final simulated state of the cluster.
+
+=== Simulation scenario tool
+The autoscaling command-line tool supports also the execution of end-to-end simulation scenarios consisting of
+several cluster- and collection-level operations and events.
+
+This tool can be invoked using `bin/solr autoscaling -scenario <FILE>`. All other command-line options are ignored in this mode.
+
+The file describing a scenario to test uses a simple plain text (UTF-8 encoded) line-oriented format, where
+each line of text uses the following syntax:
+
+[source,text]
+----
+line := command whitespace params | '#'
+params := [ path, '?' ] key, '=', value { '&', key, '=', value } *
+----
+
+Keys and values additionally use www-urlencoded format to avoid meta-characters and non-ascii characters.
+
+The `params` part of the line closely follows a regular Solr params representation on purpose - in many cases
+the content of this part of the command is passed directly to the respective collection- or cluster-level API.
+
+==== Scenario context
+Scenario has a context, which is simply a map of key-value pairs. Before executing each command the context is
+updated to contain the current values for the following properties:
+
+* `_random_node_` - randomly selected node name, or null if no node is live
+* `_overseer_leader_` - node name of the current Overseer leader node, or absent if there's no Overseer
+* `_live_nodes_` - a list of current live nodes, or absent if there are no live nodes
+* `_collections_` - a list of existing collections, or absent if there are no collections (or no live nodes)
+* `_suggestions_` - a list of autoscaling suggestions generated using CREATE_SUGGESTIONS command.
+* `_responses_` - a list of SolrResponse-s resulting from SOLR_REQUEST commands.
+* `_loop_iter_` - current loop iteration, or absent outside of loop.
+* `_trigger_event_<triggerName>` - last trigger event captured by WAIT_EVENT
+
+Command parameters support variable expansion using string values from the current context (non-string values, including numeric, are ignored)
+and from system properties, with the context values taking precedence if set.
+
+For example, assuming a system property is set 'foo=bar', the following command will load a snapshot from
+`/tmp/bar`:
+[source,text]
+----
+load_snapshot path=/tmp/${foo}
+----
+
+==== Scenario commands
+The following commands are supported (command names are case insensitive, but params aren't):
+
+* `create_cluster numNodes=N[&disableMetricsHistory=false&timeSourcee=simTime:50]` - create a simulated cluster with N nodes
+* `load_snapshot (path=/some/path | zkHost=ZK_CONNECT_STRING)` - create a simulated cluster from an autoscaling snapshot or from a live cluster.
+* `save_snapshot path=/some/path[&redact=false]` - save an autoscaling snapshot of the current simulated cluster state.
+* `calculate_suggestions` - calculate autoscaling suggestions based on the current cluster state and the policy.
+* `apply_suggestions` - apply previously calculated suggestions.
+* `kill_nodes (numNodes=N | nodes=node1,node2,...)` - kill a number of randomly selected nodes, or specific nodes.
+* `add_nodes numNodes=N` - add a number of new nodes.
+* `load_autoscaling (path=/some/path | json={...}` - load `autoscaling.json` config from a path or from the supplied JSON string, and apply this config to the simulated cluster.
+* `loop_start [iterations=N]`, `loop_end` - iterate commands enclosed in `loop_start` / `loop_end` N times, or until a loop abort is requested.
+* `set_op_delays op1=delayMs1&op2=delayMs2...` - set operation delays for specific collection commands to simulate slow execution.
+* `solr_request /admin/handler?httpMethod=POST&stream.body={'json':'body'}&other=params` - execute one of SolrRequest types supported by `SimCloudManager`.
+* `run [time=60000]` - run the simulator for some time, allowing background tasks to execute (eg. trigger event processing).
+* `wait_collection collection=test&shards=N&replicas=M[&withInactive=false&requireLeaders=true&wait=90]` - wait until the collection shape matches the criteria or the wait time elapses (in which case an error is thrown).
+* `event_listener trigger=triggerName&stage=SUCCEEDED[&beforeAction=foo | &afterAction=bar]` - prepare to listen for a specific trigger event.
+* `wait_event trigger=triggerName[&wait=90]` - wait until an event specified in `event_listener` is captured or a wait time elapses (in which cases an error is thrown).
+* `ctx_set key=myKey&value=myValue` - set a key / value pair in the scenario's context.
+* `ctx_remove key=myKey` - remove a key / value pair from the scenario's context.
+* `dump [redact=false&withData=false&withStats=false&withSuggestions=false&withDiagnostics=false&withNodeState=false&withClusterState=false&withManagerState=false]` - dump the simulator state to the console.
+* `set_node_metrics nodeset=node1,node2...&aKey1=aValue1&aKey2=aValue2...` - set node metrics.
+* `set_shard_metrics collection=test&shard=shard1[&delta=false&divide=false]&aKey1=aValue1&aKey2=aValue2...` - set per-shard metrics, optionally expressed as delta change from existing values and optionally with the values divided across existing replicas for a shard.
+* `index_docs numDocs=NNN[&start=XXX]` - simulate bulk indexing of a large number of documents.
+* `assert condition=(equals | not_equals | null | not_null)&(key=objectPath | value=myValue)[&expected=value]` - assert a condition. When `key` is specified then it can be an object path to complex values present in the scenario's context.
+
+==== Example scenarios
+Example scenario testing the behavior of `.autoAddReplicas` trigger:
+[source,text]
+----
+# standard comment
+// java comment
+create_cluster numNodes=2 // inline comment
+// load autoscaling config from a json string. Notice that the value must be URL-encoded
+load_autoscaling json={'cluster-policy'+:+[{'replica'+:+'<3',+'shard'+:+'#EACH',+'collection'+:+'testCollection','node':'#ANY'}]}&defaultWaitFor=10
+solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=2
+wait_collection collection=testCollection&shards=2&replicas=2
+// prepare a listener for trigger events and the processing state SUCCEEDED
+event_listener trigger=.auto_add_replicas&stage=SUCCEEDED
+// kill a random node
+kill_nodes node=${_random_node_}
+// wait for the listener to capture the event
+wait_event trigger=.auto_add_replicas&wait=60
+// the collection should have the same shape as before
+wait_collection collection=testCollection&shards=2&replicas=2
+save_snapshot path=${snapshotPath}
+----
+
+Example scenario testing the behavior of `indexSize` trigger. Notice the use of POST SolrRequest and the use of
+`assert` command with an object path:
+
+[source,text]
+----
+create_cluster numNodes=100
+solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=2
+wait_collection collection=testCollection&shards=2&replicas=2
+// example of defining a trigger config
+solr_request /admin/autoscaling?httpMethod=POST&stream.body={'set-trigger':{'name':'indexSizeTrigger','event':'indexSize','waitFor':'10s','aboveDocs':1000,'enabled':true,'actions':[{'name':'compute_plan','class':'solr.ComputePlanAction'},{'name':'execute_plan','class':'solr.ExecutePlanAction'}]}}
+// prepare an event listener
+event_listener trigger=indexSizeTrigger&stage=SUCCEEDED
+// add documents
+index_docs collection=testCollection&numDocs=3000
+// run for 60 sec
+run
+// wait for a trigger event (as defined in the listener)
+wait_event trigger=indexSizeTrigger&wait=60
+// even is stored in the context
+assert condition=not_null&key=_trigger_event_indexSizeTrigger
+assert condition=equals&key=_trigger_event_indexSizeTrigger/eventType&expected=INDEXSIZE
+assert condition=equals&key=_trigger_event_indexSizeTrigger/properties/requestedOps[0]/action&expected=SPLITSHARD
+wait_collection collection=testCollection&shards=6&withInactive=true&requireLeaders=false&replicas=2
+----
+
+Example scenario where context variables are used for conditional execution of loops. Depending on the value of
+`iterative` and `justCalc` the two loops will execute 0 or more times. Notice also how the scenario picks up
+a random node to consistently add replicas to it.
+
+[source,text]
+----
+create_cluster numNodes=2
+solr_request /admin/collections?action=CREATE&autoAddReplicas=true&name=testCollection&numShards=2&replicationFactor=2&maxShardsPerNode=10
+wait_collection collection=testCollection&shards=2&replicas=2
+ctx_set key=myNode&value=${_random_node_}
+solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}
+solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}
+loop_start iterations=${iterative}
+  calculate_suggestions
+  apply_suggestions
+  solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}
+  solr_request /admin/collections?action=ADDREPLICA&collection=testCollection&shard=shard1&node=${myNode}
+loop_end
+loop_start iterations=${justCalc}
+  calculate_suggestions
+loop_end
+dump redact=true
+----
\ No newline at end of file
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index a1f507c..a2a699e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -331,7 +331,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
           }
         }
         
-        if (cnt == retries) {
+        if (cnt == retries || rsp == null) {
           throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get remote info after many retries on NoHttpResponseException");
         }