You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2019/01/08 17:30:29 UTC

lucene-solr:branch_8x: SOLR-13118: Fix various nodeAdded/nodeLost trigger (integration) tests related to restoriung state

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_8x a5972cedf -> 5a60c3e0d


SOLR-13118: Fix various nodeAdded/nodeLost trigger (integration) tests related to restoriung state

This includes some cleanup and refactoring of unrelated test methods in the same classes to use new helper methods

(cherry picked from commit 5a513fab8345cd0397435e7ce830268cd3763651)


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

Branch: refs/heads/branch_8x
Commit: 5a60c3e0db26ec3dba119d6a6a44facf2089c77d
Parents: a5972ce
Author: Chris Hostetter <ho...@apache.org>
Authored: Mon Jan 7 17:53:04 2019 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Tue Jan 8 10:18:27 2019 -0700

----------------------------------------------------------------------
 .../cloud/autoscaling/ScheduledTriggers.java    |  13 +
 .../solr/cloud/autoscaling/TriggerBase.java     |  13 +-
 .../org/apache/solr/cloud/CloudTestUtils.java   |  23 +-
 .../NodeAddedTriggerIntegrationTest.java        | 246 +++---
 .../NodeLostTriggerIntegrationTest.java         | 262 ++++---
 .../cloud/autoscaling/sim/SimCloudManager.java  |  16 +-
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  12 +
 .../sim/TestSimTriggerIntegration.java          | 746 ++++++++++---------
 8 files changed, 759 insertions(+), 572 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 05a8020..768d8a7 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
@@ -511,6 +511,19 @@ public class ScheduledTriggers implements Closeable {
     return Collections.unmodifiableSet(new HashSet<>(scheduledTriggerWrappers.keySet())); // shallow copy
   }
 
