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 2018/01/22 11:45:31 UTC

[1/2] lucene-solr:branch_7x: SOLR-11730: Collect more stats in the benchmark. Add simulation framework package docs.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 194e56180 -> 6752e4c72


SOLR-11730: Collect more stats in the benchmark. Add simulation framework package docs.


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

Branch: refs/heads/branch_7x
Commit: 6752e4c72f8f98c6ddca2669e4ac34aa93b19294
Parents: 3c1163c
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 8 15:08:17 2018 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 22 11:21:08 2018 +0100

----------------------------------------------------------------------
 .../cloud/autoscaling/sim/SimCloudManager.java  | 17 +++++
 .../autoscaling/sim/SimSolrCloudTestCase.java   | 15 +---
 .../cloud/autoscaling/sim/TestLargeCluster.java | 68 ++++++++++++++---
 .../cloud/autoscaling/sim/package-info.java     | 79 +++++++++++++++++++-
 4 files changed, 152 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6752e4c7/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index 1433fa9..2310a14 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -26,10 +26,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.solr.client.solrj.SolrClient;
@@ -305,6 +307,21 @@ public class SimCloudManager implements SolrCloudManager {
     return systemColl;
   }
 
+  public Map<String, Map<String, AtomicInteger>> simGetEventCounts() {
+    TreeMap<String, Map<String, AtomicInteger>> counts = new TreeMap<>();
+    synchronized (systemColl) {
+      for (SolrInputDocument d : systemColl) {
+        if (!"autoscaling_event".equals(d.getFieldValue("type"))) {
+          continue;
+        }
+        counts.computeIfAbsent((String)d.getFieldValue("event.source_s"), s -> new TreeMap<>())
+            .computeIfAbsent((String)d.getFieldValue("stage_s"), s -> new AtomicInteger())
+            .incrementAndGet();
+      }
+    }
+    return counts;
+  }
+
   /**
    * Get a {@link SolrClient} implementation where calls are forwarded to this
    * instance of the cluster.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6752e4c7/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
index 48d46b6..122ff2c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -35,7 +35,6 @@ import java.util.function.Predicate;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
-import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.DocCollection;
@@ -137,19 +136,7 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
       log.info("######### Final Solr op counts ##########");
       cluster.simGetOpCounts().forEach((k, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-14s  %4d", k, cnt.get())));
       log.info("######### Autoscaling event counts ###########");
-      TreeMap<String, Map<String, AtomicInteger>> counts = new TreeMap<>();
-
-      List<SolrInputDocument> solrInputDocuments = cluster.simGetSystemCollection();
-      synchronized (solrInputDocuments) {
-        for (SolrInputDocument d : solrInputDocuments) {
-          if (!"autoscaling_event".equals(d.getFieldValue("type"))) {
-            continue;
-          }
-          counts.computeIfAbsent((String)d.getFieldValue("event.source_s"), s -> new TreeMap<>())
-              .computeIfAbsent((String)d.getFieldValue("stage_s"), s -> new AtomicInteger())
-              .incrementAndGet();
-        }
-      }
+      Map<String, Map<String, AtomicInteger>> counts = cluster.simGetEventCounts();
       counts.forEach((trigger, map) -> {
         log.info("## * Trigger: " + trigger);
         map.forEach((s, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-11s  %4d", s, cnt.get())));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6752e4c7/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
index 15cb226..3adf652 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -30,6 +30,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -61,6 +62,7 @@ import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAut
 /**
  *
  */
+@TimeoutSuite(millis = 4 * 3600 * 1000)
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
 public class TestLargeCluster extends SimSolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -315,6 +317,11 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
   private static final int[] renard5x = new int[] {
       1, 2, 3, 4, 6,
       10, 16, 25, 40, 63,
+      100
+  };
+  private static final int[] renard5xx = new int[] {
+      1, 2, 3, 4, 6,
+      10, 16, 25, 40, 63,
       100, 158, 251, 398, 631,
       1000, 1585, 2512, 3981, 6310,
       10000
@@ -330,12 +337,18 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
       100
   };
 
