Skip to content

Conversation

@BryceKan3
Copy link
Contributor

Description

Currently, as part of DirectoryReader.open() Lucene will sequentially create segment readers for each segment.

This can be a very slow operation due to the I/O on the SegmentReader creation. By adding support for an ExecutorService to be passed in to DirectoryReader.open() we can submit the segment reader creations into the threadpool and achieve significant performance gains in DirectoryReader.open() times. The implementation is fully backwards compatible and allows for the users to pass in their own executor services.

I have tested the changes and validated the performance improvement that can be possible by utilizing parallelism for the opening of the directory readers.

Optimization P50 (ms) P90 (ms) P99 (ms) P50 Reduction %
Baseline 995 1020 1041 N/A
Concurrent SegmentReader Initialization 171 178 188 82.81%

Fixes #15387

@github-actions github-actions bot added this to the 11.0.0 milestone Nov 14, 2025
Copy link
Contributor

@vigyasharma vigyasharma left a comment

Choose a reason for hiding this comment

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

Thanks for these changes Bryce. I'm curious about the use-cases where opening segments become a bottleneck that should be parallelized. Have you seen some in production?

createMultiSegmentIndex(dir, 5);

ExecutorService executor =
Executors.newFixedThreadPool(1, new NamedThreadFactory("TestDirectoryReader"));
Copy link
Contributor

Choose a reason for hiding this comment

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

Any reason to only test with 1 thread in these tests? Can we add more threads to test for edge cases and races, like only one thread sees an exception while opening a segment reader?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hey, thanks for the review! Yes we have seen cases where this is a bottleneck. I have added a test case that covers scenarios where a single thread observes the failure.

@vigyasharma
Copy link
Contributor

I was wondering if this concurrency would also benefit the openIfChanged calls? But looks like the benefit comes from doing segment reading work in parallel, and for openIfChanged calls, we try to use old readers as much as possible?

Also curious if you tried using virtual threads for the executor in this use-case, and if they helped.

