Skip to content

SharedMergeScheduler using shared thread pool for multi-tenant merge scheduling #14900

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

Open
wants to merge 8 commits into
base: main
Choose a base branch
from

Conversation

N624-debu
Copy link
Contributor

[Draft] SharedMergeScheduler using shared thread pool for multi-tenant merge scheduling

This draft PR introduces a prototype SharedMergeScheduler, which extends MergeScheduler and routes all merge tasks through a shared thread pool across IndexWriter instances.

Motivation

In multi-tenant environments (e.g., Solr, Elasticsearch), many IndexWriters may coexist in the same JVM. The default Lucene behavior assigns each writer its own ConcurrentMergeScheduler and thread pool, which can lead to resource oversubscription and inefficient coordination.

This implementation introduces a new merge scheduler that centralizes merge execution via a shared thread pool. This idea builds on feedback from GitHub issue #13883, where contributors suggested exploring a dedicated scheduler based on Java's executor framework, rather than modifying the existing ConcurrentMergeScheduler.

Implementation Highlights

  • Adds a new class SharedMergeScheduler in lucene.index
  • Implements the merge(MergeSource, MergeTrigger) method using the public MergeSource API
  • Uses a static Executors.newFixedThreadPool(4) as a prototype shared executor
  • Keeps ConcurrentMergeScheduler unchanged for easier evaluation and iteration

Next Steps

  • Add lifecycle management and graceful shutdown to the executor
  • Evaluate fairness or throttling strategies across writers
  • Potentially combine with a centralized merge manager
  • Seek feedback on architecture fit and maintainability

This PR is opened to propose and evaluate a shared-thread-pool merge scheduler design, and to gather feedback for further development.

Copy link

github-actions bot commented Jul 5, 2025

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

Copy link

github-actions bot commented Jul 5, 2025

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

@vigyasharma
Copy link
Contributor

This seems to be in the right direction. Let's make it a singleton? As a next step, should we add support for prioritising small merges over larger ones (by backing the thread pool with a priority blocking queue)?

Also, since this is a draft PR, let's set it as "draft" on GitHub. You can also add a comment anywhere in your PR that says // no commit, which will ensure that the PR is not merged until it's ready (some GitHub checks will fail).

@jpountz
Copy link
Contributor

jpountz commented Jul 6, 2025

@vigyasharma what would you think of the following:

  • Instead of making it a singleton, add the executor as a constructor argument so that the same executor can be used for indexing and merging? (plus add javadocs to recommend using a fixed thread pool and sharing it with indexing)
  • Tasks get wrapped when submitted to the executor, similarly to what TaskExecutor does with TaskExecutor.Task, to allow merges to run in the current thread if they don't get picked up by the executor in a timely fashion.
  • To mimic CMS, maybe this Task wrapper should keep polling pending merges that are below MIN_BIG_MERGE_MB=50MB without waiting for them to go through the executor queue so that they always run almost immediately? (this requires tracking pending merges in a separate queue)

@vigyasharma
Copy link
Contributor

@jpountz I feel the scheduler should have some control on how the executor is created, its backing queue etc, so that it can prioritize how merges get scheduled, e.g. pick smaller merges before large ones, or have merges smaller than MIN_BIG_MERGE_MB so straight through without waiting. Otherwise, all the "merge scheduling" logic really gets offloaded to how the thread pool manages its task queue. That could be hard to get right, but maybe it's okay for expert users?

We could have the multi-tenant CMS accept a constructor argument for a custom MergeTasksExecutorService defined in Lucene. It will maintain a fixed thread pool, use a fixed size priority blocking queue that prefers small merges over big ones, and add custom logic to directly execute small merges, have bounded wait time on merges, use calling thread when needed etc. Something on the lines of Lucene's TaskExecutor that you mentioned. On similar lines, we can also create an IndexingMergeSharedExecutorService to share threads across indexing and merge tasks. We'll probably need to write a custom executor service; I'm not sure if wrapping an executor will give us the hooks we need.

@vigyasharma
Copy link
Contributor

