You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2019/01/14 10:40:13 UTC

[lucene-solr] branch branch_8x updated (74ee4dd -> b33df8d)

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

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


    from 74ee4dd  LUCENE-8623: Decrease I/O pressure when merging high dimensional points
     new 229a089  SOLR-13072: Wait for autoscaling config refresh to finish before modifying the cluster and enable the tests for now.
     new b33df8d  SOLR-13072: Use the same wait in other simulated tests where the same race condition may occur.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../cloud/autoscaling/OverseerTriggerThread.java   |  13 +++
 .../autoscaling/sim/SimClusterStateProvider.java   |   4 +
 .../autoscaling/sim/SimSolrCloudTestCase.java      |  15 +++
 .../autoscaling/sim/TestSimComputePlanAction.java  |   9 ++
 .../autoscaling/sim/TestSimExecutePlanAction.java  |   2 +
 .../autoscaling/sim/TestSimExtremeIndexing.java    |   2 +
 .../cloud/autoscaling/sim/TestSimLargeCluster.java |   6 ++
 .../autoscaling/sim/TestSimTriggerIntegration.java | 101 +++++++++++++++------
 8 files changed, 123 insertions(+), 29 deletions(-)


[lucene-solr] 02/02: SOLR-13072: Use the same wait in other simulated tests where the same race condition may occur.

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b33df8dc0ff387e999348a03a748d466c2e6de50
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Jan 8 19:12:38 2019 +0100

    SOLR-13072: Use the same wait in other simulated tests where the same race condition may occur.
---
 .../solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java  | 15 +++++++++++++++
 .../cloud/autoscaling/sim/TestSimComputePlanAction.java   |  9 +++++++++
 .../cloud/autoscaling/sim/TestSimExecutePlanAction.java   |  2 ++
 .../cloud/autoscaling/sim/TestSimExtremeIndexing.java     |  2 ++
 .../solr/cloud/autoscaling/sim/TestSimLargeCluster.java   |  6 ++++++
 .../cloud/autoscaling/sim/TestSimTriggerIntegration.java  | 12 ------------
 6 files changed, 34 insertions(+), 12 deletions(-)

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 49a7bbd..6cb3f1a 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
@@ -21,14 +21,17 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 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;
 import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -63,6 +66,18 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
     cluster = null;
   }
 
+  protected static void assertAutoscalingUpdateComplete() throws Exception {
+    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+        .waitFor("OverseerTriggerThread never caught up to the latest znodeVersion", () -> {
+          try {
+            AutoScalingConfig autoscalingConfig = cluster.getDistribStateManager().getAutoScalingConfig();
+            return autoscalingConfig.getZkVersion() == cluster.getOverseerTriggerThread().getProcessedZnodeVersion();
+          } catch (Exception e) {
+            throw new RuntimeException("FAILED", e);
+          }
+        });
+  }
+
   @Override
   public void tearDown() throws Exception {
     super.tearDown();
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
index f82c5fe..19f9b8d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
@@ -100,6 +100,7 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     rsp = cluster.request(req);
     response = rsp.getResponse();
     assertEquals(response.get("result").toString(), "success");
+    assertAutoscalingUpdateComplete();
     cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
   }
 
@@ -138,6 +139,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeLost",
         "conf",1, 2);
     create.process(solrClient);
@@ -201,6 +204,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeWithMultipleReplicasLost",
         "conf",2, 3);
     create.setMaxShardsPerNode(2);
@@ -285,6 +290,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeAdded",
         "conf",1, 4).setMaxShardsPerNode(-1);
     create.process(solrClient);
@@ -304,6 +311,8 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     // start a node so that the 'violation' created by the previous policy update is fixed
     String newNode = cluster.simAddNode();
     assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
index e42510c..a0c18a9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
@@ -173,6 +173,8 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     String collectionName = "testIntegration";
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
         "conf", 1, 2);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
index 3c996a1..56cfdf6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
@@ -120,6 +120,8 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     long batchSize = BATCH_SIZE;
     for (long i = 0; i < NUM_BATCHES; i++) {
       addDocs(collectionName, i * batchSize, batchSize);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
index 7834c70..fed7b14 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
@@ -174,6 +174,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     cluster.getTimeSource().sleep(5000);
 
     // pick a few random nodes
@@ -269,6 +271,8 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
+
     // create a collection with more than 1 replica per node
     String collectionName = "testNodeAdded";
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
@@ -472,6 +476,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
 
     // create a collection with 1 replica per node
     String collectionName = "testNodeLost";
@@ -671,6 +676,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
+    assertAutoscalingUpdateComplete();
 
     boolean await = triggerFinishedLatch.await(waitForSeconds * 45000 / SPEED, TimeUnit.MILLISECONDS);
     assertTrue("The trigger did not fire at all", await);
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 711b333..3056d33 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
@@ -1424,16 +1424,4 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     return event;
   }
 
