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: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org