Based on all this, I think we get the following structure:

  1. A merge scheduler that accepts an ExecutorService in its ctor. It will offload most of the scheduling / queueing / throttling logic to the executor service and its backing queue implementation. We will recommend using MergeTasksExecutorService or IndexingMergeSharedExecutorService but users are free to pass in their own executors.
  2. A MergeTasksExecutorService which does CMS like scheduling but for merge tasks only. The fixed thread pool is shared across all submitted merges across all writers. This executor service assumes requires a separate indexing thread pool, and puts back-pressure on indexing when merges accumulate e.g. by making the merge run on the indexing thread which tries to submit it.
  3. A IndexingMergeSharedExecutorService that can effectively handle sharing threads b/w indexing and merge tasks.

FWIW, I'm not really sure if sharing a thread pool b/w indexing and merging would be simpler than having separate thread pools and applying backpressure on indexing. I still need to grok all the details. But if we think backpressure is the way to go, and we'll always only use the MergeTasksExecutorService, then writing this executor as part of the scheduler and making it a singleton might make the code simpler?

OTOH, if we want a shared indexing/merging thread pool, or foresee a need to have different thread pools for sets of writers (shards in Elasticsearch / OpenSearch), then the ctor arg for executor service makes sense.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

public long getMergeSize() {
return mergeSize;
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: we keep an empty line at the end of file. There might be an editor/formatter setting you want to add that brings it in automatically.


public class MergeTaskWrapper {
private final Runnable mergeTask;
private final Object sourceWriter; // Can be IndexWriter or its ID
Copy link
Contributor

Choose a reason for hiding this comment

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

You can directly use IndexWriter or String for ID instead of using Object. Keeping the type as specific as possible helps catch a lot of bugs before runtime.

* version, the shared executor should be properly shut down.
*/
@Override
public void close() {
Copy link
Contributor

Choose a reason for hiding this comment

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

This impl. for close() will shut down the entire executor. Since this is a shared merge scheduler, you only want to address merges related to the closing IndexWriter, like cancelling all pending merges and waiting for running merges to complete. (See close() in CMS as well).

@@ -0,0 +1,25 @@
package org.apache.lucene.index;

public class MergeTaskWrapper {
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm curious how we intend to use this wrapper, looking forward to the next iteration of this PR.

@jpountz
Copy link
Contributor

jpountz commented Jul 17, 2025

FWIW, I'm not really sure if sharing a thread pool b/w indexing and merging would be simpler than having separate thread pools and applying backpressure on indexing

To me the reason for sharing the thread pools is not to make things simpler but rather to make it easier to control the overall number of active indexing+merging threads. For CPU-bound workloads (which indexing tends to be), the best approach is to create a thread pool sized based on the number of cores of the machine. You can't do this if indexing and merging don't use the same thread pool.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

Copy link

This PR does not have an entry in lucene/CHANGES.txt. Consider adding one. If the PR doesn't need a changelog entry, then add the skip-changelog label to it and you will stop receiving this reminder on future updates to the PR.

*/
@Override
public void merge(MergeSource mergeSource, MergeTrigger trigger) throws IOException {
while (true) {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we use hasPendingMerges() instead?

MergeTaskWrapper wrappedTask = new MergeTaskWrapper(mergeRunnable, (IndexWriter) mergeSource, merge.totalBytesSize());

// Registering this task under the writer
writerToMerges.computeIfAbsent((IndexWriter) mergeSource, k -> new CopyOnWriteArraySet<>()).add(wrappedTask);
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think you can cast mergeSource as an IndexWriter. However, you might not need to. This CMS is no longer a singleton, only the executor is shared. The CMS is actually owned by an indexWriter, which means all calls to merge() come from the same IW.

So instead of a writer -> merges mapping that you'd need in a singleton, all you need here is the set of pending and running merges submitted to this CMS. They are all from the same writer! When writer closes, instead of shutting down the executor, you could update the merge objects in the set (MergeTaskWrappers), and set an aborted flag. And update your runnable to skip the merge if "aborted" flag has been set.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants