You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2015/08/08 15:40:00 UTC

svn commit: r1694798 [2/3] - in /lucene/dev/trunk: lucene/tools/forbiddenApis/ solr/ solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/ solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/ solr/con...

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Sat Aug  8 13:39:58 2015
@@ -35,6 +35,7 @@ import org.apache.solr.common.SolrExcept
 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.SuppressForbidden;
 import org.apache.solr.core.SolrConfig.UpdateHandlerInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -142,7 +143,7 @@ public class DirectUpdateHandler2 extend
   }
 
   private void deleteAll() throws IOException {
-    SolrCore.log.info(core.getLogId()+"REMOVING ALL DOCUMENTS FROM INDEX");
+    SolrCore.log.info(core.getLogId() + "REMOVING ALL DOCUMENTS FROM INDEX");
     RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
     try {
       iw.get().deleteAll();
@@ -493,6 +494,15 @@ public class DirectUpdateHandler2 extend
     return rc;
   }
 
+  @SuppressForbidden(reason = "Need currentTimeMillis, commit time should be used only for debugging purposes, " +
+      " but currently suspiciously used for replication as well")
+  private void setCommitData(IndexWriter iw) {
+    final Map<String,String> commitData = new HashMap<>();
+    commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY,
+        String.valueOf(System.currentTimeMillis()));
+    iw.setCommitData(commitData);
+  }
+
   public void prepareCommit(CommitUpdateCommand cmd) throws IOException {
 
     boolean error=true;
@@ -501,10 +511,7 @@ public class DirectUpdateHandler2 extend
       log.info("start "+cmd);
       RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
       try {
-        final Map<String,String> commitData = new HashMap<>();
-        commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY,
-            String.valueOf(System.currentTimeMillis()));
-        iw.get().setCommitData(commitData);
+        setCommitData(iw.get());
         iw.get().prepareCommit();
       } finally {
         iw.decref();
@@ -579,10 +586,7 @@ public class DirectUpdateHandler2 extend
           // SolrCore.verbose("writer.commit() start writer=",writer);
 
           if (writer.hasUncommittedChanges()) {
-            final Map<String,String> commitData = new HashMap<>();
-            commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY,
-                String.valueOf(System.currentTimeMillis()));
-            writer.setCommitData(commitData);
+            setCommitData(writer);
             writer.commit();
           } else {
             log.info("No uncommitted changes. Skipping IW.commit.");
@@ -767,9 +771,7 @@ public class DirectUpdateHandler2 extend
           }
 
           // todo: refactor this shared code (or figure out why a real CommitUpdateCommand can't be used)
-          final Map<String,String> commitData = new HashMap<>();
-          commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, String.valueOf(System.currentTimeMillis()));
-          writer.setCommitData(commitData);
+          setCommitData(writer);
           writer.commit();
 
           synchronized (solrCoreState.getUpdateLock()) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateLog.java Sat Aug  8 13:39:58 2015
