Skip to content
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

LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges. #633

Merged
merged 60 commits into from
Jul 6, 2022

Conversation

vigyasharma
Copy link
Contributor

@vigyasharma vigyasharma commented Feb 1, 2022

Description

The addIndexes(CodecReader...) API today merges all provided readers into a single merge, in one large blocking call. We want to add concurrency here by invoking multiple parallel merges on subsets of readers, in a way that is configurable by users. The merged segments created, can later be merged further in the regular, non-blocking, background merges triggered by Lucene. Currently, users are responsible for managing their API run times, by invoking it multiple times with subsets of readers.
JIRA - https://issues.apache.org/jira/browse/LUCENE-10216

Solution

In this change, we leverage the configured MergeScheduler to invoke all merges required by the addIndexes API. MergePolicy also exposes a findMerges(List<CodecReader> readers) API that users can override to configure how their readers should be clustered into merges. We add a default implementation that retains current behavior of adding all readers into a single merge.

Tests

  • Existing tests passing
  • Pending: New tests to be added

Checklist

Please review the following and check all that apply:

  • I have reviewed the guidelines for How to Contribute and my code conforms to the standards described there to the best of my ability.
  • I have created a Jira issue and added the issue ID to my pull request title.
  • I have given Lucene maintainers access to contribute to my PR branch. (optional but recommended)
  • I have developed this patch against the main branch.
  • I have run ./gradlew check.
  • I have added tests for my changes.

@vigyasharma
Copy link
Contributor Author

The addIndexes(CodecReaders[]) API is transactional in nature - either all readers get added or all of them fail. Retaining this behavior with concurrent background merges needs some work.

My plan is to defer adding the newly created merged segments to IndexWriter until all merges finish. MergeScheduler threads will do all the work for merge right upto creating segment files, and return. At addIndexes() API, we'll wait for all merges kicked off by the API to complete, and add them to IndexWriter.segmentInfos in a sync block only when all of them succeed. For partial failures, we'll delete the interim files created.

Added code for this change to get some feedback on this approach. Tests are failing right now and I'm working on fixing them and adding new ones. Have put nocommit comments meanwhile.

Copy link
Member

@mikemccand mikemccand left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I really love this approach, of letting MergePolicy guide exactly how merging of addIndexes(CodecReader[]) will happen.

What remains to bring this out of WIP? I see a few // nocommits still to fix ... can you post the failing tests details here maybe? This is certainly tricky code; thank you for tackling it!

@@ -352,6 +352,14 @@ public FieldDimensions(int dimensionCount, int indexDimensionCount, int dimensio
this.softDeletesFieldName = softDeletesFieldName;
}

public void verifyFieldInfo(FieldInfo fi) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this really be public? Or is it only for asserting, in which case maybe make it package private and rename to boolean assertFieldInfo or so?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method makes sure that incoming index fields are compatible with the destination index, e.g. vectors have the same dimensions and use the same similarity function.

Changed access to package private. I retained method name - verifyFieldInfo(), as it internally calls verifySoftDeletedFieldName and verifySameSchema, which have a similar naming style.

I can change it if assert*() is more in line with the Lucene convention. In which case, I also think that it should only return a condition and the asserts should be done by the caller.. Let me know..

@@ -39,6 +40,11 @@ public MergeSpecification findMerges(
return null;
}

@Override
public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
return null;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm should null even be allowed by this new API? Won't this throw NPE if someone tries to addIndexes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In my current impl, addIndexes simply completes without adding anything if the API returns null. I now realize, this is misleading for the user. Callers of addIndexes may not know about the configured merge policy. I should probably throw an exception if I get a null here?
On similar lines, what should we do if the API call had some CodecReaders, but findMerges returns an empty spec.? Should we error out? (Will add a log for this either way.)

Re: letting the API return null, I see that other variants of findMerges in this policy, all return null, and all callers of findMerges seem to be doing the null checks. Personally, I prefer disallowing the null return values, but I wanted to keep this in sync with the other variants.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are consumers of NoMergePolicy that expect addIndexes(CodecReaders...) to work, like IndexRearranger. My understanding is that NoMergePolicy is meant to turn off any merges on existing segments. Since this API uses merges to add new readers to the index, I've changed it here to return the default base class impl.