+  /**
+   * For use in white/grey box testing: The Trigger returned may be inspected, 
+   * but should not be modified in any way.
+   *
+   * @param name the name of an existing trigger
+   * @return the current scheduled trigger with that name, or null if none exists
+   * @lucene.internal
+   */
+  public synchronized AutoScaling.Trigger getTrigger(String name) {
+    TriggerWrapper w = scheduledTriggerWrappers.get(name);
+    return (null == w) ? null : w.trigger;
+  }
+  
   @Override
   public void close() throws IOException {
     synchronized (this) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
index 93fb353..46fd8e5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
@@ -222,9 +222,20 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
    */
   protected abstract void setState(Map<String,Object> state);
 
+  /**
+   * Returns an immutable deep copy of this trigger's state, suitible for saving.
+   * This method is public only for tests that wish to do grey-box introspection
+   *
+   * @see #getState
+   * @lucene.internal
+   */
+  public Map<String,Object> deepCopyState() {
+    return Utils.getDeepCopy(getState(), 10, false, true);
+  }
+  
   @Override
   public void saveState() {
-    Map<String,Object> state = Utils.getDeepCopy(getState(), 10, false, true);
+    Map<String,Object> state = deepCopyState();
     if (lastState != null && lastState.equals(state)) {
       // skip saving if identical
       return;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
index 498cdba..07a9219 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
@@ -200,6 +200,7 @@ public class CloudTestUtils {
    *
    * @param cloudManager current instance of {@link SolrCloudManager}
    * @param triggerName the name of the trigger we need to see sheduled in order to return successfully
+   * @see #suspendTrigger
    */
   public static long waitForTriggerToBeScheduled(final SolrCloudManager cloudManager,
                                                  final String triggerName)
@@ -230,17 +231,33 @@ public class CloudTestUtils {
    *
    * @param cloudManager current instance of {@link SolrCloudManager}
    * @param triggerName the name of the trigger to suspend.  This must already be scheduled.
+   * @see #assertAutoScalingRequest
+   * @see #waitForTriggerToBeScheduled
    */
   public static void suspendTrigger(final SolrCloudManager cloudManager,
                                     final String triggerName) throws IOException {
+    assertAutoScalingRequest(cloudManager, "{'suspend-trigger' : {'name' : '"+triggerName+"'} }");
+  }
+
+  /**
+   * Creates &amp; executes an autoscaling request against the current cluster, asserting that 
+   * the result is a success.
+   * 
+   * @param cloudManager current instance of {@link SolrCloudManager}
+   * @param json The request to POST to the AutoScaling Handler
+   * @see AutoScalingRequest#create
+   */
+  public static void assertAutoScalingRequest(final SolrCloudManager cloudManager,
+                                              final String json) throws IOException {
+    // TODO: a lot of code that directly uses AutoScalingRequest.create should use this method
     
-    final SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST,
-                                                      "{'suspend-trigger' : {'name' : '"+triggerName+"'} }");
+    final SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, json);
     final SolrResponse rsp = cloudManager.request(req);
     final String result = rsp.getResponse().get("result").toString();
-    Assert.assertEquals("Unexpected 'result' in response: " + rsp,
+    Assert.assertEquals("Unexpected result from auto-scaling command: " + json + " -> " + rsp,
                         "success", result);
   }
+
   
   /**
    * Helper class for sending (JSON) autoscaling requests that can randomize between V1 and V2 requests

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
index 9d798e2..cd4485e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
@@ -18,57 +18,51 @@
 package org.apache.solr.cloud.autoscaling;
 
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
-import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.autoscaling.ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
 import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
 
+// TODO: this class shares duplicated code with NodeLostTriggerIntegrationTest ... merge?
+
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static CountDownLatch actionConstructorCalled;
-  private static CountDownLatch actionInitCalled;
-  private static CountDownLatch triggerFiredLatch;
-  private static int waitForSeconds = 1;
-  private static AtomicBoolean triggerFired;
-  private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
-  private static SolrCloudManager cloudManager;
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  private static volatile CountDownLatch actionConstructorCalled;
+  private static volatile CountDownLatch actionInitCalled;
+  private static volatile CountDownLatch triggerFiredLatch;
+  private static volatile int waitForSeconds = 1;
+  private static volatile AtomicBoolean triggerFired;
+  private static volatile Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
+  private static volatile SolrCloudManager cloudManager;
 
-  }
-  
   @After 
   public void after() throws Exception {
     shutdownCluster();
@@ -84,23 +78,18 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
         .addConfig("conf", configset("cloud-minimal"))
         .configure();
     
+    final Overseer overseer = cluster.getOpenOverseer();
+    assertNotNull(overseer);
+    cloudManager = overseer.getSolrCloudManager();
+    assertNotNull(cloudManager);
+      
     // disable .scheduled_maintenance (once it exists)
-    CloudTestUtils.waitForTriggerToBeScheduled(cluster.getOpenOverseer().getSolrCloudManager(), ".scheduled_maintenance");
-    CloudTestUtils.suspendTrigger(cluster.getOpenOverseer().getSolrCloudManager(), ".scheduled_maintenance");
-
-    NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
-    String overseerLeader = (String) overSeerStatus.get("leader");
-    int overseerLeaderIndex = 0;
-    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
-      JettySolrRunner jetty = cluster.getJettySolrRunner(i);
-      if (jetty.getNodeName().equals(overseerLeader)) {
-        overseerLeaderIndex = i;
-        break;
-      }
-    }
-    Overseer overseer = cluster.getJettySolrRunner(overseerLeaderIndex).getCoreContainer().getZkController().getOverseer();
-    ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
+    CloudTestUtils.waitForTriggerToBeScheduled(cloudManager, ".scheduled_maintenance");
+    CloudTestUtils.suspendTrigger(cloudManager, ".scheduled_maintenance");
+
     // aggressively remove all active scheduled triggers
+    final ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
+    // TODO: is this really safe? is it possible overseer is still in process of adding some to schedule?
     scheduledTriggers.removeAll();
 
     // clear any persisted auto scaling configuration
@@ -116,7 +105,6 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
     triggerFired = new AtomicBoolean(false);
     events.clear();
 
-    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
     // clear any events or markers
     // todo: consider the impact of such cleanup on regular cluster restarts
     deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
@@ -131,78 +119,108 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
 
   @Test
   public void testNodeAddedTriggerRestoreState() throws Exception {
-    // for this test we want to update the trigger so we must assert that the actions were created twice
-    actionInitCalled = new CountDownLatch(2);
-
-    CloudSolrClient solrClient = cluster.getSolrClient();
-    waitForSeconds = 5;
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_restore_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '5s'," + // should be enough for us to update the trigger
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
-    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
-      Thread.sleep(200);
-    }
-    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+    
+    final String triggerName = "node_added_restore_trigger";
+
+    // should be enough to ensure trigger doesn't fire any actions until we replace the trigger
+    waitForSeconds = 500000;
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
 
     // start a new node
-    JettySolrRunner newNode = cluster.startJettySolrRunner();
+    final JettySolrRunner newNode = cluster.startJettySolrRunner();
+    final String nodeName = newNode.getNodeName();
+
+    // poll the internal state of the trigger until it run()s at least once and updates
+    // it's internal state to know the node we added is live
+    //
+    // (this should run roughly once a second)
+    (new TimeOut(30, TimeUnit.SECONDS, cloudManager.getTimeSource()))
+    .waitFor("initial trigger never ran to detect new live node", () ->
+             (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+              .contains(nodeName)));
     
-    cluster.waitForAllNodes(30);
-
-    // ensure that the old trigger sees the new node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
-
-    waitForSeconds = 0;
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_restore_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    // wait until the second instance of action is created
-    if (!actionInitCalled.await(3, TimeUnit.SECONDS)) {
-      fail("Two TriggerAction instances should have been created by now");
-    }
+    // since we know the nodeAdded event has been detected, we can recored the current timestamp
+    // (relative to the cluster's time source) and later assert that (restored state) correctly
+    // tracked that the event happened prior to "now"
+    final long maxEventTimeNs = cloudManager.getTimeSource().getTimeNs();
+    
+    //
+    // now replace the trigger with a new instance to test that the state gets copied over correctly
+    //
+    
+    // reset the actionInitCalled counter so we can confirm the second instances is inited
+    actionInitCalled = new CountDownLatch(1);
+    // use a low waitTime to ensure it processes the event quickly.
+    // (this updated property also ensures the set-trigger won't be treated as a No-Op)
+    waitForSeconds = 0 + random().nextInt(3);
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
 
-    boolean await = triggerFiredLatch.await(5, TimeUnit.SECONDS);
-    assertTrue("The trigger did not fire at all", await);
-    assertTrue(triggerFired.get());
-    NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) events.iterator().next();
-    assertNotNull(nodeAddedEvent);
-    List<String> nodeNames = (List<String>) nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
-    assertTrue(nodeNames.contains(newNode.getNodeName()));
+    // the trigger actions should now (eventually) record that the node was added
+    assertTrue("Second instance of our trigger never fired the action to process the event",
+               triggerFiredLatch.await(30, TimeUnit.SECONDS));
+    
+    assertEquals("Wrong number of events recorded: " + events.toString(),
+                 1, events.size());
+    
+    final TriggerEvent event = events.iterator().next();
+    assertNotNull("null event???", event);
+    assertTrue("Event should have been a nodeAdded event: " + event.getClass(),
+               event instanceof NodeAddedTrigger.NodeAddedEvent);
+
+    assertNotNull("event is missing NODE_NAMES: " + event, event.getProperty(TriggerEvent.NODE_NAMES));
+    assertEquals("event has incorrect NODE_NAMES: " + event,
+                 Collections.singletonList(nodeName),
+                 event.getProperty(TriggerEvent.NODE_NAMES));
+    
+    assertTrue("event TS is too late, should be before (max) expected TS @ "
+               + maxEventTimeNs + ": " + event,
+               event.getEventTime() < maxEventTimeNs);
+    
+    assertNotNull("event is missing EVENT_TIMES: " + event, event.getProperty(TriggerEvent.EVENT_TIMES));
+    assertEquals("event has unexpeted number of EVENT_TIMES: " + event,
+                 1, ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).size());
+    assertEquals("event's TS doesn't match EVENT_TIMES: " + event,
+                 event.getEventTime(),
+                 ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).iterator().next());
   }
 
   @Test
   public void testNodeAddedTrigger() throws Exception {
     CloudSolrClient solrClient = cluster.getSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     if (!actionInitCalled.await(3, TimeUnit.SECONDS)) {
       fail("The TriggerAction should have been created by now");
@@ -223,17 +241,16 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
     actionInitCalled = new CountDownLatch(1);
 
     // update the trigger with exactly the same data
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     // this should be a no-op so the action should have been created but init should not be called
     if (!actionConstructorCalled.await(3, TimeUnit.SECONDS)) {
@@ -277,4 +294,17 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
       super.init();
     }
   }
+  
+  /** 
+   * Helper method for getting a copy of the current (internal) trigger state of a scheduled trigger. 
+   */
+  private Map<String, Object> getTriggerState(final String name) {
+    final Overseer overseer = cluster.getOpenOverseer();
+    final ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
+    final AutoScaling.Trigger t = scheduledTriggers.getTrigger(name);
+    assertNotNull(name + " is not a currently scheduled trigger", t);
+    assertTrue(name + " is not a TriggerBase w/state: " + t.getClass(),
+               t instanceof TriggerBase);
+    return ((TriggerBase)t).deepCopyState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
index d0d1958..f88c448 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
@@ -18,20 +18,21 @@
 package org.apache.solr.cloud.autoscaling;
 
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
-import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -43,31 +44,26 @@ import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.autoscaling.ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
 import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
 
+// TODO: this class shares duplicated code with NodeAddedTriggerIntegrationTest ... merge?
+
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static CountDownLatch actionConstructorCalled;
-  private static CountDownLatch actionInitCalled;
-  private static CountDownLatch triggerFiredLatch;
-  private static int waitForSeconds = 1;
-  private static AtomicBoolean triggerFired;
-  private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
-  private static SolrCloudManager cloudManager;
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
- 
-  }
+  private static volatile CountDownLatch actionConstructorCalled;
+  private static volatile CountDownLatch actionInitCalled;
+  private static volatile CountDownLatch triggerFiredLatch;
+  private static volatile int waitForSeconds = 1;
+  private static volatile AtomicBoolean triggerFired;
+  private static volatile Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
+  private static volatile SolrCloudManager cloudManager;
 
   private static CountDownLatch getTriggerFiredLatch() {
     return triggerFiredLatch;
@@ -80,33 +76,26 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
     .addConfig("conf", configset("cloud-minimal"))
     .configure();
     
+    final Overseer overseer = cluster.getOpenOverseer();
+    assertNotNull(overseer);
+    cloudManager = overseer.getSolrCloudManager();
+    assertNotNull(cloudManager);
+      
     // disable .scheduled_maintenance (once it exists)
-    CloudTestUtils.waitForTriggerToBeScheduled(cluster.getOpenOverseer().getSolrCloudManager(), ".scheduled_maintenance");
-    CloudTestUtils.suspendTrigger(cluster.getOpenOverseer().getSolrCloudManager(), ".scheduled_maintenance");
+    CloudTestUtils.waitForTriggerToBeScheduled(cloudManager, ".scheduled_maintenance");
+    CloudTestUtils.suspendTrigger(cloudManager, ".scheduled_maintenance");
 
-    NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
-    String overseerLeader = (String) overSeerStatus.get("leader");
-    int overseerLeaderIndex = 0;
-    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
-      JettySolrRunner jetty = cluster.getJettySolrRunner(i);
-      if (jetty.getNodeName().equals(overseerLeader)) {
-        overseerLeaderIndex = i;
-        break;
-      }
-    }
-    Overseer overseer = cluster.getJettySolrRunner(overseerLeaderIndex).getCoreContainer().getZkController().getOverseer();
-    ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
     // aggressively remove all active scheduled triggers
+    final ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
+    // TODO: is this really safe? is it possible overseer is still in process of adding some to schedule?
     scheduledTriggers.removeAll();
 
     // clear any persisted auto scaling configuration
     Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
     log.info(SOLR_AUTOSCALING_CONF_PATH + " reset, new znode version {}", stat.getVersion());
 
-
     cluster.getSolrClient().setDefaultCollection(null);
 
-
     waitForSeconds = 1 + random().nextInt(3);
     actionConstructorCalled = new CountDownLatch(1);
     actionInitCalled = new CountDownLatch(1);
@@ -114,7 +103,6 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
     triggerFired = new AtomicBoolean(false);
     events.clear();
 
-    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
     // clear any events or markers
     // todo: consider the impact of such cleanup on regular cluster restarts
     deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
@@ -134,71 +122,114 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
 
   @Test
   public void testNodeLostTriggerRestoreState() throws Exception {
-    // for this test we want to update the trigger so we must assert that the actions were created twice
-    actionInitCalled = new CountDownLatch(2);
-
-    // start a new node
-    JettySolrRunner newNode = cluster.startJettySolrRunner();
-    String nodeName = newNode.getNodeName();
-
-    CloudSolrClient solrClient = cluster.getSolrClient();
-    waitForSeconds = 5;
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_restore_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '5s'," + // should be enough for us to update the trigger
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
-    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
-      Thread.sleep(200);
-    }
-    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
-
-    List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
-    int index = -1;
-    for (int i = 0; i < jettySolrRunners.size(); i++) {
-      JettySolrRunner runner = jettySolrRunners.get(i);
-      if (runner == newNode) index = i;
-    }
-    assertFalse(index == -1);
-    JettySolrRunner j = cluster.stopJettySolrRunner(index);
-    cluster.waitForJettyToStop(j);
-
-    // ensure that the old trigger sees the stopped node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
-
-    waitForSeconds = 0;
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_restore_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
 
-    // wait until the second instance of action is created
-    if (!actionInitCalled.await(3, TimeUnit.SECONDS)) {
-      fail("Two TriggerAction instances should have been created by now");
-    }
+    final String triggerName = "node_lost_restore_trigger";
 
-    boolean await = triggerFiredLatch.await(5, TimeUnit.SECONDS);
-    assertTrue("The trigger did not fire at all", await);
-    assertTrue(triggerFired.get());
-    NodeLostTrigger.NodeLostEvent nodeLostEvent = (NodeLostTrigger.NodeLostEvent) events.iterator().next();
-    assertNotNull(nodeLostEvent);
-    List<String> nodeNames = (List<String>) nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
-    assertTrue(nodeNames.contains(nodeName));
+    // start a new node
+    final JettySolrRunner newNode = cluster.startJettySolrRunner();
+    final String nodeName = newNode.getNodeName();
+
+    // should be enough to ensure trigger doesn't fire any actions until we replace the trigger
+    waitForSeconds = 500000;
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
+
+    // poll the internal state of the trigger until it run()s at least once and updates
+    // it's internal state to know the node we added is live
+    //
+    // (this should run roughly once a second)
+    (new TimeOut(30, TimeUnit.SECONDS, cloudManager.getTimeSource()))
+    .waitFor("initial trigger never ran to detect new live node", () ->
+             (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+              .contains(nodeName)));
+
+    // kill our node
+    cluster.stopJettySolrRunner(newNode);
+    cluster.waitForJettyToStop(newNode);
+
+    // poll the internal state of the trigger until it run()s at least once (more) and updates
+    // it's internal state to know the node we killed is no longer alive
+    //
+    // (this should run roughly once a second of simulated time)
+    (new TimeOut(30, TimeUnit.SECONDS, cloudManager.getTimeSource()))
+    .waitFor("initial trigger never ran to detect lost node", () ->
+             ! (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+                .contains(nodeName)));
+
+    // since we know the nodeLost event has been detected, we can recored the current timestamp
+    // (relative to the cluster's time source) and later assert that (restored state) correctly
+    // tracked that the event happened prior to "now"
+    final long maxEventTimeNs = cloudManager.getTimeSource().getTimeNs();
+    
+    // even though our trigger has detected a lost node, the *action* we registered should not have
+    // been run yet, due to the large waitFor configuration...
+    assertEquals("initial trigger action should not have fired", false, triggerFired.get());
+    assertEquals("initial trigger action latch should not have counted down",
+                 1, triggerFiredLatch.getCount());
+    assertEquals("initial trigger action should not have recorded any events: " + events.toString(),
+                 0, events.size());
+
+    //
+    // now replace the trigger with a new instance to test that the state gets copied over correctly
+    //
+    
+    // reset the actionInitCalled counter so we can confirm the second instances is inited
+    actionInitCalled = new CountDownLatch(1);
+    // use a low waitTime to ensure it processes the event quickly.
+    // (this updated property also ensures the set-trigger won't be treated as a No-Op)
+    waitForSeconds = 0 + random().nextInt(3);
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
+    
+    // the trigger actions should now (eventually) record that the node is lost
+    assertTrue("Second instance of our trigger never fired the action to process the event",
+               triggerFiredLatch.await(30, TimeUnit.SECONDS));
+    
+    assertEquals("Wrong number of events recorded: " + events.toString(),
+                 1, events.size());
+    
+    final TriggerEvent event = events.iterator().next();
+    assertNotNull("null event???", event);
+    assertTrue("Event should have been a nodeLost event: " + event.getClass(),
+               event instanceof NodeLostTrigger.NodeLostEvent);
+
+    assertNotNull("event is missing NODE_NAMES: " + event, event.getProperty(TriggerEvent.NODE_NAMES));
+    assertEquals("event has incorrect NODE_NAMES: " + event,
+                 Collections.singletonList(nodeName),
+                 event.getProperty(TriggerEvent.NODE_NAMES));
+    
+    assertTrue("event TS is too late, should be before (max) expected TS @ "
+               + maxEventTimeNs + ": " + event,
+               event.getEventTime() < maxEventTimeNs);
+    
+    assertNotNull("event is missing EVENT_TIMES: " + event, event.getProperty(TriggerEvent.EVENT_TIMES));
+    assertEquals("event has unexpeted number of EVENT_TIMES: " + event,
+                 1, ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).size());
+    assertEquals("event's TS doesn't match EVENT_TIMES: " + event,
+                 event.getEventTime(),
+                 ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).iterator().next());
   }
 
   @Test
@@ -221,9 +252,7 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
         nonOverseerLeaderIndex = i;
       }
     }
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    CloudTestUtils.assertAutoScalingRequest(cloudManager, setTriggerCommand); 
 
     if (!actionInitCalled.await(3, TimeUnit.SECONDS)) {
       fail("The TriggerAction should have been created by now");
@@ -247,17 +276,16 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
     actionInitCalled = new CountDownLatch(1);
 
     // update the trigger with exactly the same data
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    CloudTestUtils.assertAutoScalingRequest
+      (cloudManager,
+       "{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_lost_trigger'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     // this should be a no-op so the action should have been created but init should not be called
     if (!actionConstructorCalled.await(3, TimeUnit.SECONDS)) {
@@ -301,4 +329,18 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
       super.init();
     }
   }
+
+  /** 
+   * Helper method for getting a copy of the current (internal) trigger state of a scheduled trigger. 
+   */
+  private Map<String, Object> getTriggerState(final String name) {
+    final Overseer overseer = cluster.getOpenOverseer();
+    final ScheduledTriggers scheduledTriggers = ((OverseerTriggerThread) overseer.getTriggerThread().getThread()).getScheduledTriggers();
+    final AutoScaling.Trigger t = scheduledTriggers.getTrigger(name);
+    assertNotNull(name + " is not a currently scheduled trigger", t);
+    assertTrue(name + " is not a TriggerBase w/state: " + t.getClass(),
+               t instanceof TriggerBase);
+    return ((TriggerBase)t).deepCopyState();
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/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 20e778b..ba14fad 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
@@ -532,10 +532,12 @@ public class SimCloudManager implements SolrCloudManager {
   }
 
   /**
-   * Simulate the effect of restarting Overseer leader - in this case this means restarting the
-   * OverseerTriggerThread and optionally killing a node. All background tasks currently in progress
-   * will be interrupted.
+   * Simulate the effect of restarting Overseer leader - in this case this means closing the current
+   * {@link OverseerTriggerThread} (and optionally killing a node) then starting a new 
+   * {@link OverseerTriggerThread}.
+   * All background tasks currently in progress will be interrupted.
    * @param killNodeId optional nodeId to kill. If null then don't kill any node, just restart the thread
+   * @see #getOverseerTriggerThread
    */
   public void simRestartOverseer(String killNodeId) throws Exception {
     log.info("=== Restarting OverseerTriggerThread and clearing object cache...");
@@ -900,4 +902,12 @@ public class SimCloudManager implements SolrCloudManager {
     IOUtils.closeQuietly(objectCache);
     simCloudManagerPool.shutdownNow();
   }
+
+  /**
+   * Direct access to the current {@link OverseerTriggerThread}
+   * @see #simRestartOverseer
+   */
+  public OverseerTriggerThread getOverseerTriggerThread() {
+    return ((OverseerTriggerThread) triggerThread.getThread());
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/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 b593168..49a7bbd 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
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.function.Predicate;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
@@ -121,4 +122,15 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
     fail("Couldn't get random replica that matched conditions\n" + slice.toString());
     return null;  // just to keep the compiler happy - fail will always throw an Exception
   }
+
+  /**
+   * Creates &amp; executes an autoscaling request against the current cluster, asserting that 
+   * the result is a success
+   * 
+   * @param json The request to send
+   * @see CloudTestUtils#assertAutoScalingRequest
+   */
+  public void assertAutoScalingRequest(final String json) throws IOException {
+    CloudTestUtils.assertAutoScalingRequest(cluster, json);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a60c3e0/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
index 8dab189..d7e792e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
@@ -17,11 +17,10 @@
 
 package org.apache.solr.cloud.autoscaling.sim;
 
-import static org.apache.solr.cloud.autoscaling.ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS;
-
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -38,7 +37,6 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
@@ -46,15 +44,17 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
-import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
 import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
 import org.apache.solr.cloud.autoscaling.CapturedEvent;
 import org.apache.solr.cloud.autoscaling.ComputePlanAction;
 import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
+import org.apache.solr.cloud.autoscaling.NodeAddedTrigger;
 import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
 import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
 import org.apache.solr.cloud.autoscaling.SearchRateTrigger;
 import org.apache.solr.cloud.autoscaling.TriggerActionBase;
+import org.apache.solr.cloud.autoscaling.TriggerBase;
 import org.apache.solr.cloud.autoscaling.TriggerEvent;
 import org.apache.solr.cloud.autoscaling.TriggerEventQueue;
 import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
@@ -62,7 +62,6 @@ import org.apache.solr.cloud.autoscaling.TriggerValidationException;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.LiveNodesListener;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
@@ -158,30 +157,25 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     SolrClient solrClient = cluster.simGetSolrClient();
 
     // first trigger
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger1'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '0s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    // second trigger
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger2'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '0s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger1'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '0s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+       "}}");
+
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger2'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '0s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+       "}}");
 
     // wait until the two instances of action are created
     if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
@@ -199,29 +193,25 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     actionInitCalled = new CountDownLatch(2);
     triggerFiredLatch = new CountDownLatch(2);
 
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_trigger1'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '0s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_trigger2'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '0s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_lost_trigger1'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '0s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+       "}}");
+
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_lost_trigger2'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '0s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+       "}}");
 
     // wait until the two instances of action are created
     if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
