You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2020/06/26 23:04:24 UTC

[lucene-solr] branch jira/lucene-8962 updated: LUCENE-8962: add some IndexWriter verbosity about what commit-on-merge is doing; fix test case to use deterministic merging

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

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


The following commit(s) were added to refs/heads/jira/lucene-8962 by this push:
     new ae8ac0c  LUCENE-8962: add some IndexWriter verbosity about what commit-on-merge is doing; fix test case to use deterministic merging
ae8ac0c is described below

commit ae8ac0cba211b0dba497944cf40b526cb697fe65
Author: Mike McCandless <mi...@apache.org>
AuthorDate: Fri Jun 26 19:03:06 2020 -0400

    LUCENE-8962: add some IndexWriter verbosity about what commit-on-merge is doing; fix test case to use deterministic merging
---
 .../core/src/java/org/apache/lucene/index/IndexWriter.java | 14 ++++++++++++++
 .../org/apache/lucene/search/TestPhraseWildcardQuery.java  |  4 +++-
 2 files changed, 17 insertions(+), 1 deletion(-)

diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 55425f2..6c97794 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -3268,8 +3268,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       }
 
       if (onCommitMerges != null) {
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "now run merges during commit: " + onCommitMerges.segString(directory));
+        }
         mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
         onCommitMerges.await(maxCommitMergeWaitMillis, TimeUnit.MILLISECONDS);
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "done waiting for merges during commit");
+        }
         synchronized (this) {
           // we need to call this under lock since mergeFinished above is also called under the IW lock
           includeInCommit.set(false);
@@ -3334,6 +3340,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
                 && committed
                 && includeInCommit.get()) {
 
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "now apply merge during commit: " + toWrap.segString());
+              }
+
               // make sure onMergeComplete really was called:
               assert origInfo != null;
 
@@ -3355,6 +3365,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
               long segmentCounter = Long.parseLong(origInfo.info.name.substring(1), Character.MAX_RADIX);
               committingSegmentInfos.counter = Math.max(committingSegmentInfos.counter, segmentCounter + 1);
               committingSegmentInfos.applyMergeChanges(applicableMerge, false);
+            } else {
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "skip apply merge during commit: " + toWrap.segString());
+              }
             }
             toWrap.mergeFinished(committed, false);
             super.mergeFinished(committed, segmentDropped);
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestPhraseWildcardQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestPhraseWildcardQuery.java
index 552d750..263adf2 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestPhraseWildcardQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestPhraseWildcardQuery.java
@@ -67,7 +67,9 @@ public class TestPhraseWildcardQuery extends LuceneTestCase {
     super.setUp();
     directory = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), directory,
-        newIndexWriterConfig().setMaxCommitMergeWaitMillis(0)); // want to keep segments separated
+                                                 newIndexWriterConfig().setMergePolicy(newLogMergePolicy())); // do not accidentally merge
+                                                                                                              // the two segments we create
+                                                                                                              // here
     iw.setDoRandomForceMerge(false); // Keep the segments separated.
     addSegments(iw);
     reader = iw.getReader();