It's a little off because the name is NoMergePolicy, but I think(hope) it aligns with the intent behind that class. I've added some comments to call this out. Let me know if we should instead create a different MergePolicy that only allows fineMerges(readers).. (it would be exactly like NoMergePolicy except for one api).

@vigyasharma
Copy link
Contributor Author

Thanks for reviewing this @mikemccand ! I've addressed the ones that had minor changes, and am working on the larger ones.

What remains to bring this out of WIP?

There are 2 existing tests that are breaking:

- org.apache.lucene.index.TestAddIndexes.testAddIndicesWithSoftDeletes (:lucene:core)
- org.apache.lucene.index.TestIndexSorting.testAddIndexesWithDeletions (:lucene:core)

I've been looking at testAddIndicesWithSoftDeletes. IW does a rollbackInternal() on shutDown, where it is failing on this assert - assert pendingNumDocs.get() == segmentInfos.totalMaxDoc(). I'm guessing my code is reserving more docs than it should.. is that the right direction?
Here's a longer error stack trace -

java.lang.AssertionError: pendingNumDocs 7 != 5 totalMaxDoc
   at __randomizedtesting.SeedInfo.seed([63C0554849BFDEC5:4F94AA0CFCBAFADD]:0)
   at org.apache.lucene.index.IndexWriter.rollbackInternal(IndexWriter.java:2428)
   at org.apache.lucene.index.IndexWriter.shutdown(IndexWriter.java:1334)
   at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1362)
   at org.apache.lucene.util.IOUtils.close(IOUtils.java:85)
   at org.apache.lucene.util.IOUtils.close(IOUtils.java:72)
   at org.apache.lucene.index.TestAddIndexes.testAddIndicesWithSoftDeletes(TestAddIndexes.java:1521)

Outside of fixing existing tests, I want to add some new tests, especially around cases where one of the merges fails, causing the whole API to fail (i.e. test that transactionality was retained).

@vigyasharma vigyasharma force-pushed the vs-10216 branch 2 times, most recently from edab6df to 84cfe00 Compare March 15, 2022 18:01
@vigyasharma
Copy link
Contributor Author

New Changes:

  1. addIndexes(CodecReader...) merges are abortable now.
    • Earlier, addIndexes(CodecReader...) triggered merges directly via SegmentMerger. As a result, the running merges could be tracked in Set<SegmentMerger> runningAddIndexesMerges, but any pending merge operations could not be aborted preemptively.
    • With this change, merges are defined via MergePolicy.OneMerge objects, and scheduled by the MergeScheduler. These are aborted when IndexWriter is rolled back or closed.
    • removed corresponding TODO - “TODO: somehow we should fix this merge so it’s abortable so that IW.close(false) is able to stop it”
  2. Updated NoMergePolicy to return the default MergeSpec for findMerges(List<CodecReader> readers), as it is required by consumers of addIndexes that do use this policy to avoid background segment merges, like IndexRearranger.

Tests Added:

  1. Test addIndexes with a MergePolicy that creates one merge per CodecReader.
  2. Test transactionality when only a subset of triggered merges succeed.
  3. Test interim files are deleted when a partial subset of merges fails, causing the API to fail.
  4. Tests for null and empty MergeSpec returned from MergePolicy.
  5. Test no merges are triggered if empty readers provided.
  6. Test cascading merges get triggered after each addIndexes(CodecReader...) call.

Existing Tests Expanded:

  1. Added a test MergePolicy that creates one merge per CodecReader. This is used with multiple existing concurrency tests, that check for steady state calls, concurrent operations overlapping with addIndexes, IW closed (with and without wait) with a conc. running addIndexes, and IW rollback.
  2. Tested that pending merges get aborted when IW is rolled back.

@vigyasharma
Copy link
Contributor Author

I think this PR has reasonable test coverage and can come out of WIP now. Will continue to iterate on the PR comments and suggestions..