@@ -277,140 +267,184 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  // commentted 190-Dec-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
   public void testNodeLostTriggerRestoreState() throws Exception {
-    // for this test we want to update the trigger so we must assert that the actions were created twice
-    actionInitCalled = new CountDownLatch(2);
-
-    // start a new node
-    String nodeName = cluster.simAddNode();
-
-    SolrClient solrClient = cluster.simGetSolrClient();
-    waitForSeconds = 5;
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_restore_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '5s'," + // should be enough for us to update the trigger
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
-    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
-      timeOut.sleep(200);
-    }
-    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
-
+    
+    final String triggerName = "node_lost_restore_trigger";
+      
+    // should be enough to ensure trigger doesn't fire any actions until we replace the trigger
+    waitForSeconds = 500000;
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
+    
+    // start a new node that we can kill later
+    final String nodeName = cluster.simAddNode();
+    
+    // poll the internal state of the trigger until it run()s at least once and updates
+    // it's internal state to know the node we added is live
+    //
+    // (this should run roughly once a second of simulated time)
+    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+    .waitFor("initial trigger never ran to detect new live node", () ->
+             (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+              .contains(nodeName)));
+    
+    // kill our node
     cluster.simRemoveNode(nodeName, false);
-
-    // ensure that the old trigger sees the stopped node, todo find a better way to do this
-    timeOut.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
-
-    waitForSeconds = 0;
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_restore_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    // wait until the second instance of action is created
-    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
-      fail("Two TriggerAction instances should have been created by now");
-    }
-
-    boolean await = triggerFiredLatch.await(90000 / SPEED, TimeUnit.MILLISECONDS);
-    assertTrue("The trigger did not fire at all", await);
-    assertTrue(triggerFired.get());
-    NodeLostTrigger.NodeLostEvent nodeLostEvent = (NodeLostTrigger.NodeLostEvent) events.iterator().next();
-    assertNotNull(nodeLostEvent);
-    List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
-    assertTrue(nodeNames.contains(nodeName));
+    
+    // poll the internal state of the trigger until it run()s at least once (more) and updates
+    // it's internal state to know the node we killed is no longer alive
+    //
+    // (this should run roughly once a second of simulated time)
+    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+    .waitFor("initial trigger never ran to detect lost node", () ->
+             ! (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+                .contains(nodeName)));
+    
+    // since we know the nodeLost event has been detected, we can recored the current timestamp
+    // (relative to the cluster's time source) and later assert that (restored state) correctly
+    // tracked that the event happened prior to "now"
+    final long maxEventTimeNs = cluster.getTimeSource().getTimeNs();
+    
+    // even though our trigger has detected a lost node, the *action* we registered should not have
+    // been run yet, due to the large waitFor configuration...
+    assertEquals("initial trigger action should not have fired", false, triggerFired.get());
+    assertEquals("initial trigger action latch should not have counted down",
+                 1, triggerFiredLatch.getCount());
+    assertEquals("initial trigger action should not have recorded any events: " + events.toString(),
+                 0, events.size());
+
+    //
+    // now replace the trigger with a new instance to test that the state gets copied over correctly
+    //
+    
+    // reset the actionInitCalled counter so we can confirm the second instances is inited
+    actionInitCalled = new CountDownLatch(1);
+    // use a low waitTime to ensure it processes the event quickly.
+    // (this updated property also ensures the set-trigger won't be treated as a No-Op)
+    waitForSeconds = 0 + random().nextInt(3);
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
+               
+    // the trigger actions should now (eventually) record that the node is lost
+    assertTrue("Second instance of our trigger never fired the action to process the event",
+               triggerFiredLatch.await(30, TimeUnit.SECONDS));
+    
+    final TriggerEvent event = assertSingleEvent(nodeName, maxEventTimeNs);
+    assertTrue("Event should have been a nodeLost event: " + event.getClass(),
+               event instanceof NodeLostTrigger.NodeLostEvent);
+    
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
   public void testNodeAddedTriggerRestoreState() throws Exception {
-    // for this test we want to update the trigger so we must assert that the actions were created twice
-    actionInitCalled = new CountDownLatch(2);
-
-    SolrClient solrClient = cluster.simGetSolrClient();
-    waitForSeconds = 5;
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_restore_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '5s'," + // should be enough for us to update the trigger
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
-    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
-      timeOut.sleep(200);
-    }
-    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+    
+    final String triggerName = "node_added_restore_trigger";
+      
+    // should be enough to ensure trigger doesn't fire any actions until we replace the trigger
+    waitForSeconds = 500000;
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
 
     // start a new node
