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
This commit is contained in:
Marios Trivyzas 2022-09-07 00:55:01 +03:00 committed by Robert Muir
parent 299ba5e154
commit fb40a43c0d
No known key found for this signature in database
GPG Key ID: 817AE1DD322D7ECA
31 changed files with 141 additions and 127 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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);
}
}

View File

@ -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] + "-----");

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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();

View File

@ -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;
}
}

View File

@ -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

View File

@ -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) {

View File

@ -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();

View File

@ -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();

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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();

View File

@ -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());

View File

@ -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);
}
};

View File

@ -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);

View File

@ -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;
}

View File

@ -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);

View File

@ -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);

View File

@ -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();
}

View File

@ -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();

View File

@ -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();

View File

@ -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");
}
}
}

View File

@ -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

View File

@ -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;

View File

@ -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 {

View File

@ -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]");
}
}

View File

@ -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;
}
}