benwtrent commented on PR #13124:
URL: https://github.com/apache/lucene/pull/13124#issuecomment-1961417895

   OK, my head is spinning a bit trying to grok the side-effects of the CMS. I 
think I understand most of it now. 
   
   Currently we have two adjustable parameters. 
   
   `maxThreadCount` dictates merge through put, or now many merges we have at a 
time.
   
   `mergeWorkerCount` dictates back pressure, or when we stop allowing merges 
to be queued at all & block upstream indexing from continuing. 
   
   Throughout the life of a merge, it can become (un)paused numerous times, 
this is controlled via the `RateLimitedIndexOutput` which only rate limits 
WRITING results, not reading or anything else related to threads. 
   
   Now we want to add a new configuration, `maxMergeThreads`, which will 
control merge latency by adding parallelism to each individual thread. 
   
   Since merge thread pausing only has to do with index output, I am not sure 
we need to add any individual thread throttling other than what’s already 
there. The Directory wrapper will pause/throttle all writing occurring for the 
merge. This is acceptable even if the merge is using multiple threads. 
   
   I also think that small merges (<50MB) should never be allowed to run over 
multiple threads. Similar to how we never throttle those because they are so 
small, the benefit in latency reduction will be minuscule and we should reserve 
the extra thread usage to larger merges. 
   
   What I am stuck on is on this:

    - Should we use a common pool for all merges? Thus restricting the total 
threads used by merging to be `maxThreadCount` + `maxMergeThreads`? This will 
simplify the logic in the CMS significantly as a single task executor can be 
used for all merges.
    - Or should we use an individual executor/pool per merge? Thus restricting 
total threads by `maxMergeThreads` * `maxMergeThreads` (or some fraction of 
`maxMergeThreads`)? This could get interesting… How do we determine how many 
threads each merge can get? Are we ok with creating a new task executor on 
every larger merge and then closing it?
   
   What do you think @jpountz?
   
   One other thing we need to fix is this idea of “numWorkers” in HNSW. It 
seems like it should just pick optimal slices given the number of vectors it 
has (similar to multi-segment search stuff). Chunk itself into those slices and 
then be at the behest of the task executor. What say you @zhaih ? It seems 
weird to use `numWorkers` as a way to say “only use these many threads” when we 
have no knowledge of how many threads the task executor actually has.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org

Reply via email to