-    String newNode = cluster.simAddNode();
-
-    // ensure that the old trigger sees the new node, todo find a better way to do this
-    cluster.getTimeSource().sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
-
-    waitForSeconds = 0;
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_restore_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    // wait until the second instance of action is created
-    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
-      fail("Two TriggerAction instances should have been created by now");
-    }
+    final String nodeName = cluster.simAddNode();
+
+    // poll the internal state of the trigger until it run()s at least once and updates
+    // it's internal state to know the node we added is live
+    //
+    // (this should run roughly once a second of simulated time)
+    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+    .waitFor("initial trigger never ran to detect new live node", () ->
+             (((Collection<String>) getTriggerState(triggerName).get("lastLiveNodes"))
+              .contains(nodeName)));
+
+    // since we know the nodeAddded event has been detected, we can recored the current timestamp
+    // (relative to the cluster's time source) and later assert that (restored state) correctly
+    // tracked that the event happened prior to "now"
+    final long maxEventTimeNs = cluster.getTimeSource().getTimeNs();
+    
+    // even though our trigger has detected an added node, the *action* we registered should not have
+    // been run yet, due to the large waitFor configuration...
+    assertEquals("initial trigger action should not have fired", false, triggerFired.get());
+    assertEquals("initial trigger action latch should not have counted down",
+                 1, triggerFiredLatch.getCount());
+    assertEquals("initial trigger action should not have recorded any events: " + events.toString(),
+                 0, events.size());
+
+    //
+    // now replace the trigger with a new instance to test that the state gets copied over correctly
+    //
+    
+    // reset the actionInitCalled counter so we can confirm the second instances is inited
+    actionInitCalled = new CountDownLatch(1);
+    // use a low waitTime to ensure it processes the event quickly.
+    // (this updated property also ensures the set-trigger won't be treated as a No-Op)
+    waitForSeconds = 0 + random().nextInt(3);
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : '"+triggerName+"'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '"+waitForSeconds+"s'," + 
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
+    assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
+               actionInitCalled.await(60, TimeUnit.SECONDS));
+    
+    // the trigger actions should now (eventually) record that the new node is added
+    assertTrue("Second instance of our trigger never fired the action to process the event",
+               triggerFiredLatch.await(30, TimeUnit.SECONDS));
+    
+    final TriggerEvent event = assertSingleEvent(nodeName, maxEventTimeNs);
+    assertTrue("Event should have been a nodeAdded event: " + event.getClass(),
+               event instanceof NodeAddedTrigger.NodeAddedEvent);
 