-  private static void assertAutoscalingUpdateComplete() throws Exception {
-    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
-        .waitFor("OverseerTriggerThread never caught up to the latest znodeVersion", () -> {
-          try {
-            AutoScalingConfig autoscalingConfig = cluster.getDistribStateManager().getAutoScalingConfig();
-            return autoscalingConfig.getZkVersion() == cluster.getOverseerTriggerThread().getProcessedZnodeVersion();
-          } catch (Exception e) {
-            throw new RuntimeException("FAILED", e);
-          }
-        });
-  }
-  
 }


[lucene-solr] 01/02: SOLR-13072: Wait for autoscaling config refresh to finish before modifying the cluster and enable the tests for now.

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 229a0894fbcb152db4ca08119da085a002953943
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Jan 8 16:16:55 2019 +0100

    SOLR-13072: Wait for autoscaling config refresh to finish before modifying the cluster
    and enable the tests for now.
---
 .../cloud/autoscaling/OverseerTriggerThread.java   |  13 +++
 .../autoscaling/sim/SimClusterStateProvider.java   |   4 +
 .../autoscaling/sim/TestSimTriggerIntegration.java | 111 +++++++++++++++------
 3 files changed, 100 insertions(+), 28 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
index 41ae59b..de875fb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
@@ -74,6 +74,8 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
 
   private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>();
 
+  private volatile int processedZnodeVersion = -1;
+
   private volatile boolean isClosed = false;
 
   private AutoScalingConfig autoScalingConfig;
@@ -109,6 +111,16 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
     return scheduledTriggers;
   }
 
+  /**
+   * For tests, to ensure that all processing has been completed in response to an update of /autoscaling.json.
+   * @lucene.internal
+   * @return version of /autoscaling.json for which all configuration updates &amp; processing have been completed.
+   *  Until then this value will always be smaller than the current znodeVersion of /autoscaling.json.
+   */
+  public int getProcessedZnodeVersion() {
+    return processedZnodeVersion;
+  }
+
   @Override
   public boolean isClosed() {
     return isClosed;
@@ -248,6 +260,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
       log.debug("-- cleaning old nodeLost / nodeAdded markers");
       removeMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
       removeMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+      processedZnodeVersion = znodeVersion;
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 5265594..784c0cb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -372,6 +372,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
   }
 
+  public synchronized String simGetOverseerLeader() {
+    return overseerLeader;
+  }
+
   // this method needs to be called under a lock
   private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
     List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
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 d7e792e..711b333 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
@@ -35,13 +35,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 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.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.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.autoscaling.ActionContext;
@@ -147,7 +145,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testTriggerThrottling() throws Exception  {
     // for this test we want to create two triggers so we must assert that the actions were created twice
     actionInitCalled = new CountDownLatch(2);
@@ -177,6 +175,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     // wait until the two instances of action are created
     if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
@@ -213,6 +213,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     // wait until the two instances of action are created
     if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
@@ -285,6 +287,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
 
     assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
                actionInitCalled.await(60, TimeUnit.SECONDS));
+
+    assertAutoscalingUpdateComplete();
     
     // start a new node that we can kill later
     final String nodeName = cluster.simAddNode();
@@ -371,7 +375,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'enabled' : true," +
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
-    
+
+    assertAutoscalingUpdateComplete();
+
     assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
                actionInitCalled.await(60, TimeUnit.SECONDS));
 
@@ -418,7 +424,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'enabled' : true," +
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
-    
+
+    assertAutoscalingUpdateComplete();
+
     assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
                actionInitCalled.await(60, TimeUnit.SECONDS));
     
@@ -433,7 +441,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testNodeAddedTrigger() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     assertAutoScalingRequest
@@ -446,6 +454,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
@@ -474,6 +484,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     // 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");
@@ -483,8 +495,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  // commented 4-Sep-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 26-Mar-2018
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testNodeLostTrigger() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     assertAutoScalingRequest
@@ -497,6 +508,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
@@ -525,7 +538,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'enabled' : true," +
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
-    
+
+    assertAutoscalingUpdateComplete();
+
     // 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");
@@ -654,7 +669,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   public static long eventQueueActionWait = 5000;
 
   @Test
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // this test fails easily
+  //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072") // this test fails easily
   public void testEventQueue() throws Exception {
     waitForSeconds = 1;
     SolrClient solrClient = cluster.simGetSolrClient();
@@ -670,6 +685,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
@@ -692,6 +709,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     // kill overseer
     cluster.simRestartOverseer(overseerLeader);
     cluster.getTimeSource().sleep(5000);
+
+    assertAutoscalingUpdateComplete();
+
     // new overseer leader should be elected and run triggers
     await = actionInterrupted.await(3000 / SPEED, TimeUnit.MILLISECONDS);
     assertTrue("action wasn't interrupted", await);
@@ -707,8 +727,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  // commented 4-Sep-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") //2018-03-10
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
+ // @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testEventFromRestoredState() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     assertAutoScalingRequest
@@ -721,6 +740,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
        "}}");
 
+    assertAutoscalingUpdateComplete();
+
     if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
