-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Return MergeObserver from IndexWriter.forceMergeDeletes() (#14515) #15378
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
Return MergeObserver from IndexWriter.forceMergeDeletes() (#14515) #15378
Conversation
78dedde to
9c6683f
Compare
IndexWriter.forceMergeDeletes() now returns MergePolicy.MergeObserver instead of void, allowing applications to monitor merge progress and wait for completion. This enables coordination between merge completion and other application logic, and supports both synchronous (await) and asynchronous (CompletableFuture) waiting patterns. Key capabilities: - Query merge status: hasNewMerges(), numMerges() - Wait synchronously: await(), await(timeout, unit) - Wait asynchronously: awaitAsync() returns CompletableFuture<Void> - Inspect individual merges: getMerge(int) Changes: - Add MergePolicy.MergeObserver nested class - Update IndexWriter.forceMergeDeletes() methods to return MergeObserver - Update RandomIndexWriter to propagate return type - Add comprehensive tests (blocking/non-blocking modes, timeout handling) Backward compatible: existing code that ignores the return value continues to work without modification.
9c6683f to
53dbb7f
Compare
Measure actual merge time in testForceMergeDeletesWithObserver and testMergeObserverAwaitWithTimeout, failing if merges take longer than 30 seconds. This helps detect stuck merges or performance regressions while being generous enough for slow CI machines. Both tests maintain their upper timeouts (implicit for await() and 10 minutes for await(timeout, unit)) to prevent indefinite hangs, but now also fail fast at 30 seconds if something is wrong.
vigyasharma
left a comment
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 like this approach, it let's us observe merge progress without exposing mutable objects. Thanks for making these changes @salvatorecampagna
| * @return the merge at index i | ||
| * @throws IndexOutOfBoundsException if index is invalid or no merges exist | ||
| */ | ||
| public MergePolicy.OneMerge getMerge(int i) { |
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'm not sure if providing this method is really useful. The caller can't know the "index" location of a OneMerge unless they already have access to the MergeSpec.
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.
Also, I don't think that the OneMerge object is immutable?
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.
You're right on both points. My original intent was to provide detailed observability, but as you noted, the caller can't meaningfully use index-based access without already having the MergeSpec. And OneMerge is definitely mutable, so exposing it breaks encapsulation.
The umMerges()/numCompletedMerges() approach you suggested is much better, as it provides progress tracking without exposing internals.
| * | ||
| * @return number of merges, or 0 if no merges were scheduled | ||
| */ | ||
| public int numMerges() { |
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.
How about we also expose the no. of completed merges? You will need to run through all the OneMerge objects in merges and check for mergeCompleted. It could be useful to track overall merge progress.
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.
Excellent suggestion! This is a far better way to provide progress tracking without exposing mutable OneMerge objects. I'll add numCompletedMerges() that iterates through the merges and checks mergeCompleted.isDone().
Users can then track progress as: 100 * numCompletedMerges() / numMerges()
I'll also update toString() to show the completion count for better debugging.
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.
Later I decided not to include completed merges in toString for performance implications with a large number of merges.
| * | ||
| * @return {@code true} if merges were scheduled, {@code false} if no merges needed | ||
| */ | ||
| public boolean hasNewMerges() { |
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 naming can get confusing: is it still a "new" merge once the merge is underway, does it count completed ones, etc.? Instead of this API, can we just use numMerges() > 0 ? Or how about an isEmpty api?
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.
You're right, the naming is confusing. "New" is ambiguous: does it mean not yetstarted? Does it include in-progress or completed merges?
numMerges() > 0 is much clearer and eliminates the ambiguity.
I'll remove hasNewMerges() and update all test call sites to use numMerges() > 0 instead.
One less method to maintain and better clarity.
| dir.close(); | ||
| } | ||
|
|
||
| public void testMergeObserverGetMerges() throws IOException { |
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.
We can remove this test if we remove the getMerges api.
|
|
||
| assertTrue( | ||
| "Merge took too long: " + elapsedMillis + "ms (expected < 30000ms)", | ||
| elapsedMillis < 30_000); |
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 30_000 a random magic number for timeout upperbound? Why don't we just use it in the await timeout value? Then assertTrue will fail if we don't meet this limit.
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.
Correct. Calling await() without timeout then checking elapsed time separately doesn't make sense. Much cleaner to put the timeout directly in await():
assertTrue(observer.await(30_000, MILLISECONDS))
This mimics real usage where users will call await() with a reasonable timeout. The 30_000ms is just a test value ensuring await() doesn't hang indefinitely. Since we're exposing await() APIs, testing they actually work seems reasonable to me.
|
|
||
| MergePolicy.MergeObserver observer = iw.forceMergeDeletes(true); | ||
| assertTrue("Should have scheduled merges", observer.hasNewMerges()); | ||
| assertTrue("Should have completed merges", observer.numMerges() > 0); |
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.
You can add an assert here that tests that numMerges == completedMerges
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.
Sure! Since forceMergeDeletes(true) blocks until completion, all merges should be done when it returns. I'll add an assetion. That would verify the blocking behavior actually works.
lucene/CHANGES.txt
Outdated
|
|
||
| * GITHUB#15124: Use RamUsageEstimator to calculate size for non-accountable queries. (Sagar Upadhyaya) | ||
|
|
||
| * GITHUB#14515: IndexWriter.forceMergeDeletes() now returns MergePolicy.MergeObserver, |
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 change can go into 10.4. Let's move this entry to the "API Changes" section under 10.4?
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 see the PR is targeting main (milestone 11.0.0). According to CONTRIBUTING.md
You should open a pull request against the main branch. Committers will backport it to the maintenance branches once the change is merged into main.
Should I keep the CHANGES.txt entry in 11.0.0 for now, and then it gets moved to 10.4 in a backport PR if needed? Or should I change it to 10.4 now and update the milestone as well?
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.
We add the changes entry in the Lucene version we are targeting. All changes go to main branch regardless, but the ones for a specific target branch (like 10.4 in this case) additionally get backported to 10.x.
So you should just add the entry to 10.4 in this PR itself and remove it from 11.0. After I merge this into main, i'll backport the commit to 10.x branch. Will reach out to you if I need any help with backport.
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.
Quick question for future contributions: How do I know upfront which version a change is targeting?
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.
Generally default to the next minor version. If it's a breaking change or a major change in existing behavior, we defer it for a major version release.
| @Override | ||
| public void close() {} | ||
| } | ||
|
|
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.
Appreciate the thorough tests below, nice work!
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.
Thank you! I'm a fan of good tests, they're as important as the code itself, documenting behavior and preventing regressions.
This is my first Lucene contribution, so I really appreciate your detailed feedback on the API design. The suggestions made the API much cleaner and safer.
The getMerge(int) method exposed mutable OneMerge objects, breaking encapsulation. Callers also cannot meaningfully use an index-based API without already having the MergeSpecification. Refactored IndexWriter to work with MergeSpecification directly for merge registration and waiting. Removed associated tests.
The hasNewMerges() method is confusing - it's unclear whether it counts in-progress merges, completed merges, etc. Users can simply check numMerges() > 0 instead for better clarity. This change: - Removes the hasNewMerges() method from MergeObserver - Updates all test cases to use numMerges() > 0 instead - Removes redundant assertions where both hasNewMerges() and numMerges() were checked
This method allows tracking merge progress by checking how many merges have completed out of the total. Users can calculate progress as: int progress = (int)(100.0 * observer.numCompletedMerges() / observer.numMerges()); This is particularly useful when monitoring long-running merge operations such as forceMerge or forceMergeDeletes with non-blocking mode. Also updates toString() to display the completion count for better debugging visibility.
Instead of using magic numbers for timeout assertions, use the actual timeout value in the await() call. This makes the tests clearer and ensures the timeout is enforced properly. Also adds assertions to verify that all merges are completed after await() returns true, leveraging the new numCompletedMerges() API. Changes: - testForceMergeDeletesWithObserver: Use 30s timeout in await() call - testMergeObserverAwaitWithTimeout: Use 30s timeout in await() call - Both tests: Add assertion that numMerges == numCompletedMerges
The forceMergeDeletes() API change should be documented in the 10.4.0 release (the target version for this feature) under API Changes, not in the 11.0.0 Improvements section. Moved the entry from: - 11.0.0 → Improvements To: - 10.4.0 → API Changes
The toString() method was calling numCompletedMerges(), which iterates through all merges to count completed ones. This is problematic because toString() is frequently called by debuggers, logging frameworks, and IDE watches, making the O(n) iteration too expensive for a debugging method. Users who need the completion count can explicitly call the numCompletedMerges() method. The toString() should remain minimal and cheap for debugging purposes.
The previous refactoring in commit 05b8266 changed more than necessary in IndexWriter, including switching from indexed for-loops to enhanced for-loops and removing the newMergesFound variable. This commit restores the original code structure while still avoiding the removed getMerge() method. Changes: - Restore newMergesFound boolean variable as in original code - Use indexed for-loops as in original code - Access spec.merges directly instead of removed getMerge() - Only add the return statement needed for the new API This keeps the diff minimal and preserves the original code patterns, making the change easier to review and maintain.
Reformats assertTrue statements to single line as per project style.
Use String.format() with Locale.ROOT to avoid locale-dependent formatting. This is the standard pattern used throughout Lucene codebase.
| if (spec == null) { | ||
| return "MergeObserver: no merges"; | ||
| } | ||
| return String.format(Locale.ROOT, "MergeObserver: %d merges", numMerges()); |
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.
We could include spec.toString() here. It has some useful info about the segments in each 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.
According to a test I executed locally the outout should be something like. Are we sure we want this level of detail? I guess the output might be long if there are meny merges.
MergeObserver: 3 merges
MergeSpec:
1: _0(11.0.0):c5 _1(11.0.0):c5 into _k
2: _2(11.0.0):c5 _3(11.0.0):c5 into _l
3: _4(11.0.0):c5 _5(11.0.0):c5 into _m
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.
My concern is that spec.toString() was internal before, so we controlled where it's used. Now that MergeObserver is public API, users might call toString() frequently (logging, monitoring) without realizing it could be expensive with many merges.
Update MergeObserver.toString() to include the underlying MergeSpec details by appending spec.toString() to the merge count. This provides useful information about segments in each merge. Sample output with 3 merges: ``` MergeObserver: 3 merges MergeSpec: 1: _0(11.0.0):c5 _1(11.0.0):c5 into _k 2: _2(11.0.0):c5 _3(11.0.0):c5 into _l 3: _4(11.0.0):c5 _5(11.0.0):c5 into _m ```
Splits long String.format() call across multiple lines for readability as per project style guidelines.
vigyasharma
left a comment
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.
Looks great, thanks @salvatorecampagna . And congratulations on your first contribution!
Return MergeObserver from IndexWriter.forceMergeDeletes() IndexWriter.forceMergeDeletes() now returns MergePolicy.MergeObserver instead of void, allowing applications to monitor merge progress and wait for completion. Addresses (#14515).
Summary
Changes
IndexWriter.forceMergeDeletes()to returnMergePolicy.MergeObserverinstead of void, allowing applications to monitor merge progress and wait for completion without exposing mutable internal objects.Fixes #14515
Motivation
Currently when calling
forceMergeDeletes(false), there's no way to monitor whether merges were scheduled or wait for them to complete. This makes it difficult to coordinate merge completion with other operations or implement custom scheduling strategies.Changes
Added
MergePolicy.MergeObserveras a new public API with methods to:numMerges())numCompletedMerges())await(),await(timeout, unit))awaitAsync())The observer is thread-safe and handles cases where no merges are needed. It provides observability without exposing mutable
OneMergeobjects, maintaining proper encapsulation.Updated
IndexWriter.forceMergeDeletes()methods to return the observer instead of void. Also updatedRandomIndexWriterto propagate the new return type.Testing
Added comprehensive tests covering:
numCompletedMerges()All tests pass with randomized seeds across multiple iterations.
Backward Compatibility
Fully backward compatible. Existing code that ignores the return value continues to work unchanged. Changing void to a return type is not a breaking change in Java.
Files Modified
MergePolicy.java- New MergeObserver class with progress trackingIndexWriter.java- Updated return types and javadocRandomIndexWriter.java- Propagated return typeTestIndexWriterMerging.java- Comprehensive test coverageCHANGES.txt- Added entry under 10.4.0 API Changes