Plumb merge parallelism via ConcurrentMergeScheduler instead of codec#507
Plumb merge parallelism via ConcurrentMergeScheduler instead of codec#507abernardi597 wants to merge 2 commits intomikemccand:mainfrom
ConcurrentMergeScheduler instead of codec#507Conversation
cbf0b90 to
8ec4176
Compare
|
This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution! |
8ec4176 to
6c00719
Compare
mikemccand
left a comment
There was a problem hiding this comment.
Phew, this was hard to think about. I like the motivation for this change (use Lucene's defaults, stop creating an extra executor with confusing knobs to tune), but it's too forceful (overloading CMS hard limit on merge debt and HNSW max merge concurrency).
Could we keep the HNSW merge concurrency parameter separate? Rename it to -hnswMergeWorkerCount or so, and pass that to Lucene99HnswVectorsFormat when we getCodec(...). The javadocs for that class state that you must provide and executor when numMergeWorkers > 1 but I think that's not true? It seems to (correctly) fallback to CMS's intra-merge thread pool? Maybe make upstream PR to fix that if so.
And then change what -numMergeThread does as you do here? It becomes the soft limit (maxMergeThreads) for CMS, which is also what it sizes its intra-merge thread pool to, so that will be consistent with how KnnGraphTester works today. Since you must also set CMS's hard limit on merge debt, pick something dynamic ... maybe max(numMergeThread+2, 3*numMergeThread/2) ish? And add a new parameter (-maxMergeDebt maybe?) so user could override that dynamic default if they want?
| numQueryVectors = 1000; | ||
| dim = 256; | ||
| topK = 100; | ||
| numMergeThread = 1; |
There was a problem hiding this comment.
OK we are also switching to Lucene's default concurrency with this change right? Previously knnPerfTest.py was defaulting to 1 and 1 here?
There was a problem hiding this comment.
Yes, it makes more sense to me that omitting an option should use whatever the Lucene default is.
It's also more ergonomic in my opinion if you want the Lucene default; otherwise you'd specify -1 on the CLI which can be ambiguous.
src/main/knn/KnnGraphTester.java
Outdated
| numMergeThread = 1; | ||
| numMergeWorker = 1; | ||
| numMergeThread = ConcurrentMergeScheduler.AUTO_DETECT_MERGES_AND_THREADS; | ||
| numMergeWorker = ConcurrentMergeScheduler.AUTO_DETECT_MERGES_AND_THREADS; |
There was a problem hiding this comment.
OK what actually is the difference between worker and thread here, even before your change?
It looks like numMergeWorker is how much concurrency a single HNSW merge will use, and numMergeThread is the size of the globally shared (ConcurrentMergeScheduler's intraMergeExecutor) thread pool.
I don't get why these are separately configurable. If I have only one merge to run, wouldn't I want to use all available concurrency? So I should set numMergeWorker >= numMergeThread. But then it's silly to expect/ask for more concurrency than you can possibly execute, so then numMergeWorker should be <= numMergeThread. Intersecting the two ... I should just always set them to the same value?
Or, perhaps we lose throughput if we try to do too many concurrent tasks for a single HNSW merge (thread sync/context switch overhead), in which case maybe I limit workers and expect multiple merges to typically be running to soak up all the concurrent threads?
Anyway it's all intensely confusing, and this PR is great progress (eliminating extra confusion added on top by creating another executor, and falling back to Lucene's preferred default path)...
There was a problem hiding this comment.
If I follow correctly, it seems like we can just set numMergeWorker to cms.getMaxThreadCount() instead of cms.getMaxMergeCount().
I'm not concerned about thread overhead since the HNSW merges are batched anyway to mitigate the overhead of over-provisioning concurrency.
src/main/knn/KnnGraphTester.java
Outdated
| ).createIndex(); | ||
| ).createIndex(iwc -> { | ||
| ConcurrentMergeScheduler cms = (ConcurrentMergeScheduler) iwc.getMergeScheduler(); | ||
| cms.setMaxMergesAndThreads(numMergeWorker, numMergeThread); |
There was a problem hiding this comment.
Hmmm ... this isn't quite right ... it's overloading two very different settings.
setMaxMergesAndThreads takes two parameters, maxMergeCount and maxThreadCount. Think of maxMergeCount as a hard limit on number of running merges (merge debt), and maxThreadCount as a soft limit.
When merge debt crosses the soft limit, ConcurrentMergeScheduler (CMS) begins pausing/unpausing enough merges to keep the running merge count at the soft limit. If merge debt continues to grow, CMS takes the even more drastic step of forcefully stalling the incoming indexing threads (the threads causing new segments to poof into existence, the "producers", or the "mutator threads" in GC-speak). These are CMS's two backpressure mechanisms against the adversarial mutators.
Whereas, numMergeWorkers here is a very different concept: it's how much concurrency the HNSW merger, for a single segment, is able to take advantage of. It really should not be a configurable parameter, I think -- it should be "as much concurrency as you have to offer" (numMergeThreads here).
There was a problem hiding this comment.
See above, I think we can use getMaxThreadCount() for numMergeWorker. Really, -numMergeWorker should be -maxMergeCount and -numMergeThread should be -maxMergeThreads?
And add an override flag for hnswMergeThreads, perhaps.
Does it make sense to overload the flag to control whether the intra-merge executor is used or not?
That is:
- if you omit the flag then the intra-merge executor is used for hnsw merges at whatever the
getMaxThreadCount()is - if you provide a value for the flag then it will use a separate executor
As far as I can tell, there isn't much else that uses the intra-merge executor so it seems redundant to even support the separate executor. I wonder why this escape hatch was added instead of just using the intra-merge executor?
|
This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution! |
6c00719 to
dc50806
Compare
dc50806 to
9591d33
Compare
|
The latest revision of this PR:
|
By leaving the HNSW vector codec merge executor as
null, the codecs will leverage theintraMergeTaskExecutorprovided at merge-time to parallelize the merges.To populate the field, the corresponding parameters are properly set using
ConcurrentMergeScheduler.setMaxMergesAndThreads(int, int).This change results in a bit of a dependency issue with constructing the codec when using the default value
ConcurrentMergeScheduler.AUTO_DETECT_MERGES_AND_THREADS, as the auto-detected value only becomes available after setting it on the CMS.I modified
KnnIndexer.createIndexto take a lambda for setting non-default options in order to get the CMS and use its values to construct the codec, but it's admittedly a bit janky.I also modified some of the reported merge timings and print statements.
These changes were originally made as part of #502.