-    boolean await = triggerFiredLatch.await(60000 / SPEED, TimeUnit.MILLISECONDS);
-    assertTrue("The trigger did not fire at all", await);
-    assertTrue(triggerFired.get());
-    TriggerEvent nodeAddedEvent = events.iterator().next();
-    assertNotNull(nodeAddedEvent);
-    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
-    assertTrue(nodeNames.toString(), nodeNames.contains(newNode));
   }
 
   @Test
   @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18
   public void testNodeAddedTrigger() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
@@ -430,17 +464,15 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     actionInitCalled = new CountDownLatch(1);
 
     // update the trigger with exactly the same data
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     // this should be a no-op so the action should have been created but init should not be called
     if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
@@ -455,17 +487,15 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testNodeLostTrigger() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_lost_trigger'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
@@ -486,18 +516,16 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     actionInitCalled = new CountDownLatch(1);
 
     // update the trigger with exactly the same data
-    setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_lost_trigger'," +
-        "'event' : 'nodeLost'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_lost_trigger'," +
+       "'event' : 'nodeLost'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
+    
     // this should be a no-op so the action should have been created but init should not be called
     if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
@@ -630,20 +658,17 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   public void testEventQueue() throws Exception {
     waitForSeconds = 1;
     SolrClient solrClient = cluster.simGetSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger1'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
-        "}}";
-
     String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
 
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger1'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
+       "}}");
 
     if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
