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