+  private static final AtomicInteger ZERO = new AtomicInteger(0);
+
   //@Test
   public void benchmarkNodeLost() throws Exception {
     List<String> results = new ArrayList<>();
-    for (int wait : renard5) {
-      for (int delay : renard5) {
-        SummaryStatistics stat = new SummaryStatistics();
+    for (int wait : renard5x) {
+      for (int delay : renard5x) {
+        SummaryStatistics totalTime = new SummaryStatistics();
+        SummaryStatistics ignoredOurEvents = new SummaryStatistics();
+        SummaryStatistics ignoredOtherEvents = new SummaryStatistics();
+        SummaryStatistics startedOurEvents = new SummaryStatistics();
+        SummaryStatistics startedOtherEvents = new SummaryStatistics();
         for (int i = 0; i < 5; i++) {
           if (cluster != null) {
             cluster.close();
@@ -344,14 +357,29 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
           setUp();
           setupTest();
           long total = doTestNodeLost(wait, delay * 1000, 0);
-          stat.addValue(total);
+          totalTime.addValue(total);
+          // get event counts
+          Map<String, Map<String, AtomicInteger>> counts = cluster.simGetEventCounts();
+          Map<String, AtomicInteger> map = counts.remove("node_lost_trigger");
+          startedOurEvents.addValue(map.getOrDefault("STARTED", ZERO).get());
+          ignoredOurEvents.addValue(map.getOrDefault("IGNORED", ZERO).get());
+          int otherStarted = 0;
+          int otherIgnored = 0;
+          for (Map<String, AtomicInteger> m : counts.values()) {
+            otherStarted += m.getOrDefault("STARTED", ZERO).get();
+            otherIgnored += m.getOrDefault("IGNORED", ZERO).get();
+          }
+          startedOtherEvents.addValue(otherStarted);
+          ignoredOtherEvents.addValue(otherIgnored);
         }
-        results.add(String.format(Locale.ROOT, "%d\t%d\t%6.0f\t%6.0f\t%6.0f\t%6.0f\t%6.0f", wait, delay,
-            stat.getMin(), stat.getMax(), stat.getMean(), stat.getVariance(), stat.getStandardDeviation()));
+        results.add(String.format(Locale.ROOT, "%d\t%d\t%4.0f\t%4.0f\t%4.0f\t%4.0f\t%6.0f\t%6.0f\t%6.0f\t%6.0f\t%6.0f",
+            wait, delay, startedOurEvents.getMean(), ignoredOurEvents.getMean(),
+            startedOtherEvents.getMean(), ignoredOtherEvents.getMean(),
+            totalTime.getMin(), totalTime.getMax(), totalTime.getMean(), totalTime.getStandardDeviation(), totalTime.getVariance()));
       }
     }
     log.info("===== RESULTS ======");
-    log.info("waitFor\tkillDelay\tmin\tmax\tmean\tvar\tstdev");
+    log.info("waitFor\tdelay\tSTRT\tIGN\toSTRT\toIGN\tmin\tmax\tmean\tstdev\tvar");
     results.forEach(s -> log.info(s));
   }
 
@@ -364,6 +392,7 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
         "'waitFor' : '" + waitFor + "s'," +
         "'enabled' : true," +
         "'actions' : [" +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}," +
         "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
         "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}" +
         "]" +
@@ -391,6 +420,11 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
       cluster.simRemoveNode(nodes.get(i), false);
       cluster.getTimeSource().sleep(killDelay);
     }
+    // wait for the trigger to fire
+    boolean await = triggerFiredLatch.await(10 * waitFor * 1000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("trigger did not fire within timeout, " +
+        "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+        await);
     List<SolrInputDocument> systemColl = cluster.simGetSystemCollection();
     int startedEventPos = -1;
     for (int i = 0; i < systemColl.size(); i++) {
@@ -404,7 +438,9 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
         break;
       }
     }
-    assertTrue("no STARTED event: " + systemColl, startedEventPos > -1);
+    assertTrue("no STARTED event: " + systemColl + ", " +
+            "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+          startedEventPos > -1);
     SolrInputDocument startedEvent = systemColl.get(startedEventPos);
     int ignored = 0;
     int lastIgnoredPos = startedEventPos;
@@ -420,9 +456,13 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
         }
       }
     }
