-
Notifications
You must be signed in to change notification settings - Fork 1.1k
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-9662: CheckIndex should be concurrent - parallelizing index check across segments #128
LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments #128
Conversation
@@ -810,6 +976,8 @@ public Status checkIndex(List<String> onlySegments) throws IOException { | |||
result.newSegments.add(info.clone()); | |||
} | |||
|
|||
executor.shutdown(); |
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.
Add await; shutdown is non-blocking.
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 sorry. Thought it would be ok with the join
call before shutdown
, but yeah even the idle threads may take time to shutdown. Added.
segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast); | ||
CompletableFuture<Void> testPointvalues = | ||
CompletableFuture.supplyAsync( | ||
() -> { |
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 add a utility method that accepts a callable and wraps in completion exception? This is a very repetitive block.
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 also feel this is quite repetitive and cumbersome to look at. I gave it some more tries but it seems to be a bit difficult actually. The main issue here is the testXXX
methods all throw checked IOException
, and thus any callable I define that wrap around the method will need to handle the IOException there as well, instead of handling it inside the utility method, or the compiler may not be happy:
utilityMethod(() -> {
try {
testXXX
} catch (IOException) {
handle IOE
})
Maybe some refactoring in those testXXX
methods can work around the issue?
For the code in thenAccept
, we should be able to simplify with generics and creating a new CheckIndexStatus
that all status classes extend (right now they don't share a super class).
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 added a commit that shows what I meant (and replaces three calls). Sorry for not replacing all of them. Maybe it can be made even less verbose if you fold all those blocks into a single function that accepts executor, first callable, follow-up callable and just returns completable future from the composition of these.
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.
Ah ok I see. I think I encountered the issue earlier when I tried to use Supplier
directly instead of Callable
there. I've updated it according to your suggestion (I used a Function
for the follow-up lambda as it takes one argument)
@@ -605,6 +610,15 @@ public Status checkIndex(List<String> onlySegments) throws IOException { | |||
result.newSegments.clear(); | |||
result.maxSegmentName = -1; | |||
|
|||
// nocommit number of threads should be set dynamically |
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'd say: set dynamically based on CPU count, but provide an option (command line) to override?
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 the checkIndex() method just take ExecutorService as a parameter? This way, e.g. the calling main
method could configure the ExecutorService (possibly based on commandline options), unit tests can use a single thread, etc.
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.
Great suggestions! I've updated it to work for both checkIndex
invoked from main
, as well as invoked from other classes / methods.
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException { | |||
* quite a long time to run. | |||
*/ | |||
public Status checkIndex(List<String> onlySegments) 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.
please don't overload it here. force callers to pass executor.
I am still worried that tests are using multiple threads here which must not happen: on an 8 core machine we don't want to use 8 jvms * 8 threads each.
I am also concerned about newly-created synchronization issues here (e.g. with output). If checkindex fails it is really important that we can read this output.
All these changes leave my concerned that this is the right way to go. At the minimal, if we are going to make these changes to CheckIndex, then it needs to be done in a more cautious/defensive way (e.g. default threadcount to 1 instead of numcpus).
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.
please don't overload it here. force callers to pass executor.
I took a quick look at these methods' current usage. checkIndex()
has 4 references in tests and 1 in luke module, and checkIndex(onlySegments)
has 6 references in tests and 1 in CheckIndex
main, so it should be straightforward to remove these convenience methods and have all callers to invoke checkIndex(onlySegments, executorService)
. Having said that, it does seems to be a burden to put on each caller to handle executor creation and shutdown each time this method gets called though. Maybe we could keep these methods and use the 1 default threadcount executor like you suggested if one is not explicitly passed in (I also updated the default threadCount to be 1 in my latest commit) ?
I am still worried that tests are using multiple threads here which must not happen: on an 8 core machine we don't want to use 8 jvms * 8 threads each.
Oh I didn't realize it would spawn multiple jvm processes, or are you suggesting that if this gets run in multiple jvm processes for some reasons then the overall overhead will be high (which I agree!) ? Shall we do something like cpu / constant
to reduce it a bit, or any other mechanism we can use to better control the number of threads based on whole system's overhead?
I am also concerned about newly-created synchronization issues here (e.g. with output). If checkindex fails it is really important that we can read this output.
Yeah definitely agree with this. For the output messages synchronization issue, I put in some nocommit comment earlier and proposed to also include segment / part id in these messages as a cheap way to allow manual sorting / grep-ing after the messages were written, to essentially work around this issue. Not sure if this will be good enough?
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 post the CheckIndex
output on a real-ish (multi-segment) index as an example? Actually I will do that on the nightly benchmark (English Wikipedia) index!
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.
Oh I didn't realize it would spawn multiple jvm processes,
This is just how Lucene's test infrastructure runs tests -- it spawns multiple JVMs, each of which is running one Lucene test at a time, but that test may use (often uses?) its own threads, including here in CheckIndex
if we make it concurrent by default.
I am excited to see what happens to |
Thanks Michael for the comment! Just curious, do you have any suggestion to my nocommit questions in this PR with regard to how to parallelize some of the section of code that currently depends on ordering (message printing inside each segment / index part, error handling etc)? I could also give it a try first and see how folks feel about the solution. |
…gardless of its value, and the fail faster from current segment part / skip the rest segment parts benefit no long exists in concurrent setting
For now I've gone ahead and implemented the idea to add segment / part ids to logging messages so that they are still usable (with some log parsing) in concurrent setting. I also removed What do you think about these changes? @rmuir @dweiss @mikemccand |
try { | ||
executorService.awaitTermination(5, TimeUnit.SECONDS); | ||
} catch ( | ||
@SuppressWarnings("unused") |
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.
Why ignore? Seems like something went wrong there - log it at least?
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.
Yeah agreed. Updated.
// Test Livedocs | ||
segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast); | ||
CompletableFuture<Void> testliveDocs = | ||
runAysncSegmentPartCheck( |
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.
Typo in aysnc.
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. Fixed.
|
||
testFieldInfos.join(); | ||
if (segInfoStat.fieldInfoStatus.error != null) { | ||
throw new RuntimeException(segmentId + "Field Info test failed"); |
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 the cause should be added to those runtime exceptions.
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.
Ah good catch. Updated.
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.
[Disclaimer: @zacharymorn and I both work at Amazon, though in completely different teams!]
I love this change -- CheckIndex
is often run in the "heat of the moment" so time is of the essence to recovering your Lucene index and getting things back online. Yet it is very slow today, even on highly concurrent boxes.
I left some small comments that I think are fine to do in a followon PR. This change is already massive enough (GitHub was at first not willing to even render the CheckIndex
diffs!) and impactful enough that we can do the improvements after.
@@ -216,6 +225,9 @@ | |||
|
|||
/** Status of vectors */ | |||
public VectorValuesStatus vectorValuesStatus; | |||
|
|||
/** Status of soft deletes */ | |||
public SoftDeletsStatus softDeletesStatus; |
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.
Whoa, were we failing to CheckIndex
soft deletes before this?
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.
It was checked before, but was done in a way that's different (not using status class for example) from the rest
lucene/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
Lines 786 to 789 in 65820e5
final String softDeletesField = reader.getFieldInfos().getSoftDeletesField(); | |
if (softDeletesField != null) { | |
checkSoftDeletes(softDeletesField, info, reader, infoStream, failFast); | |
} |
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.
Ahh, that's great! Thanks.
SoftDeletsStatus() {} | ||
|
||
/** Exception thrown during soft deletes test (null on success) */ | ||
public Throwable error = null; |
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 we don't need the = null
-- it is already java's default.
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.
Removed, as well as the same ones used in other status classes.
@@ -468,6 +495,10 @@ private static void msg(PrintStream out, String msg) { | |||
if (out != null) out.println(msg); | |||
} | |||
|
|||
private static void msg(PrintStream out, String id, String msg) { | |||
if (out != null) out.println(id + " " + msg); |
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 you break this into separate lines and add { ... }
?
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.
final Status.LiveDocStatus status = new Status.LiveDocStatus(); | ||
|
||
try { | ||
if (infoStream != null) infoStream.print(" test: check live docs....."); | ||
if (infoStream != null) infoStream.print(segmentPartId + " test: check live docs....."); |
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.
Sorry about not answering the // nocommit
question before.
Ideally, all infoStream.print
for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output? (So that we don't see confusing interleaved across segments/parts checks)?
But I think it is OK to make this (cosmetic) improvement as a followon PR ... this change is already awesome enough.
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.
Sorry about not answering the // nocommit question before.
No problem, and thanks again for the review and feedback!
Ideally, all infoStream.print for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output? (So that we don't see confusing interleaved across segments/parts checks)?
Oh I see, haven't thought about this approach before, and it sounds interesting! I assume by "per-part log" you meant an array of in-memory, per part buffers that accumulate messages over concurrent check right? If we were to combine these buffers at the end of / after the concurrent index check, we should be ok to just print them out to main InfoStream without locking?
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.
Ideally, all infoStream.print for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output? (So that we don't see confusing interleaved across segments/parts checks)?
Oh I see, haven't thought about this approach before, and it sounds interesting! I assume by "per-part log" you meant an array of in-memory, per part buffers that accumulate messages over concurrent check right? If we were to combine these buffers at the end of / after the concurrent index check, we should be ok to just print them out to main InfoStream without locking?
Yes, exactly! So we won't see the logged output coming out in real-time as the checks happen, like CheckIndex
does today, but rather all things are running concurrently, and then, once you've joined all those concurrent checker threads back to main thread, the main thread prints all per-part output messages to the console. So then the user would still see the same (coherent looking) output, just with some delay since we wait for all concurrent checks to finish.
Or, alternatively, once any concurrent check finishes, you immediately acquire "console printing lock", and print its full output. This is a bit better because you see the output as each part finishes, and the long-pole slow checker parts won't delay the output of the fast parts. Less nail-biting for the user ...
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've implemented it here 57f542f . The per part messages should be printed as soon as each concurrent check finishes, and without locking since the shared PrintStream
object already locks internally.
final int numDocs = reader.numDocs(); | ||
if (reader.hasDeletions()) { | ||
Bits liveDocs = reader.getLiveDocs(); | ||
if (liveDocs == null) { | ||
throw new RuntimeException("segment should have deletions, but liveDocs is null"); | ||
throw new RuntimeException( |
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.
Should we maybe make a new RuntimeException
subclass, that takes this segmentPartId
as its own String
parameter, and the exception message, and maybe a Throwable cause
?
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.
* | ||
* @lucene.experimental | ||
*/ | ||
public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream) |
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, did this just move elsewhere? This is a helpful API to test just postings ...
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 I accidentally removed it...I've restored it as well as another one.
long startNS = System.nanoTime(); | ||
String segmentPartId = segmentId + "[StoredFields]"; |
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 maybe the custom RuntimeException
subclass could take CheckIndexFailure(String segmentId, String indexPart, String message, Throwable rootCause) {...}
?
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 (I used CheckIndexException
instead of CheckIndexFailure
for naming consistency). I also replaced all RuntimeException
in CheckIndex
with this new exception class.
@@ -2795,12 +2972,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { | |||
* @lucene.experimental | |||
*/ | |||
public static Status.DocValuesStatus testDocValues( | |||
CodecReader reader, PrintStream infoStream, boolean failFast) 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.
This is net/net an API break, but given that CheckIndex
is internal/experimental, I think it is OK (to backport to Lucene 8.x too)?
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 also think it should be ok to backport. The only thing I would like to mention is that, in addition to the API change, the more subtle change is that these methods would now no longer throw unchecked RuntimeException when the check find index integrity error, and failFast
set to true
. For any application that (should not have) relied on this behavior, it may now find the check would continue processing instead of aborting with exception when the check finds an error.
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, but on finding a problem in the index, with this change, and with failFast=true
, CheckIndex
will still throw an exception, right? It may just take a bit longer since it waits for all concurrent checking to finish for that one segment?
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.
Yes. If it's invoked via CheckIndex#checkIndex
, and failFast = true
, error from segment part check will be stored inside each Status
object, and then after join
CheckIndexException
will be thrown via these code
if (segInfoStat.liveDocStatus.error != null) {
throw new CheckIndexException(segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
}
which will be caught in the outside catch
block and rethrown via
if (failFast) {
throw IOUtils.rethrowAlways(t);
}
Thanks Michael for the review and feedback. I think as far as the original scope of the jira ticket goes, there's also the parallelization across segments that has not been implemented yet. But agree that this PR is already big and should already provide a good speed up on powerful concurrent boxes (up to 11 concurrent checks for each segment), so we can probably let it run for a while and see if parallelization across segments is still needed, which from my quick in-mind coding will definitely require much more changes for concurrency control to get it right. One thing I'm still researching is that, it seems there's limited direct test coverage for this |
Hi Zach. You can also "git add -A ." (stage your changes for commit); or just commit them in. Then there's no need for the fail-on-modified flag to be turned off. :) |
Ha yes I came to realize that also some time ago, but kinda formed the habit of passing it in by default now (mostly from past command search) so that I don't need to worry about where my changes are. But yeah that's good tip. Thanks Dawid! |
@@ -321,6 +321,7 @@ public static void syncConcurrentMerges(MergeScheduler ms) { | |||
checker.setDoSlowChecks(doSlowChecks); | |||
checker.setFailFast(failFast); | |||
checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false); | |||
checker.setThreadCount(RandomNumbers.randomIntBetween(new Random(), 1, 5)); |
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.
PREDICTABLE_RANDOM: This random generator (java.util.Random) is predictable (details)
(at-me in a reply with help
or ignore
)
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 should use randomized testing's context random, otherwise tests are not reproducible. Use this static method from RandomizedTest: https://github.com/randomizedtesting/randomizedtesting/blob/master/randomized-runner/src/main/java/com/carrotsearch/randomizedtesting/RandomizedTest.java#L161.
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.
@dweiss also opened a good follow-on issue to add new Random()
to forbidden APIs for test. Ahh, it was already there, but something else was wrong in how test-framework
invokes forbidden APIs. Anyway it looks like @dweiss is fixing this, thanks ;)
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.
Ah thanks @dweiss and @mikemccand for the context and solution! I used this one instead https://github.com/randomizedtesting/randomizedtesting/blob/8be8a09e3b3bf1a57947d5367a1afe68322ac4dc/randomized-runner/src/main/java/com/carrotsearch/randomizedtesting/RandomizedTest.java#L178 since the other one was marked as deprecated.
More tests are always welcome! But I don't think that should block this change -- we can open a follow-on issue to get better direct unit tests for |
Our nightly benchmarks also exercise |
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 looks great! Thank you for folding all the feedback in @zacharymorn!
I do think we need to ensure the output remains intelligible, due to the concurrency, and also the segment
/partId
change. I'll test the PR on a nightly benchmark index and post the output, and maybe we can iterate to make it look closer to how it looks with the single-threaded impl today?
@@ -305,7 +317,7 @@ private FieldNormStatus() {} | |||
public long totVectors = 0; | |||
|
|||
/** Exception thrown during term vector test (null on success) */ | |||
public Throwable error = null; |
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 for fixing all of these!
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException { | |||
* quite a long time to run. | |||
*/ | |||
public Status checkIndex(List<String> onlySegments) 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.
Maybe post the CheckIndex
output on a real-ish (multi-segment) index as an example? Actually I will do that on the nightly benchmark (English Wikipedia) index!
throw new RuntimeException( | ||
throw new CheckIndexException( | ||
segmentId, | ||
"", |
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 could maybe pass "live docs"
as the partId
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.
I've removed the use of segmentId
and partId
as part of 57f542f.
testTermVectors.join(); | ||
if (segInfoStat.termVectorStatus.error != null) { | ||
throw new CheckIndexException( | ||
segmentId, "", "Term Vector test failed", segInfoStat.termVectorStatus.error); |
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 instead of empty string for partId
param we can pass e.g. "field infos"
, "term vectors"`, etc.
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've removed the use of segmentId
and partId
as part of 57f542f.
segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error); | ||
} | ||
|
||
testFieldInfos.join(); |
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, here is where we could always e.g.System.out.print(segInfoState.fieldInfoMessages.toString())
or so?
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've handled it inside each of the future blocks in 57f542f, so that the per part messages can be printed out as soon as they are available, and don't need to wait for join
of other futures.
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.
Woot!
testVectors.join(); | ||
if (segInfoStat.vectorValuesStatus.error != null) { | ||
throw new CheckIndexException( | ||
segmentId, "", "Vectors test failed", segInfoStat.vectorValuesStatus.error); |
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 partId
can be "vectors"
.
throw IOUtils.rethrowAlways(e); | ||
} | ||
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]"); | ||
msg(infoStream, segmentId + partId, "ERROR [" + String.valueOf(e.getMessage()) + "]"); |
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 if partId
is a simple string like "term vectors"
but then sometimes empty string, maybe we need to make a helper method to carefully join them? E.g. if partId
is null
or ""
then we leave it off, but if it is not then we maybe print something like _7 [term vectors]: ...
?
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've removed the use of segmentId
and partId
as part of 57f542f.
OK I ran
I think the issue is that |
Thank you for all the awesome iterations here @zacharymorn! To get the best speedup, even at But we can add such complexity later -- the PR now ("thread per segment") is surely a great step forward too :) And +1 to spinoff a separate issue to change |
You're right -- it is able to do that. It's random, so if it flips a "really important" bit, such that the |
Hmm, could we instead just output the full segment's output as each finishes? So we would presumably see small segments finish first, then all checks for that small segment are printed? |
This fine-grained concurrency (one task for "part X segment") would then mean we can get the postings check of the big segments all going, early, concurrently across all segments, and likely the biggest segments postings check would be the long pole, and {{CheckIndex}} would finish in (about) that much time. Versus "thread per segment" concurrency, where the long pole will be one thread checking the biggest segment. The really fun part of concurrency across and within segments will be figure out the rough heuristic of how to assign estimated cost to each task :) But this all can come later! "thread per segment" is already a big step forward! |
Also, note that the "first segment" might not necessarily always be the largest! The way our default merge policy works, that tends to be the case, but users can customize the merge policy, and we recently added a feature to let |
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.
Thanks @zacharymorn -- I left a few comments!
if (diagnostics.size() > 0) { | ||
msg(infoStream, " diagnostics = " + diagnostics); | ||
// checks segments sequentially | ||
if (executorService == null) { |
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 we should make a single threaded executor so we don't have to bifurcate the code?
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 actually done to avoid deadlock during test (I put a comment in MockDirectoryWrapper#close
on passing a flag to have null executorService
, but it might not be immediately obvious here).
Essentially, the deadlock can be formed in test as such even with a single threaded executor:
- At the end of the tests that use directory,
MockDirectoryWrapper#close
was called, which would hold directory's monitor as the method issynchronized
MockDirectoryWrapper#close
would callTestUtil#checkIndex
and passed in itself for directory reference as 1st argument- With concurrent execution across segments in
TestUtil#checkIndex
, another thread checking segment would be making call back to methods from directory, such asMockDirectoryWrapper#fileLength
, which again require directory's monitor access as they are alsosynchronized
- Deadlock occurred as another thread is waiting for directory's monitor, which is held by the main thread waiting for the other thread to complete
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 that is really sneaky deadlock! Maybe MDW#close
should not hold its monitor lock when it calls CheckIndex
, but let's not try to solve that here. We can refactor in the future. This change is already a good step forwards -- progress not perfection!
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.
Sounds good! I've created a new issue for following up on this https://issues.apache.org/jira/browse/LUCENE-10071
throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc()); | ||
} | ||
|
||
int toLoseDocCount = info.info.maxDoc(); |
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, sometimes this seems to be maxDoc
(here) and other times numDocs
(accounting for deleted documents properly) -- let's try to be consistent with what it was before (I think numDocs
)?
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 part was actually copied / extracted into method from existing logic:
lucene/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
Lines 637 to 699 in c46bcf7
int toLoseDocCount = info.info.maxDoc(); | |
SegmentReader reader = null; | |
try { | |
msg(infoStream, " version=" + (version == null ? "3.0" : version)); | |
msg(infoStream, " id=" + StringHelper.idToString(info.info.getId())); | |
final Codec codec = info.info.getCodec(); | |
msg(infoStream, " codec=" + codec); | |
segInfoStat.codec = codec; | |
msg(infoStream, " compound=" + info.info.getUseCompoundFile()); | |
segInfoStat.compound = info.info.getUseCompoundFile(); | |
msg(infoStream, " numFiles=" + info.files().size()); | |
Sort indexSort = info.info.getIndexSort(); | |
if (indexSort != null) { | |
msg(infoStream, " sort=" + indexSort); | |
} | |
segInfoStat.numFiles = info.files().size(); | |
segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.); | |
msg(infoStream, " size (MB)=" + nf.format(segInfoStat.sizeMB)); | |
Map<String, String> diagnostics = info.info.getDiagnostics(); | |
segInfoStat.diagnostics = diagnostics; | |
if (diagnostics.size() > 0) { | |
msg(infoStream, " diagnostics = " + diagnostics); | |
} | |
if (!info.hasDeletions()) { | |
msg(infoStream, " no deletions"); | |
segInfoStat.hasDeletions = false; | |
} else { | |
msg(infoStream, " has deletions [delGen=" + info.getDelGen() + "]"); | |
segInfoStat.hasDeletions = true; | |
segInfoStat.deletionsGen = info.getDelGen(); | |
} | |
long startOpenReaderNS = System.nanoTime(); | |
if (infoStream != null) infoStream.print(" test: open reader........."); | |
reader = new SegmentReader(info, sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT); | |
msg( | |
infoStream, | |
String.format( | |
Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startOpenReaderNS))); | |
segInfoStat.openReaderPassed = true; | |
long startIntegrityNS = System.nanoTime(); | |
if (infoStream != null) infoStream.print(" test: check integrity....."); | |
reader.checkIntegrity(); | |
msg( | |
infoStream, | |
String.format( | |
Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startIntegrityNS))); | |
if (reader.maxDoc() != info.info.maxDoc()) { | |
throw new RuntimeException( | |
"SegmentReader.maxDoc() " | |
+ reader.maxDoc() | |
+ " != SegmentInfo.maxDoc " | |
+ info.info.maxDoc()); | |
} | |
final int numDocs = reader.numDocs(); | |
toLoseDocCount = numDocs; |
but I can update it as well to make it consistent.
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 took a closer look at this code. I think the reason of inconsistency here is that, between toLoseDocCount = info.info.maxDoc()
and toLoseDocCount = reader.numDocs()
, there are a few places where exception can be thrown (e.g. reader = new SegmentReader
may throw IOException), but we still want to catch the exception and add the value into result via segInfoStat.toLoseDocCount = toLoseDocCount
(using maxDoc count as the upper bound). So I guess this inconsistency is intended?
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.
Hi @mikemccand, just want to check back on this and see if you have any further concern?
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.
Oh hello, sorry this nearly fell past the event horizon of my TODO list!
Since these inconsistencies of the "to lost doc count" are pre-existing, let's not fret about it here/now. This change is already big/exciting enough!
} | ||
|
||
final int numDocs = reader.numDocs(); | ||
toLoseDocCount = numDocs; |
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.
Here is it numDocs
(taking deletions into account).
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.
Same as above #128 (comment), this might be intended behavior?
@@ -181,6 +193,9 @@ | |||
/** True if we were able to open a CodecReader on this segment. */ | |||
public boolean openReaderPassed; | |||
|
|||
/** doc count in this segment */ |
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.
Once we decide whether this is maxDoc
or docCount
(taking deletions into account) let's update this javadoc?
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.
Same as above #128 (comment), this might be intended behavior?
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.
Yeah let's just keep it the (inconsistent) way it was before for now.
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.
Sounds good.
} | ||
|
||
// capped threadCount at 4 | ||
private int threadCount = Math.min(Runtime.getRuntime().availableProcessors(), 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.
Whoa, why 4
:) Could we maybe use java's Runtime.getRuntime().availableProcessorts()
?
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.
Ah I thought we would like to cap it to 4 (or 8) per earlier comment. But I think in latest changes I actually removed away the usage of this default, as when the user does not pass in -threadCount
, the execution will use 0 for threadCount
and fallback to sequential check. Should we give user this flexibility to not use concurrent check?
// printed in order | ||
stream = new PrintStream(output, true, IOUtils.UTF_8); | ||
} else { | ||
// optimize for first segment to print real-time |
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 it would be better to 1) buffer all segment's output, and 2) print each segment's full output once it is done. This way the tiny segments which finish quickly would produce their output, and the large segments would be the long poles, finally finishing and printing theirs.
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, could we instead just output the full segment's output as each finishes? So we would presumably see small segments finish first, then all checks for that small segment are printed?
Also, note that the "first segment" might not necessarily always be the largest! The way our default merge policy works, that tends to be the case, but users can customize the merge policy, and we recently added a feature to let IndexWriter customize how it sorts the segments. We should maybe sort the segments by descending docCount and work on them (concurrently) in that order?
Hmm ok, and good to know about the custom segment sorting feature! The current implementation actually ensures the output to follow the same segment order as if it were executed sequentially, hence the little trick there to print first segment output early. But yeah I guess this order requirement is not really needed, especially under concurrent setting. Let me update that to give better output (and maybe even fail faster if smaller segment encounters error early!).
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've implemented the above by sorting segments by the file size they contain in increasing order. Here are the test results:
Full check on good index
5:12:03 PM: Executing task 'CheckIndex.main()'...
> Task :buildSrc:compileJava UP-TO-DATE
> Task :buildSrc:compileGroovy NO-SOURCE
> Task :buildSrc:processResources NO-SOURCE
> Task :buildSrc:classes UP-TO-DATE
> Task :buildSrc:jar UP-TO-DATE
> Task :buildSrc:assemble UP-TO-DATE
> Task :buildSrc:compileTestJava NO-SOURCE
> Task :buildSrc:compileTestGroovy NO-SOURCE
> Task :buildSrc:processTestResources NO-SOURCE
> Task :buildSrc:testClasses UP-TO-DATE
> Task :buildSrc:test NO-SOURCE
> Task :buildSrc:check UP-TO-DATE
> Task :buildSrc:build UP-TO-DATE
> Configure project :
IntelliJ Idea IDE detected.
> Task :errorProneSkipped
WARNING: errorprone disabled (skipped on non-nightly runs)
> Task :lucene:core:processResources UP-TO-DATE
> Task :lucene:core:compileJava
> Task :lucene:core:classes
> Task :lucene:core:CheckIndex.main()
NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/wikibigall.lucene_baseline.facets.taxonomy:Date.taxonomy:Month.taxonomy:DayOfYear.sortedset:Month.sortedset:DayOfYear.Lucene90.Lucene90.nd6.64758M/index
Checking index with async threadCount: 12
0.00% total deletions; 6647577 documents; 0 deletions
Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
1 of 15: name=_h2 maxDoc=11248
version=9.0.0
id=59c6he3dhebad46x7proh30nm
codec=Lucene90
compound=false
numFiles=17
size (MB)=10.617
diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102791291, os=Mac OS X, java.vendor=AdoptOpenJDK}
no deletions
test: open reader.........OK [took 0.167 sec]
test: check integrity.....OK [took 0.097 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.007 sec]
test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.771 sec]
test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.057 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.111 sec]
test: points..............OK [2 fields, 22496 points] [took 0.016 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
2 of 15: name=_h1 maxDoc=11979
version=9.0.0
id=59c6he3dhebad46x7proh30nj
codec=Lucene90
compound=false
numFiles=17
size (MB)=12.824
diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102788648, os=Mac OS X, java.vendor=AdoptOpenJDK}
no deletions
test: open reader.........OK [took 0.166 sec]
test: check integrity.....OK [took 0.108 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.004 sec]
test: terms, freq, prox...OK [290488 terms; 1843478 terms/docs pairs; 4383419 tokens] [took 0.880 sec]
test: stored fields.......OK [35937 total field count; avg 3.0 fields per doc] [took 0.021 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.083 sec]
test: points..............OK [2 fields, 23958 points] [took 0.005 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
...
...
14 of 15: name=_65 maxDoc=1197893
version=9.0.0
id=59c6he3dhebad46x7proh2zqv
codec=Lucene90
compound=false
numFiles=17
size (MB)=1,539.981
diagnostics = {os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622100810971, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1}
no deletions
test: open reader.........OK [took 0.018 sec]
test: check integrity.....OK [took 14.172 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.045 sec]
test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 74.763 sec]
test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.910 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.224 sec]
test: points..............OK [2 fields, 2395786 points] [took 0.183 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
15 of 15: name=_32 maxDoc=1197893
version=9.0.0
id=59c6he3dhebad46x7proh2zhm
codec=Lucene90
compound=false
numFiles=17
size (MB)=2,531.843
diagnostics = {os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622100146526, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1}
no deletions
test: open reader.........OK [took 0.020 sec]
test: check integrity.....OK [took 21.075 sec]
test: check live docs.....OK [took 0.014 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.041 sec]
test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 111.604 sec]
test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 1.112 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.648 sec]
test: points..............OK [2 fields, 2395786 points] [took 0.207 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
No problems were detected with this index.
Took 138.332 sec total.
BUILD SUCCESSFUL in 2m 22s
4 actionable tasks: 3 executed, 1 up-to-date
5:14:26 PM: Task execution finished 'CheckIndex.main()'.
Full check on bad index
> Task :lucene:core:CheckIndex.main()
NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
Checking index with async threadCount: 12
0.00% total deletions; 6647577 documents; 0 deletions
Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
1 of 15: name=_h2 maxDoc=11248
version=9.0.0
id=59c6he3dhebad46x7proh30nm
codec=Lucene90
compound=false
numFiles=17
size (MB)=10.617
diagnostics = {os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102791291, os=Mac OS X, java.vendor=AdoptOpenJDK, os.version=10.15.5, java.runtime.version=11.0.9+11}
no deletions
test: open reader.........OK [took 0.101 sec]
test: check integrity.....OK [took 0.047 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.005 sec]
test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.863 sec]
test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.059 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.133 sec]
test: points..............OK [2 fields, 22496 points] [took 0.015 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
...
...
6 of 15: name=_gb maxDoc=119789
version=9.0.0
id=59c6he3dhebad46x7proh30ld
codec=Lucene90
compound=false
numFiles=17
size (MB)=125.605
diagnostics = {os=Mac OS X, timestamp=1622102690942, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
no deletions
test: open reader.........OK [took 0.101 sec]
test: check integrity.....OK [took 0.678 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.007 sec]
test: terms, freq, prox...OK [1773712 terms; 20129621 terms/docs pairs; 51648295 tokens] [took 5.681 sec]
test: stored fields.......OK [359367 total field count; avg 3.0 fields per doc] [took 0.168 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.856 sec]
test: points..............OK [2 fields, 239578 points] [took 0.037 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
7 of 15: name=_gx maxDoc=119789
version=9.0.0
id=59c6he3dhebad46x7proh30n7
codec=Lucene90
compound=false
numFiles=17
size (MB)=129.046
diagnostics = {os=Mac OS X, timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
no deletions
test: open reader.........OK [took 0.101 sec]
test: check integrity.....FAILED
WARNING: exorciseIndex() would remove reference to this segment; full exception:
org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:925)
at org.apache.lucene.index.CheckIndex.lambda$checkIndex$1(CheckIndex.java:756)
at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$2(CheckIndex.java:854)
at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:834)
14 of 15: name=_65 maxDoc=1197893
version=9.0.0
id=59c6he3dhebad46x7proh2zqv
codec=Lucene90
compound=false
numFiles=17
size (MB)=1,539.981
diagnostics = {os=Mac OS X, timestamp=1622100810971, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
no deletions
test: open reader.........OK [took 0.011 sec]
test: check integrity.....OK [took 11.858 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.044 sec]
test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 83.292 sec]
test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.903 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.599 sec]
test: points..............OK [2 fields, 2395786 points] [took 0.210 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
15 of 15: name=_32 maxDoc=1197893
version=9.0.0
id=59c6he3dhebad46x7proh2zhm
codec=Lucene90
compound=false
numFiles=17
size (MB)=2,531.843
diagnostics = {os=Mac OS X, timestamp=1622100146526, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
no deletions
test: open reader.........OK [took 0.016 sec]
test: check integrity.....OK [took 19.048 sec]
test: check live docs.....OK [took 0.000 sec]
test: field infos.........OK [17 fields] [took 0.000 sec]
test: field norms.........OK [2 fields] [took 0.047 sec]
test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 118.554 sec]
test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 1.099 sec]
test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.914 sec]
test: points..............OK [2 fields, 2395786 points] [took 0.214 sec]
test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
WARNING: 1 broken segments (containing 119789 documents) detected
Took 143.432 sec total.
WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
> Task :lucene:core:CheckIndex.main() FAILED
Execution failed for task ':lucene:core:CheckIndex.main()'.
> Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
Check with -segment _gx
flag
> Task :lucene:core:CheckIndex.main() FAILED
NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
Checking index with async threadCount: 12
0.00% total deletions; 6647577 documents; 0 deletions
Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
Checking only these segments: _gx:
7 of 15: name=_gx maxDoc=119789
version=9.0.0
id=59c6he3dhebad46x7proh30n7
codec=Lucene90
compound=false
numFiles=17
size (MB)=129.046
diagnostics = {os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge}
no deletions
test: open reader.........OK [took 0.058 sec]
test: check integrity.....FAILED
WARNING: exorciseIndex() would remove reference to this segment; full exception:
org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:925)
at org.apache.lucene.index.CheckIndex.lambda$checkIndex$1(CheckIndex.java:756)
at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$2(CheckIndex.java:854)
at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:834)
WARNING: 1 broken segments (containing 119789 documents) detected
Took 0.347 sec total.
WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
Execution failed for task ':lucene:core:CheckIndex.main()'.
> Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
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've implemented the above by sorting segments by the file size they contain in increasing order.
Hmm, shouldn't we sort them by decreasing size, so that the biggest segments get the most time to check concurrently? I.e. the biggest segments will take the longest to check so I think we want them to kick off first?
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.
Yup this indeed will be both fun and challenging to implement! I think maybe it's also worth looking into breaking posting check into parallelize-able parts, as they alone can often take up 80% - 90% of the entire segment check time? |
I've created a spinoff issue to track the default setting change https://issues.apache.org/jira/browse/LUCENE-9984 |
Hi @mikemccand, I have addressed the comment above with some additional changes and posted some updated results. Could you please let me know if they look ready? |
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.
Sorry for taking so long to respond @zacharymorn! This change looks great -- I left a bunch of minor comments. Thank you for getting the output to match what we see today, and adding a test case confirming that :) Too bad about the MDW deadlock, but we can defer solving that one.
if (concurrent) { | ||
checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5)); | ||
} else { | ||
checker.setThreadCount(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.
Hmm, shouldn't we use 1
instead of 0
to mean "check sequentially"? And maybe 0
should not be allowed?
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.
Currently in the code, it is using 0
to signal turning off concurrent checking (using the main thread to check index), and it is also used as default when users don't explicitly pass in -threadCount
flag.
In CheckIndex#doCheck(Options)
, here's the logic to overwrite default thread count with user provided value (0
when not specified)
// when threadCount was not provided via command line, overrides it with 0 to turn off concurrent
// check
setThreadCount(opts.threadCount);
and later in CheckIndex#checkIndex(List)
, it is used as such
public Status checkIndex(List<String> onlySegments) throws IOException {
ExecutorService executorService = null;
if (threadCount > 0) {
executorService =
Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index"));
}
try {
return checkIndex(onlySegments, executorService);
} finally {
...
}
}
If we were to use 1
to signal checking sequentially / turning off concurrency, then I would imagine users will need to use -threadCount 2
when they meant "create 1 more thread in addition to main thread to check index", and in general N+1 when they were thinking about creating N additional threads right? I feel this might be a bit unintuitive?
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 I still don't think we should special-case 0
to mean "use a single thread" -- I think that's kinda counter-intuitive.
In fact I think 0
should not even be allowed (user should get an IllegalArgumentException
) -- how can one make progress with no threads :)
I think future users would indeed understand that -threadCount 1
means single-threaded, and that that most closely matches how CheckIndex
functions before this awesome addition of optional concurrency.
I suppose we could debate whether -threadCount 1
means "use the incoming (main) thread to do the checking" or it means "make an executor with one worker thread", but I think that's really an implementation detail. From the uer's standpoint, only one thread is making progress on checking.
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 after some more thoughts I do think that makes sense, particularly around the idea of this (main
vs. executor
) is really an implementation detail that users shouldn't need to worry about, and so -threadCount 1
to represent single-threaded execution would be the most intuitive approach here. I've pushed a new commit to update this accordingly.
Also, in the latest commit I've made it the default that when users don't specify -threadCount
value via command line, concurrent index checking will be used on machines with more than 1 core, but the number of threads will be capped at 4 via Math.min(Runtime.getRuntime().availableProcessors(), 4)
. I think this default behavior was discussed in multiple places in this PR, but would like to double confirm that this is the preferred default setting we would like to have (versus sequential index checking)?
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.
Great thanks @zacharymorn -- that sounds like a good approach.
@@ -450,6 +480,14 @@ public void setChecksumsOnly(boolean v) { | |||
|
|||
private boolean checksumsOnly; | |||
|
|||
/** Set threadCount used for parallelizing index integrity checking. */ | |||
public void setThreadCount(int tc) { | |||
threadCount = tc; |
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 validate the argument? It must be >= 1 at least?
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.
@@ -181,6 +193,9 @@ | |||
/** True if we were able to open a CodecReader on this segment. */ | |||
public boolean openReaderPassed; | |||
|
|||
/** doc count in this segment */ |
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.
Yeah let's just keep it the (inconsistent) way it was before for now.
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException { | |||
* quite a long time to run. | |||
*/ | |||
public Status checkIndex(List<String> onlySegments) 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.
Oh I didn't realize it would spawn multiple jvm processes,
This is just how Lucene's test infrastructure runs tests -- it spawns multiple JVMs, each of which is running one Lucene test at a time, but that test may use (often uses?) its own threads, including here in CheckIndex
if we make it concurrent by default.
@@ -54,4 +65,137 @@ public void testChecksumsOnlyVerbose() throws IOException { | |||
public void testObtainsLock() throws IOException { | |||
testObtainsLock(directory); | |||
} | |||
|
|||
@Test | |||
public void testCheckIndexAllValid() throws Exception { |
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 for this nice unit test! Confirming the textual output syntax from CheckIndex
.
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.
No problem! As I iterated through the changes, I see the need to create a test to guard against any unintentional bug, hence creating one here.
@@ -3622,6 +3860,7 @@ public static void main(String[] args) throws IOException, InterruptedException | |||
boolean doSlowChecks = false; |
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.
[Pre-existing] We could remove all these = false
and = null
. Hmm maybe there is a static ecj
checker for this.
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've created a spin-off issue available here https://issues.apache.org/jira/browse/LUCENE-10074.
|
||
if (checksumsOnly == false) { | ||
// This redundant assignment is done to make compiler happy | ||
SegmentReader finalReader = reader; |
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.
Egads, why does compiler insist on that? I don't see any anonymous classes / lambda bodies trying to reference reader
/finalReader
? Is this a leftover?
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.
Ah yes this is left over when I was previously using lambda for each test. I've removed this.
if (diagnostics.size() > 0) { | ||
msg(infoStream, " diagnostics = " + diagnostics); | ||
// checks segments sequentially | ||
if (executorService == null) { |
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 that is really sneaky deadlock! Maybe MDW#close
should not hold its monitor lock when it calls CheckIndex
, but let's not try to solve that here. We can refactor in the future. This change is already a good step forwards -- progress not perfection!
|
||
segInfoStat.openReaderPassed = true; | ||
// sort segmentCommitInfos by segment size, as smaller segment tends to finish faster, and |
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.
Ahhh OK I see -- this is a tricky tradeoff of seeing output sooner, versus finishing the overall CheckIndex
sooner :) Let's leave it as it is here (seeing output sooner)?
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.
Sounds good!
// TestUtil#checkIndex checks segment concurrently using another thread, but making | ||
// call back to synchronized methods such as MockDirectoryWrapper#fileLength. | ||
// Hence passing concurrent = false to this method to turn off concurrent checks. | ||
TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true, false, null); |
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 open a follow-on issue to fix this sync situation so that we could, randomly, sometimes use concurrency in CheckIndex
from tests? Maybe we could start by making some of the TestUtil.checkIndex
use concurrency, just not the one that MDW invokes?
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 open a follow-on issue to fix this sync situation so that we could, randomly, sometimes use concurrency in CheckIndex from tests?
Sounds good. I've created this issue https://issues.apache.org/jira/browse/LUCENE-10071 for following up on this.
Maybe we could start by making some of the TestUtil.checkIndex use concurrency, just not the one that MDW invokes?
Yes this is already done in 138b72e, where CheckIndex#checkIndex
got a new parameter concurrent
, and most of the invocations except the one in MDW would pass in true
to enable concurrency.
No worry there, and as usual appreciate your time and effort for reviewing my changes, and providing great suggestions and feedback! |
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.
Thanks @zacharymorn -- the changes look awesome! I'm looking forward to faster CheckIndex
!
Thanks @mikemccand again for the review and approval, and I look forward to the speedup as well! I'll merge in a few days just in case if anyone else may have further comment. |
Hi @mikemccand, I've merged this PR and will wait for an update on the nightly check index time page. Once the result there looks good, I believe we should backport this change to 8x as well? |
From the nightly check index time page, looks like the changes reduced the index checking time by a bit more than half! Woo hoo! |
Lucene 8.11.x has a new feature (apache/lucene#128) to check segments concurrently. Since this is executed in a threadpool, new thread name needs to be whitelisted.
Description
Parallelize CheckIndex across segments
Tests
Passed
./gradlew clean; ./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false
without nocommit comment