@@ -686,17 +711,15 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
   public void testEventFromRestoredState() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '10s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '10s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+       "}}");
 
     if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
@@ -849,29 +872,25 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     // set up triggers
 
     log.info("====== ADD TRIGGERS");
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '1s'," +
-        "'enabled' : true," +
-        "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    setTriggerCommand = "{" +
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '1s'," +
+       "'enabled' : true," +
+       "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
+       "}}");
+
+    assertAutoScalingRequest
+      ("{" +
         "'set-trigger' : {" +
         "'name' : 'node_lost_trigger'," +
         "'event' : 'nodeLost'," +
         "'waitFor' : '1s'," +
         "'enabled' : true," +
         "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
-        "}}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+       "}}");
 
     overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
 
@@ -949,54 +968,48 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   @Test
   public void testListeners() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [" +
-        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}," +
-        "{'name':'test1','class':'" + TestDummyAction.class.getName() + "'}," +
-        "]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [" +
+       "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}," +
+       "{'name':'test1','class':'" + TestDummyAction.class.getName() + "'}," +
+       "]" +
+       "}}");
 
     if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
 
-    String setListenerCommand = "{" +
-        "'set-listener' : " +
-        "{" +
-        "'name' : 'foo'," +
-        "'trigger' : 'node_added_trigger'," +
-        "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
-        "'beforeAction' : 'test'," +
-        "'afterAction' : ['test', 'test1']," +
-        "'class' : '" + TestTriggerListener.class.getName() + "'" +
-        "}" +
-        "}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setListenerCommand);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    String setListenerCommand1 = "{" +
-        "'set-listener' : " +
-        "{" +
-        "'name' : 'bar'," +
-        "'trigger' : 'node_added_trigger'," +
-        "'stage' : ['FAILED','SUCCEEDED']," +
-        "'beforeAction' : ['test', 'test1']," +
-        "'afterAction' : 'test'," +
-        "'class' : '" + TestTriggerListener.class.getName() + "'" +
-        "}" +
-        "}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setListenerCommand1);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-listener' : " +
+       "{" +
+       "'name' : 'foo'," +
+       "'trigger' : 'node_added_trigger'," +
+       "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+       "'beforeAction' : 'test'," +
+       "'afterAction' : ['test', 'test1']," +
+       "'class' : '" + TestTriggerListener.class.getName() + "'" +
+       "}" +
+       "}");
+
+    assertAutoScalingRequest
+      ("{" +
+       "'set-listener' : " +
+       "{" +
+       "'name' : 'bar'," +
+       "'trigger' : 'node_added_trigger'," +
+       "'stage' : ['FAILED','SUCCEEDED']," +
+       "'beforeAction' : ['test', 'test1']," +
+       "'afterAction' : 'test'," +
+       "'class' : '" + TestTriggerListener.class.getName() + "'" +
+       "}" +
+       "}");
 
     listenerEvents.clear();
     failDummyAction = false;