@vigyasharma vigyasharma marked this pull request as ready for review March 15, 2022 18:10
@vigyasharma vigyasharma changed the title [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges. LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges. Mar 15, 2022
@vigyasharma
Copy link
Contributor Author

./gradlew check had passed on my local machine. However, some tests have randomization and don't always invoke addIndexes(CodecReaders...), and have failed in the run above.

Is there a way to ensure all code paths in the random tests get executed? I want to run the tests that invoke addIndexes(CodecReader...) for some random flag value.

@vigyasharma
Copy link
Contributor Author

  1. Found a better way to handle addIndexes failures due to background merges -
    OneMerge objects store the error they run into, which can be pulled via OneMerge.getException(). We rethrow these errors if they turn out to be the cause for addIndexes failure. This fixes tests which expect certain exceptions, like random IO exceptions thrown by MockDirectoryWrapper in BasePointsFormatTestCase#testWithExceptions.

  2. Added a test for validate we API and destination writer state when we hit IndexWriter.actualMaxDocs limit.

  3. Fixed breaking tests from the last run.

@mikemccand
Copy link
Member

Is there a way to ensure all code paths in the random tests get executed? I want to run the tests that invoke addIndexes(CodecReader...) for some random flag value.

Alas, I don't think we have a way to ensure ALL paths get executed -- that would likely be combinatorically prohibitive? But sure would be nice to have some control over this.

But we do have gradle beast, which will run the same test case over and over with different random seeds and hopefully over time provoke any latent failures that one or two successful runs failed to uncover?

@vigyasharma
Copy link
Contributor Author

Thanks @mikemccand . I figured that I had changed the exception signature for addIndexes() in my last revision. I was throwing a blanket MergePolicyException when addIndexes failed to merge all the readers in; and a lot of these tests were designed to look for specific exceptions that would've earlier gotten thrown from the SegmentMerger code running in main addIndexes thread.. This was causing the randomly invoked tests to fail.

I changed that, to rethrow the exceptions seen by any failed background OneMerges, and that appeased the gradle gods and builds began to pass again.

I think this is better, because now the API throws the actual error seen by a thread doing the merge, instead of some blanket failure message. In general, I have to say, I'm fascinated by the breadth of test cases around this API, and the code framework present to add more tests.

@mikemccand
Copy link
Member

This method makes sure that incoming index fields are compatible with the destination index, e.g. vectors have the same dimensions and use the same similarity function.

Changed access to package private. I retained method name - verifyFieldInfo(), as it internally calls verifySoftDeletedFieldName and verifySameSchema, which have a similar naming style.

I can change it if assert*() is more in line with the Lucene convention. In which case, I also think that it should only return a condition and the asserts should be done by the caller.. Let me know..

OK let's leave it as verify now -- when in Rome!

Copy link
Member

@mikemccand mikemccand left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @vigyasharma -- this is a super exciting change! It generalizes addIndexes to use MergePolicy/Scheduler so that incoming indices can be added concurrently. I think it's ready, but needs rebasing to resolve conflicts?

@vigyasharma
Copy link
Contributor Author

Thanks for reviewing these rather lengthy changes, @mikemccand! I've rebased this on to the latest main branch, and updated doc string and exception message.

@vigyasharma
Copy link
Contributor Author

The build failure is on TestTaxonomyFacetAssociations.testFloatAssociationRandom, and I'm unable to repro it without the randomization seed.

The failure (with randomization seed) is because of a mismatch in (SUM) aggregated multi-valued, float_random facet field. We accept an error delta of 1 in this aggregation, but for the failing random seed, the delta is 1.3. For the failing random seed, the test also fails on the main branch.

I'm not sure if this is related to the PR's changes (I don't see any related APIs being invoked by the test). My hunch is that it is some floating point approximation error.
@gsmiller, i see you worked on it recently, what are your thoughts? Would it make sense increase the delta in assert to 1.5?

--

Failing Test

./gradlew test --tests TestTaxonomyFacetAssociations.testFloatAssociationRandom \ 
-Dtests.seed=4DFBA8209AC82EB2 -Dtests.slow=true -Dtests.locale=fr-VU \
-Dtests.timezone=Europe/Athens -Dtests.asserts=true -Dtests.file.encoding=UTF-8

Failure:

org.apache.lucene.facet.taxonomy.TestTaxonomyFacetAssociations > testFloatAssociationRandom FAILED
    java.lang.AssertionError: expected:<2409060.8> but was:<2409059.5>
        at __randomizedtesting.SeedInfo.seed([4DFBA8209AC82EB2:1EC25C9148A9BF04]:0)
        at junit@4.13.1/org.junit.Assert.fail(Assert.java:89)
        at junit@4.13.1/org.junit.Assert.failNotEquals(Assert.java:835)
        at junit@4.13.1/org.junit.Assert.assertEquals(Assert.java:577)
        at junit@4.13.1/org.junit.Assert.assertEquals(Assert.java:701)
        at org.apache.lucene.facet.taxonomy.TestTaxonomyFacetAssociations.validateFloats(TestTaxonomyFacetAssociations.java:445)
        at org.apache.lucene.facet.taxonomy.TestTaxonomyFacetAssociations.testFloatAssociationRandom(TestTaxonomyFacetAssociations.java:256)
        at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
       ...

Passing Test:

./gradlew test --tests TestTaxonomyFacetAssociations.testFloatAssociationRandom \
-Dtests.slow=true -Dtests.locale=fr-VU -Dtests.timezone=Europe/Athens \
-Dtests.asserts=true -Dtests.file.encoding=UTF-8

@mikemccand
Copy link
Member

Thanks for digging into that failure @vigyasharma.

For the failing random seed, the test also fails on the main branch.

OK, so this is pre-existing! Could you open a separate spinoff issue and post the failure details there? It should not block this awesome change.

@mikemccand
Copy link
Member

I restarted the gradle test job.

@@ -89,6 +89,9 @@ New Features
Improvements
---------------------

* LUCENE-10216: Use MergePolicy to define and MergeScheduler to trigger the reader merges
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for adding CHANGES entry, but we usually append to the end of the section not the start :)

Copy link
Member

@mikemccand mikemccand left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @vigyasharma -- I had a couple questions about some changes to existing unit tests.

I think it would be worth implementing the new MergePolicy method in either MockRandomMergePolicy or AlcoholicMergePolicy or maybe both to exercise concurrent addIndexes across any tests today or future tests that use addIndexes?

@@ -352,15 +352,15 @@ public void testAddIndexOnDiskFull() throws IOException {
done = true;
}

} catch (IllegalStateException | IOException e) {
} catch (IllegalStateException | IOException | MergePolicy.MergeException e) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm can you give an example exception when we now throw MergePolicy.MergeException inside the try?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the merge threads triggered for addIndexes readers fail, they throw a merge exception, which is silently recorded in that OneMerge object. After all threads join, we check if merge passed on all threads, and if not, we rethrow this MergeException, to surface the actual cause of API failure. This is done here. If all threads passed, we proceed to register the new segmentinfo.

