You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2017/12/27 15:04:00 UTC

[07/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeOut.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeOut.java b/solr/core/src/java/org/apache/solr/util/TimeOut.java
index fd91045..bcc29961 100644
--- a/solr/core/src/java/org/apache/solr/util/TimeOut.java
+++ b/solr/core/src/java/org/apache/solr/util/TimeOut.java
@@ -18,26 +18,34 @@ package org.apache.solr.util;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
+
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
 public class TimeOut {
 
   private final long timeoutAt, startTime;
+  private final TimeSource timeSource;
 
-  public TimeOut(long interval, TimeUnit unit) {
-    startTime = System.nanoTime();
+  public TimeOut(long interval, TimeUnit unit, TimeSource timeSource) {
+    this.timeSource = timeSource;
+    startTime = timeSource.getTime();
     this.timeoutAt = startTime + NANOSECONDS.convert(interval, unit);
   }
 
   public boolean hasTimedOut() {
-    return System.nanoTime() > timeoutAt;
+    return timeSource.getTime() > timeoutAt;
+  }
+
+  public void sleep(long ms) throws InterruptedException {
+    timeSource.sleep(ms);
   }
 
   public long timeLeft(TimeUnit unit) {
-    return unit.convert(timeoutAt - System.nanoTime(), NANOSECONDS);
+    return unit.convert(timeoutAt - timeSource.getTime(), NANOSECONDS);
   }
 
   public long timeElapsed(TimeUnit unit) {
-    return unit.convert(System.nanoTime() - startTime, NANOSECONDS);
+    return unit.convert(timeSource.getTime() - startTime, NANOSECONDS);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeSource.java b/solr/core/src/java/org/apache/solr/util/TimeSource.java
deleted file mode 100644
index a0c7bc0..0000000
--- a/solr/core/src/java/org/apache/solr/util/TimeSource.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.util;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.common.util.SuppressForbidden;
-
-/**
- * Source of timestamps.
- */
-public abstract class TimeSource {
-
-  /** Implementation that uses {@link System#currentTimeMillis()}. */
-  public static final class CurrentTimeSource extends TimeSource {
-
-    @Override
-    @SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
-    public long getTime() {
-      return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
-    }
-  }
-
-  /** Implementation that uses {@link System#nanoTime()}. */
-  public static final class NanoTimeSource extends TimeSource {
-
-    @Override
-    public long getTime() {
-      return System.nanoTime();
-    }
-  }
-
-  /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
-  public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
-
-  /** This instance uses {@link NanoTimeSource} for generating timestamps. */
-  public static final TimeSource NANO_TIME = new NanoTimeSource();
-
-  /**
-   * Return a timestamp, in nanosecond unit.
-   */
-  public abstract long getTime();
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
index 76b7285..05c6c9f 100644
--- a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
+++ b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -120,7 +121,7 @@ public class TransformerProvider {
     
     lastFilename = filename;
     lastTemplates = result;
-    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
+    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
index 5463f5b..21a53d8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
@@ -21,7 +21,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Test;
 
 public class ActionThrottleTest extends SolrTestCaseJ4 {
@@ -40,6 +40,16 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
       return returnValues.get(index++);
     }
 
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   // use the same time source as ActionThrottle

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
index 99d785a..07b59aa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
@@ -73,7 +73,7 @@ public class AddReplicaTest extends SolrCloudTestCase {
         success = true;
         break;
       }
-      assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
+      assertFalse(rsp.toString(), rsp.getRequestStatus() == RequestStatusState.FAILED);
       Thread.sleep(500);
     }
     assertTrue(success);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 21c001c..cf26de4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -75,11 +75,11 @@ public class AssignTest extends SolrTestCaseJ4 {
         zkClientData.get(invocation.getArgument(0)));
     // TODO: fix this to be independent of ZK
     ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
-    String nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    String nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node2", nodeName);
   }
 
@@ -145,8 +145,8 @@ public class AssignTest extends SolrTestCaseJ4 {
       slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
 
       DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
-      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
-      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
+      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
+      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildSolrCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
     } finally {
       server.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
index c032f6c..67668c9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -34,6 +34,7 @@ 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.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -264,7 +265,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
         }
       }
       
-      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS));
+      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //      waitForAllWarmingSearchers();
       
       Set<String> addFails = getAddFails(indexTreads);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
index ce77996..cee7a35 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
@@ -32,6 +32,7 @@ import org.apache.solr.common.SolrInputDocument;
 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.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -206,7 +207,7 @@ public class ChaosMonkeySafeLeaderWithPullReplicasTest extends AbstractFullDistr
     
     log.info("collection state: " + printClusterStateInfo(DEFAULT_COLLECTION));
     
-    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS));
+    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //    waitForAllWarmingSearchers();
 
     checkShardConsistency(batchSize == 1, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index 4ae2b70..5615918 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -66,6 +66,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean.Category;
@@ -458,7 +459,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
         .add("id", "7")
         .add("id", "8")
         .commit(cluster.getSolrClient(), collectionName);
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       try {
         long numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -514,7 +515,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
   private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
 
 
-    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     boolean allTimesAreCorrect = false;
     while (! timeout.hasTimedOut()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
index bc3b9c5..57d38cd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -140,7 +141,7 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
     
     @Override
     public void run() {
-      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
+      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       while (! timeout.hasTimedOut() && failure.get() == null) {
         doWork();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
index 31f8418..8847cec 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory;
 import org.apache.solr.util.TimeOut;
@@ -175,7 +176,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
                                 SolrParams params)
       throws SolrServerException, InterruptedException, IOException {
 
-    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = cluster.getSolrClient().query(COLLECTION, params).getResults().getNumFound();
     while (0L < numFound && ! timeout.hasTimedOut()) {
       Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
index 8136d3e..0423428 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -153,7 +154,7 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
       // shutdown the original leader
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
-      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       waitTillNodesActive();
       log.info("Updating mappings from zk");
       updateMappingsFromZk(jettys, clients, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
index b57d06f..57ff0b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.RoutingRule;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
 
     @Override
     public void run() {
-      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       for (int id = 26*3; id < 500 && ! timeout.hasTimedOut(); id++) {
         String shardKey = "" + (char) ('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
         SolrInputDocument doc = new SolrInputDocument();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index b6e7415..8a46808 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -44,7 +44,9 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -87,6 +89,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   private static ClusterState clusterStateMock;
   private static SolrZkClient solrZkClientMock;
   private static DistribStateManager stateManagerMock;
+  private static ObjectCache objectCache;
   private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
   private final Map zkMap = new HashMap();
   private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
@@ -136,6 +139,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     overseerMock = mock(Overseer.class);
     zkControllerMock = mock(ZkController.class);
     cloudDataProviderMock = mock(SolrCloudManager.class);
+    objectCache = new ObjectCache();
     clusterStateProviderMock = mock(ClusterStateProvider.class);
     stateManagerMock = mock(DistribStateManager.class);
   }
@@ -173,6 +177,9 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     reset(overseerMock);
     reset(zkControllerMock);
     reset(cloudDataProviderMock);
+    objectCache.clear();
+    when(cloudDataProviderMock.getObjectCache()).thenReturn(objectCache);
+    when(cloudDataProviderMock.getTimeSource()).thenReturn(TimeSource.NANO_TIME);
     reset(clusterStateProviderMock);
     reset(stateManagerMock);
 
@@ -537,7 +544,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   }
   
   protected void waitForEmptyQueue(long maxWait) throws Exception {
-    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS);
+    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (queue.peek() != null) {
       if (timeout.hasTimedOut())
         fail("Queue not empty within " + maxWait + " ms");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index 72b5142..edc6695 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -51,7 +52,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
   }
 
   private void waitForNewOverseer(int seconds, Predicate<String> state) throws Exception {
-    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     String current = null;
     while (timeout.hasTimedOut() == false) {
       current = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index 7ff5334..3dac890 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -48,6 +48,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.TimeOut;
@@ -162,7 +163,7 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
       log.info("Updating mappings from zk");
-      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       updateMappingsFromZk(jettys, clients, true);
       assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
index 41c7c40..457af81 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
@@ -54,6 +54,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -389,7 +390,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
   }
 
   private void assertSliceAndReplicaCount(String collection, int numSlices, int numReplicas, int timeOutInMs) throws InterruptedException {
-    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS);
+    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
       Collection<Slice> slices = clusterState.getCollection(collection).getActiveSlices();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
index b6efa53..0fbc0a1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
@@ -24,6 +24,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.util.MockCoreContainer.MockCoreDescriptor;
 import org.apache.solr.util.TimeOut;
@@ -98,7 +99,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // kill the replica
     int children = cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size();
     ChaosMonkey.stop(notLeader.jetty);
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (children > cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size()) {
         break;
@@ -173,7 +174,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // this should have published a down state so assert that cversion has incremented
     assertTrue(getOverseerCversion() > cversion);
 
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica r = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName());
       if (r.getState() == Replica.State.DOWN) {
@@ -217,7 +218,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     thread = new LeaderInitiatedRecoveryThread(zkController, coreContainer,
         DEFAULT_COLLECTION, SHARD1, replicaCoreNodeProps, 1, coreContainer.getCores().iterator().next().getCoreDescriptor());
     thread.publishDownState(replicaCoreNodeProps.getCoreName(), replica.getName(), replica.getNodeName(), replicaCoreNodeProps.getCoreUrl(), false);
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica.State state = zkController.getLeaderInitiatedRecoveryState(DEFAULT_COLLECTION, SHARD1, replica.getName());
       if (state == Replica.State.DOWN) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 04e22f7..7d652b8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -52,6 +52,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 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.SolrCore;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
@@ -229,7 +230,7 @@ public class TestPullReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.PULL))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient readOnlyReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -500,7 +501,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
         while (true) {
@@ -521,7 +522,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
index 12b3ef0..dae0b4c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
@@ -43,6 +43,7 @@ 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.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -86,7 +87,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       proxies.put(proxy.getUrl(), proxy);
       jettys.put(proxy.getUrl(), jetty);
     }
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (true) {
       try {
         CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false");
@@ -159,7 +160,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       assertNumberOfReplicas(numShards, 0, numShards, true, true);// Replica should still be active, since it doesn't disconnect from ZooKeeper
       {
         long numFound = 0;
-        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (numFound < 20 && !t.hasTimedOut()) {
           Thread.sleep(200);
           numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -233,7 +234,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void assertNumDocs(int numDocs, SolrClient client, int timeoutSecs) throws InterruptedException, SolrServerException, IOException {
-    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = -1;
     while (!t.hasTimedOut()) {
       Thread.sleep(200);
@@ -299,7 +300,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
index 798a8a8..b47424f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -118,7 +119,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
   // 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue.
   // 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader.
   void checkConsistency() throws InterruptedException, KeeperException {
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     boolean checkAppearOnce = false;
     boolean checkElectionZero = false;
     boolean checkZkLeadersAgree = false;
@@ -323,7 +324,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
 
   boolean waitForAllPreferreds() throws KeeperException, InterruptedException {
     boolean goAgain = true;
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (! timeout.hasTimedOut()) {
       goAgain = false;
       Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index ff9fbb8..4ac9f02 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -60,6 +60,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 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.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrIndexWriter;
@@ -232,7 +233,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.TLOG))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient tlogReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -400,7 +401,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
     docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
     
     // Wait until a new leader is elected
-    TimeOut t = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!t.hasTimedOut()) {
       docCollection = getCollectionState(collectionName);
       Replica leader = docCollection.getSlice("shard1").getLeader();
@@ -488,7 +489,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
 
     waitForNumDocsInAllActiveReplicas(2);
     // There are a small delay between new searcher and copy over old updates operation
-    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (assertCopyOverOldUpdates(1, timeCopyOverPerCores)) {
         break;
@@ -748,7 +749,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query, int timeout) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       if (!r.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
         continue;
@@ -772,7 +773,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       try {
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
index 28a0a4e..16673da 100644
--- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -73,7 +74,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
   private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
       boolean shouldBePresent, int expectedSliceCount) throws Exception {
-    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     Boolean isPresent = null; // null meaning "don't know"
     while (null == isPresent || shouldBePresent != isPresent.booleanValue()) {
       final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
index 600cd65..e3df94b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
@@ -36,6 +36,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
index b3147a4..d31bba5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,6 +45,7 @@ import org.junit.Test;
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 
 public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(3)
@@ -141,7 +143,7 @@ public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
index 985d3aa..cf3bff3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
@@ -69,7 +70,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
   }
 
   private static void testAutoAddReplicas() throws Exception {
-    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
       ZkNodeProps loaded = ZkNodeProps.load(data);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
new file mode 100644
index 0000000..e08d37b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud.autoscaling;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+
+/**
+ *
+ */
+public class CapturedEvent {
+  public final AutoScalingConfig.TriggerListenerConfig config;
+  public final TriggerEventProcessorStage stage;
+  public final String actionName;
+  public final TriggerEvent event;
+  public final String message;
+  public final Map<String, Object> context = new HashMap<>();
+  public final long timestamp;
+
+  public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
+                       TriggerEvent event, String message) {
+    if (context != null) {
+      context.toMap(this.context);
+    }
+    this.config = config;
+    this.stage = stage;
+    this.actionName = actionName;
+    this.event = event;
+    this.message = message;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String toString() {
+    return "CapturedEvent{" +
+        "timestamp=" + timestamp +
+        ", stage=" + stage +
+        ", actionName='" + actionName + '\'' +
+        ", event=" + event +
+        ", context=" + context +
+        ", config=" + config +
+        ", message='" + message + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
index 837c683..2b80ec3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -38,11 +39,13 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -139,6 +142,19 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
   }
 
+  @After
+  public void printState() throws Exception {
+    log.debug("-------------_ FINAL STATE --------------");
+    SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
+      Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
+      log.debug("* Node values: " + node + "\n" + Utils.toJSONString(values));
+    }
+    log.debug("* Live nodes: " + cloudManager.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
+    state.forEachCollection(coll -> log.debug("* Collection " + coll.getName() + " state: " + coll));
+  }
+
   @Test
   public void testNodeLost() throws Exception  {
     // let's start a node so that we have at least two

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
index 74a1a82..fa826b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
@@ -41,7 +41,7 @@ import org.apache.solr.common.params.CollectionParams;
 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.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index f2f3f74..d477594 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index 8bca296..dad81e0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index cacf39c..c104a99 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -56,7 +56,7 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.data.Stat;
@@ -87,6 +87,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   private static AtomicBoolean triggerFired;
   private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
   private static ZkStateReader zkStateReader;
+  private static SolrCloudManager cloudManager;
 
   // use the same time source as triggers use
   private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
@@ -146,6 +147,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       // lets start a node
       cluster.startJettySolrRunner();
     }
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
   }
 
   private void deleteChildrenRecursively(String path) throws Exception {
@@ -322,7 +324,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -338,7 +340,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     cluster.stopJettySolrRunner(index);
 
     // ensure that the old trigger sees the stopped node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -386,7 +388,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -396,7 +398,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     JettySolrRunner newNode = cluster.startJettySolrRunner();
 
     // ensure that the old trigger sees the new node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -954,52 +956,22 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(TriggerEventType.NODELOST, ev.getEventType());
   }
 
-  private static class TestEvent {
-    final AutoScalingConfig.TriggerListenerConfig config;
-    final TriggerEventProcessorStage stage;
-    final String actionName;
-    final TriggerEvent event;
-    final String message;
-    final long timestamp;
-
-    TestEvent(AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName, TriggerEvent event, String message) {
-      this.config = config;
-      this.stage = stage;
-      this.actionName = actionName;
-      this.event = event;
-      this.message = message;
-      this.timestamp = timeSource.getTime();
-    }
-
-    @Override
-    public String toString() {
-      return "TestEvent{" +
-          "timestamp=" + timestamp +
-          ", config=" + config +
-          ", stage=" + stage +
-          ", actionName='" + actionName + '\'' +
-          ", event=" + event +
-          ", message='" + message + '\'' +
-          '}';
-    }
-  }
-
-  static Map<String, List<TestEvent>> listenerEvents = new HashMap<>();
+  static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
   static CountDownLatch listenerCreated = new CountDownLatch(1);
   static boolean failDummyAction = false;
 
   public static class TestTriggerListener extends TriggerListenerBase {
     @Override
-    public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-      super.init(dataProvider, config);
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
       listenerCreated.countDown();
     }
 
     @Override
     public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
                                      ActionContext context, Throwable error, String message) {
-      List<TestEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
-      lst.add(new TestEvent(config, stage, actionName, event, message));
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(timeSource.getTime(), context, config, stage, actionName, event, message));
     }
   }
 
@@ -1079,38 +1051,38 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    List<TestEvent> testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 5, testEvents.size());
+    List<CapturedEvent> capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 5, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(4).stage);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(3).stage);
 
     // reset
     triggerFired.set(false);
@@ -1125,37 +1097,37 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 4, testEvents.size());
+    capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
   }
 
   @Test
@@ -1200,7 +1172,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(1000);
 
-    List<TestEvent> capturedEvents = listenerEvents.get("bar");
+    List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
     // we may get a few IGNORED events if other tests caused events within cooldown period
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
     long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
@@ -1220,11 +1192,11 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     capturedEvents = listenerEvents.get("bar");
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
     for (int i = 0; i < capturedEvents.size() - 1; i++) {
-      TestEvent ev = capturedEvents.get(i);
+      CapturedEvent ev = capturedEvents.get(i);
       assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
       assertTrue(ev.toString(), ev.message.contains("cooldown"));
     }
-    TestEvent ev = capturedEvents.get(capturedEvents.size() - 1);
+    CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
     assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
     // the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
     // must be larger than cooldown period
@@ -1380,8 +1352,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       Map<String, Object> props = map("waitFor", 0L, "actions", Collections.singletonList(map("name","throttler", "class", ThrottlingTesterAction.class.getName())));
       scheduledTriggers.add(new NodeAddedTrigger("y1", props, resourceLoader, solrCloudManager));
       scheduledTriggers.add(new NodeAddedTrigger("y2", props, resourceLoader, solrCloudManager));
+      scheduledTriggers.resetActionThrottle();
       JettySolrRunner newNode = cluster.startJettySolrRunner();
-      assertTrue(getTriggerFiredLatch().await(10, TimeUnit.SECONDS));
+      assertTrue(getTriggerFiredLatch().await(20, TimeUnit.SECONDS));
       for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
         if (cluster.getJettySolrRunner(i) == newNode) {
           cluster.stopJettySolrRunner(i);
@@ -1439,7 +1412,12 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
   public void testSearchRate() throws Exception {
+    // start a few more jetty-s
+    for (int i = 0; i < 3; i++) {
+      cluster.startJettySolrRunner();
+    }
     CloudSolrClient solrClient = cluster.getSolrClient();
     String COLL1 = "collection1";
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
@@ -1453,6 +1431,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'enabled' : true," +
         "'rate' : 1.0," +
         "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
         "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
         "]" +
         "}}";
@@ -1466,6 +1446,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'name' : 'srt'," +
         "'trigger' : 'search_rate_trigger'," +
         "'stage' : ['FAILED','SUCCEEDED']," +
+        "'afterAction': ['compute', 'execute', 'test']," +
         "'class' : '" + TestTriggerListener.class.getName() + "'" +
         "}" +
         "}";
@@ -1481,7 +1462,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(2000);
     assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
-    TestEvent ev = listenerEvents.get("srt").get(0);
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
     long now = timeSource.getTime();
     // verify waitFor
     assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
new file mode 100644
index 0000000..c1c070d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.autoscaling.sim;
+
+/**
+ * Interface that helps simulating action errors.
+ */
+public interface ActionError {
+  boolean shouldFail(String... args);
+}