-    assertTrue("should be at least " + minIgnored + " IGNORED events", ignored >= minIgnored);
+    assertTrue("should be at least " + minIgnored + " IGNORED events, " +
+            "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+            ignored >= minIgnored);
     // make sure some replicas have been moved
-    assertTrue("no MOVEREPLICA ops?", cluster.simGetOpCount("MOVEREPLICA") > 0);
+    assertTrue("no MOVEREPLICA ops? " +
+            "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+            cluster.simGetOpCount("MOVEREPLICA") > 0);
 
     log.info("Ready after " + waitForState(collectionName, 20 * NUM_NODES, TimeUnit.SECONDS, clusterShape(NUM_NODES / 5, NUM_NODES / 10)) + " ms");
 
@@ -450,12 +490,16 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
       }
     }
 
-    assertTrue("did not finish processing changes", finishedEvent != null);
+    assertTrue("did not finish processing changes, " +
+            "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+            finishedEvent != null);
     long delta = (Long)finishedEvent.getFieldValue("event.time_l") - (Long)startedEvent.getFieldValue("event.time_l");
     delta = TimeUnit.NANOSECONDS.toMillis(delta);
     log.info("#### System stabilized after " + delta + " ms");
     long ops = cluster.simGetOpCount("MOVEREPLICA");
-    assertTrue("unexpected number of MOVEREPLICA ops: " + ops, ops >= 40);
+    assertTrue("unexpected number of MOVEREPLICA ops: " + ops + ", " +
+            "waitFor=" + waitFor + ", killDelay=" + killDelay + ", minIgnored=" + minIgnored,
+            ops >= 40);
     return delta;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6752e4c7/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
index 0b412cb..8c1ee86 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
@@ -15,7 +15,84 @@
  * limitations under the License.
  */
 /**
- * Simulated environment for autoscaling tests.
+ * <h1>Simulated environment for autoscaling tests.</h1>
+ *
+ * <h2>Goals</h2>
+ *  <ul>
+ *    <li>Use the actual unchanged autoscaling code for cluster state monitoring and autoscaling plan execution.</li>
+ *    <li>Support testing large clusters (&gt; 100 nodes).</li>
+ *    <li>Support fast testing using accelerated time (eg. 100x faster).</li>
+ *    <li>Support enough of other Solr functionality for the test results to be meaningful.</li>
+ *  </ul>
+ *
+ *  <h2>Simulated SolrCloudManager - {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager}</h2>
+ *  This implementation of {@link org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager}
+ *  uses the following simulated components:
+ *  <ul>
+ *     <li>{@link org.apache.solr.cloud.autoscaling.sim.SimDistribStateManager} - in-memory ZK look-alike, with support for Watcher-s, ephemeral and sequential nodes.</li>
+ *     <li>{@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider} - manages collection, replica infos, states and replica metrics.</li>
+ *     <li>{@link org.apache.solr.cloud.autoscaling.sim.SimNodeStateProvider} - manages node metrics.</li>
+ *     <li>{@link org.apache.solr.cloud.autoscaling.sim.GenericDistributedQueue} - DistributedQueue that uses SimDistribStateManager.</li>
+ *  </ul>
+ *  SimCloudManager also maintains an up-to-date /live_nodes in SimDistribStateManager, provides a SolrClient instance for use in tests,
+ *  and provides several convenience methods for setting up simulated clusters, populating node and replica metrics, collecting
+ *  autoscaling-related event history, collecting autoscaling event statistics, etc.
+ *
+ *  SimCloudManager runs actual {@link org.apache.solr.cloud.autoscaling.OverseerTriggerThread} so that it
+ *  uses real trigger and trigger action implementations, as well as real event scheduling and processing code.
+ *  It also provides methods for simulating Overseer leader change.
+ *
+ *  An important part of the SimCloudManager is also a request handler that processes common autoscaling
+ *  and collection admin requests. Autoscaling requests are processes by an instance of
+ *  {@link org.apache.solr.cloud.autoscaling.AutoScalingHandler} (and result in changes in respective
+ *  data stored in {@link org.apache.solr.cloud.autoscaling.sim.SimDistribStateManager}). Collection
+ *  admin commands are simulated, ie. they don't use actual {@link org.apache.solr.handler.admin.CollectionsHandler}
+ *  due to the complex dependencies on real components.
+ *
+ *  <h2>{@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider}</h2>
+ *  This components maintains collection and replica states:
+ *  <ul>
+ *    <li>Simulates delays between request and the actual cluster state changes</li>
+ *    <li>Marks replicas as down when a node goes down (optionally preserving the replica metrics in order to simulate a node coming back), and keeps track of per-node cores and disk space.</li>
+ *    <li>Runs a shard leader election look-alike on collection state updates.</li>
+ *    <li>Maintains up-to-date /clusterstate.json and /clusterprops.json in SimDistribStateManager (which in turn notifies Watcher-s about collection updates).
+ *    Currently for simplicity it uses the old single /clusterstate.json format for representing ClusterState.</li>
+ *  </ul>
+ *
+ *  <h2>{@link org.apache.solr.cloud.autoscaling.sim.SimNodeStateProvider}</h2>
+ *  This component maintains node metrics. When a simulated cluster is set up using eg.
+ *  {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager#createCluster(int, org.apache.solr.common.util.TimeSource)}
+ *  method, each simulated node is initialized with some basic metrics that are expected by the autoscaling
+ *  framework, such as node name, fake system load average, heap usage and disk usage.
+ *
+ *  The number of cores and disk space metrics may be used in autoscaling calculations, so they are
+ *  tracked and adjusted by {@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider} according
+ *  to the currently active replicas located on each node.
+ *
+ *  <h2>Limitations of the simulation framework</h2>
+ *  Currently the simulation framework is limited to testing the core autoscaling API in a single JVM.
+ *  Using it for other purposes would require extensive modifications in Solr and in the framework code.
+ *
+ *  Specifically, the framework supports testing the following autoscaling components:
+ *  <ul>
+ *    <li>OverseerTriggerThread and components that it uses.</li>
+ *    <li>Autoscaling config, triggers, trigger listeners, ScheduledTriggers, trigger event queues, ComputePlanAction / ExecutePlanAction, etc.</li>
+ *  </ul>
+ *  Overseer and CollectionsHandler Cmd implementations are NOT used, so cannot be properly tested - some of their functionality is simulated.
+ *  Other SolrCloud components make too many direct references to ZkStateReader, or direct HTTP requests, or rely on too many other components and require much more complex functionality - they may be refactored later but the effort may be too high.
+ *
+ *  Simulation framework definitely does not support the following functionality:
+ *  <ul>
+ *    <li>Solr searching and indexing</li>
+ *    <li>Any component that uses ZkController (eg. CoreContainer)</li>
+ *    <li>Any component that uses ShardHandler (eg. CollectionsHandler Cmd-s)</li>
+ *  </ul>
+ *
+ *  <h2>Testing framework</h2>
+ *  A base class {@link org.apache.solr.cloud.autoscaling.sim.SimSolrCloudTestCase} is provided, which
+ *  provides similar helper methods to the ones in the {@link org.apache.solr.cloud.SolrCloudTestCase}.
+ *
  */
 package org.apache.solr.cloud.autoscaling.sim;
 
+


[2/2] lucene-solr:branch_7x: SOLR-11730 Add a nodeLost benchmark.

Posted by ab...@apache.org.
SOLR-11730 Add a nodeLost benchmark.


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

Branch: refs/heads/branch_7x
Commit: 3c1163cf0a14b2f17e08cc5a31a6bb6dc7659289
Parents: 194e561
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Wed Jan 3 13:27:55 2018 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 22 11:21:08 2018 +0100

----------------------------------------------------------------------
 .../cloud/autoscaling/sim/TestLargeCluster.java | 64 ++++++++++++++++++--
 1 file changed, 60 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c1163cf/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
index 6758987..15cb226 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -29,6 +30,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
@@ -302,12 +304,64 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
 
   @Test
   public void testNodeLost() throws Exception {
+    doTestNodeLost(waitForSeconds, 5000, 1);
+  }
+
+  // Renard R5 series - evenly covers a log10 range
+  private static final int[] renard5 = new int[] {
+      1, 2, 3, 4, 6,
+      10
+  };
+  private static final int[] renard5x = new int[] {
+      1, 2, 3, 4, 6,
+      10, 16, 25, 40, 63,
+      100, 158, 251, 398, 631,
+      1000, 1585, 2512, 3981, 6310,
+      10000
+  };
+  // Renard R10 series
+  private static final double[] renard10 = new double[] {
+      1, 1.3, 1.6, 2, 2.5, 3.2, 4, 5, 6.3, 7.9,
+      10
+  };
+  private static final double[] renard10x = new double[] {
+      1, 1.3, 1.6, 2, 2.5, 3.2, 4, 5, 6.3, 7.9,
+      10, 12.6, 15.8, 20, 25.1, 31.6, 39.8, 50.1, 63.1, 79.4,
+      100
+  };
+
+  //@Test
+  public void benchmarkNodeLost() throws Exception {
+    List<String> results = new ArrayList<>();
+    for (int wait : renard5) {
+      for (int delay : renard5) {
+        SummaryStatistics stat = new SummaryStatistics();
+        for (int i = 0; i < 5; i++) {
+          if (cluster != null) {
+            cluster.close();
+          }
+          setupCluster();
+          setUp();
+          setupTest();
+          long total = doTestNodeLost(wait, delay * 1000, 0);
+          stat.addValue(total);
+        }
+        results.add(String.format(Locale.ROOT, "%d\t%d\t%6.0f\t%6.0f\t%6.0f\t%6.0f\t%6.0f", wait, delay,
+            stat.getMin(), stat.getMax(), stat.getMean(), stat.getVariance(), stat.getStandardDeviation()));
+      }
+    }
+    log.info("===== RESULTS ======");
+    log.info("waitFor\tkillDelay\tmin\tmax\tmean\tvar\tstdev");
+    results.forEach(s -> log.info(s));
+  }
+
+  private long doTestNodeLost(int waitFor, long killDelay, int minIgnored) throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     String setTriggerCommand = "{" +
         "'set-trigger' : {" +
         "'name' : 'node_lost_trigger'," +
         "'event' : 'nodeLost'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'waitFor' : '" + waitFor + "s'," +
         "'enabled' : true," +
         "'actions' : [" +
         "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
@@ -335,7 +389,7 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
       // this may also select a node where a replica is moved to, so the total number of
       // MOVEREPLICA may vary
       cluster.simRemoveNode(nodes.get(i), false);
-      cluster.getTimeSource().sleep(4000);
+      cluster.getTimeSource().sleep(killDelay);
     }
     List<SolrInputDocument> systemColl = cluster.simGetSystemCollection();
     int startedEventPos = -1;
@@ -366,7 +420,7 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
         }
       }
     }
-    assertTrue("no IGNORED events", ignored > 0);
+    assertTrue("should be at least " + minIgnored + " IGNORED events", ignored >= minIgnored);
     // make sure some replicas have been moved
     assertTrue("no MOVEREPLICA ops?", cluster.simGetOpCount("MOVEREPLICA") > 0);
 
@@ -398,9 +452,11 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
 
     assertTrue("did not finish processing changes", finishedEvent != null);
     long delta = (Long)finishedEvent.getFieldValue("event.time_l") - (Long)startedEvent.getFieldValue("event.time_l");
-    log.info("#### System stabilized after " + TimeUnit.NANOSECONDS.toMillis(delta) + " ms");
+    delta = TimeUnit.NANOSECONDS.toMillis(delta);
+    log.info("#### System stabilized after " + delta + " ms");
     long ops = cluster.simGetOpCount("MOVEREPLICA");
     assertTrue("unexpected number of MOVEREPLICA ops: " + ops, ops >= 40);
+    return delta;
   }
 
   @Test