// parallelize segment reader initialization
futures.add(
(executor)
.submit(
Copy link

Choose a reason for hiding this comment

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

Have you considered using the existing TaskExecutor pattern and instead implementing this as such:

TaskExecutor taskExecutor = new TaskExecutor(executorService);  
List<Callable<SegmentReader>> tasks = new ArrayList<>(sis.size());  
        
for (int i = 0; i < sis.size(); i++) {  
    final int index = i;  
    tasks.add(() -> new SegmentReader(  
        sis.info(index), sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT));  
}  

Based on the comments in that class, there are some optimizations we could inherit here:

// try to execute as many tasks as possible on the current thread to minimize context
// switching in case of long running concurrent
// tasks as well as dead-locking if the current thread is part of #executor for executors that
// have limited or no parallelism

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hey yeah I took a look at this, I went with an ExecutorService here directly to ensure we could gracefully close out the readers in the event of an exception. With the TaskExecutor we would get the exception bubbled up and we would not be able to close the SegmentReaders.

@mikemccand
Copy link
Member

I was wondering if this concurrency would also benefit the openIfChanged calls? But looks like the benefit comes from doing segment reading work in parallel, and for openIfChanged calls, we try to use old readers as much as possible?

+1

But, if there are enough new segments (e.g. it's been a long time since the last openIfChanged), then concurrency might still be helpful.

@BryceKan3
Copy link
Contributor Author

I was wondering if this concurrency would also benefit the openIfChanged calls? But looks like the benefit comes from doing segment reading work in parallel, and for openIfChanged calls, we try to use old readers as much as possible?

+1

But, if there are enough new segments (e.g. it's been a long time since the last openIfChanged), then concurrency might still be helpful.

Yeah that makes sense, I will add this support to the corresponding openIfChanged APIs so we can get the benefits there as well.

@vigyasharma
Copy link
Contributor

Would it make sense to use virtual threads and create a local executor internally, instead of making the callers pass one? Maybe create a helper function that opens and returns a SegmentReader[] array from a provided SegmentInfos...

how about something like –

// pass a null array when there are no old readers?
private static SegmentReader[] createSegmentReaders(SegmentInfos sis, SegmentReader[] oldReaders) {
  final SegmentReader[] readers = new SegmentReader[sis.size()];
  try (var executor = Executors.newVirtualThreadPerTaskExecutor()) {
    List<Future<SegmentReader>> futures = new ArrayList<>();
    for (int i = sis.size() - 1; i >= 0; i--) {
      // TODO: add check for cases where old reader should be reused?
      final int index = i;
      futures.add(
        (executor)
        .submit(
          () ->
          new SegmentReader(
            sis.info(index),
            sis.getIndexCreatedVersionMajor(),
            IOContext.DEFAULT)));
    }
    RuntimeException firstException = null;
    for (int i = 0; i < futures.size(); i++) {
      try {
        readers[sis.size() - 1 - i] = futures.get(i).get();
      } catch (ExecutionException | InterruptedException e) {
      // If there is an exception creating the reader we still process
      // the rest of the completed futures to allow us to close created readers
        if (firstException == null) firstException = new RuntimeException(e);
      }
    }
    if (firstException != null) throw firstException;
  } finally {
    return readers;
  }
}

@BryceKan3
Copy link
Contributor Author

BryceKan3 commented Nov 18, 2025

Would it make sense to use virtual threads and create a local executor internally, instead of making the callers pass one? Maybe create a helper function that opens and returns a SegmentReader[] array from a provided SegmentInfos...

how about something like –

// pass a null array when there are no old readers?
private static SegmentReader[] createSegmentReaders(SegmentInfos sis, SegmentReader[] oldReaders) {
  final SegmentReader[] readers = new SegmentReader[sis.size()];
  try (var executor = Executors.newVirtualThreadPerTaskExecutor()) {
    List<Future<SegmentReader>> futures = new ArrayList<>();
    for (int i = sis.size() - 1; i >= 0; i--) {
      // TODO: add check for cases where old reader should be reused?
      final int index = i;
      futures.add(
        (executor)
        .submit(
          () ->
          new SegmentReader(
            sis.info(index),
            sis.getIndexCreatedVersionMajor(),
            IOContext.DEFAULT)));
    }
    RuntimeException firstException = null;
    for (int i = 0; i < futures.size(); i++) {
      try {
        readers[sis.size() - 1 - i] = futures.get(i).get();
      } catch (ExecutionException | InterruptedException e) {
      // If there is an exception creating the reader we still process
      // the rest of the completed futures to allow us to close created readers
        if (firstException == null) firstException = new RuntimeException(e);
      }
    }
    if (firstException != null) throw firstException;
  } finally {
    return readers;
  }
}

I just did some testing with passing in Executors.newVirtualThreadPerTaskExecutor() as the executor service and was still able to see the benefits. Upon a deep dive into virtual threads - I have some concerns around implementing them as the default for everything as there are some issues around pinning and mmap that could affect lucene users. With the ability to pass in an executor if the user would like to use virtual threads they can still pass it in otherwise their behavior will remain unchanged - this would allow full backwards compatibility and a new API that can be utilized with virtual threads. Let me know your thoughts @vigyasharma

@jainankitk
Copy link
Contributor

Upon a deep dive into virtual threads - I have some concerns around implementing them as the default for everything as there are some issues around pinning and mmap that could affect lucene users.

Yeah there are scenarios with virtual threads around mmap / pinning, that make them less effective for file I/O use cases in Lucene. The same reason we are looking at building better I/O concurrency in OpenSearch - opensearch-project/OpenSearch#18841, else it could have been delegated to the virtual threads in Java!

readers[i] =
new SegmentReader(
sis.info(i), sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Right now, we are not using the caller thread to create segment readers. I am wondering if we can/should do that similar to this PR - #13472. Although that makes the code change slightly complex, but might add slightly better concurrency.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's an interesting optimization! - I would like to keep the scope of this PR to just implementing the base concurrency with the ExecutorService and then we can have future improvements like this one that we can add. Let me know your thoughts!


* GITHUB#15187: Restrict visibility of PerFieldKnnVectorsFormat.FieldsReader (Simon Cooper)

* GITHUB##15428: Add support for ExecutorServices to be passed into DirectoryReader.open() API (Bryce Kane)
Copy link
Contributor

Choose a reason for hiding this comment

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

Are we not targeting this change for Lucene 10.4 release? If yes, we should move the entry to 10.4 section.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will update!

@vigyasharma
Copy link
Contributor

Upon a deep dive into virtual threads - I have some concerns around implementing them as the default for everything as there are some issues around pinning and mmap that could affect lucene users.

Yes, I was thinking about that too. It's a frustrating limitation with virtual threads; we can't always know if the underlying IO will use FFM in Lucene and are unable to use them.

Thinking out loud here: a thread getting pinned means it doesn't unmount, which is the same as using a platform thread for the entire task? The only difference is that there is no "pooling" for virtual threads. You create a lot of them (one per task). This is a problem if the executor lives for long and can schedule many virtual threads that get pinned - eating up all the available platform threads. I would assume that in this case, the thread pins only until the reader is opened (which is not long)? And doesn't stay pinned until the reader is eventually closed?!

Anyway, I agree that letting users pass an executor is the safer option here. This is a frequently invoked public API and we don't want to change default behavior in ways that can break for some users.

Maybe we can write it such that it accepts an executor and passes it to the helper function, so that we can use it for both open and openIfChanged ?

@uschindler
Copy link
Contributor

Hi,

Would it make sense to use virtual threads and create a local executor internally, instead of making the callers pass one? Maybe create a helper function that opens and returns a SegmentReader[] array from a provided SegmentInfos...

We don't want to create threads in Lucene "automatucally". Of course a virtual thread would not consume resources, but it is not helpful here. Lucene does not hand over threads back to the VM because it does not do "native" IO. It primarily uses MMap and when there are no syscalls, the virtual thraeds won't run in parallel. Please remember; Lucene's CPU usage is bound to CPU and theres no waiting time (e.g., for resources or similar), so virtual threads don't help. You need real threads here, so a Executor is the only way to go.

Uwe

@uschindler
Copy link
Contributor

What is the problem with allowing the use to pass an Executor? Lucene should not do anything "aitomagically". If somebody uses NIOFSDir, they can pass virtual threads. Anybody calling Lucene should always have some thread pools laying around, so only use an externally provided thread pool.

@vigyasharma
Copy link
Contributor

What is the problem with allowing the use to pass an Executor?

Yes, we aligned above on having users pass an Executor with an externally provided thread pool. It is the right, safe way to make this change.

@uschindler
Copy link
Contributor

P.S.: You may still see a small improvement when using virtual threads, but this is because the virtual threads can use the "waiting" time when opening files or listing directory contents. At thos eplace the JVM invokes the callback to the virtual threads management.

@github-actions github-actions bot modified the milestones: 11.0.0, 10.4.0 Nov 19, 2025
@BryceKan3
Copy link
Contributor Author

@vigyasharma I have pulled the SegmentReader creation logic into a helper function that is used now by both the open and openIfChanged APIs. Additionally I have added support for an ExecutorService to be passed into openIfChanged as well. Let me know your thoughts!

@dianjifzm
Copy link

Hello everyone,

I've also been looking forward to the parallel initialization capability for SegmentReader.
In fact, I've been anticipating this for several years. It's great to see the discussion sparked by the issues submitted by BryceKan3 today.

Let me explain the application scenario:
Normally, SegmentReader initialization is expected to be fast. However, since Lucene supports StoredFieldsFormat extensions,
we utilized StoredFieldsFormat to compress and store all forward column data in memory.
This move significantly improved IO performance (with about a 30% performance boost),
but it resulted in extremely slow SegmentReader initialization, taking as long as over 10 minutes.
This only affects the startup time and does not impact subsequent segment update mechanisms or other operations, which is why we've been using it for years.

Another suggestion: using Executors.newFixedThreadPool is not ideal.
It would be better to use ForkJoinPool.commonPool(), as the number of threads matches the number of CPUs, fully utilizing CPU performance and avoiding excessive context switching.

From my understanding, there's no need to consider openIfChanged or virtual threads.
SegmentReader only needs to be initialized in parallel during the startup phase. Once initialized, it no longer supports other asynchronous behaviors, which would minimize risks.
The simplest way is to use Arrays.parallelSetAll to initialize the SegmentReader[] readers.

@BryceKan3
Copy link
Contributor Author

Hey dianjifzm,

Thanks so much for the feedback! Very happy to hear that this feature will be something that you will benefit from as well.

ForkJoinPool is an ExecutorService - with these changes you will be able to directly pass this in the commonPool() as your ExecutorService if you want! Some clients need fine grained threading control and for those they usually manage threadpools themselves, by allowing support to pass in an ExecutorService these users will be able to benefit from this change. As for using Arrays.parallelSetAll - this will utilize the ForkJoinPool, we don't want to do this by default as it can cause unexpected resource contention if the client is managing their own threads.

As far as openIfChanged() - this method does create SegmentReaders and as Mike and Vigya mentioned above there can be scenarios where if we have enough new segments we will be able to benefit from this change as well in that API.

SegmentReader initialization taking 10 minutes seems extremely slow! In my profiling I have found most of the overhead is checksumming files, curious if that is the case for you as well... I wonder if there is some room on the table for some additional optimization in those code paths.

Copy link
Contributor

@vigyasharma vigyasharma left a comment

Choose a reason for hiding this comment

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

Looks good @BryceKan3 , thanks for addressing all the suggestions and generalizing it to openIfChanged. I have some minor comments, looks good to ship otherwise.

* @return DirectoryReader that covers entire index plus all changes made so far by this
* IndexWriter instance, or null if there are no new changes
* @param writer The IndexWriter to open from
* @param executorService Provides intra-open concurrency
Copy link
Contributor

Choose a reason for hiding this comment

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

Minor: "intra-open concurrency" feels a little hard to understand. I have context on this change so I think i know what this means, but let's simplify for the java doc? How about something like:

/**
* ...
* @param executorService used to open segment readers in parallel 
*/

SegmentInfos sis =
SegmentInfos.readCommit(directory, segmentFileName, minSupportedMajorVersion);
final SegmentReader[] readers = new SegmentReader[sis.size()];
SegmentReader[] readers = new SegmentReader[sis.size()];
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need to allocate this array here anymore? looks like createSegmentReader allocates its own array?

@BryceKan3
Copy link
Contributor Author

Thanks @vigyasharma for the review, I had addressed your comments.

@vigyasharma vigyasharma merged commit 7d4e224 into apache:main Nov 21, 2025
12 checks passed
@vigyasharma
Copy link
Contributor

Thanks @BryceKan3 ! I tried backporting it for 10.4 but ran into some conflicts. Would you be able to take a look at these conflicts and create another backporting PR against branch_10x ?

@BryceKan3
Copy link
Contributor Author

Thanks @vigyasharma - so to confirm it is in 11.x currently but needs to be backported into the branch_10x to go out with 10.4?

@vigyasharma
Copy link
Contributor

It is merged into the main branch. Needs to additionally be backported to branch_10x to be released with the next 10.x minor version.

BryceKan3 added a commit to BryceKan3/lucene that referenced this pull request Nov 22, 2025
…en() to enable concurrent segment reader initialization (apache#15428)
@BryceKan3
Copy link
Contributor Author

Created #15445 for the backport

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.

[RFC] Add support for Concurrent SegmentReader Initialization in DirectoryReader.open()

7 participants