You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/04/29 20:35:13 UTC

[lucene-solr] branch master updated: Fix tests to survive nightly runs with many documents

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 207d240  Fix tests to survive nightly runs with many documents
207d240 is described below

commit 207d240ae2427e15af929fa44f1b36d4f8e5a250
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Wed Apr 29 22:11:42 2020 +0200

    Fix tests to survive nightly runs with many documents
---
 .../lucene/index/TestNRTReaderWithThreads.java     |  4 +-
 .../apache/lucene/search/TestSearcherManager.java  |  4 +-
 .../org/apache/lucene/index/RandomIndexWriter.java | 54 +++++++++++-----------
 .../ThreadedIndexingAndSearchingTestCase.java      | 24 +---------
 .../org/apache/lucene/util/LuceneTestCase.java     | 26 +++++++++++
 5 files changed, 59 insertions(+), 53 deletions(-)

diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java b/lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java
index 45ceb5a..3a41d3a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java
@@ -36,9 +36,9 @@ public class TestNRTReaderWithThreads extends LuceneTestCase {
     }
     IndexWriter writer = new IndexWriter(
         mainDir,
-        newIndexWriterConfig(new MockAnalyzer(random()))
+        ensureSaneIWCOnNightly(newIndexWriterConfig(new MockAnalyzer(random()))
            .setMaxBufferedDocs(10)
-           .setMergePolicy(newLogMergePolicy(false,2))
+           .setMergePolicy(newLogMergePolicy(false,2)))
     );
     IndexReader reader = writer.getReader(); // start pooling readers
     reader.close();
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 9d5e087..b923866 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
@@ -556,7 +556,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     AtomicReference<IndexWriter> writerRef = new AtomicReference<>();
     final MockAnalyzer analyzer = new MockAnalyzer(random());
     analyzer.setMaxTokenLength(IndexWriter.MAX_TERM_LENGTH);
-    writerRef.set(new IndexWriter(dir, ensureSaneIWCOnNigtly(newIndexWriterConfig(analyzer))));
+    writerRef.set(new IndexWriter(dir, ensureSaneIWCOnNightly(newIndexWriterConfig(analyzer))));
 
     AtomicReference<SearcherManager> mgrRef = new AtomicReference<>();
     mgrRef.set(new SearcherManager(writerRef.get(), null));
@@ -580,7 +580,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
                 } else {
                   w.rollback();
                 }
-                writerRef.set(new IndexWriter(dir, ensureSaneIWCOnNigtly(newIndexWriterConfig(analyzer))));
+                writerRef.set(new IndexWriter(dir, ensureSaneIWCOnNightly(newIndexWriterConfig(analyzer))));
               }
             }
             if (VERBOSE) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
