You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2022/09/06 22:11:41 UTC

[lucene] branch branch_9x updated (299ba5e1541 -> 2895887cacb)

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

rmuir pushed a change to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


    from 299ba5e1541 Add tidy after addVersion is called. (#11748)
     new fb40a43c0de LUCENE-10423: Remove usages of System.currentTimeMillis() from tests (#11749)
     new 2895887cacb sync CHANGES.txt with trunk

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


Summary of changes:
 .../validation/forbidden-apis/defaults.tests.txt   |  2 +-
 lucene/CHANGES.txt                                 | 23 ++------------
 .../lucene/analysis/hunspell/TestPerformance.java  |  4 +--
 .../pattern/TestPatternReplaceCharFilter.java      |  4 +--
 .../analysis/sinks/TestTeeSinkTokenFilter.java     | 12 ++++----
 .../lucene/analysis/ja/TestJapaneseTokenizer.java  |  8 ++---
 .../benchmark/byTask/TestPerfTasksLogic.java       |  3 +-
 .../benchmark/byTask/feeds/TestDocMaker.java       |  2 +-
 .../byTask/tasks/CountingSearchTestTask.java       | 14 ++++-----
 .../Test20NewsgroupsClassification.java            | 18 +++++------
 .../test/org/apache/lucene/index/Test2BTerms.java  | 10 +++----
 .../apache/lucene/index/TestDeletionPolicy.java    |  6 ++--
 .../index/TestDocumentsWriterStallControl.java     |  6 ++--
 .../apache/lucene/index/TestIndexWriterCommit.java |  6 ++--
 .../lucene/index/TestIndexWriterExceptions.java    |  6 ++--
 .../org/apache/lucene/index/TestNRTThreads.java    |  8 ++---
 .../org/apache/lucene/index/TestNeverDelete.java   |  7 +++--
 .../lucene/index/TestSnapshotDeletionPolicy.java   |  6 ++--
 .../apache/lucene/index/TestStressIndexing.java    |  7 ++---
 .../org/apache/lucene/index/TestTermdocPerf.java   | 14 +++++----
 .../org/apache/lucene/index/TestTransactions.java  |  8 ++---
 .../search/TestControlledRealTimeReopenThread.java | 10 +++----
 .../apache/lucene/search/TestSearcherManager.java  |  7 +++--
 .../org/apache/lucene/util/SelectorBenchmark.java  |  3 +-
 .../org/apache/lucene/util/SorterBenchmark.java    |  3 +-
 .../lucene/util/fst/TestFSTDirectAddressing.java   | 18 +++++------
 .../test/org/apache/lucene/util/fst/TestFSTs.java  | 18 +++++------
 .../idversion/TestIDVersionPostingsFormat.java     |  9 +++---
 .../lucene/search/spell/TestSpellChecker.java      |  3 --
 .../ThreadedIndexingAndSearchingTestCase.java      | 35 ++++++++++++----------
 .../org/apache/lucene/tests/search/QueryUtils.java |  4 +--
 31 files changed, 136 insertions(+), 148 deletions(-)


[lucene] 02/02: sync CHANGES.txt with trunk

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

rmuir pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 2895887cacb4d34b0fb45b59545be6ab4a716a71
Author: Robert Muir <rm...@apache.org>
AuthorDate: Tue Sep 6 18:06:26 2022 -0400

    sync CHANGES.txt with trunk
---
 lucene/CHANGES.txt | 26 --------------------------
 1 file changed, 26 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 00d3a07011d..b93e2ecd7ea 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -5,32 +5,6 @@ http://s.apache.org/luceneversions
 
 ======================== Lucene 9.5.0 =======================
 
-API Changes
----------------------
-(No changes)
-
-New Features
----------------------
-(No changes)
-
-Improvements
----------------------
-(No changes)
-
-Optimizations
----------------------
-(No changes)
-
-Bug Fixes
----------------------
-(No changes)
-
-Other
----------------------
-(No changes)
-
-======================== Lucene 9.5.0 =======================
-
 Other
 ---------------------
 * LUCENE-10423: Remove usages of System.currentTimeMillis() from tests. (Marios Trivyzas)


[lucene] 01/02: LUCENE-10423: Remove usages of System.currentTimeMillis() from tests (#11749)

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

rmuir pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit fb40a43c0de65950e51380e9bc0b409030c53068
Author: Marios Trivyzas <50...@users.noreply.github.com>
AuthorDate: Wed Sep 7 00:55:01 2022 +0300

    LUCENE-10423: Remove usages of System.currentTimeMillis() from tests (#11749)
    
    * Remove usages of System.currentTimeMillis() from tests
    
    - Use Random from `RandomizedRunner` to be able to use a Seed to
      reproduce tests, instead of a seed coming from wall clock.
    - Replace time based tests, using wall clock to determine periods
      with counter of repetitions, to have a consistent reproduction.
    
    Closes: #11459
    
    * address comments
    
    * tune iterations
    
    * tune iterations for nightly
---
 .../validation/forbidden-apis/defaults.tests.txt   |  2 +-
 lucene/CHANGES.txt                                 |  7 +++++
 .../lucene/analysis/hunspell/TestPerformance.java  |  4 +--
 .../pattern/TestPatternReplaceCharFilter.java      |  4 +--
 .../analysis/sinks/TestTeeSinkTokenFilter.java     | 12 ++++----
 .../lucene/analysis/ja/TestJapaneseTokenizer.java  |  8 ++---
 .../benchmark/byTask/TestPerfTasksLogic.java       |  3 +-
 .../benchmark/byTask/feeds/TestDocMaker.java       |  2 +-
 .../byTask/tasks/CountingSearchTestTask.java       | 14 ++++-----
 .../Test20NewsgroupsClassification.java            | 18 +++++------
 .../test/org/apache/lucene/index/Test2BTerms.java  | 10 +++----
 .../apache/lucene/index/TestDeletionPolicy.java    |  6 ++--
 .../index/TestDocumentsWriterStallControl.java     |  6 ++--
 .../apache/lucene/index/TestIndexWriterCommit.java |  6 ++--
 .../lucene/index/TestIndexWriterExceptions.java    |  6 ++--
 .../org/apache/lucene/index/TestNRTThreads.java    |  8 ++---
 .../org/apache/lucene/index/TestNeverDelete.java   |  7 +++--
 .../lucene/index/TestSnapshotDeletionPolicy.java   |  6 ++--
 .../apache/lucene/index/TestStressIndexing.java    |  7 ++---
 .../org/apache/lucene/index/TestTermdocPerf.java   | 14 +++++----
 .../org/apache/lucene/index/TestTransactions.java  |  8 ++---
 .../search/TestControlledRealTimeReopenThread.java | 10 +++----
 .../apache/lucene/search/TestSearcherManager.java  |  7 +++--
 .../org/apache/lucene/util/SelectorBenchmark.java  |  3 +-
 .../org/apache/lucene/util/SorterBenchmark.java    |  3 +-
 .../lucene/util/fst/TestFSTDirectAddressing.java   | 18 +++++------
 .../test/org/apache/lucene/util/fst/TestFSTs.java  | 18 +++++------
 .../idversion/TestIDVersionPostingsFormat.java     |  9 +++---
 .../lucene/search/spell/TestSpellChecker.java      |  3 --
 .../ThreadedIndexingAndSearchingTestCase.java      | 35 ++++++++++++----------
 .../org/apache/lucene/tests/search/QueryUtils.java |  4 +--
 31 files changed, 141 insertions(+), 127 deletions(-)

diff --git a/gradle/validation/forbidden-apis/defaults.tests.txt b/gradle/validation/forbidden-apis/defaults.tests.txt
index 3eb9de50b79..bb491804e06 100644
--- a/gradle/validation/forbidden-apis/defaults.tests.txt
+++ b/gradle/validation/forbidden-apis/defaults.tests.txt
@@ -17,7 +17,7 @@ java.util.Random#<init>() @ Use RandomizedRunner's random() instead
 java.lang.Math#random() @ Use RandomizedRunner's random().nextDouble() instead
 
 # TODO: fix tests that do this!
-#java.lang.System#currentTimeMillis() @ Don't depend on wall clock times
+java.lang.System#currentTimeMillis() @ Don't depend on wall clock times
 #java.lang.System#nanoTime() @ Don't depend on wall clock times
 
 java.text.Collator#compare(**) @ Use LuceneTestCase.collate instead, which can avoid JDK-8071862
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 395859e25a2..00d3a07011d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -29,6 +29,13 @@ Other
 ---------------------
 (No changes)
 
+======================== Lucene 9.5.0 =======================
+
+Other
+---------------------
+* LUCENE-10423: Remove usages of System.currentTimeMillis() from tests. (Marios Trivyzas)
+
+
 ======================== Lucene 9.4.0 =======================
 
 API Changes
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
index 61410c9939e..0342b3a671b 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
@@ -257,9 +257,9 @@ public class TestPerformance extends LuceneTestCase {
 
     List<Long> times = new ArrayList<>();
     for (int i = 0; i < 7; i++) {
-      long start = System.currentTimeMillis();
+      long start = System.nanoTime();
       iteration.run(consumer);
-      times.add(System.currentTimeMillis() - start);
+      times.add(System.nanoTime() - start / 100_000);
     }
     System.out.println(
         what
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java
index 119859bc775..d73ef8f7230 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java
@@ -257,11 +257,11 @@ public class TestPatternReplaceCharFilter extends BaseTokenStreamTestCase {
         "[;<!--aecbbaa--><    febcfdc fbb = \"fbeeebff\" fc = dd   >\\';<eefceceaa e= babae\" eacbaff =\"fcfaccacd\" = bcced>>><  bccaafe edb = ecfccdff\"   <?</script><    edbd ebbcd=\"faacfcc\" aeca= bedbc ceeaac =adeafde aadccdaf = \"afcc ffda=aafbe &#x16921ed5\"1843785582']";
     for (int i = 0; i < input.length(); i++) {
       Matcher matcher = p.matcher(input.substring(0, i));
-      long t = System.currentTimeMillis();
+      long t = System.nanoTime();
       if (matcher.find()) {
         System.out.println(matcher.group());
       }
-      System.out.println(i + " > " + (System.currentTimeMillis() - t) / 1000.0);
+      System.out.println(i + " > " + (System.nanoTime() - t) / 1_000_000_000.0);
     }
   }
 
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
index 74863fd1264..36e7914c307 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
@@ -183,7 +183,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
       // simulate two fields, each being analyzed once, for 20 documents
       for (int j = 0; j < modCounts.length; j++) {
         int tfPos = 0;
-        long start = System.currentTimeMillis();
+        long start = System.nanoTime();
         for (int i = 0; i < 20; i++) {
           stream = standardTokenizer(buffer);
           PositionIncrementAttribute posIncrAtt =
@@ -197,12 +197,12 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
             tfPos += posIncrAtt.getPositionIncrement();
           }
         }
-        long finish = System.currentTimeMillis();
+        long finish = System.nanoTime();
         System.out.println(
-            "ModCount: " + modCounts[j] + " Two fields took " + (finish - start) + " ms");
+            "ModCount: " + modCounts[j] + " Two fields took " + (finish - start) / 100_000 + " ms");
         int sinkPos = 0;
         // simulate one field with one sink
-        start = System.currentTimeMillis();
+        start = System.nanoTime();
         for (int i = 0; i < 20; i++) {
           teeStream = new TeeSinkTokenFilter(standardTokenizer(buffer));
           sink = new ModuloTokenFilter(teeStream.newSinkTokenStream(), modCounts[j]);
@@ -217,9 +217,9 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
             sinkPos += posIncrAtt.getPositionIncrement();
           }
         }
-        finish = System.currentTimeMillis();
+        finish = System.nanoTime();
         System.out.println(
-            "ModCount: " + modCounts[j] + " Tee fields took " + (finish - start) + " ms");
+            "ModCount: " + modCounts[j] + " Tee fields took " + (finish - start) / 100_000 + " ms");
         assertTrue(sinkPos + " does not equal: " + tfPos, sinkPos == tfPos);
       }
       System.out.println("- End Tokens: " + tokCount[k] + "-----");
diff --git a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java
index 3cb5870180c..cc7798402ea 100644
--- a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java
+++ b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java
@@ -779,7 +779,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
     }
     */
 
-    long totalStart = System.currentTimeMillis();
+    long totalStart = System.nanoTime();
     for (int i = 0; i < numIterations; i++) {
       try (TokenStream ts = analyzer.tokenStream("ignored", line)) {
         ts.reset();
@@ -790,11 +790,11 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
     }
     String[] sentences = line.split("、|。");
     if (VERBOSE) {
-      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+      System.out.println("Total time : " + (System.nanoTime() - totalStart) / 100_000);
       System.out.println(
           "Test for Bocchan with pre-splitting sentences (" + sentences.length + " sentences)");
     }
-    totalStart = System.currentTimeMillis();
+    totalStart = System.nanoTime();
     for (int i = 0; i < numIterations; i++) {
       for (String sentence : sentences) {
         try (TokenStream ts = analyzer.tokenStream("ignored", sentence)) {
@@ -806,7 +806,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
       }
     }
     if (VERBOSE) {
-      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+      System.out.println("Total time : " + (System.nanoTime() - totalStart) / 100_000);
     }
   }
 
diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
index 4b8b390f4c6..d83a3d960bb 100644
--- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
+++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
@@ -125,7 +125,8 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
     CountingSearchTestTask.numSearches = 0;
     execBenchmark(algLines);
     assertTrue(CountingSearchTestTask.numSearches > 0);
-    long elapsed = CountingSearchTestTask.prevLastMillis - CountingSearchTestTask.startMillis;
+    long elapsed =
+        (CountingSearchTestTask.prevLastNanos - CountingSearchTestTask.startNanos) / 1_000_000;
     assertTrue("elapsed time was " + elapsed + " msec", elapsed <= 1500);
   }
 
diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TestDocMaker.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TestDocMaker.java
index 33ce32d61b8..b3d76edaefb 100644
--- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TestDocMaker.java
+++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TestDocMaker.java
@@ -165,7 +165,7 @@ public class TestDocMaker extends BenchmarkTestCase {
     // leading to a file handle leak.
     Path f = getWorkDir().resolve("docMakerLeak.txt");
     PrintStream ps = new PrintStream(Files.newOutputStream(f), true, IOUtils.UTF_8);
-    ps.println("one title\t" + System.currentTimeMillis() + "\tsome content");
+    ps.println("one title\t" + random().nextLong() + "\tsome content");
     ps.close();
 
     Properties props = new Properties();
diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/CountingSearchTestTask.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/CountingSearchTestTask.java
index 2fc75487493..8238ac08d5a 100644
--- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/CountingSearchTestTask.java
+++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/CountingSearchTestTask.java
@@ -22,9 +22,9 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
 public class CountingSearchTestTask extends SearchTask {
 
   public static int numSearches = 0;
-  public static long startMillis;
-  public static long lastMillis;
-  public static long prevLastMillis;
+  public static long startNanos;
+  public static long lastNanos;
+  public static long prevLastNanos;
 
   public CountingSearchTestTask(PerfRunData runData) {
     super(runData);
@@ -38,15 +38,15 @@ public class CountingSearchTestTask extends SearchTask {
   }
 
   private static synchronized void incrNumSearches() {
-    prevLastMillis = lastMillis;
-    lastMillis = System.currentTimeMillis();
+    prevLastNanos = lastNanos;
+    lastNanos = System.nanoTime();
     if (0 == numSearches) {
-      startMillis = prevLastMillis = lastMillis;
+      startNanos = prevLastNanos = lastNanos;
     }
     numSearches++;
   }
 
   public long getElapsedMillis() {
-    return lastMillis - startMillis;
+    return lastNanos - startNanos / 1_000_000;
   }
 }
diff --git a/lucene/classification/src/test/org/apache/lucene/classification/Test20NewsgroupsClassification.java b/lucene/classification/src/test/org/apache/lucene/classification/Test20NewsgroupsClassification.java
index 1cb03f618dd..12eb045b4fd 100644
--- a/lucene/classification/src/test/org/apache/lucene/classification/Test20NewsgroupsClassification.java
+++ b/lucene/classification/src/test/org/apache/lucene/classification/Test20NewsgroupsClassification.java
@@ -123,15 +123,15 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
 
         System.out.println("Indexing 20 Newsgroups...");
 
-        long startIndex = System.currentTimeMillis();
+        long startIndex = System.nanoTime();
         IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(analyzer));
 
         Path indexDir = Paths.get(INDEX_DIR);
         int docsIndexed = buildIndex(indexDir, indexWriter);
 
-        long endIndex = System.currentTimeMillis();
+        long endIndex = System.nanoTime();
         System.out.println(
-            "Indexed " + docsIndexed + " docs in " + (endIndex - startIndex) / 1000 + "s");
+            "Indexed " + docsIndexed + " docs in " + (endIndex - startIndex) / 1_000_000_000 + "s");
 
         indexWriter.close();
       }
@@ -145,7 +145,7 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
       if (index && split) {
         System.out.println("Splitting the index...");
 
-        long startSplit = System.currentTimeMillis();
+        long startSplit = System.nanoTime();
         DatasetSplitter datasetSplitter = new DatasetSplitter(0.2, 0);
         datasetSplitter.split(
             reader,
@@ -160,8 +160,8 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
             CATEGORY_FIELD);
         reader.close();
         reader = DirectoryReader.open(train); // using the train index from now on
-        long endSplit = System.currentTimeMillis();
-        System.out.println("Splitting done in " + (endSplit - startSplit) / 1000 + "s");
+        long endSplit = System.nanoTime();
+        System.out.println("Splitting done in " + (endSplit - startSplit) / 1_000_000_000 + "s");
       }
 
       classifiers.add(
@@ -359,7 +359,7 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
     futures.add(
         service.submit(
             () -> {
-              final long startTime = System.currentTimeMillis();
+              final long startTime = System.nanoTime();
               ConfusionMatrixGenerator.ConfusionMatrix confusionMatrix;
               if (split) {
                 confusionMatrix =
@@ -370,8 +370,8 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
                     ConfusionMatrixGenerator.getConfusionMatrix(
                         ar, classifier, CATEGORY_FIELD, BODY_FIELD, 60000 * 30);
               }
-              final long endTime = System.currentTimeMillis();
-              final int elapse = (int) (endTime - startTime) / 1000;
+              final long endTime = System.nanoTime();
+              final int elapse = (int) (endTime - startTime) / 1_000_000_000;
 
               return " * "
                   + classifier
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
index 7d7872c7d17..27c08adb545 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
@@ -199,10 +199,10 @@ public class Test2BTerms extends LuceneTestCase {
       System.out.println("numDocs=" + numDocs);
 
       for (int i = 0; i < numDocs; i++) {
-        final long t0 = System.currentTimeMillis();
+        final long t0 = System.nanoTime();
         w.addDocument(doc);
         System.out.println(
-            i + " of " + numDocs + " " + (System.currentTimeMillis() - t0) + " msec");
+            i + " of " + numDocs + " " + (System.nanoTime() - t0) / 1_000_000 + " msec");
       }
       savedTerms = ts.savedTerms;
 
@@ -260,14 +260,14 @@ public class Test2BTerms extends LuceneTestCase {
     for (int iter = 0; iter < 10 * terms.size(); iter++) {
       final BytesRef term = terms.get(random().nextInt(terms.size()));
       System.out.println("TEST: search " + term);
-      final long t0 = System.currentTimeMillis();
+      final long t0 = System.nanoTime();
       final long count = s.count(new TermQuery(new Term("field", term)));
       if (count <= 0) {
         System.out.println("  FAILED: count=" + count);
         failed = true;
       }
-      final long t1 = System.currentTimeMillis();
-      System.out.println("  took " + (t1 - t0) + " millis");
+      final long t1 = System.nanoTime();
+      System.out.println("  took " + (t1 - t0) / 1_000_000 + " millis");
 
       TermsEnum.SeekStatus result = termsEnum.seekCeil(term);
       if (result != TermsEnum.SeekStatus.FOUND) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
index 83524d4f02f..0e7fadffa85 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
@@ -236,7 +236,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     ExpirationTimeDeletionPolicy policy =
         (ExpirationTimeDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     Map<String, String> commitData = new HashMap<>();
-    commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+    commitData.put("commitTime", String.valueOf(System.nanoTime()));
     writer.setLiveCommitData(commitData.entrySet());
     writer.commit();
     writer.close();
@@ -246,7 +246,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     while (policy.numDelete < targetNumDelete) {
       // Record last time when writer performed deletes of
       // past commits
-      lastDeleteTime = System.currentTimeMillis();
+      lastDeleteTime = System.nanoTime();
       conf =
           newIndexWriterConfig(new MockAnalyzer(random()))
               .setOpenMode(OpenMode.APPEND)
@@ -259,7 +259,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
         addDoc(writer);
       }
       commitData = new HashMap<>();
-      commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+      commitData.put("commitTime", String.valueOf(System.nanoTime()));
       writer.setLiveCommitData(commitData.entrySet());
       writer.commit();
       writer.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
index c6a7b29793e..9327e86469a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
@@ -73,12 +73,12 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
           };
     }
     start(stallThreads);
-    long time = System.currentTimeMillis();
     /*
-     * use a 100 sec timeout to make sure we not hang forever. join will fail in
+     * use a 100 maximum iterations check to make sure we not hang forever. join will fail in
      * that case
      */
-    while ((System.currentTimeMillis() - time) < 100 * 1000 && !terminated(stallThreads)) {
+    int iterations = 0;
+    while (++iterations < 100 && !terminated(stallThreads)) {
       ctrl.updateStalled(false);
       if (random().nextBoolean()) {
         Thread.yield();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
index 1cba793c8c8..600dbbec870 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
@@ -346,7 +346,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
   // index
   public void testCommitThreadSafety() throws Throwable {
     final int NUM_THREADS = 5;
-    final double RUN_SEC = 0.5;
+    final int maxIterations = 10;
     final Directory dir = newDirectory();
     final RandomIndexWriter w =
         new RandomIndexWriter(
@@ -357,7 +357,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
     w.commit();
     final AtomicBoolean failed = new AtomicBoolean();
     Thread[] threads = new Thread[NUM_THREADS];
-    final long endTime = System.currentTimeMillis() + ((long) (RUN_SEC * 1000));
     for (int i = 0; i < NUM_THREADS; i++) {
       final int finalI = i;
       threads[i] =
@@ -369,6 +368,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
                 DirectoryReader r = DirectoryReader.open(dir);
                 Field f = newStringField("f", "", Field.Store.NO);
                 doc.add(f);
+                int iterations = 0;
                 int count = 0;
                 do {
                   if (failed.get()) break;
@@ -384,7 +384,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
                     r = r2;
                     assertEquals("term=f:" + s + "; r=" + r, 1, r.docFreq(new Term("f", s)));
                   }
-                } while (System.currentTimeMillis() < endTime);
+                } while (++iterations < maxIterations);
                 r.close();
               } catch (Throwable t) {
                 failed.set(true);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
index ee634b49be8..ec904f5531e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
@@ -170,8 +170,8 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
       final Field idField = newField(r, "id", "", DocCopyIterator.custom2);
       doc.add(idField);
 
-      final long stopTime = System.currentTimeMillis() + 500;
-
+      final int maxIterations = 250;
+      int iterations = 0;
       do {
         if (VERBOSE) {
           System.out.println(Thread.currentThread().getName() + ": TEST: IndexerThread: cycle");
@@ -219,7 +219,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
           failure = t;
           break;
         }
-      } while (System.currentTimeMillis() < stopTime);
+      } while (++iterations < maxIterations);
     }
   }
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java b/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java
index 86278809283..91bc8fa3f5d 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java
@@ -42,13 +42,13 @@ public class TestNRTThreads extends ThreadedIndexingAndSearchingTestCase {
   }
 
   @Override
-  protected void doSearching(ExecutorService es, long stopTime) throws Exception {
+  protected void doSearching(ExecutorService es, int maxIterations) throws Exception {
 
     boolean anyOpenDelFiles = false;
 
     DirectoryReader r = DirectoryReader.open(writer);
-
-    while (System.currentTimeMillis() < stopTime && !failed.get()) {
+    int iterations = 0;
+    while (++iterations < maxIterations && !failed.get()) {
       if (random().nextBoolean()) {
         if (VERBOSE) {
           System.out.println("TEST: now reopen r=" + r);
@@ -84,7 +84,7 @@ public class TestNRTThreads extends ThreadedIndexingAndSearchingTestCase {
       if (r.numDocs() > 0) {
         fixedSearcher = new IndexSearcher(r, es);
         smokeTestSearcher(fixedSearcher);
-        runSearchThreads(System.currentTimeMillis() + 100);
+        runSearchThreads(100);
       }
     }
     r.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
index 5461b78061b..ccb7da61db1 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
@@ -46,7 +46,7 @@ public class TestNeverDelete extends LuceneTestCase {
 
     w.commit();
     Thread[] indexThreads = new Thread[random().nextInt(4)];
-    final long stopTime = System.currentTimeMillis() + atLeast(1000);
+    final int stopIterations = atLeast(100);
     for (int x = 0; x < indexThreads.length; x++) {
       indexThreads[x] =
           new Thread() {
@@ -54,7 +54,7 @@ public class TestNeverDelete extends LuceneTestCase {
             public void run() {
               try {
                 int docCount = 0;
-                while (System.currentTimeMillis() < stopTime) {
+                while (docCount < stopIterations) {
                   final Document doc = new Document();
                   doc.add(newStringField("dc", "" + docCount, Field.Store.YES));
                   doc.add(newTextField("field", "here is some text", Field.Store.YES));
@@ -77,7 +77,8 @@ public class TestNeverDelete extends LuceneTestCase {
     final Set<String> allFiles = new HashSet<>();
 
     DirectoryReader r = DirectoryReader.open(d);
-    while (System.currentTimeMillis() < stopTime) {
+    int iterations = 0;
+    while (++iterations < stopIterations) {
       final IndexCommit ic = r.getIndexCommit();
       if (VERBOSE) {
         System.out.println("TEST: check files: " + ic.getFileNames());
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
index 9635bdcda81..19e1c89901b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
@@ -104,8 +104,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
   }
 
   private void runTest(Random random, Directory dir) throws Exception {
-    // Run for ~1 seconds at night
-    final long stopTime = System.currentTimeMillis() + (TEST_NIGHTLY ? 1000 : 100);
+    final int maxIterations = TEST_NIGHTLY ? 100 : 10;
 
     SnapshotDeletionPolicy dp = getDeletionPolicy();
     final IndexWriter writer =
@@ -128,6 +127,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
         new Thread() {
           @Override
           public void run() {
+            int iterations = 0;
             Document doc = new Document();
             FieldType customType = new FieldType(TextField.TYPE_STORED);
             customType.setStoreTermVectors(true);
@@ -155,7 +155,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
               } catch (InterruptedException ie) {
                 throw new ThreadInterruptedException(ie);
               }
-            } while (System.currentTimeMillis() < stopTime);
+            } while (++iterations < maxIterations);
           }
         };
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
index 41f4baed4e1..ae921e83734 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
@@ -27,7 +27,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
 public class TestStressIndexing extends LuceneTestCase {
   private abstract static class TimedThread extends Thread {
     volatile boolean failed;
-    private static int RUN_TIME_MSEC = atLeast(1000);
+    private static int RUN_ITERATIONS = atLeast(100);
     private TimedThread[] allThreads;
 
     public abstract void doWork() throws Throwable;
@@ -38,13 +38,12 @@ public class TestStressIndexing extends LuceneTestCase {
 
     @Override
     public void run() {
-      final long stopTime = System.currentTimeMillis() + RUN_TIME_MSEC;
-
+      int iterations = 0;
       try {
         do {
           if (anyErrors()) break;
           doWork();
-        } while (System.currentTimeMillis() < stopTime);
+        } while (++iterations < RUN_ITERATIONS);
       } catch (Throwable e) {
         System.out.println(Thread.currentThread() + ": exc");
         e.printStackTrace(System.out);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java b/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java
index 58a9b3a8233..d8c0dd0d2bb 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java
@@ -113,17 +113,18 @@ public class TestTermdocPerf extends LuceneTestCase {
   public int doTest(int iter, int ndocs, int maxTF, float percentDocs) throws IOException {
     Directory dir = newDirectory();
 
-    long start = System.currentTimeMillis();
+    long start = System.nanoTime();
     addDocs(random(), dir, ndocs, "foo", "val", maxTF, percentDocs);
-    long end = System.currentTimeMillis();
+    long end = System.nanoTime();
     if (VERBOSE)
-      System.out.println("milliseconds for creation of " + ndocs + " docs = " + (end - start));
+      System.out.println(
+          "milliseconds for creation of " + ndocs + " docs = " + (end - start) / 1_000_000);
 
     IndexReader reader = DirectoryReader.open(dir);
 
     TermsEnum tenum = MultiTerms.getTerms(reader, "foo").iterator();
 
-    start = System.currentTimeMillis();
+    start = System.nanoTime();
 
     int ret = 0;
     PostingsEnum tdocs = null;
@@ -136,9 +137,10 @@ public class TestTermdocPerf extends LuceneTestCase {
       }
     }
 
-    end = System.currentTimeMillis();
+    end = System.nanoTime();
     if (VERBOSE)
-      System.out.println("milliseconds for " + iter + " TermDocs iteration: " + (end - start));
+      System.out.println(
+          "milliseconds for " + iter + " TermDocs iteration: " + (end - start) / 1_000_000);
 
     return ret;
   }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTransactions.java b/lucene/core/src/test/org/apache/lucene/index/TestTransactions.java
index fd3905c4152..5fde3977e7c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTransactions.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTransactions.java
@@ -48,7 +48,7 @@ public class TestTransactions extends LuceneTestCase {
 
   private abstract static class TimedThread extends Thread {
     volatile boolean failed;
-    private static float RUN_TIME_MSEC = atLeast(500);
+    private static int MAX_ITERATIONS = atLeast(100);
     private TimedThread[] allThreads;
 
     public abstract void doWork() throws Throwable;
@@ -59,13 +59,13 @@ public class TestTransactions extends LuceneTestCase {
 
     @Override
     public void run() {
-      final long stopTime = System.currentTimeMillis() + (long) (RUN_TIME_MSEC);
-
       try {
+        int iterations = 0;
         do {
+          System.out.println(++iterations);
           if (anyErrors()) break;
           doWork();
-        } while (System.currentTimeMillis() < stopTime);
+        } while (iterations < MAX_ITERATIONS);
       } catch (Throwable e) {
         System.out.println(Thread.currentThread() + ": exc");
         e.printStackTrace(System.out);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
index 2488c1a5429..1fadd9af22c 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
@@ -289,8 +289,8 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
   }
 
   @Override
-  protected void doSearching(ExecutorService es, long stopTime) throws Exception {
-    runSearchThreads(stopTime);
+  protected void doSearching(ExecutorService es, int maxIterations) throws Exception {
+    runSearchThreads(maxIterations);
   }
 
   @Override
@@ -555,11 +555,11 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
       Document d = new Document();
       d.add(new TextField("count", i + "", Field.Store.NO));
       d.add(new TextField("content", content, Field.Store.YES));
-      long start = System.currentTimeMillis();
+      long start = System.nanoTime();
       long l = iw.addDocument(d);
       controlledRealTimeReopenThread.waitForGeneration(l);
-      long wait = System.currentTimeMillis() - start;
-      assertTrue("waited too long for generation " + wait, wait < (maxStaleSecs * 1000));
+      long wait = System.nanoTime() - start;
+      assertTrue("waited too long for generation " + wait, wait < (maxStaleSecs * 1_000_000_000L));
       IndexSearcher searcher = sm.acquire();
       TopDocs td = searcher.search(new TermQuery(new Term("count", i + "")), 10);
       sm.release(searcher);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
index 52a0cc7519d..435c386b934 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
@@ -106,7 +106,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
   }
 
   @Override
-  protected void doSearching(ExecutorService es, final long stopTime) throws Exception {
+  protected void doSearching(ExecutorService es, final int maxIterations) throws Exception {
 
     Thread reopenThread =
         new Thread() {
@@ -118,7 +118,8 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
                     "[" + Thread.currentThread().getName() + "]: launch reopen thread");
               }
 
-              while (System.currentTimeMillis() < stopTime) {
+              int iterations = 0;
+              while (++iterations < maxIterations) {
                 Thread.sleep(TestUtil.nextInt(random(), 1, 100));
                 writer.commit();
                 Thread.sleep(TestUtil.nextInt(random(), 1, 5));
@@ -143,7 +144,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     reopenThread.setDaemon(true);
     reopenThread.start();
 
-    runSearchThreads(stopTime);
+    runSearchThreads(maxIterations);
 
     reopenThread.join();
   }
diff --git a/lucene/core/src/test/org/apache/lucene/util/SelectorBenchmark.java b/lucene/core/src/test/org/apache/lucene/util/SelectorBenchmark.java
index ec9f6a19d50..669bcd36514 100644
--- a/lucene/core/src/test/org/apache/lucene/util/SelectorBenchmark.java
+++ b/lucene/core/src/test/org/apache/lucene/util/SelectorBenchmark.java
@@ -72,9 +72,10 @@ public class SelectorBenchmark {
 
   public static void main(String[] args) throws Exception {
     assert false : "Disable assertions to run the benchmark";
-    Random random = new Random(System.currentTimeMillis());
+    Random random = new Random(System.nanoTime());
     long seed = random.nextLong();
 
+    System.out.println("SEED: " + seed);
     System.out.println("WARMUP");
     benchmarkSelectors(Strategy.RANDOM, random, seed);
     System.out.println();
diff --git a/lucene/core/src/test/org/apache/lucene/util/SorterBenchmark.java b/lucene/core/src/test/org/apache/lucene/util/SorterBenchmark.java
index 9b62cb03ccb..b02579bfeed 100644
--- a/lucene/core/src/test/org/apache/lucene/util/SorterBenchmark.java
+++ b/lucene/core/src/test/org/apache/lucene/util/SorterBenchmark.java
@@ -64,9 +64,10 @@ public class SorterBenchmark {
 
   public static void main(String[] args) throws Exception {
     assert false : "Disable assertions to run the benchmark";
-    Random random = new Random(System.currentTimeMillis());
+    Random random = new Random(System.nanoTime());
     long seed = random.nextLong();
 
+    System.out.println("SEED: " + seed);
     System.out.println("WARMUP");
     benchmarkSorters(Strategy.RANDOM, random, seed);
     System.out.println();
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDirectAddressing.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDirectAddressing.java
index 25e780e371f..d54f572d27c 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDirectAddressing.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDirectAddressing.java
@@ -267,24 +267,24 @@ public class TestFSTDirectAddressing extends LuceneTestCase {
         new InputStreamDataInput(newInputStream(Paths.get(fstFilePath)))) {
 
       System.out.println("Reading FST");
-      long startTimeMs = System.currentTimeMillis();
+      long startTimeMs = System.nanoTime();
       FST<CharsRef> originalFst = new FST<>(in, in, CharSequenceOutputs.getSingleton());
-      long endTimeMs = System.currentTimeMillis();
-      System.out.println("time = " + (endTimeMs - startTimeMs) + " ms");
+      long endTimeMs = System.nanoTime();
+      System.out.println("time = " + (endTimeMs - startTimeMs) / 100_000 + " ms");
 
       for (float oversizingFactor : List.of(0f, 0f, 0f, 1f, 1f, 1f)) {
         System.out.println("\nFST construction (oversizingFactor=" + oversizingFactor + ")");
-        startTimeMs = System.currentTimeMillis();
+        startTimeMs = System.nanoTime();
         FST<CharsRef> fst = recompile(originalFst, oversizingFactor);
-        endTimeMs = System.currentTimeMillis();
-        System.out.println("time = " + (endTimeMs - startTimeMs) + " ms");
+        endTimeMs = System.nanoTime();
+        System.out.println("time = " + (endTimeMs - startTimeMs) / 100_000 + " ms");
         System.out.println("FST RAM = " + fst.ramBytesUsed() + " B");
 
         System.out.println("FST enum");
-        startTimeMs = System.currentTimeMillis();
+        startTimeMs = System.nanoTime();
         walk(fst);
-        endTimeMs = System.currentTimeMillis();
-        System.out.println("time = " + (endTimeMs - startTimeMs) + " ms");
+        endTimeMs = System.nanoTime();
+        System.out.println("time = " + (endTimeMs - startTimeMs) / 100_000 + " ms");
       }
     }
   }
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index fc6b75fe707..d4b48c142eb 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -528,7 +528,7 @@ public class TestFSTs extends LuceneTestCase {
       BufferedReader is = Files.newBufferedReader(wordsFileIn, StandardCharsets.UTF_8);
       try {
         final IntsRefBuilder intsRefBuilder = new IntsRefBuilder();
-        long tStart = System.currentTimeMillis();
+        long tStart = System.nanoTime();
         int ord = 0;
         while (true) {
           String w = is.readLine();
@@ -543,7 +543,7 @@ public class TestFSTs extends LuceneTestCase {
             System.out.printf(
                 Locale.ROOT,
                 "%6.2fs: %9d...",
-                ((System.currentTimeMillis() - tStart) / 1000.0),
+                ((System.nanoTime() - tStart) / 1_000_000_000.0),
                 ord);
           }
           if (ord >= limit) {
@@ -551,13 +551,13 @@ public class TestFSTs extends LuceneTestCase {
           }
         }
 
-        long tMid = System.currentTimeMillis();
-        System.out.println(((tMid - tStart) / 1000.0) + " sec to add all terms");
+        long tMid = System.nanoTime();
+        System.out.println(((tMid - tStart) / 1_000_000_000.0) + " sec to add all terms");
 
         assert fstCompiler.getTermCount() == ord;
         FST<T> fst = fstCompiler.compile();
-        long tEnd = System.currentTimeMillis();
-        System.out.println(((tEnd - tMid) / 1000.0) + " sec to finish/pack");
+        long tEnd = System.nanoTime();
+        System.out.println(((tEnd - tMid) / 1_000_000_000.0) + " sec to finish/pack");
         if (fst == null) {
           System.out.println("FST was fully pruned!");
           System.exit(0);
@@ -598,7 +598,7 @@ public class TestFSTs extends LuceneTestCase {
         is = Files.newBufferedReader(wordsFileIn, StandardCharsets.UTF_8);
 
         ord = 0;
-        tStart = System.currentTimeMillis();
+        tStart = System.nanoTime();
         while (true) {
           String w = is.readLine();
           if (w == null) {
@@ -622,14 +622,14 @@ public class TestFSTs extends LuceneTestCase {
           ord++;
           if (ord % 500000 == 0) {
             System.out.println(
-                ((System.currentTimeMillis() - tStart) / 1000.0) + "s: " + ord + "...");
+                ((System.nanoTime() - tStart) / 1_000_000_000.0) + "s: " + ord + "...");
           }
           if (ord >= limit) {
             break;
           }
         }
 
-        double totSec = ((System.currentTimeMillis() - tStart) / 1000.0);
+        double totSec = ((System.nanoTime() - tStart) / 1_000_000_000.0);
         System.out.println(
             "Verify took "
                 + totSec
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/idversion/TestIDVersionPostingsFormat.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/idversion/TestIDVersionPostingsFormat.java
index 99483493c18..fad34e02f41 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/idversion/TestIDVersionPostingsFormat.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/idversion/TestIDVersionPostingsFormat.java
@@ -27,6 +27,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
@@ -733,8 +734,9 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
       }
     }
 
-    // Run for .5 sec in normal tests, else 60 seconds for nightly:
-    final long stopTime = System.currentTimeMillis() + (TEST_NIGHTLY ? 60000 : 500);
+    // Run for 20k iterations in normal tests, else 2m iterations for nightly:
+    final AtomicInteger iterations = new AtomicInteger(0);
+    final int stopIterations = TEST_NIGHTLY ? 2_000_000 : 20_000;
 
     for (int i = 0; i < threads.length; i++) {
       threads[i] =
@@ -752,8 +754,7 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
               startingGun.await();
               PerThreadVersionPKLookup lookup = null;
               IndexReader lookupReader = null;
-              while (System.currentTimeMillis() < stopTime) {
-
+              while (iterations.incrementAndGet() < stopIterations) {
                 // Intentionally pull version first, and then sleep/yield, to provoke version
                 // conflicts:
                 long newVersion;
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java b/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java
index 7028a02a28f..c5bbfdaed70 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java
@@ -364,10 +364,7 @@ public class TestSpellChecker extends LuceneTestCase {
   }
 
   private void addwords(IndexReader r, SpellChecker sc, String field) throws IOException {
-    long time = System.currentTimeMillis();
     sc.indexDictionary(new LuceneDictionary(r, field), newIndexWriterConfig(null), false);
-    time = System.currentTimeMillis() - time;
-    // System.out.println("time to build " + field + ": " + time);
   }
 
   private int numdoc() throws IOException {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/ThreadedIndexingAndSearchingTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/ThreadedIndexingAndSearchingTestCase.java
index 2708276d40c..e65ba808b81 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/ThreadedIndexingAndSearchingTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/ThreadedIndexingAndSearchingTestCase.java
@@ -103,7 +103,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
   protected void releaseSearcher(IndexSearcher s) throws Exception {}
 
   // Called once to run searching
-  protected abstract void doSearching(ExecutorService es, long stopTime) throws Exception;
+  protected abstract void doSearching(ExecutorService es, int maxIterations) throws Exception;
 
   protected Directory getDirectory(Directory in) {
     return in;
@@ -137,7 +137,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
   private Thread[] launchIndexingThreads(
       final LineFileDocs docs,
       int numThreads,
-      final long stopTime,
+      final int maxIterations,
       final Set<String> delIDs,
       final Set<String> delPackIDs,
       final List<SubDocs> allSubDocs) {
@@ -151,7 +151,8 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
               // deleting anothers added docs works:
               final List<String> toDeleteIDs = new ArrayList<>();
               final List<SubDocs> toDeleteSubDocs = new ArrayList<>();
-              while (System.currentTimeMillis() < stopTime && !failed.get()) {
+              int iterations = 0;
+              while (++iterations < maxIterations && !failed.get()) {
                 try {
 
                   // Occasional longish pause if running
@@ -372,7 +373,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
     return threads;
   }
 
-  protected void runSearchThreads(final long stopTimeMS) throws Exception {
+  protected void runSearchThreads(final int maxIterations) throws Exception {
     final int numThreads = TEST_NIGHTLY ? TestUtil.nextInt(random(), 1, 5) : 2;
     final Thread[] searchThreads = new Thread[numThreads];
     final AtomicLong totHits = new AtomicLong();
@@ -389,7 +390,8 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
               if (VERBOSE) {
                 System.out.println(Thread.currentThread().getName() + ": launch search thread");
               }
-              while (System.currentTimeMillis() < stopTimeMS && !failed.get()) {
+              int iterations = 0;
+              while (++iterations < maxIterations && !failed.get()) {
                 try {
                   final IndexSearcher s = getCurrentSearcher();
                   try {
@@ -434,7 +436,8 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
                         trigger = totTermCount.get() / 30;
                         shift = random().nextInt(trigger);
                       }
-                      while (System.currentTimeMillis() < stopTimeMS) {
+                      int iters = 0;
+                      while (++iters < maxIterations) {
                         BytesRef term = termsEnum.next();
                         if (term == null) {
                           totTermCount.set(seenTermCount);
@@ -495,7 +498,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
     delCount.set(0);
     packCount.set(0);
 
-    final long t0 = System.currentTimeMillis();
+    final long t0 = System.nanoTime();
 
     Random random = new Random(random().nextLong());
     final LineFileDocs docs = new LineFileDocs(random);
@@ -565,33 +568,33 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
 
     final int NUM_INDEX_THREADS = TestUtil.nextInt(random(), 2, 4);
 
-    final int RUN_TIME_MSEC = LuceneTestCase.TEST_NIGHTLY ? 300000 : 100 * RANDOM_MULTIPLIER;
+    final int MAX_ITERATIONS = LuceneTestCase.TEST_NIGHTLY ? 200 : 10 * RANDOM_MULTIPLIER;
 
     final Set<String> delIDs = Collections.synchronizedSet(new HashSet<String>());
     final Set<String> delPackIDs = Collections.synchronizedSet(new HashSet<String>());
     final List<SubDocs> allSubDocs = Collections.synchronizedList(new ArrayList<SubDocs>());
 
-    final long stopTime = System.currentTimeMillis() + RUN_TIME_MSEC;
-
     final Thread[] indexThreads =
-        launchIndexingThreads(docs, NUM_INDEX_THREADS, stopTime, delIDs, delPackIDs, allSubDocs);
+        launchIndexingThreads(
+            docs, NUM_INDEX_THREADS, MAX_ITERATIONS, delIDs, delPackIDs, allSubDocs);
 
     if (VERBOSE) {
       System.out.println(
           "TEST: DONE start "
               + NUM_INDEX_THREADS
               + " indexing threads ["
-              + (System.currentTimeMillis() - t0)
+              + (System.nanoTime() - t0) / 1_000_000
               + " ms]");
     }
 
     // Let index build up a bit
     Thread.sleep(100);
 
-    doSearching(es, stopTime);
+    doSearching(es, MAX_ITERATIONS);
 
     if (VERBOSE) {
-      System.out.println("TEST: all searching done [" + (System.currentTimeMillis() - t0) + " ms]");
+      System.out.println(
+          "TEST: all searching done [" + (System.nanoTime() - t0) / 1_000_000 + " ms]");
     }
 
     for (Thread thread : indexThreads) {
@@ -601,7 +604,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
     if (VERBOSE) {
       System.out.println(
           "TEST: done join indexing threads ["
-              + (System.currentTimeMillis() - t0)
+              + (System.nanoTime() - t0) / 1_000_000
               + " ms]; addCount="
               + addCount
               + " delCount="
@@ -762,7 +765,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
     dir.close();
 
     if (VERBOSE) {
-      System.out.println("TEST: done [" + (System.currentTimeMillis() - t0) + " ms]");
+      System.out.println("TEST: done [" + (System.nanoTime() - t0) / 1_000_000 + " ms]");
     }
   }
 
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/QueryUtils.java
index c098785304d..ec15851ce66 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/QueryUtils.java
@@ -541,7 +541,7 @@ public class QueryUtils {
           public void collect(int doc) throws IOException {
             float score = scorer.score();
             try {
-              long startMS = System.currentTimeMillis();
+              long startNS = System.nanoTime();
               for (int i = lastDoc[0] + 1; i <= doc; i++) {
                 Weight w = s.createWeight(rewritten, ScoreMode.COMPLETE, 1);
                 Scorer scorer = w.scorer(context.get(leafPtr));
@@ -571,7 +571,7 @@ public class QueryUtils {
 
                 // Hurry things along if they are going slow (eg
                 // if you got SimpleText codec this will kick in):
-                if (i < doc && System.currentTimeMillis() - startMS > 5) {
+                if (i < doc && System.nanoTime() - startNS > 5_000_000) {
                   i = doc - 1;
                 }
               }