@@ -1113,32 +1126,28 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     SolrClient solrClient = cluster.simGetSolrClient();
     failDummyAction = false;
     waitForSeconds = 1;
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'node_added_cooldown_trigger'," +
-        "'event' : 'nodeAdded'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'actions' : [" +
-        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
-        "]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    String setListenerCommand1 = "{" +
-        "'set-listener' : " +
-        "{" +
-        "'name' : 'bar'," +
-        "'trigger' : 'node_added_cooldown_trigger'," +
-        "'stage' : ['FAILED','SUCCEEDED', 'IGNORED']," +
-        "'class' : '" + TestTriggerListener.class.getName() + "'" +
-        "}" +
-        "}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setListenerCommand1);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'node_added_cooldown_trigger'," +
+       "'event' : 'nodeAdded'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'actions' : [" +
+       "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+       "]" +
+       "}}");
+
+    assertAutoScalingRequest
+      ("{" +
+       "'set-listener' : " +
+       "{" +
+       "'name' : 'bar'," +
+       "'trigger' : 'node_added_cooldown_trigger'," +
+       "'stage' : ['FAILED','SUCCEEDED', 'IGNORED']," +
+       "'class' : '" + TestTriggerListener.class.getName() + "'" +
+       "}" +
+       "}");
 
     listenerCreated = new CountDownLatch(1);
     listenerEvents.clear();