testAddIndexOnDiskFull() simulates this by randomly filling up the disk and causing these merge threads to fail.

@vigyasharma
Copy link
Contributor Author

I think it would be worth implementing the new MergePolicy method in either MockRandomMergePolicy or AlcoholicMergePolicy or maybe both to exercise concurrent addIndexes across any tests today or future tests that use addIndexes?

Great idea @mikemccand, thanks for the input, and for reviewing this PR. I added the new merge policy to MockRandomMergePolicy, configured to trigger with 50% probability, and found some new failures in existing tests. I'm working on fixing them on my box, will update this PR soon (along with your other suggestions).

For AlcoholicMergePolicy, I think it is mostly about randomizing the size of participating segments, while the findMerges(CodecReader[]) that I'm currently proposing, doesn't really take sizes into account. I'll leave it as is for now.

@vigyasharma
Copy link
Contributor Author

vigyasharma commented Apr 29, 2022

Updated the MockRandomMergePolicy to randomly use the more concurrent version of findMerges(CodecReaders[]) with a 50% probability. This uncovered a failing test that was asserting directly on segments within a writer, and expected a single segment out of the API. Fixed broken test.

Kudos to Lucene's randomized testing for catching such edge cases!

@vigyasharma
Copy link
Contributor Author

I could either wrap the runningMerges update with a synchronized (IndexWriter.this) {}, or make runningMerges a synchronizedSet. I like the second approach as it automatically fixes this at all other places.

I decided to go with the first approach of wrapping critical sections with a synchronized (IndexWriter.this) {}. This feels simpler to reason about as we're synchronizing on a single object - IndexWriter.this.