@@ -58,6 +58,7 @@ import org.apache.solr.update.processor.
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.RTimer;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.slf4j.Logger;
@@ -1571,7 +1572,7 @@ public class UpdateLog implements Plugin
    */
   protected Long seedBucketsWithHighestVersion(SolrIndexSearcher newSearcher, VersionInfo versions) {
     Long highestVersion = null;
-    long startMs = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     RecentUpdates recentUpdates = null;
     try {
@@ -1596,9 +1597,8 @@ public class UpdateLog implements Plugin
         recentUpdates.close();
     }
 
-    long tookMs = (System.currentTimeMillis() - startMs);
-    log.info("Took {} ms to seed version buckets with highest version {}",
-        tookMs, String.valueOf(highestVersion));
+    log.info("Took {}ms to seed version buckets with highest version {}",
+        timer.getTime(), String.valueOf(highestVersion));
 
     return highestVersion;
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/VersionInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/VersionInfo.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/VersionInfo.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/VersionInfo.java Sat Aug  8 13:39:58 2015
@@ -35,6 +35,7 @@ import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -150,15 +151,14 @@ public class VersionInfo {
   // Time-based lamport clock.  Good for introducing some reality into clocks (to the degree
   // that times are somewhat synchronized in the cluster).
   // Good if we want to relax some constraints to scale down to where only one node may be
-  // up at a time.  Possibly harder to detect missing messages (because versions are not contiguous.
-  long vclock;
-  long time;
+  // up at a time.  Possibly harder to detect missing messages (because versions are not contiguous).
+  private long vclock;
   private final Object clockSync = new Object();
 
-
+  @SuppressForbidden(reason = "need currentTimeMillis just for getting realistic version stamps, does not assume monotonicity")
   public long getNewClock() {
     synchronized (clockSync) {
-      time = System.currentTimeMillis();
+      long time = System.currentTimeMillis();
       long result = time << 20;
       if (result <= vclock) {
         result = vclock + 1;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Sat Aug  8 13:39:58 2015
@@ -524,8 +524,7 @@ public class DistributedUpdateProcessor
         if (routeKey != null) {
           RoutingRule rule = routingRules.get(routeKey + "!");
           if (rule != null) {
-            // TODO: look at using nanoTime
-            if (rule.getExpireAt() >= System.currentTimeMillis()) {
+            if (! rule.isExpired()) {
               List<DocRouter.Range> ranges = rule.getRouteRanges();
               if (ranges != null && !ranges.isEmpty()) {
                 int hash = compositeIdRouter.sliceHash(id, doc, null, coll);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SimplePostTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SimplePostTool.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SimplePostTool.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SimplePostTool.java Sat Aug  8 13:39:58 2015
@@ -60,6 +60,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 import java.util.zip.GZIPInputStream;
@@ -172,7 +173,7 @@ public class SimplePostTool {
    * This method delegates to the correct mode method.
    */
   public void execute() {
-    final long startTime = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
     if (DATA_MODE_FILES.equals(mode) && args.length > 0) {
       doFilesMode();
     } else if(DATA_MODE_ARGS.equals(mode) && args.length > 0) {
@@ -188,8 +189,7 @@ public class SimplePostTool {
     
     if (commit)   commit();
     if (optimize) optimize();
-    final long endTime = System.currentTimeMillis();
-    displayTiming(endTime - startTime);
+    displayTiming((long) timer.getTime());
   }
   
   /**

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java Sat Aug  8 13:39:58 2015
@@ -51,8 +51,11 @@ public class SolrLogLayout extends Layou
   public static interface TG {
     public String getTag();
   }
-  
-  long startTime = System.currentTimeMillis();
+
+  @SuppressForbidden(reason = "Need currentTimeMillis to compare against log event timestamp. " +
+    "This is inaccurate but unavoidable due to interface limitations, in any case this is just for logging.")
+  final long startTime = System.currentTimeMillis();
+
   long lastTime = startTime;
   Map<Method,String> methodAlias = new HashMap<>();
   

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/TimeOut.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/TimeOut.java?rev=1694798&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/TimeOut.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/TimeOut.java Sat Aug  8 13:39:58 2015
@@ -0,0 +1,37 @@
+package org.apache.solr.util;
+
+/*
+ * 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.
+ */
+
+import java.util.concurrent.TimeUnit;
+
+public class TimeOut {
+
+  private final long timeoutAt;
+
+  public TimeOut(long interval, TimeUnit unit) {
+    this.timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(interval, unit);
+  }
+
+  public boolean hasTimedOut() {
+    return System.nanoTime() > timeoutAt;
+  }
+
+  public long timeLeft(TimeUnit unit) {
+    return unit.convert(timeoutAt - System.nanoTime(), TimeUnit.NANOSECONDS);
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/stats/Clock.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/stats/Clock.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/stats/Clock.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/stats/Clock.java Sat Aug  8 13:39:58 2015
@@ -24,6 +24,8 @@ package org.apache.solr.util.stats;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadMXBean;
 
+import org.apache.solr.common.util.SuppressForbidden;
+
 /**
  * An abstraction for how time passes. It is passed to {@link Timer} to track timing.
  */
@@ -40,6 +42,7 @@ public abstract class Clock {
    *
    * @return time in milliseconds
    */
+  @SuppressForbidden(reason = "Need currentTimeMillis, API used by ExponentiallyDecayingSample for suspect reasons")
   public long getTime() {
     return System.currentTimeMillis();
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java Sat Aug  8 13:39:58 2015
@@ -19,6 +19,9 @@ package org.apache.solr.util.xslt;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.commons.io.IOUtils;
@@ -45,8 +48,8 @@ import org.apache.solr.core.SolrConfig;
 public class TransformerProvider {
   private String lastFilename;
   private Templates lastTemplates = null;
-  private long cacheExpires = 0;
-  
+  private TimeOut cacheExpiresTimeout;
+
   private static final Logger log = LoggerFactory.getLogger(TransformerProvider.class.getName());
   private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
   
@@ -69,7 +72,8 @@ public class TransformerProvider {
   public synchronized Transformer getTransformer(SolrConfig solrConfig, String filename,int cacheLifetimeSeconds) throws IOException {
     // For now, the Templates are blindly reloaded once cacheExpires is over.
     // It'd be better to check the file modification time to reload only if needed.
-    if(lastTemplates!=null && filename.equals(lastFilename) && System.currentTimeMillis() < cacheExpires) {
+    if(lastTemplates!=null && filename.equals(lastFilename) &&
+        cacheExpiresTimeout != null && ! cacheExpiresTimeout.hasTimedOut()) {
       if(log.isDebugEnabled()) {
         log.debug("Using cached Templates:" + filename);
       }
@@ -117,8 +121,8 @@ public class TransformerProvider {
     
     lastFilename = filename;
     lastTemplates = result;
-    cacheExpires = System.currentTimeMillis() + (cacheLifetimeSeconds * 1000);
-    
+    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
+
     return result;
   }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java Sat Aug  8 13:39:58 2015
@@ -192,7 +192,7 @@ public class DistributedIntervalFaceting
   }
 
   private String getFieldWithKey(String field) {
-    return "{!key='_some_key_for_" + field + "_" + System.currentTimeMillis() + "'}" + field;
+    return "{!key='_some_key_for_" + field + "_" + random().nextInt() + "'}" + field;
   }
 
   /**

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java Sat Aug  8 13:39:58 2015
@@ -48,7 +48,6 @@ public class SolrInfoMBeanTest extends S
   public void testCallMBeanInfo() throws Exception {
     List<Class> classes = new ArrayList<>();
     classes.addAll(getClassesForPackage(StandardRequestHandler.class.getPackage().getName()));
-    classes.addAll(getClassesForPackage(SearchHandler.class.getPackage().getName()));
     classes.addAll(getClassesForPackage(SearchComponent.class.getPackage().getName()));
     classes.addAll(getClassesForPackage(LukeRequestHandler.class.getPackage().getName()));
     classes.addAll(getClassesForPackage(DefaultSolrHighlighter.class.getPackage().getName()));

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java Sat Aug  8 13:39:58 2015
@@ -52,6 +52,7 @@ import org.apache.solr.common.util.Execu
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -318,11 +319,11 @@ public class BasicDistributedZkTest exte
     long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("commitWithin", 10);
-    add(cloudClient, params , getDoc("id", 300));
-    
-    long timeout = System.currentTimeMillis() + 45000;
+    add(cloudClient, params, getDoc("id", 300));
+
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
     while (cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound() != before + 1) {
-      if (timeout <= System.currentTimeMillis()) {
+      if (timeout.hasTimedOut()) {
         fail("commitWithin did not work");
       }
       Thread.sleep(100);
@@ -374,10 +375,10 @@ public class BasicDistributedZkTest exte
     } catch (Exception e) {
       
     }
-    
-    long timeout = System.currentTimeMillis() + 15000;
+
+    TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS);
     while (cloudClient.getZkStateReader().getZkClient().exists("/collections/the_core_collection", true)) {
-      if (timeout <= System.currentTimeMillis()) {
+      if (timeout.hasTimedOut()) {
         fail(cloudClient.getZkStateReader().getZkClient().getChildren("/collections", null, true).toString() + " Collection zk node still exists");
       }
       Thread.sleep(100);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java Sat Aug  8 13:39:58 2015
@@ -80,6 +80,7 @@ import org.apache.solr.core.CoreContaine
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean.Category;
 import org.apache.solr.servlet.SolrDispatchFilter;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
@@ -280,10 +281,10 @@ public class CollectionsAPIDistributedZk
     request.setPath("/admin/collections");
     
     makeRequest(baseUrl, request);
-    
-    long timeout = System.currentTimeMillis() + 10000;
+
+    TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
     while (cloudClient.getZkStateReader().getClusterState().hasCollection("halfdeletedcollection2")) {
-      if (System.currentTimeMillis() > timeout) {
+      if (timeout.hasTimedOut()) {
         throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");
       }
       
@@ -934,12 +935,12 @@ public class CollectionsAPIDistributedZk
   }
 
   private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
-    
-    
-    long timeoutAt = System.currentTimeMillis() + 45000;
+
+
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
 
     boolean allTimesAreCorrect = false;
-    while (System.currentTimeMillis() < timeoutAt) {
+    while (! timeout.hasTimedOut()) {
       Map<String,Long> urlToTimeAfter = new HashMap<>();
       collectStartTimes(collectionName, urlToTimeAfter);
       
@@ -1164,10 +1165,10 @@ public class CollectionsAPIDistributedZk
       addReplica.setProperties(props);
     }
     client.request(addReplica);
-    long timeout = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     Replica newReplica = null;
 
-    for (; System.currentTimeMillis() < timeout; ) {
+    for (; ! timeout.hasTimedOut(); ) {
       Slice slice = client.getZkStateReader().getClusterState().getSlice(collectionName, shard);
       newReplica = slice.getReplica(newReplicaName);
     }
@@ -1218,9 +1219,9 @@ public class CollectionsAPIDistributedZk
     request.setPath("/admin/collections");
     client.request(request);
 
-    long timeOut = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     boolean changed = false;
-    while(System.currentTimeMillis() <timeOut){
+    while(! timeout.hasTimedOut()){
       Thread.sleep(10);
       changed = Objects.equals(val,client.getZkStateReader().getClusterProps().get(name));
       if(changed) break;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java Sat Aug  8 13:39:58 2015
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slic
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
@@ -45,6 +46,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
 
@@ -290,10 +292,10 @@ public class CollectionsAPISolrJTests ex
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    long timeout = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     Replica newReplica = null;
 
-    while (System.currentTimeMillis() < timeout && newReplica == null) {
+    while (! timeout.hasTimedOut() && newReplica == null) {
       Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
       newReplica = slice.getReplica(newReplicaName);
     }
@@ -313,10 +315,10 @@ public class CollectionsAPISolrJTests ex
     response = deleteReplicaRequest.process(cloudClient);
 
     assertEquals(0, response.getStatus());
-    
-    timeout = System.currentTimeMillis() + 3000;
-    
-    while (System.currentTimeMillis() < timeout && newReplica != null) {
+
+    timeout = new TimeOut(3, TimeUnit.SECONDS);
+
+    while (! timeout.hasTimedOut() && newReplica != null) {
       Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
       newReplica = slice.getReplica(newReplicaName);
     }
@@ -332,10 +334,10 @@ public class CollectionsAPISolrJTests ex
 
     assertEquals(0, response.getStatus());
 
-    long timeOut = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     boolean changed = false;
     
-    while(System.currentTimeMillis() < timeOut){
+    while(! timeout.hasTimedOut()){
       Thread.sleep(10);
       changed = Objects.equals("false",
           cloudClient.getZkStateReader().getClusterProps().get(ZkStateReader.LEGACY_CLOUD));
@@ -349,9 +351,9 @@ public class CollectionsAPISolrJTests ex
             .setPropertyValue(null);
     clusterPropRequest.process(cloudClient);
 
-    timeOut = System.currentTimeMillis() + 3000;
+    timeout = new TimeOut(3, TimeUnit.SECONDS);
     changed = false;
-    while(System.currentTimeMillis() < timeOut){
+    while(! timeout.hasTimedOut()) {
       Thread.sleep(10);
       changed = (cloudClient.getZkStateReader().getClusterProps().get(ZkStateReader.LEGACY_CLOUD) == null);
       if(changed)  
@@ -423,11 +425,11 @@ public class CollectionsAPISolrJTests ex
             .setPropertyValue("true").process(cloudClient);
     assertEquals(0, response.getStatus());
 
-    long timeout = System.currentTimeMillis() + 20000;
+    TimeOut timeout = new TimeOut(20, TimeUnit.SECONDS);
     String propertyValue = null;
     
     String replicaName = replica.getName();
-    while (System.currentTimeMillis() < timeout) {
+    while (! timeout.hasTimedOut()) {
       ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
       replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
       propertyValue = replica.getStr("property.preferredleader"); 
@@ -448,10 +450,10 @@ public class CollectionsAPISolrJTests ex
             .setPropertyName("property.preferredleader").process(cloudClient);
     assertEquals(0, response.getStatus());
 
-    timeout = System.currentTimeMillis() + 20000;
+    timeout = new TimeOut(20, TimeUnit.SECONDS);
     boolean updated = false;
 
-    while (System.currentTimeMillis() < timeout) {
+    while (! timeout.hasTimedOut()) {
       ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
       replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
       updated = replica.getStr("property.preferredleader") == null;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java Sat Aug  8 13:39:58 2015
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
@@ -142,8 +143,8 @@ public class ConcurrentDeleteAndCreateCo
     
     @Override
     public void run() {
-      final long timeToStop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(timeToRunSec);
-      while (System.currentTimeMillis() < timeToStop && failure.get() == null) {
+      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
+      while (! timeout.hasTimedOut() && failure.get() == null) {
         doWork();
       }
     }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteInactiveReplicaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteInactiveReplicaTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteInactiveReplicaTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteInactiveReplicaTest.java Sat Aug  8 13:39:58 2015
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -37,6 +38,7 @@ import org.apache.solr.common.params.Cor
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 public class DeleteInactiveReplicaTest extends AbstractFullDistribZkTestBase{
@@ -66,9 +68,9 @@ public class DeleteInactiveReplicaTest e
       StringBuilder sb = new StringBuilder();
       Replica replica1 = null;
       Slice shard1 = null;
-      long timeout = System.currentTimeMillis() + 3000;
+      TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
       DocCollection testcoll = null;
-      while (!stopped && System.currentTimeMillis() < timeout) {
+      while (!stopped && ! timeout.hasTimedOut()) {
         testcoll = client.getZkStateReader().getClusterState().getCollection(collectionName);
         for (JettySolrRunner jetty : jettys)
           sb.append(jetty.getBaseUrl()).append(",");
@@ -102,9 +104,9 @@ public class DeleteInactiveReplicaTest e
             + " jettys: " + sb);
       }
 
-      long endAt = System.currentTimeMillis() + 3000;
+      timeout = new TimeOut(20, TimeUnit.SECONDS);
       boolean success = false;
-      while (System.currentTimeMillis() < endAt) {
+      while (! timeout.hasTimedOut()) {
         testcoll = client.getZkStateReader()
             .getClusterState().getCollection(collectionName);
         if (testcoll.getSlice(shard1.getName()).getReplica(replica1.getName()).getState() != Replica.State.ACTIVE) {

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java Sat Aug  8 13:39:58 2015
@@ -29,12 +29,14 @@ import org.apache.solr.common.cloud.ZkSt
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
@@ -89,10 +91,10 @@ public class DeleteLastCustomShardedRepl
     SolrRequest request = new QueryRequest(params);
     request.setPath("/admin/collections");
     client.request(request);
-    long endAt = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     boolean success = false;
     DocCollection testcoll = null;
-    while (System.currentTimeMillis() < endAt) {
+    while (! timeout.hasTimedOut()) {
       testcoll = getCommonCloudSolrClient().getZkStateReader()
           .getClusterState().getCollection(COLL_NAME);
       // In case of a custom sharded collection, the last replica deletion would also lead to

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java Sat Aug  8 13:39:58 2015
@@ -32,6 +32,7 @@ import org.apache.solr.common.cloud.ZkSt
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 import java.io.File;
@@ -39,6 +40,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.ONLY_IF_DOWN;
@@ -133,10 +135,10 @@ public class DeleteReplicaTest extends A
     SolrRequest request = new QueryRequest(params);
     request.setPath("/admin/collections");
     client.request(request);
-    long endAt = System.currentTimeMillis() + 3000;
+    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
     boolean success = false;
     DocCollection testcoll = null;
-    while (System.currentTimeMillis() < endAt) {
+    while (! timeout.hasTimedOut()) {
       testcoll = client.getZkStateReader()
           .getClusterState().getCollection(COLL_NAME);
       success = testcoll.getSlice(shard).getReplica(replica.getName()) == null;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java Sat Aug  8 13:39:58 2015
@@ -26,6 +26,7 @@ import org.apache.solr.common.params.Mod
 import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory; // jdoc
 import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactoryTest;
 
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,6 +37,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.HashSet;
 import java.util.HashMap;
+import java.util.concurrent.TimeUnit;
 
 /** Test of {@link DocExpirationUpdateProcessorFactory} in a cloud setup */
 @Slow // Has to do some sleeping to wait for a future expiration
@@ -171,10 +173,10 @@ public class DistribDocExpirationUpdateP
                                 SolrParams params)
       throws SolrServerException, InterruptedException, IOException {
 
-    final long giveUpAfter = System.currentTimeMillis() + (1000L * maxTimeLimitSeconds);
+    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
     long numFound = cloudClient.query(params).getResults().getNumFound();
-    while (0L < numFound && System.currentTimeMillis() < giveUpAfter) {
-      Thread.sleep(Math.min(5000, giveUpAfter - System.currentTimeMillis()));
+    while (0L < numFound && ! timeout.hasTimedOut()) {
+      Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));
       numFound = cloudClient.query(params).getResults().getNumFound();
     }
     assertEquals("Give up waiting for no results: " + params,

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java Sat Aug  8 13:39:58 2015
@@ -40,6 +40,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.update.UpdateLog;
+import org.apache.solr.util.RTimer;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -520,7 +521,7 @@ public class HttpPartitionTest extends A
   }
 
   protected void waitToSeeReplicasActive(String testCollectionName, String shardId, Set<String> replicasToCheck, int maxWaitSecs) throws Exception {
-    long startMs = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     ZkStateReader zkr = cloudClient.getZkStateReader();
     zkr.updateClusterState(); // force the state to be fresh
@@ -565,7 +566,6 @@ public class HttpPartitionTest extends A
       fail("Didn't see replicas "+ replicasToCheck +
           " come up within " + maxWaitMs + " ms! ClusterState: " + printClusterStateInfo(testCollectionName));
 
-    long diffMs = (System.currentTimeMillis() - startMs);
-    log.info("Took " + diffMs + " ms to see replicas ["+replicasToCheck+"] become active.");
+    log.info("Took {} ms to see replicas [{}] become active.", timer.getTime(), replicasToCheck);
   }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java Sat Aug  8 13:39:58 2015
@@ -23,6 +23,7 @@ import org.apache.solr.client.solrj.impl
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.util.RTimer;
 import org.junit.Test;
 
 import java.io.File;
@@ -157,13 +158,12 @@ public class LeaderInitiatedRecoveryOnCo
   protected void sendCommitWithRetry(Replica replica) throws Exception {
     String replicaCoreUrl = replica.getCoreUrl();
     log.info("Sending commit request to: "+replicaCoreUrl);
-    long startMs = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
     try (HttpSolrClient client = new HttpSolrClient(replicaCoreUrl)) {
       try {
         client.commit();
 
-        long tookMs = System.currentTimeMillis() - startMs;
-        log.info("Sent commit request to "+replicaCoreUrl+" OK, took: "+tookMs);
+        log.info("Sent commit request to {} OK, took {}ms", replicaCoreUrl, timer.getTime());
       } catch (Exception exc) {
         Throwable rootCause = SolrException.getRootCause(exc);
         if (rootCause instanceof NoHttpResponseException) {

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java Sat Aug  8 13:39:58 2015
@@ -31,6 +31,7 @@ import org.apache.solr.common.cloud.Rout
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
@@ -38,6 +39,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
@@ -64,7 +66,8 @@ public class MigrateRouteKeyTest extends
   private boolean waitForRuleToExpire(String splitKey, long finishTime) throws KeeperException, InterruptedException, SolrServerException, IOException {
     ClusterState state;Slice slice;
     boolean ruleRemoved = false;
-    while (System.currentTimeMillis() - finishTime < 60000) {
+    long expiryTime = finishTime + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
+    while (System.nanoTime() < expiryTime) {
       getCommonCloudSolrClient().getZkStateReader().updateClusterState();
       state = getCommonCloudSolrClient().getZkStateReader().getClusterState();
       slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
@@ -74,7 +77,7 @@ public class MigrateRouteKeyTest extends
         break;
       }
       SolrInputDocument doc = new SolrInputDocument();
-      doc.addField("id", splitKey + System.currentTimeMillis());
+      doc.addField("id", splitKey + random().nextInt());
       cloudClient.add(doc);
       Thread.sleep(1000);
     }
@@ -160,7 +163,7 @@ public class MigrateRouteKeyTest extends
       assertEquals("DocCount on target collection does not match", 0, collectionClient.query(solrQuery).getResults().getNumFound());
 
       invokeMigrateApi(AbstractDistribZkTestBase.DEFAULT_COLLECTION, splitKey + "/" + BIT_SEP + "!", targetCollection);
-      long finishTime = System.currentTimeMillis();
+      long finishTime = System.nanoTime();
 
       indexer.join();
       splitKeyCount[0] += indexer.getSplitKeyCount();
@@ -207,8 +210,8 @@ public class MigrateRouteKeyTest extends
 
     @Override
     public void run() {
-      long start = System.currentTimeMillis();
-      for (int id = 26*3; id < 500 && System.currentTimeMillis() - start <= seconds*1000; id++) {
+      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+      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();
         doc.addField("id", shardKey + (bitSep != -1 ? "/" + bitSep : "") + "!" + id);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java Sat Aug  8 13:39:58 2015
@@ -36,6 +36,7 @@ import org.apache.solr.handler.component
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.CreateMode;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
@@ -59,6 +60,7 @@ import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import static org.easymock.EasyMock.anyBoolean;
 import static org.easymock.EasyMock.anyObject;
@@ -562,10 +564,10 @@ public class OverseerCollectionProcessor
   }
   
   protected void waitForEmptyQueue(long maxWait) throws Exception {
-    long start = System.currentTimeMillis();
+    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS);
     while (queue.peek() != null) {
-      if ((System.currentTimeMillis() - start) > maxWait) fail(" Queue not empty within "
-          + maxWait + " ms" + System.currentTimeMillis());
+      if (timeout.hasTimedOut())
+        fail("Queue not empty within " + maxWait + " ms");
       Thread.sleep(100);
     }
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java Sat Aug  8 13:39:58 2015
@@ -31,6 +31,7 @@ import org.apache.solr.common.params.Col
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Test;
 
@@ -40,6 +41,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.solr.cloud.OverseerCollectionProcessor.getLeaderNode;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.getSortedOverseerNodeNames;
@@ -83,9 +85,9 @@ public class OverseerRolesTest  extends
     String s = (String) m.get("id");
     String leader = LeaderElector.getNodeName(s);
     Overseer.getInQueue(zk).offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower())));
-    long timeout = System.currentTimeMillis()+10000;
+    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
     String newLeader=null;
-    for(;System.currentTimeMillis() < timeout;){
+    for(;! timeout.hasTimedOut();){
       newLeader = OverseerCollectionProcessor.getLeaderNode(zk);
       if(newLeader!=null && !newLeader.equals(leader)) break;
       Thread.sleep(100);
@@ -118,10 +120,10 @@ public class OverseerRolesTest  extends
     log.info("overseerDesignate {}",overseerDesignate);
     setOverseerRole(client, CollectionAction.ADDROLE,overseerDesignate);
 
-    long timeout = System.currentTimeMillis()+15000;
+    TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS);
 
     boolean leaderchanged = false;
-    for(;System.currentTimeMillis() < timeout;){
+    for(;!timeout.hasTimedOut();){
       if(overseerDesignate.equals(OverseerCollectionProcessor.getLeaderNode(client.getZkStateReader().getZkClient()))){
         log.info("overseer designate is the new overseer");
         leaderchanged =true;
@@ -177,9 +179,9 @@ public class OverseerRolesTest  extends
         OverseerCollectionProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient(),
             OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE));
     ChaosMonkey.stop(leaderJetty);
-    timeout = System.currentTimeMillis() + 10000;
+    timeout = new TimeOut(10, TimeUnit.SECONDS);
     leaderchanged = false;
-    for (; System.currentTimeMillis() < timeout; ) {
+    for (; !timeout.hasTimedOut(); ) {
       currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
       if (anotherOverseer.equals(currentOverseer)) {
         leaderchanged = true;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java Sat Aug  8 13:39:58 2015
@@ -24,7 +24,6 @@ import org.apache.solr.common.cloud.ZkSt
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.RuleChain;
@@ -74,7 +73,7 @@ public class SolrXmlInZkTest extends Sol
     System.setProperty("zkClientTimeout", "8000");
 
     zkDir = tmpDir.getAbsolutePath() + File.separator
-        + "zookeeper" + System.currentTimeMillis() + "/server1/data";
+        + "zookeeper" + System.nanoTime() + "/server1/data";
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
     System.setProperty("zkHost", zkServer.getZkAddress());

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java Sat Aug  8 13:39:58 2015
@@ -26,6 +26,7 @@ import org.apache.solr.common.params.Col
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
@@ -34,6 +35,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 
 public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
@@ -114,9 +116,9 @@ public class TestRebalanceLeaders extend
   // 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 {
-    long start = System.currentTimeMillis();
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
 
-    while ((System.currentTimeMillis() - start) < timeoutMs) {
+    while (! timeout.hasTimedOut()) {
       if (checkAppearOnce() &&
           checkElectionZero() &&
           checkZkLeadersAgree()) {
@@ -234,7 +236,6 @@ public class TestRebalanceLeaders extend
 
   byte[] getZkData(CloudSolrClient client, String path) {
     org.apache.zookeeper.data.Stat stat = new org.apache.zookeeper.data.Stat();
-    long start = System.currentTimeMillis();
     try {
       byte[] data = client.getZkStateReader().getZkClient().getData(path, null, stat, true);
       if (data != null) {
@@ -299,8 +300,8 @@ public class TestRebalanceLeaders extend
 
   boolean waitForAllPreferreds() throws KeeperException, InterruptedException {
     boolean goAgain = true;
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start < timeoutMs) {
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    while (! timeout.hasTimedOut()) {
       goAgain = false;
       cloudClient.getZkStateReader().updateClusterState();
       Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java Sat Aug  8 13:39:58 2015
@@ -33,6 +33,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 import java.io.File;
@@ -108,9 +109,9 @@ public class UnloadDistributedZkTest ext
 
       // there should be only one shard
       int slices = getCommonCloudSolrClient().getZkStateReader().getClusterState().getSlices(collection).size();
-      long timeoutAt = System.currentTimeMillis() + 45000;
+      final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
       while (slices != 1) {
-        if (System.currentTimeMillis() > timeoutAt) {
+        if (timeout.hasTimedOut()) {
           printLayout();
           fail("Expected to find only one slice in " + collection);
         }
@@ -127,9 +128,9 @@ public class UnloadDistributedZkTest ext
 
     //printLayout();
     // the collection should be gone
-    long timeoutAt = System.currentTimeMillis() + 30000;
+    final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
     while (getCommonCloudSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
-      if (System.currentTimeMillis() > timeoutAt) {
+      if (timeout.hasTimedOut()) {
         printLayout();
         fail("Still found collection");
       }
@@ -145,7 +146,7 @@ public class UnloadDistributedZkTest ext
   private void testCoreUnloadAndLeaders() throws Exception {
     File tmpDir = createTempDir().toFile();
 
-    String core1DataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_1n";
+    String core1DataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection1" + "_1n";
 
     // create a new collection collection
     SolrClient client = clients.get(0);
@@ -175,7 +176,7 @@ public class UnloadDistributedZkTest ext
       Create createCmd = new Create();
       createCmd.setCoreName("unloadcollection2");
       createCmd.setCollection("unloadcollection");
-      String core2dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_2n";
+      String core2dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection1" + "_2n";
       createCmd.setDataDir(getDataDir(core2dataDir));
       adminClient.request(createCmd);
     }
@@ -212,7 +213,7 @@ public class UnloadDistributedZkTest ext
       Create createCmd = new Create();
       createCmd.setCoreName("unloadcollection3");
       createCmd.setCollection("unloadcollection");
-      String core3dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_3n";
+      String core3dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection" + "_3n";
       createCmd.setDataDir(getDataDir(core3dataDir));
       adminClient.request(createCmd);
     }
@@ -282,7 +283,7 @@ public class UnloadDistributedZkTest ext
       Create createCmd = new Create();
       createCmd.setCoreName("unloadcollection4");
       createCmd.setCollection("unloadcollection");
-      String core4dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_4n";
+      String core4dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection" + "_4n";
       createCmd.setDataDir(getDataDir(core4dataDir));
       adminClient.request(createCmd);
     }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java Sat Aug  8 13:39:58 2015
@@ -186,7 +186,7 @@ public class ZkCLITest extends SolrTestC
         "list"};
     ZkCLI.main(args);
   }
-  
+
   @Test
   public void testUpConfigLinkConfigClearZk() throws Exception {
     File tmpDir = createTempDir().toFile();
@@ -218,7 +218,7 @@ public class ZkCLITest extends SolrTestC
     
     // test down config
     File confDir = new File(tmpDir,
-        "solrtest-confdropspot-" + this.getClass().getName() + "-" + System.currentTimeMillis());
+        "solrtest-confdropspot-" + this.getClass().getName() + "-" + System.nanoTime());
     assertFalse(confDir.exists());
 
     args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
@@ -268,7 +268,7 @@ public class ZkCLITest extends SolrTestC
     this.zkClient.create(getNode, data, CreateMode.PERSISTENT, true);
 
     File file = new File(tmpDir,
-        "solrtest-getfile-" + this.getClass().getName() + "-" + System.currentTimeMillis());
+        "solrtest-getfile-" + this.getClass().getName() + "-" + System.nanoTime());
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
         "getfile", getNode, file.getAbsolutePath()};
     ZkCLI.main(args);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java Sat Aug  8 13:39:58 2015
@@ -37,6 +37,7 @@ import org.apache.solr.common.params.Col
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.BadHdfsThreadsFilter;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -49,6 +50,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
 
 @Slow
 @ThreadLeakFilters(defaultFilters = true, filters = {
@@ -202,10 +204,10 @@ public class StressHdfsTest extends Basi
     QueryRequest request = new QueryRequest(params);
     request.setPath("/admin/collections");
     cloudClient.request(request);
-    
-    long timeout = System.currentTimeMillis() + 10000;
+
+    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
     while (cloudClient.getZkStateReader().getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) {
-      if (System.currentTimeMillis() > timeout) {
+      if (timeout.hasTimedOut()) {
         throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");
       }
       

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java Sat Aug  8 13:39:58 2015
@@ -27,6 +27,7 @@ import org.apache.solr.client.solrj.resp
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -40,6 +41,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -309,8 +311,6 @@ public class OpenCloseCoreStressTest ext
 }
 
 class Indexer {
-  static volatile long stopTime;
-
   static AtomicInteger idUnique = new AtomicInteger(0);
 
   static AtomicInteger errors = new AtomicInteger(0);
@@ -322,13 +322,15 @@ class Indexer {
   static AtomicInteger updateCounts = new AtomicInteger(0);
 
   static volatile int lastCount;
-  static volatile long nextTime;
+
+  static volatile TimeOut stopTimeout;
+  private static volatile TimeOut nextTimeout;
 
   ArrayList<OneIndexer> _threads = new ArrayList<>();
 
   public Indexer(OpenCloseCoreStressTest OCCST, String url, List<HttpSolrClient> clients, int numThreads, int secondsToRun, Random random) {
-    stopTime = System.currentTimeMillis() + (secondsToRun * 1000);
-    nextTime = System.currentTimeMillis() + 60000;
+    stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS);
+    nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
     docsThisCycle.set(0);
     qTimesAccum.set(0);
     updateCounts.set(0);
@@ -358,11 +360,11 @@ class Indexer {
   }
 
   synchronized static void progress(int myId, String core) {
-    if (nextTime - System.currentTimeMillis() <= 0) {
+    if (nextTimeout.hasTimedOut()) {
       SolrTestCaseJ4.log.info(String.format(Locale.ROOT, " s indexed: [run %,8d] [cycle %,8d] [last minute %,8d] Last core updated: %s. Seconds left in cycle %,4d",
-          myId, docsThisCycle.get(), myId - lastCount, core, stopTime - (System.currentTimeMillis() / 1000)));
+          myId, docsThisCycle.get(), myId - lastCount, core, stopTimeout.timeLeft(TimeUnit.SECONDS)));
       lastCount = myId;
-      nextTime += (System.currentTimeMillis() / 1000) * 60;
+      nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
     }
   }
 
@@ -385,7 +387,7 @@ class OneIndexer extends Thread {
   public void run() {
     SolrTestCaseJ4.log.info(String.format(Locale.ROOT, "Starting indexing thread: " + getId()));
 
-    while (Indexer.stopTime > System.currentTimeMillis()) {
+    while (! Indexer.stopTimeout.hasTimedOut()) {
       int myId = Indexer.idUnique.incrementAndGet();
       Indexer.docsThisCycle.incrementAndGet();
       String core = OCCST.getRandomCore(random);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Sat Aug  8 13:39:58 2015
@@ -637,7 +637,7 @@ public class TestCoreContainer extends S
 
   private long getCoreStartTime(final CoreContainer cc, final String name) {
     try (SolrCore tmp = cc.getCore(name)) {
-      return tmp.getStartTime();
+      return tmp.getStartTimeStamp().getTime();
     }
   }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java Sat Aug  8 13:39:58 2015
@@ -37,6 +37,7 @@ import org.apache.solr.common.cloud.ZkSt
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigOverlay;
+import org.apache.solr.util.RTimer;
 import org.apache.solr.util.SimplePostTool;
 import org.junit.Test;
 import org.noggit.JSONParser;
@@ -120,7 +121,7 @@ public class TestBlobHandler extends Abs
     String url;
     Map map = null;
     List l;
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
     int i = 0;
     for (; i < 150; i++) {//15 secs
       url = baseUrl + "/.system/blob/" + blobName;
@@ -137,7 +138,7 @@ public class TestBlobHandler extends Abs
       return;
     }
     fail(StrUtils.formatString("Could not successfully add blob after {0} attempts. Expecting {1} items. time elapsed {2}  output  for url is {3}",
-        i, count, System.currentTimeMillis() - start, getAsString(map)));
+        i, count, timer.getTime(), getAsString(map)));
   }
 
   public static String getAsString(Map map) {

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java Sat Aug  8 13:39:58 2015
@@ -195,7 +195,7 @@ public class CoreAdminHandlerTest extend
   @Test
   public void testDeleteInstanceDir() throws Exception  {
     File solrHomeDirectory = new File(initCoreDataDir, getClass().getName() + "-corex-"
-        + System.currentTimeMillis());
+        + System.nanoTime());
     solrHomeDirectory.mkdirs();
     copySolrHomeToTemp(solrHomeDirectory, "corex", true);
     File corex = new File(solrHomeDirectory, "corex");

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java Sat Aug  8 13:39:58 2015
@@ -31,6 +31,7 @@ import org.apache.solr.response.BinaryQu
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.util.RTimer;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -109,7 +110,7 @@ public class TestWriterPerf extends Abst
     ByteArrayOutputStream out=null;
 
     System.gc();
-    long start = System.currentTimeMillis();
+    RTimer timer = new RTimer();
     for (int i=0; i<encIter; i++) {
     if (w instanceof BinaryQueryResponseWriter) {
       BinaryQueryResponseWriter binWriter = (BinaryQueryResponseWriter) w;
@@ -126,11 +127,11 @@ public class TestWriterPerf extends Abst
     }
     }
 
-    long encodeTime = Math.max(System.currentTimeMillis() - start, 1);
+    double encodeTime = timer.getTime();
 
     byte[] arr = out.toByteArray();
 
-    start = System.currentTimeMillis();
+    timer = new RTimer();
     writerName = writerName.intern();
     for (int i=0; i<decIter; i++) {
       ResponseParser rp = null;
@@ -145,9 +146,9 @@ public class TestWriterPerf extends Abst
       rp.processResponse(in, "UTF-8");      
     }
 
-    long decodeTime = Math.max(System.currentTimeMillis() - start, 1);
+    double decodeTime = timer.getTime();
 
-    log.info("writer "+writerName+", size="+out.size()+", encodeRate="+(encodeTime==1 ? "N/A":  ""+(encIter*1000L/encodeTime)) + ", decodeRate="+(decodeTime==1 ? "N/A":  ""+(decIter*1000L/decodeTime)) );
+    log.info("writer "+writerName+", size="+out.size()+", encodeRate="+(encIter*1000L/encodeTime) + ", decodeRate="+(decIter*1000L/decodeTime));
 
     req.close();
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/AbstractCurrencyFieldTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/AbstractCurrencyFieldTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/AbstractCurrencyFieldTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/AbstractCurrencyFieldTest.java Sat Aug  8 13:39:58 2015
@@ -19,6 +19,7 @@ package org.apache.solr.schema;
 import org.apache.lucene.index.StorableField;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.RTimer;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -260,25 +261,25 @@ public abstract class AbstractCurrencyFi
     }
 
     for (int j = 0; j < 3; j++) {
-      long t1 = System.currentTimeMillis();
+      final RTimer timer = new RTimer();
       for (int i = 0; i < 1000; i++) {
         double lower = r.nextInt(10) + 1.00;
         assertQ(req("fl", "*,score", "q", field()+":[" +  lower + ",USD TO " + (lower + (9.99 - (j * 0.01))) + ",USD]"), "//*");
       }
 
-      System.out.println(System.currentTimeMillis() - t1);
+      System.out.println(timer.getTime());
     }
 
     System.out.println("---");
 
     for (int j = 0; j < 3; j++) {
-      long t1 = System.currentTimeMillis();
+      final RTimer timer = new RTimer();
       for (int i = 0; i < 1000; i++) {
         double lower = r.nextInt(10) + 1.00;
         assertQ(req("fl", "*,score", "q", field()+":[" +  lower + ",EUR TO " + (lower + (9.99 - (j * 0.01))) + ",EUR]"), "//*");
       }
 
-      System.out.println(System.currentTimeMillis() - t1);
+      System.out.println(timer.getTime());
     }
   }
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java Sat Aug  8 13:39:58 2015
@@ -19,6 +19,7 @@ package org.apache.solr.schema;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.SolrResourceLoader;
 import org.junit.Before;
 import org.junit.Test;
@@ -86,6 +87,7 @@ public class OpenExchangeRatesOrgProvide
     assertEquals("USD", oerp.rates.getBaseCurrency());
   }
 
+  @SuppressForbidden(reason = "Needs currentTimeMillis to construct rates file contents")
   @Test
   public void testReload() {
     // reminder: interval is in minutes

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/DocSetPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/DocSetPerf.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/DocSetPerf.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/DocSetPerf.java Sat Aug  8 13:39:58 2015
@@ -21,6 +21,7 @@ import java.util.Random;
 
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.SuppressForbidden;
+import org.apache.solr.util.RTimer;
 
 /**
  */
@@ -90,7 +91,7 @@ public class DocSetPerf {
       hset[i] = hds;
     }
 
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     if ("test".equals(test)) {
       for (int it=0; it<iter; it++) {
@@ -165,8 +166,7 @@ public class DocSetPerf {
       }
     }
 
-    long end = System.currentTimeMillis();
-    System.out.println("TIME="+(end-start));
+    System.out.println("TIME="+timer.getTime());
 
     System.out.println("ret="+ret);
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java Sat Aug  8 13:39:58 2015
@@ -19,6 +19,7 @@ package org.apache.solr.search;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ConcurrentLRUCache;
+import org.apache.solr.util.RTimer;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -254,7 +255,7 @@ public class TestFastLRUCache extends Lu
   }
 
   void doPerfTest(int iter, int cacheSize, int maxKey) {
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     int lowerWaterMark = cacheSize;
     int upperWaterMark = (int)(lowerWaterMark * 1.1);
@@ -276,8 +277,7 @@ public class TestFastLRUCache extends Lu
     }
     cache.destroy();
 
-    long end = System.currentTimeMillis();
-    System.out.println("time=" + (end-start) + ", minSize="+minSize+",maxSize="+maxSize);
+    System.out.println("time=" + timer.getTime() + ", minSize="+minSize+",maxSize="+maxSize);
   }
 
   /***
@@ -326,7 +326,7 @@ public class TestFastLRUCache extends Lu
 
     fillCache(sc, cacheSize, maxKey);
 
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     Thread[] threads = new Thread[nThreads];
     final AtomicInteger puts = new AtomicInteger(0);
@@ -357,8 +357,7 @@ public class TestFastLRUCache extends Lu
       }
     }
 
-    long end = System.currentTimeMillis();
-    System.out.println("time=" + (end-start) + " impl=" +sc.getClass().getSimpleName()
+    System.out.println("time=" + timer.getTime() + " impl=" +sc.getClass().getSimpleName()
                        +" nThreads= " + nThreads + " size="+cacheSize+" maxKey="+maxKey+" gets="+numGets
                        +" hitRatio="+(1-(((double)puts.get())/numGets)));
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java Sat Aug  8 13:39:58 2015
@@ -27,6 +27,7 @@ import org.apache.solr.update.processor.
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.RTimer;
 import org.junit.BeforeClass;
 
 import java.util.*;
@@ -139,7 +140,7 @@ public class TestSearchPerf extends Abst
 
     SolrIndexSearcher searcher = req.getSearcher();
 
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     int ret = 0;
     for (int i=0; i<iter; i++) {
@@ -147,8 +148,8 @@ public class TestSearchPerf extends Abst
       ret += set.size();
     }
 
-    long end = System.currentTimeMillis();
-    System.out.println("ret="+ret+ " time="+(end-start)+" throughput="+iter*1000/(end-start+1));
+    double elapsed = timer.getTime();
+    System.out.println("ret="+ret+ " time="+elapsed+" throughput="+iter*1000/(elapsed+1));
 
     req.close();
     assertTrue(ret>0);  // make sure we did some work
@@ -160,7 +161,7 @@ public class TestSearchPerf extends Abst
 
     SolrIndexSearcher searcher = req.getSearcher();
 
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     // These aren't public in SolrIndexSearcher
     int NO_CHECK_QCACHE       = 0x80000000;
@@ -174,8 +175,8 @@ public class TestSearchPerf extends Abst
       ret += l.matches();
     }
 
-    long end = System.currentTimeMillis();
-    System.out.println("ret="+ret+ " time="+(end-start)+" throughput="+iter*1000/(end-start+1));
+    double elapsed = timer.getTime();
+    System.out.println("ret="+ret+ " time="+elapsed+" throughput="+iter*1000/(elapsed+1));
 
     req.close();
     assertTrue(ret>0);  // make sure we did some work

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSolrJ.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSolrJ.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSolrJ.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSolrJ.java Sat Aug  8 13:39:58 2015
@@ -24,6 +24,7 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.RTimer;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -64,7 +65,7 @@ public class TestSolrJ extends SolrTestC
     client.deleteByQuery("*:*");
     client.commit();
 
-    long start = System.currentTimeMillis();
+    final RTimer timer = new RTimer();
 
     final int docsPerThread = nDocs / nProducers;
 
@@ -98,8 +99,8 @@ public class TestSolrJ extends SolrTestC
       concurrentClient.blockUntilFinished();
     }
 
-    long end = System.currentTimeMillis();
-    System.out.println("time="+(end-start) + " throughput="+(nDocs*1000/(end-start)) + " Exception="+ex);
+    double elapsed = timer.getTime();
+    System.out.println("time="+elapsed + " throughput="+(nDocs*1000/elapsed) + " Exception="+ex);
 
     // should server threads be marked as daemon?
     // need a server.close()!!!
@@ -166,7 +167,7 @@ public class TestSolrJ extends SolrTestC
 
     try (HttpSolrClient client = new HttpSolrClient("http://127.0.0.1:8983/solr")) {
 
-      long start = System.currentTimeMillis();
+      final RTimer timer = new RTimer();
 
       for (int i = 0; i < 10000; i++) {
         SolrInputDocument doc = new SolrInputDocument();
@@ -175,8 +176,7 @@ public class TestSolrJ extends SolrTestC
         client.commit(true, true, true);
       }
 
-      long end = System.currentTimeMillis();
-      System.out.println("TIME: " + (end-start));
+      System.out.println("TIME: " + timer.getTime());
     }
 
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java Sat Aug  8 13:39:58 2015
@@ -17,17 +17,6 @@
 
 package org.apache.solr.search.function;
 
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.search.similarities.DefaultSimilarity;
-import org.apache.lucene.search.similarities.TFIDFSimilarity;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Ignore;
-
 import java.io.FileOutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
@@ -37,6 +26,13 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.search.similarities.DefaultSimilarity;
+import org.apache.lucene.search.similarities.TFIDFSimilarity;
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
 /**
  * Tests some basic functionality of Solr while demonstrating good
  * Best Practices for using AbstractSolrTestCase
@@ -49,7 +45,8 @@ public class TestFunctionQuery extends S
 
   
   String base = "external_foo_extf";
-  static long start = System.currentTimeMillis();
+
+  static long start = System.nanoTime();
   
   void makeExternalFile(String field, String contents) {
     String dir = h.getCore().getDataDir();

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java Sat Aug  8 13:39:58 2015
@@ -23,6 +23,7 @@ import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.impl.cookie.DateUtils;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -77,6 +78,7 @@ public class CacheHeaderTest extends Cac
     checkVetoHeaders(response, false);
   }
 
+  @SuppressForbidden(reason = "Needs currentTimeMillis to check against expiry headers from Solr")
   protected void checkVetoHeaders(HttpResponse response, boolean checkExpires) throws Exception {
     Header head = response.getFirstHeader("Cache-Control");
     assertNotNull("We got no Cache-Control header", head);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/NoCacheHeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/NoCacheHeaderTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/NoCacheHeaderTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/NoCacheHeaderTest.java Sat Aug  8 13:39:58 2015
@@ -22,6 +22,7 @@ import org.apache.http.Header;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.impl.cookie.DateUtils;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -58,7 +59,8 @@ public class NoCacheHeaderTest extends C
     doCacheControl("HEAD");
     doCacheControl("POST");
   }
-  
+
+  @SuppressForbidden(reason = "Needs currentTimeMillis for testing caching headers")
   @Override
   protected void doLastModified(String method) throws Exception {
     // We do a first request to get the last modified

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java Sat Aug  8 13:39:58 2015
@@ -20,6 +20,7 @@ package org.apache.solr.update;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.common.params.MapSolrParams;
@@ -34,6 +35,7 @@ import org.apache.solr.response.SolrQuer
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 
 class NewSearcherListener implements SolrEventListener {
@@ -85,9 +87,9 @@ class NewSearcherListener implements Sol
     // log.info("TEST: trigger reset");
   }
 
-  boolean waitForNewSearcher(int timeout) {
-    long timeoutTime = System.currentTimeMillis() + timeout;
-    while (System.currentTimeMillis() < timeoutTime) {
+  boolean waitForNewSearcher(int timeoutMs) {
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    while (! timeout.hasTimedOut()) {
       if (triggered) {
         // check if the new searcher has been registered yet
         RefCounted<SolrIndexSearcher> registeredSearcherH = newSearcher.getCore().getSearcher();

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java?rev=1694798&r1=1694797&r2=1694798&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java Sat Aug  8 13:39:58 2015
@@ -371,13 +371,13 @@ public class DirectUpdateHandlerTest ext
 
     @Override
     public void postSoftCommit() {
-      postSoftCommitAt.set(System.currentTimeMillis());
+      postSoftCommitAt.set(System.nanoTime());
     }
 
     @Override
     public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
       newSearcherCount.incrementAndGet();
-      newSearcherOpenedAt.set(newSearcher.getOpenTime());
+      newSearcherOpenedAt.set(newSearcher.getOpenNanoTime());
     }
 
     @Override