You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/06/03 19:50:28 UTC

[GitHub] [lucene-solr] msokolov opened a new pull request #1552: LUCENE-8962

msokolov opened a new pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552


   This PR revisits the merge-on-commit patch submitted by @msfroh a little while ago. The only change from that earlier PR is a fix for failures uncovered by TestIndexWriter.testRandomOperations, some whitespace cleanups, and a rebase on the current master branch. The problem was that updateSegmentInfosOnMergeFinish would incorrectly decRef a merged segments' files if that segment was modified by deletions (or updates) while it was being merged.
   
   With this fix, I ran the failing test case several thousands of times with no failures, whereas before it would routinely fail after a few hundred test runs.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r440298695



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4483,6 +4593,7 @@ public int length() {
         // Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
         assert merge.info.info.maxDoc() == 0;
         commitMerge(merge, mergeState);
+        success = true;

Review comment:
       Phew, I dit some git archaeology (thanks @msokolov for the pointers!) and uncovered the branch commit for this "merge small segments on commit" feature where we added this controversial `success = true`: https://github.com/apache/lucene-solr/commit/cab5ef5e6f2bdcda59fd669a298ec1377777af9d
   
   +1 to pull the bugfix out into its own issue; I will open one.
   
   The above commit has a dedicated test case, but the problem is that test case (in the above commit) relies on this new feature (it uses the new `MergePolicy.findFullFlushMerges`).  So we would need a new test case based on clean `master` branch showing the bug ... it looks like a test that merged 100% deleted segments ought to then incorrectly double-call `closeMergedReaders` (first with `suppressExceptions = false` then again with `true`) due to this missing `success = true` so it really should be easy to reproduce.  Though, actually I'm surprised none of our random testing uncovered this.  Not sure I full understand the bug yet :)  I will open an issue!

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3257,6 +3320,52 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
+
+      if (mergeAwaitLatchRef != null) {
+        CountDownLatch mergeAwaitLatch = mergeAwaitLatchRef.get();
+        // If we found and registered any merges above, within the flushLock, then we want to ensure that they
+        // complete execution. Note that since we released the lock, other merges may have been scheduled. We will
+        // block until  the merges that we registered complete. As they complete, they will update toCommit to
+        // replace merged segments with the result of each merge.
+        config.getIndexWriterEvents().beginMergeOnCommit();
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+        long mergeWaitStart = System.nanoTime();
+        int abandonedCount = 0;
+        long waitTimeMillis = (long) (config.getMaxCommitMergeWaitSeconds() * 1000.0);
+        try {
+          if (mergeAwaitLatch.await(waitTimeMillis, TimeUnit.MILLISECONDS) == false) {
+            synchronized (this) {
+              // Need to do this in a synchronized block, to make sure none of our commit merges are currently
+              // executing mergeFinished (since mergeFinished itself is called from within the IndexWriter lock).
+              // After we clear the value from mergeAwaitLatchRef, the merges we schedule will still execute as
+              // usual, but when they finish, they won't attempt to update toCommit or modify segment reference
+              // counts.
+              mergeAwaitLatchRef.set(null);
+              for (MergePolicy.OneMerge commitMerge : commitMerges) {
+                if (runningMerges.contains(commitMerge) || pendingMerges.contains(commitMerge)) {
+                  abandonedCount++;
+                }
+              }
+            }
+          }
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        } finally {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", String.format(Locale.ROOT, "Waited %.1f ms for commit merges",
+                (System.nanoTime() - mergeWaitStart)/1_000_000.0));
+            infoStream.message("IW", "After executing commit merges, had " + toCommit.size() + " segments");
+            if (abandonedCount > 0) {
+              infoStream.message("IW", "Abandoned " + abandonedCount + " commit merges after " + waitTimeMillis + " ms");
+            }
+          }
+          if (abandonedCount > 0) {
+            config.getIndexWriterEvents().abandonedMergesOnCommit(abandonedCount);

Review comment:
       OK let's remove this part and leave it for another day.  I'll open a separate issue.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -109,6 +110,9 @@
   
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+  /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+  public static final double DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 30.0;

Review comment:
       > maybe 0 as a default and if somebody want's to wait they can set it?
   
   +1
   
   The feature is already disabled by default anyways (until you implement `findFullFlushMerges` in your `MergePolicy`), but making this `0` by default would make it even clearer that the feature is off by default.
   
   > The success=true added above was needed in order to fix a test failure caught by @dnhatn 's new unit test (testRandomOperations), so they belong together.
   
   Ahh thanks for the context @msokolov!  However, staring at the code (maybe for not long enough!), it looks like it really ought to be a pre-existing bug, and should be unit-testable without this new feature.  But I am confused why none of our random tests have tripped up on this yet.  I will open a separate Jira issue for this.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3228,15 +3268,38 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges) {
+                // Find any merges that can execute on commit (per MergePolicy).
+                MergePolicy.MergeSpecification mergeSpec =

Review comment:
       Thanks @s1monw.  I'll review your new PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439646326



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -109,6 +110,9 @@
   
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+  /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+  public static final double DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 30.0;

Review comment:
       maybe 0 as a default and if somebody want's to wait they can set it?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r438153575



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4483,6 +4593,7 @@ public int length() {
         // Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
         assert merge.info.info.maxDoc() == 0;
         commitMerge(merge, mergeState);
+        success = true;

Review comment:
       this seems unnecessary?
   

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3152,6 +3154,42 @@ public final boolean flushNextBuffer() throws IOException {
     }
   }
 
+  private MergePolicy.OneMerge updateSegmentInfosOnMergeFinish(MergePolicy.OneMerge merge, final SegmentInfos toCommit,
+                                                                AtomicReference<CountDownLatch> mergeLatchRef) {
+    return new MergePolicy.OneMerge(merge.segments) {
+      public void mergeFinished() throws IOException {
+        super.mergeFinished();
+        CountDownLatch mergeAwaitLatch = mergeLatchRef.get();
+        if (mergeAwaitLatch == null) {
+          // Commit thread timed out waiting for this merge and moved on. No need to manipulate toCommit.
+          return;
+        }
+        if (committed) {
+          deleter.incRef(this.info.files());

Review comment:
       I am trying to understand why we need to do any extra work here. What makes this special to any other merge such that we need to do all this work. If this needs to be done only if we include this merged segment in the commit, can't we do it outside of this mergeFinished and only use mergeFinished to signal which merge finished in time etc? Then we also might not need the latch construct and can use a simple callback that we can ignore on the commit end? 

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3228,15 +3268,38 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges) {
+                // Find any merges that can execute on commit (per MergePolicy).
+                MergePolicy.MergeSpecification mergeSpec =

Review comment:
       I do wonder if we need all these changes here in such a fragile part of the code. Wouldn't it be possible to simply call `maybeMerge(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments)` with a MergePolicy wrapper that does all the magic like wrapping segments etc. Then we could pick up the callback idea from above and just wait here until all merges called back? I think we should try to reuse most of the current infrastructure in IW instead of special casing. There was a lot of work put into this to reduce special casing I think we should try hard to reduce it more and try harder to not add any.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3257,6 +3320,52 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
+
+      if (mergeAwaitLatchRef != null) {
+        CountDownLatch mergeAwaitLatch = mergeAwaitLatchRef.get();
+        // If we found and registered any merges above, within the flushLock, then we want to ensure that they
+        // complete execution. Note that since we released the lock, other merges may have been scheduled. We will
+        // block until  the merges that we registered complete. As they complete, they will update toCommit to
+        // replace merged segments with the result of each merge.
+        config.getIndexWriterEvents().beginMergeOnCommit();
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+        long mergeWaitStart = System.nanoTime();
+        int abandonedCount = 0;
+        long waitTimeMillis = (long) (config.getMaxCommitMergeWaitSeconds() * 1000.0);
+        try {
+          if (mergeAwaitLatch.await(waitTimeMillis, TimeUnit.MILLISECONDS) == false) {
+            synchronized (this) {
+              // Need to do this in a synchronized block, to make sure none of our commit merges are currently
+              // executing mergeFinished (since mergeFinished itself is called from within the IndexWriter lock).
+              // After we clear the value from mergeAwaitLatchRef, the merges we schedule will still execute as
+              // usual, but when they finish, they won't attempt to update toCommit or modify segment reference
+              // counts.
+              mergeAwaitLatchRef.set(null);
+              for (MergePolicy.OneMerge commitMerge : commitMerges) {
+                if (runningMerges.contains(commitMerge) || pendingMerges.contains(commitMerge)) {
+                  abandonedCount++;
+                }
+              }
+            }
+          }
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        } finally {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", String.format(Locale.ROOT, "Waited %.1f ms for commit merges",
+                (System.nanoTime() - mergeWaitStart)/1_000_000.0));
+            infoStream.message("IW", "After executing commit merges, had " + toCommit.size() + " segments");
+            if (abandonedCount > 0) {
+              infoStream.message("IW", "Abandoned " + abandonedCount + " commit merges after " + waitTimeMillis + " ms");
+            }
+          }
+          if (abandonedCount > 0) {
+            config.getIndexWriterEvents().abandonedMergesOnCommit(abandonedCount);

Review comment:
       what is this used for? Can we use testPoints for it if it's necesary?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -459,6 +463,31 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
     return this;
   }
 
+  /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.
+   */
+  public IndexWriterConfig setMaxCommitMergeWaitSeconds(double maxCommitMergeWaitSeconds) {
+    this.maxCommitMergeWaitSeconds = maxCommitMergeWaitSeconds;
+    return this;
+  }
+
+  /**
+   * Set the callback that gets invoked when IndexWriter performs various actions.
+   */
+  public IndexWriterConfig setIndexWriterEvents(IndexWriterEvents indexWriterEvents) {

Review comment:
       this method is never called in this entire PR. Also the Interface seems to be unused or rather never implemented. I think it should be removed.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -459,6 +463,31 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
     return this;
   }
 
+  /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.
+   */
+  public IndexWriterConfig setMaxCommitMergeWaitSeconds(double maxCommitMergeWaitSeconds) {

Review comment:
       I really don't like this setting. While I am convinced we should be very very careful adding more settings here, we should if possible use a parameter on a method to pass information like this. I personally would feel much better if we had a new method on IW called `prepareCommit(double maxCommitMergeWaitSeconds)` Maybe we can even go further and also pass a function to select the merges such that we don't need to add more stuff to mergePolicy?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-646029945


   @msokolov you are more than welcome. I think it's a great example how OSS works or should work thanks for being so patient with me :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] bruno-roustant commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-639631017


   The 2nd - TestPhraseWildcardQuery failure should be ignored, it's not a real issue. I'll fix that fragile test.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439492017



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4483,6 +4593,7 @@ public int length() {
         // Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
         assert merge.info.info.maxDoc() == 0;
         commitMerge(merge, mergeState);
+        success = true;

Review comment:
       I think this was a small pre-existing bug.
   
   I.e. the merge has in fact succeeded on this path.  Before this change we are calling `closeMergeReaders` twice (once in the line above this, then again on line 4720 below.  Maybe that is harmless, but code-wise I think this path did succeed.
   
   If necessary, we could pull this out into its own PR?  But I think it's a good, if subtle, catch.  The merge did succeed in this path.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3228,15 +3268,38 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges) {
+                // Find any merges that can execute on commit (per MergePolicy).
+                MergePolicy.MergeSpecification mergeSpec =

Review comment:
       I think what makes this tricky is that this is a combination of `MergePolicy` (to pick the small merges) and `MergeScheduler` (to run them and await their completion, subject to a time limit) purposes.
   
   I do not think you can achieve this by just wrapping in `MergePolicy`, but I agree it would be better if we could find a simpler way to achieve it.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -459,6 +463,31 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
     return this;
   }
 
+  /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.
+   */
+  public IndexWriterConfig setMaxCommitMergeWaitSeconds(double maxCommitMergeWaitSeconds) {

Review comment:
       I think @msfroh had considered a separate `IndexWriter` method before but something went wrong with that approach?
   
   I don't think this should be a separate method, actually.
   
   We have a `MergePolicy` that governs which merges should happen upon which events/triggers and what this change is adding is a new trigger (on commit) at which merging could conceivably occur.  If we added this method, the implication to fresh eyes would be that the existing `prepareCommit` will also wait for merges with some default parameter, while this new method lets you change the default.
   
   Anyway, let's hear from @msfroh if there was some wrinkle on making a dedicated method for this, but I still think that's a messy API.  We should rather use our existing `MergePolicy` API correctly.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3257,6 +3320,52 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
+
+      if (mergeAwaitLatchRef != null) {
+        CountDownLatch mergeAwaitLatch = mergeAwaitLatchRef.get();
+        // If we found and registered any merges above, within the flushLock, then we want to ensure that they
+        // complete execution. Note that since we released the lock, other merges may have been scheduled. We will
+        // block until  the merges that we registered complete. As they complete, they will update toCommit to
+        // replace merged segments with the result of each merge.
+        config.getIndexWriterEvents().beginMergeOnCommit();
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+        long mergeWaitStart = System.nanoTime();
+        int abandonedCount = 0;
+        long waitTimeMillis = (long) (config.getMaxCommitMergeWaitSeconds() * 1000.0);
+        try {
+          if (mergeAwaitLatch.await(waitTimeMillis, TimeUnit.MILLISECONDS) == false) {
+            synchronized (this) {
+              // Need to do this in a synchronized block, to make sure none of our commit merges are currently
+              // executing mergeFinished (since mergeFinished itself is called from within the IndexWriter lock).
+              // After we clear the value from mergeAwaitLatchRef, the merges we schedule will still execute as
+              // usual, but when they finish, they won't attempt to update toCommit or modify segment reference
+              // counts.
+              mergeAwaitLatchRef.set(null);
+              for (MergePolicy.OneMerge commitMerge : commitMerges) {
+                if (runningMerges.contains(commitMerge) || pendingMerges.contains(commitMerge)) {
+                  abandonedCount++;
+                }
+              }
+            }
+          }
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        } finally {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", String.format(Locale.ROOT, "Waited %.1f ms for commit merges",
+                (System.nanoTime() - mergeWaitStart)/1_000_000.0));
+            infoStream.message("IW", "After executing commit merges, had " + toCommit.size() + " segments");
+            if (abandonedCount > 0) {
+              infoStream.message("IW", "Abandoned " + abandonedCount + " commit merges after " + waitTimeMillis + " ms");
+            }
+          }
+          if (abandonedCount > 0) {
+            config.getIndexWriterEvents().abandonedMergesOnCommit(abandonedCount);

Review comment:
       The idea with this is to be able to track externally to `IndexWriter` how many merges completed during the commit window, how many did not, how long the commit waited for small merges to finish, etc.  This is useful telemetry for understanding how the feature is actually working in your production cluster.  I don't think we can get the equivalent telemetry by e.g. wrapping `MergePolicy` or `MergeScheduler`, because it is `IndexWriter` that knows how long it will wait and knows which merges made it and which did not.
   
   I think `testPoints` only run under assertion, and are really designed for unit tests to do interesting things.  But maybe we could somehow abuse it for this use case?
   
   I agree it should be tested better if we want to include it here.
   
   If it is really controversial then +1 to remove the `IndexWriterEvents` entirely here and pursue it as a separate issue.  It was useful for us (Amazon product search) to understand how this feature impacts our production clusters, and to help us tune a reasonable timeout value.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -109,6 +110,9 @@
   
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+  /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+  public static final double DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 30.0;

Review comment:
       Maybe 5.0 or 10.0 seconds instead?  A "typical" commit, writing new Lucene segments and `fsync`ing many files can often take several seconds, but 30 seems high.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3152,6 +3154,42 @@ public final boolean flushNextBuffer() throws IOException {
     }
   }
 
+  private MergePolicy.OneMerge updateSegmentInfosOnMergeFinish(MergePolicy.OneMerge merge, final SegmentInfos toCommit,
+                                                                AtomicReference<CountDownLatch> mergeLatchRef) {
+    return new MergePolicy.OneMerge(merge.segments) {
+      public void mergeFinished() throws IOException {
+        super.mergeFinished();
+        CountDownLatch mergeAwaitLatch = mergeLatchRef.get();
+        if (mergeAwaitLatch == null) {
+          // Commit thread timed out waiting for this merge and moved on. No need to manipulate toCommit.
+          return;
+        }
+        if (committed) {
+          deleter.incRef(this.info.files());

Review comment:
       I think @msfroh might remember some context on why he settled on this approach.
   
   This is inherently a complex problem: we want to let `MergePolicy` pick the "right" merges to do on commit (the smallish ones).  But, it may pick poorly, or maybe the box is IO starved currently, and so we want to 1) ask `MergeScheduler` to kick off the merges it had requested, but 2) those merges that complete it time will make it into the commit point, while those that do not will still be allowed to run to completion and be switched in the live `SegmentInfos` (no wasted work), but will not be in the current commit point.
   
   Anyway, +1 if we can re-use existing code that "merges the merged segments" into a live or the pending commit `SegmentInfos`.  Maybe we can factor out the common code to reduce the added complexity here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -459,6 +463,31 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
     return this;
   }
 
+  /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.
+   */
+  public IndexWriterConfig setMaxCommitMergeWaitSeconds(double maxCommitMergeWaitSeconds) {
+    this.maxCommitMergeWaitSeconds = maxCommitMergeWaitSeconds;
+    return this;
+  }
+
+  /**
+   * Set the callback that gets invoked when IndexWriter performs various actions.
+   */
+  public IndexWriterConfig setIndexWriterEvents(IndexWriterEvents indexWriterEvents) {

Review comment:
       Yeah for sure if we include it in this PR it should be tested by the PR.
   
   See my comment above about this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dnhatn commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
dnhatn commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-638557223


   @s1monw Can you please take a look at this PR? You already left some [comments](https://issues.apache.org/jira/browse/LUCENE-8962?focusedCommentId=17053231&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17053231) for it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-639613345


   CC @bruno-roustant see TestPhraseWildcardQuery above


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442422185



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3226,15 +3235,53 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges && maxCommitMergeWaitSeconds > 0) {
+                SegmentInfos committingSegmentInfos = toCommit;
+                onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
+                    new MergePolicy.OneMerge(toWrap.segments) {
+                      @Override
+                      public void mergeFinished(boolean committed) throws IOException {

Review comment:
       I like to move stuf once necessary I think we need to adjust it there anyway so we can move it in a followup. ok?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r435592460



##########
File path: lucene/CHANGES.txt
##########
@@ -376,6 +376,8 @@ Improvements
 
 * LUCENE-9253: KoreanTokenizer now supports custom dictionaries(system, unknown). (Namgyu Kim)
 
+* LUCENE-8962: Add ability to selectively merge on commit (Michael Froh)

Review comment:
       Maybe `Add IndexWriter merge-on-commit feature to selectively merge small segments on commit, subject to a configurable timeout, to improve search performance by reducing the number of small segments for searching`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -459,6 +463,31 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
     return this;
   }
 
+  /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.

Review comment:
       Maybe for the last sentence: `The merges are not cancelled, and will still run to completion independent of the commit like normal segment merges`?
   
   Maybe also state that this setting has no effect unless the `MergePolicy` actually returns merges from `findFullFlushMerges`, which the default merge policy does not?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-639161416


   I'll beast this PR on my 128-core AMD Ryzen box :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439657240



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -109,6 +110,9 @@
   
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+  /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+  public static final double DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 30.0;

Review comment:
       The `success=true` added above was needed in order to fix a test failure caught by @dnhatn 's new unit test (testRandomOperations), so they belong together.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-645658854


   @msokolov @mikemccand @msfroh I merged https://github.com/apache/lucene-solr/pull/1585 and updated this PR to use it. I also went ahead and removed the IndexWriterEvents interface, cut over to use long instead of double as a config value for the time to wait and set the default to 0.  I will let you folks look at it again. I am happy to help further. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643599140


   > > @mikemccand thanks for replying to all these comments. I do understand that this change has an impact and I agree we should add this functionality. I just disagree with the how it's done and how much code is used. I will go an reply to some of your comments directly, in the meanwhile I went ahead to prototype some ideas in how this can be less intrusive and reuse code. I pushed one commit here [s1monw@3864b6c](https://github.com/s1monw/lucene-solr/commit/3864b6c2b631879fa1e995d47ed2b84aae054747) to showcase what I mean. I even think we can get away without a new method on MergePolicy but that's too much for the prototype. I'd be ok with adding a setting to IWC if we can't agree on a different way.
   > 
   > Thanks @s1monw! I would love if we could find a simple way to implement this feature as long as it keeps the "no wasted work" (merge either finishes in time, and is reflected in the commit point, or does not, but still runs to completion and is reflected later). I will review your prototype soon ... I'm mostly offline this weekend but will try to look soon.
   
   thanks @mikemccand I pushed another commit to my prototype to make it almost the same as this change but with a bit more code-reuse I think. please take a look at this here https://github.com/apache/lucene-solr/compare/master...s1monw:LUCENE-8962


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov merged pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msokolov merged pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msfroh commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msfroh commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442326229



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3226,15 +3235,53 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges && maxCommitMergeWaitSeconds > 0) {
+                SegmentInfos committingSegmentInfos = toCommit;
+                onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
+                    new MergePolicy.OneMerge(toWrap.segments) {
+                      @Override
+                      public void mergeFinished(boolean committed) throws IOException {

Review comment:
       Oh -- I guess one minor complaint about moving this into `prepareCommitInternal` is that we won't be able to reuse it (without moving it) if we decide to apply the same logic to `IndexWriter.getReader()`.
   
   That said, moving it if/when someone gets around to applying the logic there isn't a big deal. (I think the real work there is reconciling logic from StandardDirectoryReader.open() with logic in IndexWriter.prepareCommitInternal(), since the functionality is kind of duplicated.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-640678328


   > The 2nd - TestPhraseWildcardQuery failure should be ignored, it's not a real issue. I'll fix that fragile test.
   
   Great, thanks @bruno-roustant!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-644226072


   Just for my sanity to keep track of all the exciting PRs here :)  Here is the original PR (that was pushed, then reverted, then led to this PR): https://github.com/apache/lucene-solr/pull/1155
   
   And here is @s1monw's new proposed simpler approach: https://github.com/apache/lucene-solr/pull/1576


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439646861



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3257,6 +3320,52 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
+
+      if (mergeAwaitLatchRef != null) {
+        CountDownLatch mergeAwaitLatch = mergeAwaitLatchRef.get();
+        // If we found and registered any merges above, within the flushLock, then we want to ensure that they
+        // complete execution. Note that since we released the lock, other merges may have been scheduled. We will
+        // block until  the merges that we registered complete. As they complete, they will update toCommit to
+        // replace merged segments with the result of each merge.
+        config.getIndexWriterEvents().beginMergeOnCommit();
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+        long mergeWaitStart = System.nanoTime();
+        int abandonedCount = 0;
+        long waitTimeMillis = (long) (config.getMaxCommitMergeWaitSeconds() * 1000.0);
+        try {
+          if (mergeAwaitLatch.await(waitTimeMillis, TimeUnit.MILLISECONDS) == false) {
+            synchronized (this) {
+              // Need to do this in a synchronized block, to make sure none of our commit merges are currently
+              // executing mergeFinished (since mergeFinished itself is called from within the IndexWriter lock).
+              // After we clear the value from mergeAwaitLatchRef, the merges we schedule will still execute as
+              // usual, but when they finish, they won't attempt to update toCommit or modify segment reference
+              // counts.
+              mergeAwaitLatchRef.set(null);
+              for (MergePolicy.OneMerge commitMerge : commitMerges) {
+                if (runningMerges.contains(commitMerge) || pendingMerges.contains(commitMerge)) {
+                  abandonedCount++;
+                }
+              }
+            }
+          }
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        } finally {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", String.format(Locale.ROOT, "Waited %.1f ms for commit merges",
+                (System.nanoTime() - mergeWaitStart)/1_000_000.0));
+            infoStream.message("IW", "After executing commit merges, had " + toCommit.size() + " segments");
+            if (abandonedCount > 0) {
+              infoStream.message("IW", "Abandoned " + abandonedCount + " commit merges after " + waitTimeMillis + " ms");
+            }
+          }
+          if (abandonedCount > 0) {
+            config.getIndexWriterEvents().abandonedMergesOnCommit(abandonedCount);

Review comment:
       I think we should detach this discussion if we need metrics on IW from this PR. It distracts from it's actual core change IMO and if'd add metrics then we'd need some more or can consolidate them too. I'd rather have a stats object than a callback here to be honest but again that's a different discussion.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439647267



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4483,6 +4593,7 @@ public int length() {
         // Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
         assert merge.info.info.maxDoc() == 0;
         commitMerge(merge, mergeState);
+        success = true;

Review comment:
       can we fix it in a dedicated PR with a dedicated test? that would also help if we look at the history of the bugfix?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643476200


   Test case #1 above, at least does reproduce (and #4 looks like a similar stack trace; I did not try it):
   
       gradlew :lucene:core:test --tests "org.apache.lucene.index.TestIndexFileDeleter" -Ptests.seed=DC21EB3B9D4052A4 
   
   My experience with #3 (testRandomOperations) is it doesn't tend to reproduce with a given seed. It is indeed quite random. And these did not reproduce for me.
   
   #5 reproduces:
   
       gradlew :lucene:core:test  -Dtestcase=TestIndexWriterExceptions2 -Dtests.method=testBasics -Dtests.seed=AC9C0966B9BC03C8
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643498821


   Re those test failures: I was able to fix by checking for an empty merge and not submitting it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-639524684


   > I'll beast this PR on my 128-core AMD Ryzen box :)
   
   I ran this overnight, consuming a lot of electricity and generating heat in my already too hot basement ;0
   
   It finished all Lucene (core + modules) tests 889 times, and hit a few interesting failures!  I have not checked 1) whether they reproduce, nor 2) whether these failures might also occur on clean master:
   
   1st:
   
   ```
     2> NOTE: reproduce with: ant test  -Dtestcase=TestIndexFileDeleter -Dtests.method=testExcInDecRef -Dtests.seed=DC21EB3B9D4052A4 -Dtests.slow=true -Dtests.badapples\
   =true -Dtests.locale=sw -Dtests.timezone=Africa/Cairo -Dtests.asserts=true -Dtests.file.encoding=UTF-8
     2> Jun 05, 2020 4:16:02 AM com.carrotsearch.randomizedtesting.RandomizedRunner$QueueUncaughtExceptionsHandler uncaughtException
     2> WARNING: Uncaught exception in thread: Thread[Lucene Merge Thread #0,5,TGRP-TestIndexFileDeleter]
     2> org.apache.lucene.index.MergePolicy$MergeException: java.lang.RuntimeException: segments must include at least one segment
     2>    at __randomizedtesting.SeedInfo.seed([DC21EB3B9D4052A4]:0)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler.handleMergeException(ConcurrentMergeScheduler.java:703)
     2>    at org.apache.lucene.index.TestIndexFileDeleter$2.handleMergeException(TestIndexFileDeleter.java:438)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:694)
     2> Caused by: java.lang.RuntimeException: segments must include at least one segment
     2>    at org.apache.lucene.index.MergePolicy$OneMerge.<init>(MergePolicy.java:235)
     2>    at org.apache.lucene.index.IndexWriter$2.mergeFinished(IndexWriter.java:3182)
     2>    at org.apache.lucene.index.IndexWriter.closeMergeReaders(IndexWriter.java:4418)
     2>    at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:4720)
     2>    at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:4145)
     2>    at org.apache.lucene.index.IndexWriter$IndexWriterMergeSource.merge(IndexWriter.java:5706)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(ConcurrentMergeScheduler.java:624)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:682)
     2>
     2> NOTE: leaving temporary files on disk at: /tmp/lucene_tests/lucene.index.TestIndexFileDeleter_DC21EB3B9D4052A4-001
     2> NOTE: test params are: codec=Lucene86, sim=Asserting(org.apache.lucene.search.similarities.AssertingSimilarity@248b50a7), locale=sw, timezone=Africa/Cairo
     2> NOTE: Linux 5.5.6-arch1-1 amd64/Oracle Corporation 11.0.6 (64-bit)/cpus=128,threads=1,free=165714056,total=268435456
     2> NOTE: All tests run in this JVM: [TestTessellator, TestFilterLeafReader, TestNRTReaderCleanup, TestIndexFileDeleter]
   ```
   
   2nd:
   
   ```
   org.apache.lucene.search.TestPhraseWildcardQuery > testExplain FAILED
       org.junit.ComparisonFailure: expected:<...tle:"t?e b* b*") in [1]) [AssertingSimilari...> but was:<...tle:"t?e b* b*") in [4]) [AssertingSimilari...>
           at __randomizedtesting.SeedInfo.seed([698AF96B1C28F923:DF7EB48A3083036C]:0)
           at org.junit.Assert.assertEquals(Assert.java:115)
           at org.junit.Assert.assertEquals(Assert.java:144)
           at org.apache.lucene.search.TestPhraseWildcardQuery.testExplain(TestPhraseWildcardQuery.java:233)
           ...
     2> NOTE: reproduce with: ant test  -Dtestcase=TestPhraseWildcardQuery -Dtests.method=testExplain -Dtests.seed=698AF96B1C28F923 -Dtests.slow=true -Dtests.badapples=\
   true -Dtests.locale=ebu -Dtests.timezone=SystemV/CST6 -Dtests.asserts=true -Dtests.file.encoding=UTF-8
     2> NOTE: test params are: codec=Asserting(Lucene86): {other=PostingsFormat(name=Direct), author=BlockTreeOrds(blocksize=128), category=PostingsFormat(name=Assertin\
   g), title=PostingsFormat(name=Direct)}, docValues:{}, maxPointsInLeafNode=576, maxMBSortInHeap=6.318571589527405, sim=Asserting(org.apache.lucene.search.similarities\
   .AssertingSimilarity@37191dc0), locale=ebu, timezone=SystemV/CST6
     2> NOTE: Linux 5.5.6-arch1-1 amd64/Oracle Corporation 11.0.6 (64-bit)/cpus=128,threads=1,free=176758768,total=268435456
     2> NOTE: All tests run in this JVM: [TestPhraseWildcardQuery]
   ```
   
   3rd:
   
   ```
   org.apache.lucene.index.TestIndexWriter > test suite's output saved to /l/trunk/lucene/core/build/test-results/test/outputs/OUTPUT-org.apache.lucene.index.TestIndexW\
   riter.txt, copied below:
     2> xun 05, 2020 6:26:03 DE LA MAÑANA com.carrotsearch.randomizedtesting.RandomizedRunner$QueueUncaughtExceptionsHandler uncaughtException
     2> WARNING: Uncaught exception in thread: Thread[Thread-50,5,TGRP-TestIndexWriter]
     2> java.lang.AssertionError: java.nio.file.NoSuchFileException: _1y.fdt
     2>    at __randomizedtesting.SeedInfo.seed([C384705D23CB7F34]:0)
     2>    at org.apache.lucene.index.TestIndexWriter.lambda$testRandomOperations$48(TestIndexWriter.java:3886)
     2>    at java.base/java.lang.Thread.run(Thread.java:834)
     2> Caused by: java.nio.file.NoSuchFileException: _1y.fdt
     2>    at org.apache.lucene.store.ByteBuffersDirectory.deleteFile(ByteBuffersDirectory.java:148)
     2>    at org.apache.lucene.store.MockDirectoryWrapper.deleteFile(MockDirectoryWrapper.java:607)
     2>    at org.apache.lucene.store.LockValidatingDirectoryWrapper.deleteFile(LockValidatingDirectoryWrapper.java:38)
     2>    at org.apache.lucene.index.IndexFileDeleter.deleteFile(IndexFileDeleter.java:696)
     2>    at org.apache.lucene.index.IndexFileDeleter.deleteFiles(IndexFileDeleter.java:690)
     2>    at org.apache.lucene.index.IndexFileDeleter.decRef(IndexFileDeleter.java:589)
     2>    at org.apache.lucene.index.IndexFileDeleter.deleteCommits(IndexFileDeleter.java:382)
     2>    at org.apache.lucene.index.IndexFileDeleter.checkpoint(IndexFileDeleter.java:527)
     2>    at org.apache.lucene.index.IndexWriter.finishCommit(IndexWriter.java:3601)
     2>    at org.apache.lucene.index.IndexWriter.commitInternal(IndexWriter.java:3557)
     2>    at org.apache.lucene.index.IndexWriter.commit(IndexWriter.java:3507)
     2>    at org.apache.lucene.index.TestIndexWriter.lambda$testRandomOperations$48(TestIndexWriter.java:3879)
     2>    ... 1 more
   
     2> NOTE: reproduce with: ant test  -Dtestcase=TestIndexWriter -Dtests.method=testRandomOperations -Dtests.seed=C384705D23CB7F34 -Dtests.slow=true -Dtests.badapples\
   =true -Dtests.locale=ast -Dtests.timezone=Libya -Dtests.asserts=true -Dtests.file.encoding=UTF-8
     2> NOTE: test params are: codec=Asserting(Lucene86): {=PostingsFormat(name=LuceneVarGapFixedInterval), a=PostingsFormat(name=Direct), c=PostingsFormat(name=LuceneV\
   arGapDocFreqInterval), string=PostingsFormat(name=Direct), foo=BlockTreeOrds(blocksize=128), body=BlockTreeOrds(blocksize=128), version=PostingsFormat(name=LuceneVar\
   GapFixedInterval), content=PostingsFormat(name=Direct), str=PostingsFormat(name=Direct), tvtest=BlockTreeOrds(blocksize=128), field=BlockTreeOrds(blocksize=128), con\
   tent4=PostingsFormat(name=LuceneVarGapDocFreqInterval), str3=BlockTreeOrds(blocksize=128), content1=BlockTreeOrds(blocksize=128), binary=PostingsFormat(name=Direct),\
    str2=PostingsFormat(name=Direct), id=PostingsFormat(name=LuceneVarGapDocFreqInterval), myfield=BlockTreeOrds(blocksize=128)}, docValues:{numval=DocValuesFormat(name\
   =Lucene80), my_deletes=DocValuesFormat(name=Lucene80), sortedsetdv=DocValuesFormat(name=Lucene80), sortednumericdv=DocValuesFormat(name=Asserting), soft_deletes=DocV\
   aluesFormat(name=Lucene80), num=DocValuesFormat(name=Asserting), binarydv=DocValuesFormat(name=Lucene80), dv=DocValuesFormat(name=Asserting), dv_field=DocValuesForma\
   t(name=Lucene80), numericdv=DocValuesFormat(name=Lucene80), soft_delete=DocValuesFormat(name=Asserting), sorteddv=DocValuesFormat(name=Lucene80), id=DocValuesFormat(\
   name=Lucene80), soft_deletes_1=DocValuesFormat(name=Lucene80)}, maxPointsInLeafNode=95, maxMBSortInHeap=5.8271489113157, sim=Asserting(org.apache.lucene.search.simil\
   arities.AssertingSimilarity@481a5d24), locale=ast, timezone=Libya
     2> NOTE: Linux 5.5.6-arch1-1 amd64/Oracle Corporation 11.0.6 (64-bit)/cpus=128,threads=1,free=112682104,total=268435456
     2> NOTE: All tests run in this JVM: [TestForUtil, TestIndexWriter]
   ```
   
   4th:
   ```
     2> NOTE: reproduce with: ant test  -Dtestcase=TestIndexFileDeleter -Dtests.method=testExcInDecRef -Dtests.seed=DBA89BFE7C61C7AB -Dtests.slow=true -Dtests.badapples\
   =true -Dtests.locale=ln-CG -Dtests.timezone=America/Noronha -Dtests.asserts=true -Dtests.file.encoding=UTF-8
     2> yun 05, 2020 7:27:19 NTƆ́NGƆ́ com.carrotsearch.randomizedtesting.RandomizedRunner$QueueUncaughtExceptionsHandler uncaughtException
     2> WARNING: Uncaught exception in thread: Thread[Lucene Merge Thread #1,5,TGRP-TestIndexFileDeleter]
     2> org.apache.lucene.index.MergePolicy$MergeException: java.lang.RuntimeException: segments must include at least one segment
     2>    at __randomizedtesting.SeedInfo.seed([DBA89BFE7C61C7AB]:0)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler.handleMergeException(ConcurrentMergeScheduler.java:703)
     2>    at org.apache.lucene.index.TestIndexFileDeleter$2.handleMergeException(TestIndexFileDeleter.java:438)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:694)
     2> Caused by: java.lang.RuntimeException: segments must include at least one segment
     2>    at org.apache.lucene.index.MergePolicy$OneMerge.<init>(MergePolicy.java:235)
     2>    at org.apache.lucene.index.IndexWriter$2.mergeFinished(IndexWriter.java:3182)
     2>    at org.apache.lucene.index.IndexWriter.closeMergeReaders(IndexWriter.java:4418)
     2>    at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:4720)
     2>    at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:4145)
     2>    at org.apache.lucene.index.IndexWriter$IndexWriterMergeSource.merge(IndexWriter.java:5706)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(ConcurrentMergeScheduler.java:624)
     2>    at org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:682)
     2>
     2> NOTE: leaving temporary files on disk at: /tmp/lucene_tests/lucene.index.TestIndexFileDeleter_DBA89BFE7C61C7AB-001
     2> NOTE: test params are: codec=Lucene86, sim=Asserting(org.apache.lucene.search.similarities.AssertingSimilarity@44ab18a0), locale=ln-CG, timezone=America/Noronha
     2> NOTE: Linux 5.5.6-arch1-1 amd64/Oracle Corporation 11.0.6 (64-bit)/cpus=128,threads=1,free=150481312,total=268435456
     2> NOTE: All tests run in this JVM: [TestTessellator, TestFilterLeafReader, TestNRTReaderCleanup, TestIndexFileDeleter]
   ```
   
   5th:
   
   ```
      >     java.lang.RuntimeException: segments must include at least one segment
      >         at __randomizedtesting.SeedInfo.seed([AC9C0966B9BC03C8:9144A74A81525DB8]:0)
      >         at org.apache.lucene.index.MergePolicy$OneMerge.<init>(MergePolicy.java:235)
      >         at org.apache.lucene.index.IndexWriter$2.mergeFinished(IndexWriter.java:3182)
      >         at org.apache.lucene.index.IndexWriter.closeMergeReaders(IndexWriter.java:4418)
      >         at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:4720)
      >         at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:4145)
      >         at org.apache.lucene.index.IndexWriter$IndexWriterMergeSource.merge(IndexWriter.java:5706)
      >         at org.apache.lucene.index.SerialMergeScheduler.merge(SerialMergeScheduler.java:40)
      >         at org.apache.lucene.index.IndexWriter.prepareCommitInternal(IndexWriter.java:3331)
      >         at org.apache.lucene.index.IndexWriter.commitInternal(IndexWriter.java:3549)
      >         at org.apache.lucene.index.IndexWriter.commit(IndexWriter.java:3507)
      >         at org.apache.lucene.index.TestIndexWriterExceptions2.testBasics(TestIndexWriterExceptions2.java:211)
      >         ...
     2> NOTE: reproduce with: ant test  -Dtestcase=TestIndexWriterExceptions2 -Dtests.method=testBasics -Dtests.seed=AC9C0966B9BC03C8 -Dtests.slow=true -Dtests.badapple\
   s=true -Dtests.locale=et-EE -Dtests.timezone=Australia/Lord_Howe -Dtests.asserts=true -Dtests.file.encoding=UTF-8
     2> NOTE: test params are: codec=Asserting(Lucene86): {}, docValues:{}, maxPointsInLeafNode=1394, maxMBSortInHeap=6.947614894754386, sim=Asserting(org.apache.lucene\
   .search.similarities.AssertingSimilarity@52aafa77), locale=et-EE, timezone=Australia/Lord_Howe
     2> NOTE: Linux 5.5.6-arch1-1 amd64/Oracle Corporation 11.0.6 (64-bit)/cpus=128,threads=1,free=216811792,total=268435456
     2> NOTE: All tests run in this JVM: [TestLucene50StoredFieldsFormatHighCompression, Test2BPostingsBytes, TestIndexWriterExceptions2]
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643496306


   > @mikemccand thanks for replying to all these comments. I do understand that this change has an impact and I agree we should add this functionality. I just disagree with the how it's done and how much code is used. I will go an reply to some of your comments directly, in the meanwhile I went ahead to prototype some ideas in how this can be less intrusive and reuse code. I pushed one commit here [s1monw@3864b6c](https://github.com/s1monw/lucene-solr/commit/3864b6c2b631879fa1e995d47ed2b84aae054747) to showcase what I mean. I even think we can get away without a new method on MergePolicy but that's too much for the prototype. I'd be ok with adding a setting to IWC if we can't agree on a different way.
   
   Thanks @s1monw!  I would love if we could find a simple way to implement this feature as long as it keeps the "no wasted work" (merge either finishes in time, and is reflected in the commit point, or does not, but still runs to completion and is reflected later).  I will review your prototype soon ... I'm mostly offline this weekend but will try to look soon.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442167608



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3255,7 +3302,16 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
-     
+
+      if (onCommitMerges != null) {
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);

Review comment:
       yeah I mean we don't have to do that and I think its rather a rare combination. My problem is that this entire configuration of max wait time is nonsense if SerialMS is used since we block until it has merged them all and potentially a bunch of other merges to a commit / refresh could take quite a long time. On the other hand, as you stated we will call maybeMerge anyway in the commit such that it's not really making any difference and the same is true for getReader so I think we are fine as it is.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-647487636


   Thanks @s1monw @msfroh @msokolov!  What an awesome feature and great OSS collaboration bringing it to Lucene!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439647024



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3228,15 +3268,38 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges) {
+                // Find any merges that can execute on commit (per MergePolicy).
+                MergePolicy.MergeSpecification mergeSpec =

Review comment:
       I tried to showcase this here https://github.com/apache/lucene-solr/compare/master...s1monw:LUCENE-8962




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msfroh commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msfroh commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442043040



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3255,7 +3302,16 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
-     
+
+      if (onCommitMerges != null) {
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);

Review comment:
       Would it be sufficient to document the behavior in the Javadoc for `findFullFlushMerges`?
   
   I was assuming that any implementation of `findFullFlushMerges` would try to return merges that are very likely to complete within whatever timeout someone would reasonably set (e.g. a few seconds). The timeout was intended just as an extra safeguard in case a merge takes longer. 
   
   Given that lots of IndexWriter operations can have pauses with `SerialMergeScheduler` (judging by the number of calls to `maybeMerge`, especially the one from `processEvents`, in IndexWriter), blocking on this particular `merge` call doesn't feel like it introduces more risk (especially since it needs to be used in conjunction with a `MergePolicy` that implements `findFullFlushMerges`).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-644317157


   OK I opened [this issue](https://issues.apache.org/jira/browse/LUCENE-9406) to explore how to know what specifically `IndexWriter` is doing for merge-on-commit and other expert actions, and [this issue](https://issues.apache.org/jira/browse/LUCENE-9405) for the controversial pre-existing missing `success = true` `IndexWriter` bug.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643486962


   @mikemccand thanks for replying to all these comments. I do understand that this change has an impact and I agree we should add this functionality. I just disagree with the how it's done and how much code is used. I will go an reply to some of your comments directly, in the meanwhile I went ahead to prototype some ideas in how this can be less intrusive and reuse code. I pushed one commit here https://github.com/s1monw/lucene-solr/commit/3864b6c2b631879fa1e995d47ed2b84aae054747 to showcase what I mean. I even think we can get away without a new method on MergePolicy but that's too much for the prototype. I'd be ok with adding a setting to IWC if we can't agree on a different way. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw edited a comment on pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw edited a comment on pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#issuecomment-643486962


   @mikemccand thanks for replying to all these comments. I do understand that this change has an impact and I agree we should add this functionality. I just disagree with the how it's done and how much code is used. I will go an reply to some of your comments directly, in the meanwhile I went ahead to prototype some ideas in how this can be less intrusive and reuse code. I pushed one commit here https://github.com/apache/lucene-solr/compare/master...s1monw:LUCENE-8962 to showcase what I mean. I even think we can get away without a new method on MergePolicy but that's too much for the prototype. I'd be ok with adding a setting to IWC if we can't agree on a different way. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msfroh commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
msfroh commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442043040



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3255,7 +3302,16 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
-     
+
+      if (onCommitMerges != null) {
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);

Review comment:
       Would it be sufficient to document the behavior in the Javadoc for `findFullFlushMerges`?
   
   I was assuming that any implementation of `findFullFlushMerges` would try to return merges that are very likely complete within whatever timeout someone would reasonably set (e.g. a few seconds). The timeout was intended just as an extra safeguard in case a merge takes longer. 
   
   Given that lots of IndexWriter operations can have pauses with `SerialMergeScheduler` (judging by the number of calls to `maybeMerge`, especially the one from `processEvents`, in IndexWriter), blocking on this particular `merge` call doesn't feel like it introduces more risk (especially since it needs to be used in conjunction with a `MergePolicy` that implements `findFullFlushMerges`).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r442023300



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3255,7 +3302,16 @@ private long prepareCommitInternal() throws IOException {
       } finally {
         maybeCloseOnTragicEvent();
       }
-     
+
+      if (onCommitMerges != null) {
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);

Review comment:
       the last thing that I am afraid about is what if we has a MergeScheduler configured that blocks on this call like SerialMergeScheduler? I think there are multiple options like: documentation, skipping `COMMIT` merge triggers in SMS, adding a mergeAsync method to MS that has no impl in SMS... I think we should make sure that this is not trappy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1552: LUCENE-8962: merge small segments on commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1552:
URL: https://github.com/apache/lucene-solr/pull/1552#discussion_r439647024



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3228,15 +3268,38 @@ private long prepareCommitInternal() throws IOException {
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges) {
+                // Find any merges that can execute on commit (per MergePolicy).
+                MergePolicy.MergeSpecification mergeSpec =

Review comment:
       I tried to showcase this here https://github.com/s1monw/lucene-solr/commit/3864b6c2b631879fa1e995d47ed2b84aae054747




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org