index e2915e0..7c70480 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
@@ -53,6 +53,7 @@ public class RandomIndexWriter implements Closeable {
   private boolean getReaderCalled;
   private final Analyzer analyzer; // only if WE created it (then we close it)
   private final double softDeletesRatio;
+  private final LiveIndexWriterConfig config;
 
   /** Returns an indexwriter that randomly mixes up thread scheduling (by yielding at test points) */
   public static IndexWriter mockIndexWriter(Directory dir, IndexWriterConfig conf, Random r) throws IOException {
@@ -131,6 +132,7 @@ public class RandomIndexWriter implements Closeable {
       softDeletesRatio = 0d;
     }
     w = mockIndexWriter(dir, c, r);
+    config = w.getConfig();
     flushAt = TestUtil.nextInt(r, 10, 1000);
     if (closeAnalyzer) {
       analyzer = w.getAnalyzer();
@@ -151,7 +153,7 @@ public class RandomIndexWriter implements Closeable {
    * @see IndexWriter#addDocument(Iterable)
    */
   public <T extends IndexableField> long addDocument(final Iterable<T> doc) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     long seqNo;
     if (r.nextInt(5) == 3) {
       // TODO: maybe, we should simply buffer up added docs
@@ -197,7 +199,7 @@ public class RandomIndexWriter implements Closeable {
   }
 
   private void maybeFlushOrCommit() throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     if (docCount++ == flushAt) {
       if (r.nextBoolean()) {
         flushAllBuffersSequentially();
@@ -234,17 +236,17 @@ public class RandomIndexWriter implements Closeable {
   }
   
   public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     long seqNo = w.addDocuments(docs);
     maybeFlushOrCommit();
     return seqNo;
   }
 
   public long updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     long seqNo;
     if (useSoftDeletes()) {
-      seqNo = w.softUpdateDocuments(delTerm, docs, new NumericDocValuesField(w.getConfig().getSoftDeletesField(), 1));
+      seqNo = w.softUpdateDocuments(delTerm, docs, new NumericDocValuesField(config.getSoftDeletesField(), 1));
     } else {
       seqNo = w.updateDocuments(delTerm, docs);
     }
@@ -261,13 +263,13 @@ public class RandomIndexWriter implements Closeable {
    * @see IndexWriter#updateDocument(Term, Iterable)
    */
   public <T extends IndexableField> long updateDocument(Term t, final Iterable<T> doc) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     final long seqNo;
     if (useSoftDeletes()) {
       if (r.nextInt(5) == 3) {
-        seqNo = w.softUpdateDocuments(t, Arrays.asList(doc), new NumericDocValuesField(w.getConfig().getSoftDeletesField(), 1));
+        seqNo = w.softUpdateDocuments(t, Arrays.asList(doc), new NumericDocValuesField(config.getSoftDeletesField(), 1));
       } else {
-        seqNo = w.softUpdateDocument(t, doc, new NumericDocValuesField(w.getConfig().getSoftDeletesField(), 1));
+        seqNo = w.softUpdateDocument(t, doc, new NumericDocValuesField(config.getSoftDeletesField(), 1));
       }
     } else {
       if (r.nextInt(5) == 3) {
@@ -282,37 +284,37 @@ public class RandomIndexWriter implements Closeable {
   }
   
   public long addIndexes(Directory... dirs) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.addIndexes(dirs);
   }
 
   public long addIndexes(CodecReader... readers) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.addIndexes(readers);
   }
   
   public long updateNumericDocValue(Term term, String field, Long value) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.updateNumericDocValue(term, field, value);
   }
   
   public long updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.updateBinaryDocValue(term, field, value);
   }
   
   public long updateDocValues(Term term, Field... updates) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.updateDocValues(term, updates);
   }
   
   public long deleteDocuments(Term term) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.deleteDocuments(term);
   }
 
   public long deleteDocuments(Query q) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return w.deleteDocuments(q);
   }
 
@@ -321,7 +323,7 @@ public class RandomIndexWriter implements Closeable {
   }
   
   public long commit(boolean flushConcurrently) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     if (flushConcurrently) {
       List<Throwable> throwableList = new CopyOnWriteArrayList<>();
       Thread thread = new Thread(() -> {
@@ -372,7 +374,7 @@ public class RandomIndexWriter implements Closeable {
   }
 
   public DirectoryReader getReader() throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     return getReader(true, false);
   }
 
@@ -380,12 +382,12 @@ public class RandomIndexWriter implements Closeable {
   private boolean doRandomForceMergeAssert;
 
   public void forceMergeDeletes(boolean doWait) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     w.forceMergeDeletes(doWait);
   }
 
   public void forceMergeDeletes() throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     w.forceMergeDeletes();
   }
 
@@ -413,7 +415,7 @@ public class RandomIndexWriter implements Closeable {
           System.out.println("RIW: doRandomForceMerge(" + limit + ")");
         }
         w.forceMerge(limit);
-        if (limit == 1 || (w.getConfig().getMergePolicy() instanceof TieredMergePolicy) == false) {
+        if (limit == 1 || (config.getMergePolicy() instanceof TieredMergePolicy) == false) {
           assert !doRandomForceMergeAssert || w.getSegmentCount() <= limit : "limit=" + limit + " actual=" + w.getSegmentCount();
         }
       } else {
@@ -426,7 +428,7 @@ public class RandomIndexWriter implements Closeable {
   }
 
   public DirectoryReader getReader(boolean applyDeletions, boolean writeAllDeletes) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     getReaderCalled = true;
     if (r.nextInt(20) == 2) {
       doRandomForceMerge();
@@ -447,8 +449,8 @@ public class RandomIndexWriter implements Closeable {
       w.commit();
       if (r.nextBoolean()) {
         DirectoryReader reader = DirectoryReader.open(w.getDirectory());
-        if (w.getConfig().getSoftDeletesField() != null) {
-          return new SoftDeletesDirectoryReaderWrapper(reader, w.getConfig().getSoftDeletesField());
+        if (config.getSoftDeletesField() != null) {
+          return new SoftDeletesDirectoryReaderWrapper(reader, config.getSoftDeletesField());
         } else {
           return reader;
         }
@@ -467,13 +469,13 @@ public class RandomIndexWriter implements Closeable {
     boolean success = false;
     try {
       if (w.isClosed() == false) {
-        LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+        LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
       }
       // if someone isn't using getReader() API, we want to be sure to
       // forceMerge since presumably they might open a reader on the dir.
       if (getReaderCalled == false && r.nextInt(8) == 2 && w.isClosed() == false) {
         doRandomForceMerge();
-        if (w.getConfig().getCommitOnClose() == false) {
+        if (config.getCommitOnClose() == false) {
           // index may have changed, must commit the changes, or otherwise they are discarded by the call to close()
           w.commit();
         }
@@ -496,7 +498,7 @@ public class RandomIndexWriter implements Closeable {
    * @see IndexWriter#forceMerge(int)
    */
   public void forceMerge(int maxSegmentCount) throws IOException {
-    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
+    LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, config);
     w.forceMerge(maxSegmentCount);
   }
   
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
index 5691979..1a35ee7 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
@@ -424,28 +424,6 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
 
   private final Map<SegmentCoreReaders,Boolean> warmed = Collections.synchronizedMap(new WeakHashMap<SegmentCoreReaders,Boolean>());
 
-  protected static IndexWriterConfig ensureSaneIWCOnNigtly(IndexWriterConfig conf) {
-    if (LuceneTestCase.TEST_NIGHTLY) {
-      // newIWConfig makes smallish max seg size, which
-      // results in tons and tons of segments for this test
-      // when run nightly:
-      MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof TieredMergePolicy) {
-        ((TieredMergePolicy) mp).setMaxMergedSegmentMB(5000.);
-      } else if (mp instanceof LogByteSizeMergePolicy) {
-        ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1000.);
-      } else if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setMaxMergeDocs(100000);
-      }
-      // when running nightly, merging can still have crazy parameters,
-      // and might use many per-field codecs. turn on CFS for IW flushes
-      // and ensure CFS ratio is reasonable to keep it contained.
-      conf.setUseCompoundFile(true);
-      mp.setNoCFSRatio(Math.max(0.25d, mp.getNoCFSRatio()));
-    }
-    return conf;
-  }
-
   public void runTest(String testName) throws Exception {
 
     failed.set(false);
@@ -470,7 +448,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
       ((MockRandomMergePolicy)conf.getMergePolicy()).setDoNonBulkMerges(false);
     }
 
-    ensureSaneIWCOnNigtly(conf);
+    ensureSaneIWCOnNightly(conf);
 
     conf.setMergedSegmentWarmer((reader) -> {
       if (VERBOSE) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index cb32dc2..9f2cd27 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -2989,4 +2989,30 @@ public abstract class LuceneTestCase extends Assert {
     assumeTrue("hit JDK collator bug", Integer.signum(v1) == Integer.signum(v2));
     return v1;
   }
+
+
+  /**
+   * Ensures that the MergePolicy has sane values for tests that test with lots of documents.
+   */
+  protected static IndexWriterConfig ensureSaneIWCOnNightly(IndexWriterConfig conf) {
+    if (LuceneTestCase.TEST_NIGHTLY) {
+      // newIWConfig makes smallish max seg size, which
+      // results in tons and tons of segments for this test
+      // when run nightly:
+      MergePolicy mp = conf.getMergePolicy();
+      if (mp instanceof TieredMergePolicy) {
+        ((TieredMergePolicy) mp).setMaxMergedSegmentMB(5000.);
+      } else if (mp instanceof LogByteSizeMergePolicy) {
+        ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1000.);
+      } else if (mp instanceof LogMergePolicy) {
+        ((LogMergePolicy) mp).setMaxMergeDocs(100000);
+      }
+      // when running nightly, merging can still have crazy parameters,
+      // and might use many per-field codecs. turn on CFS for IW flushes
+      // and ensure CFS ratio is reasonable to keep it contained.
+      conf.setUseCompoundFile(true);
+      mp.setNoCFSRatio(Math.max(0.25d, mp.getNoCFSRatio()));
+    }
+    return conf;
+  }
 }