@@ -1226,39 +1235,35 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     create.process(solrClient);
     CloudTestUtils.waitForState(cluster, COLL1, 10, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 2, false, true));
 
-    String setTriggerCommand = "{" +
-        "'set-trigger' : {" +
-        "'name' : 'search_rate_trigger'," +
-        "'event' : 'searchRate'," +
-        "'waitFor' : '" + waitForSeconds + "s'," +
-        "'enabled' : true," +
-        "'aboveRate' : 1.0," +
-        "'aboveNodeRate' : 1.0," +
-        "'actions' : [" +
-        "{'name':'start','class':'" + StartTriggerAction.class.getName() + "'}," +
-        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
-        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
-        "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
-        "{'name':'finish','class':'" + FinishTriggerAction.class.getName() + "'}," +
-        "]" +
-        "}}";
-    SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
-    NamedList<Object> response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
-
-    String setListenerCommand1 = "{" +
-        "'set-listener' : " +
-        "{" +
-        "'name' : 'srt'," +
-        "'trigger' : 'search_rate_trigger'," +
-        "'stage' : ['FAILED','SUCCEEDED']," +
-        "'afterAction': ['compute', 'execute', 'test']," +
-        "'class' : '" + TestTriggerListener.class.getName() + "'" +
-        "}" +
-        "}";
-    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setListenerCommand1);
-    response = solrClient.request(req);
-    assertEquals(response.get("result").toString(), "success");
+    assertAutoScalingRequest
+      ("{" +
+       "'set-trigger' : {" +
+       "'name' : 'search_rate_trigger'," +
+       "'event' : 'searchRate'," +
+       "'waitFor' : '" + waitForSeconds + "s'," +
+       "'enabled' : true," +
+       "'aboveRate' : 1.0," +
+       "'aboveNodeRate' : 1.0," +
+       "'actions' : [" +
+       "{'name':'start','class':'" + StartTriggerAction.class.getName() + "'}," +
+       "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+       "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+       "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
+       "{'name':'finish','class':'" + FinishTriggerAction.class.getName() + "'}," +
+       "]" +
+       "}}");
+
+    assertAutoScalingRequest
+      ("{" +
+       "'set-listener' : " +
+       "{" +
+       "'name' : 'srt'," +
+       "'trigger' : 'search_rate_trigger'," +
+       "'stage' : ['FAILED','SUCCEEDED']," +
+       "'afterAction': ['compute', 'execute', 'test']," +
+       "'class' : '" + TestTriggerListener.class.getName() + "'" +
+       "}" +
+       "}");
 
 //    SolrParams query = params(CommonParams.Q, "*:*");
 //    for (int i = 0; i < 500; i++) {
@@ -1329,4 +1334,51 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
       assertEquals("ADDREPLICA", m._get("params.action", null));
     }
   }
+
+  /** 
+   * Helper method for getting a copy of the current (internal) trigger state of a scheduled trigger. 
+   */
+  private Map<String, Object> getTriggerState(final String name) {
+    final AutoScaling.Trigger t = cluster.getOverseerTriggerThread().getScheduledTriggers().getTrigger(name);
+    assertNotNull(name + " is not a currently scheduled trigger", t);
+    assertTrue(name + " is not a TriggerBase w/state: " + t.getClass(),
+               t instanceof TriggerBase);
+    return ((TriggerBase)t).deepCopyState();
+  }
+
+  /**
+   * Helper method for making some common assertions about {@link #events}:
+   * <ul>
+   *  <li>Exactly one event that is not null</li>
+   *  <li>Event refers to exactly one expected {@link TriggerEvent#NODE_NAMES}</li>
+   *  <li>Event has exactly one {@link TriggerEvent#EVENT_TIMES} (which matches {@link TriggerEvent#getEventTime}) which is less then the  <code>maxExpectedEventTimeNs</code></li>
+   * </ul>
+   * @return the event found so that other assertions can be made
+   */
+  private static TriggerEvent assertSingleEvent(final String expectedNodeName,
+                                                final long maxExpectedEventTimeNs) {
+    
+    assertEquals("Wrong number of events recorded: " + events.toString(),
+                 1, events.size());
+    
+    final TriggerEvent event = events.iterator().next();
+    assertNotNull("null event???", event);
+    assertNotNull("event is missing NODE_NAMES: " + event, event.getProperty(TriggerEvent.NODE_NAMES));
+    assertEquals("event has incorrect NODE_NAMES: " + event,
+                 Collections.singletonList(expectedNodeName),
+                 event.getProperty(TriggerEvent.NODE_NAMES));
+    
+    assertTrue("event TS is too late, should be before (max) expected TS @ "
+               + maxExpectedEventTimeNs + ": " + event,
+               event.getEventTime() < maxExpectedEventTimeNs);
+    
+    assertNotNull("event is missing EVENT_TIMES: " + event, event.getProperty(TriggerEvent.EVENT_TIMES));
+    assertEquals("event has unexpeted number of EVENT_TIMES: " + event,
+                 1, ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).size());
+    assertEquals("event's TS doesn't match EVENT_TIMES: " + event,
+                 event.getEventTime(),
+                 ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).iterator().next());
+    return event;
+  }
+  
 }