A runningMerges synchronizedSet, would've created three different objects that were getting locked at different places - the IndexWriter object, the synchronizedSet lock, and the AddIndexesMergeSource, which has sync functions so that running and pending merge queues can be transactionally updated. Reasoning about, and avoiding deadlocks is simpler with a single IW lock. And given the nature of these critical sections, I don't think this affects concurrency by much.

Also rebased on the latest main.

@mikemccand
Copy link
Member

I could either wrap the runningMerges update with a synchronized (IndexWriter.this) {}, or make runningMerges a synchronizedSet. I like the second approach as it automatically fixes this at all other places.

I decided to go with the first approach of wrapping critical sections with a synchronized (IndexWriter.this) {}. This feels simpler to reason about as we're synchronizing on a single object - IndexWriter.this.

A runningMerges synchronizedSet, would've created three different objects that were getting locked at different places - the IndexWriter object, the synchronizedSet lock, and the AddIndexesMergeSource, which has sync functions so that running and pending merge queues can be transactionally updated. Reasoning about, and avoiding deadlocks is simpler with a single IW lock. And given the nature of these critical sections, I don't think this affects concurrency by much.

Also rebased on the latest main.

Thanks @vigyasharma -- I'll try re-beasting. This is on the nightly Lucene benchmarking box (128 cores).

@mikemccand
Copy link
Member

My beasting has not uncovered any more failures! I think this is ready! I'll try to merge soon, just to main ... let's let it bake there for a while before backporting to 9.x? Thanks @vigyasharma.

@mikemccand mikemccand merged commit 698f40a into apache:main Jul 6, 2022
vigyasharma added a commit to vigyasharma/lucene that referenced this pull request Jul 27, 2022
…decReader[]) merges. (apache#633)

* Use merge policy and merge scheduler to run addIndexes merges

* wrapped reader does not see deletes - debug

* Partially fixed tests in TestAddIndexes

* Use writer object to invoke addIndexes merge

* Use merge object info

* Add javadocs for new methods

* TestAddIndexes passing

* verify field info schemas upfront from incoming readers

* rename flag to track pooled readers

* Keep addIndexes API transactional

* Maintain transactionality - register segments with iw after all merges complete

* fix checkstyle

* PR comments

* Fix pendingDocs - numDocs mismatch bug

* Tests with 1-1 merges and partial merge failures

* variable renaming and better comments

* add test for partial merge failures. change tests to use 1-1 findmerges

* abort pending merges gracefully

* test null and empty merge specs

* test interim files are deleted

* test with empty readers

* test cascading merges triggered

* remove nocommits

* gradle check errors

* remove unused line

* remove printf

* spotless apply

* update TestIndexWriterOnDiskFull to accept mergeException from failing addIndexes calls

* return singleton reader mergespec in NoMergePolicy

* rethrow exceptions seen in merge threads on failure

* spotless apply

* update test to new exception type thrown

* spotlessApply

* test for maxDoc limit in IndexWriter

* spotlessApply

* Use DocValuesIterator instead of DocValuesFieldExistsQuery for counting soft deletes

* spotless apply

* change exception message for closed IW

* remove non-essential comments

* update api doc string

* doc string update

* spotless

* Changes file entry

* simplify findMerges API, add 1-1 merges to MockRandomMergePolicy

* update merge policies to new api

* remove unused imports

* spotless apply

* move changes entry to end of list

* fix testAddIndicesWithSoftDeletes

* test with 1-1 merge policy always enabled

* please spotcheck

* tidy

* test - never use 1-1 merge policy

* use 1-1 merge policy randomly

* Remove concurrent addIndexes findMerges from MockRandomMergePolicy

* Bug Fix: RuntimeException in addIndexes

Aborted pending merges were slipping through the merge exception check in
API, and getting caught later in the RuntimeException check.

* tidy

* Rebase on main. Move changes to 10.0

* Synchronize IW.AddIndexesMergeSource on outer class IW object

* tidy
vigyasharma added a commit that referenced this pull request Aug 1, 2022
…decReader[]) merges. (#1051)

Use merge policy and merge scheduler to run addIndexes merges.

This is a back port of the following commits from main:
 * LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges. (#633)
 * LUCENE-10648: Fix failures in TestAssertingPointsFormat.testWithExceptions (#1012)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants