-
Notifications
You must be signed in to change notification settings - Fork 2.6k
LUCENE-8962: Add ability to selectively merge on commit #1155
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
just a few micro comments. Also, I wonder if you could paste something about the results of this on merge statistics (in the issue)?
@@ -3223,15 +3259,44 @@ private long prepareCommitInternal() throws IOException { | |||
// sneak into the commit point: | |||
toCommit = segmentInfos.clone(); | |||
|
|||
if (anyChanges) { | |||
mergeAwaitLatchRef = new AtomicReference<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
could we wait to create this until we create its CountdownLatch and simplify the null checks below?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a little bit of hackery to share state between this thread and the threads that do the merges.
We initialize the ref here, pass it to waitForMergeOnCommitPolicy
on the next line to make sure it gets shared with any computed OneMerge
s. Then, before we release the IW lock (so we're guaranteed that those OneMerge
s haven't run yet), we populate the ref with the CountdownLatch
(once we know what we're counting down).
That said, I think I could simplify things a lot by not using OneMergeWrappingMergePolicy
, but rather decorating the returned OneMerge
s (if applicable) directly. I'm going to take a stab at that approach in my next commit.
This adds a new "findCommitMerges" method to MergePolicy, which can specify merges to be executed before the IndexWriter.prepareCommitInternal method returns. If we have many index writer threads, they will flush their DWPT buffers on commit, resulting in many small segments, which can be merged before the commit returns.
lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
Outdated
Show resolved
Hide resolved
1. Made some changes to the callback to update toCommit, leveraging SegmentInfos.applyMergeChanges. 2. I realized that we'll never end up with 0 registered merges, because we throw an exception if we fail to register a merge. 3. Moved the IndexWriterEvents.beginMergeOnCommit notification to before we call MergeScheduler.merge, since we may not be merging on another thread. 4. There was an intermittent test failure due to randomness in the time it takes for merges to complete. Before doing the final commit, we wait for pending merges to finish. We may still end up abandoning the final merge, but we can detect that and assert that either the merge was abandoned (and we have > 1 segment) or we did merge down to 1 segment.
Thanks, @msfroh this looks good, and I appreciate the reduction in segment count, which can lead to nice performance gains! Let's let this bake a little longer to see if anybody has concerns, then I think we can push. @ErickErickson, @dsmiley I know you've been concerned about merge policies in the past - I just want to draw your attention to this; maybe it addresses use cases you've seen? |
Do no push; please see the parent JIRA issue @msokolov |
@@ -484,6 +487,23 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) { | |||
return this; | |||
} | |||
|
|||
/** | |||
* Expert: sets the amount of time to wait for merges returned by MergePolicy.findCommitMerges(...). | |||
* If this time is reached, we proceed with the commit based on segments merged up to that point. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe also say that any merges still running after this time will still run to completion, so they are not wasted.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
@@ -480,6 +488,22 @@ public String getSoftDeletesField() { | |||
return softDeletesField; | |||
} | |||
|
|||
/** | |||
* Expert: return the amount of time to wait for merges returned by by MergePolicy.findCommitMerges(...). | |||
* If this time is reached, those merges will be aborted and we will wait again. If this time limit is reached again, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is no longer true? We will just wrap up the commit w/ those merges that did finish, and let the still running merges finish to completion?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed!
/** | ||
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit. | ||
* | ||
* Implementers of this method should use isMergingSegment to exclude any already-merging segments from the returned |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is isMergingSegment
here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oops... that's from an earlier implementation, before I realized that MergeContext
provides information on merging segments.
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit. | ||
* | ||
* Implementers of this method should use isMergingSegment to exclude any already-merging segments from the returned | ||
* {@link MergeSpecification}. If a segment already registered in a merge is returned, then the commit will fail. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this true? Or will IW (silently) disregard that requested merge?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The code in this PR throws an exception if IW fails to register the merge:
if (registerMerge(trackedMerge) == false) {
throw new IllegalStateException("MergePolicy " + config.getMergePolicy().getClass() +
" returned merging segments from findCommitMerges");
}
I suppose we could silently disregard the merge if we remember to count down the latch for the merge that fails to register.
What would be less surprising to users? Explosive failure if an invalid MergeSpecification
is returned or letting the commit happen without trying to merge? I lean slightly toward explosive failure (with what I hope is a clear explanation), but I'm happy to write it either way.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK I agree to explosive failure! Less trappy.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this is really close, thanks @msfroh!
Once we are done here, could you open a follow-on issue to explore selecting merges on commit by default in TieredMergePolicy
?
/** | ||
* Callback interface to signal various actions taken by IndexWriter. | ||
*/ | ||
public interface IndexWriterEvents { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you add @lucene.experimental
here? That reserves the right to make breaking changes even on feature releases ...
/** | ||
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit. | ||
* | ||
* If a returned {@link OneMerge} includes a segment already included in a registered merge, then the commit will fail. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you say what exception will be thrown in that case (or add an @throws
, below)?
* Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge. | ||
* | ||
* @param segmentInfos the total set of segments in the index (while preparing the commit) | ||
* @param mergeContext the IndexWriter to find the merges on |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, maybe improve this (it's not an IndexWriter
) to say the merge policy may use this to find already merging segments?
@@ -526,6 +526,19 @@ public abstract MergeSpecification findForcedMerges( | |||
public abstract MergeSpecification findForcedDeletesMerges( | |||
SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException; | |||
|
|||
/** | |||
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you describe a bit more? E.g. explain that any merges returned here which then finish within the allotted time will be reflected in the commit, so that the number of segments in the commit point may be decreased?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
While updating the comments here, it occurred to me that if we want to leverage this same method in future to trigger a merge on NRT refresh, would it make sense to call it e.g. findFlushMerges
or findFullFlushMerges
instead?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good thinking! +1 for findFullFlushMerges
, to make it clear it's a FULL flush (commit
or refresh
) and not an ordinary flush e.g. when the in-memory segments are taking too much memory and some are flushed.
And then I guess the method could look at MergeTrigger
to differentiate whether it's a commit
or a refresh
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Renamed in latest commit and added a new MergeTrigger.
Passing the MergeTrigger
to the MergeScheduler
got me thinking that it might be possible to raise the priority of these full-flush merges, though it would be easier to manage that in the call to registerMerge
, which could insert them at the front of the pending merge queue. I think that's beyond the scope of this change, but it could make for an interesting follow-up.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The latest javadoc improvements look great, thanks @msfroh!
1. Renamed findCommitMerges -> findFullFlushMerges. 2. Added MergeTrigger.COMMIT, passed to findFullFlushMerges and to MergeScheduler when merging on commit.
* LUCENE-8962: Add ability to selectively merge on commit This adds a new "findCommitMerges" method to MergePolicy, which can specify merges to be executed before the IndexWriter.prepareCommitInternal method returns. If we have many index writer threads, they will flush their DWPT buffers on commit, resulting in many small segments, which can be merged before the commit returns. * Add missing Javadoc * Fix incorrect comment * Refactoring and fix intermittent test failure 1. Made some changes to the callback to update toCommit, leveraging SegmentInfos.applyMergeChanges. 2. I realized that we'll never end up with 0 registered merges, because we throw an exception if we fail to register a merge. 3. Moved the IndexWriterEvents.beginMergeOnCommit notification to before we call MergeScheduler.merge, since we may not be merging on another thread. 4. There was an intermittent test failure due to randomness in the time it takes for merges to complete. Before doing the final commit, we wait for pending merges to finish. We may still end up abandoning the final merge, but we can detect that and assert that either the merge was abandoned (and we have > 1 segment) or we did merge down to 1 segment. * Fix typo * Fix/improve comments based on PR feedback * More comment improvements from PR feedback * Rename method and add new MergeTrigger 1. Renamed findCommitMerges -> findFullFlushMerges. 2. Added MergeTrigger.COMMIT, passed to findFullFlushMerges and to MergeScheduler when merging on commit. * Update renamed method name in strings and comments
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was investigating some test failures in Elasticsearch after we upgraded to the 8.5 snapshot. I suspect they are caused by this PR. I think there are several concurrency issues in this PR.
@mikemccand @msokolov @msfroh Can you please take a look? Thank you.
I missed the fact that |
* Signals the start of waiting for a merge on commit, returned from | ||
* {@link MergePolicy#findFullFlushMerges(MergeTrigger, SegmentInfos, MergePolicy.MergeContext)}. | ||
*/ | ||
void beginMergeOnCommit(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not really happy with this interface. First and foremost it's only partially used in this PR. I also think it doesn't belong here but rather into a merge policy? I think IW and merge lifecycle should not be tightly coupled. Can we achieve the same with an interface a MP can provide to the IW rather than setting it on the IW config. A pull model should be used here instead IMO.
This reverts commit 90aced5. Revert "LUCENE-8962: woops, remove leftover accidental copyright (darned IDEs)" This reverts commit 3dbfd10. Revert "LUCENE-8962: Fix intermittent test failures" This reverts commit a5475de. Revert "LUCENE-8962: Add ability to selectively merge on commit (#1155)" This reverts commit a1791e7.
This reverts commit 90aced5. Revert "LUCENE-8962: woops, remove leftover accidental copyright (darned IDEs)" This reverts commit 3dbfd10. Revert "LUCENE-8962: Fix intermittent test failures" This reverts commit a5475de. Revert "LUCENE-8962: Add ability to selectively merge on commit (#1155)" This reverts commit a1791e7.
removed unrelated and whitespace changes removed unrelated and whitespace changes removed unrelated and whitespace changes removed unrelated and whitespace changes removed unrelated and whitespace changes removed unrelated and whitespace changes removed unrelated and whitespace changes removed spurious import restructure CHANGES.txt: 8.5: re-categorize issues LUCENE-9259: Fix wrong NGramFilterFactory argument name for preserveOriginal option Revert "LUCENE-8962: Split test case (apache#1313)" This reverts commit 90aced5. Revert "LUCENE-8962: woops, remove leftover accidental copyright (darned IDEs)" This reverts commit 3dbfd10. Revert "LUCENE-8962: Fix intermittent test failures" This reverts commit a5475de. Revert "LUCENE-8962: Add ability to selectively merge on commit (apache#1155)" This reverts commit a1791e7. SOLR-14073: Fix segment look ahead NPE in CollapsingQParserPlugin LUCENE-9268: Add some random tests to IndexWriter Add some tests that perform a set of operations randomly and concurrently on IndexWriter. LUCENE-9254: UniformSplit supports FST off-heap. Closes apache#1301 LUCENE-9263: Fix wrong transformation of distance in meters to radians in Geo3DPoint (apache#1318) LUCENE-9259: Fix wrong NGramFilterFactory argument name for preserveOriginal option Consolidated process event logic after CRUD action (apache#1325) Today we have duplicated logic on how to convert a seqNo into a real seqNo and process events based on this. This change consolidated the logic into a single method. SOLR-14073: Update CHANGES.txt added test to cover this api, tested to make sure the chage does not affect any boundary value analysis removed spurious changes move entry in CHANGES.txt from 8.6 to 8.5 SOLR-14139: Update CHANGE.txt Remove unused scripts in dev-tools folder (apache#1326) LUCENE-9164: process all events before closing gracefully (apache#1319) IndexWriter must process all pending events before closing the writer during rollback to prevent AlreadyClosedExceptions from being thrown during event processing which can cause the writer to be closed with a tragic event. Remove some unused lines from addBackcompatIndexes.py related to svn (apache#1322) Add 8.6 section to solr CHANGES.txt (apache#1337) SOLR-14197: SolrResourceLoader refactorings to reduce API * Remove SRL.listConfigDir (unused) * Remove SRL.getDataDir * Remove SRL.getCoreName * Remove SRL.getCoreProperties XmlConfigFile needs to be passed in the substitutableProperties IndexSchema needs to be passed in the substitutableProperties Remove redundant Properties from CoreContainer constructors * Remove SRL.newAdminHandlerInstance (unused) * Remove SRL.openSchema and openConfig * Avoid SRL.getConfigDir Also harmonized similar initialization logic between DIH Tika processor & ExtractingRequestHandler. * Ensure SRL.addToClassLoader and reloadLuceneSPI are called at most once Don't auto-load "lib" in constructor; wrong place for this logic. * Avoid SRL.getInstancePath Added SolrCore.getInstancePath instead Use CoreContainer.getSolrHome instead NodeConfig should track solrHome separate from SolrResourceLoader * Simplify some SolrCore constructors * Move locateSolrHome to new SolrPaths * Move "User Files" stuff to SolrPaths
Description
If we have many index writer threads, they will flush their DWPT buffers
on commit, resulting in many small segments, which can be merged before
the commit returns.
Solution
This adds a new "findCommitMerges" method to MergePolicy, which can specify merges to be executed as part of preparing a commit. By default, we return null for backward compatibility (i.e. don't merge on commit).
In IndexWriter, we call findCommitMerges from prepareCommitInternal, after we have cloned the SegmentInfos. If we found commit merges, we wrap each OneMerge so that on completion, they update the cloned SegmentInfos and reference counts of segment files. Outside the flush lock, we wait (up to max time specified in IndexWriterConfig) for the commit merges to complete before calling startCommit.
Also, added an IndexWriterEvents callback (configurable through IndexWriterConfig) so that consuming code can be notified of merge on commit events (to emit metrics on time spent waiting for the merges to complete, for example).
Tests
Checklist
Please review the following and check all that apply:
master
branch.ant precommit
and the appropriate test suite.