@@ -814,7 +835,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testNodeMarkersRegistration() throws Exception {
     // for this test we want to create two triggers so we must assert that the actions were created twice
     actionInitCalled = new CountDownLatch(2);
@@ -824,8 +845,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
 
     SolrClient solrClient = cluster.simGetSolrClient();
 
-    // pick overseer node
-    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
+    // get overseer node
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetOverseerLeader();
 
     // add a node
     String node = cluster.simAddNode();
@@ -842,6 +863,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     // stop overseer
     log.info("====== KILL OVERSEER 1");
     cluster.simRestartOverseer(overseerLeader);
+    assertAutoscalingUpdateComplete();
+
     if (!listener.onChangeLatch.await(10000, TimeUnit.MILLISECONDS)) {
       fail("onChange listener didn't execute on cluster change");
     }
@@ -892,7 +915,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
         "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
        "}}");
 
-    overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
+    assertAutoscalingUpdateComplete();
+    overseerLeader = cluster.getSimClusterStateProvider().simGetOverseerLeader();
 
     // create another node
     log.info("====== ADD NODE 1");
@@ -906,14 +930,10 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node1;
     assertTrue("Path " + pathAdded + " wasn't created", cluster.getDistribStateManager().hasData(pathAdded));
 
-    cluster.getTimeSource().sleep(60000);
-    // nodeAdded marker should be consumed now by nodeAdded trigger
-    assertFalse("Path " + pathAdded + " should have been deleted",
-        cluster.getDistribStateManager().hasData(pathAdded));
-
     listener.reset();
     events.clear();
-    triggerFiredLatch = new CountDownLatch(1);
+    // one nodeAdded (not cleared yet) and one nodeLost
+    triggerFiredLatch = new CountDownLatch(2);
     // kill overseer again
     log.info("====== KILL OVERSEER 2");
     cluster.simRestartOverseer(overseerLeader);
@@ -921,15 +941,32 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
       fail("onChange listener didn't execute on cluster change");
     }
 
+    assertAutoscalingUpdateComplete();
 
     if (!triggerFiredLatch.await(120000 / SPEED, TimeUnit.MILLISECONDS)) {
       fail("Trigger should have fired by now");
     }
-    assertEquals(1, events.size());
-    TriggerEvent ev = events.iterator().next();
-    List<String> nodeNames = (List<String>)ev.getProperty(TriggerEvent.NODE_NAMES);
+    assertEquals(2, events.size());
+    TriggerEvent nodeAdded = null;
+    TriggerEvent nodeLost = null;
+    for (TriggerEvent ev : events) {
+      switch (ev.getEventType()) {
+        case NODEADDED:
+          nodeAdded = ev;
+          break;
+        case NODELOST:
+          nodeLost = ev;
+          break;
+        default:
+          fail("unexpected event type: " + ev);
+      }
+    }
+    assertNotNull("expected nodeAdded event", nodeAdded);
+    assertNotNull("expected nodeLost event", nodeLost);
+    List<String> nodeNames = (List<String>)nodeLost.getProperty(TriggerEvent.NODE_NAMES);
     assertTrue(nodeNames.contains(overseerLeader));
-    assertEquals(TriggerEventType.NODELOST, ev.getEventType());
+    nodeNames = (List<String>)nodeAdded.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(node1));
   }
 
   static final Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
@@ -1011,6 +1048,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "}" +
        "}");
 
+    assertAutoscalingUpdateComplete();
+
     listenerEvents.clear();
     failDummyAction = false;
 
@@ -1121,7 +1160,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
   public void testCooldown() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     failDummyAction = false;
@@ -1149,6 +1188,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "}" +
        "}");
 
+    assertAutoscalingUpdateComplete();
+
     listenerCreated = new CountDownLatch(1);
     listenerEvents.clear();
 
@@ -1226,7 +1267,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
 
 
   @Test
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // this test is way to sensitive to timing, must be beasted before returned
+  //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072") // this test is way to sensitive to timing, must be beasted before returned
   public void testSearchRate() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     String COLL1 = "collection1";
@@ -1265,6 +1306,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
        "}" +
        "}");
 
+    assertAutoscalingUpdateComplete();
+
 //    SolrParams query = params(CommonParams.Q, "*:*");
 //    for (int i = 0; i < 500; i++) {
 //      solrClient.query(COLL1, query);
@@ -1380,5 +1423,17 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
                  ((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).iterator().next());
     return event;
   }
+
+  private static void assertAutoscalingUpdateComplete() throws Exception {
+    (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+        .waitFor("OverseerTriggerThread never caught up to the latest znodeVersion", () -> {
+          try {
+            AutoScalingConfig autoscalingConfig = cluster.getDistribStateManager().getAutoScalingConfig();
+            return autoscalingConfig.getZkVersion() == cluster.getOverseerTriggerThread().getProcessedZnodeVersion();
+          } catch (Exception e) {
+            throw new RuntimeException("FAILED", e);
+          }
+        });
+  }
   
 }