Skip to content

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

@BryceKan3

Description

@BryceKan3

Description

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

Image

This can be a very slow operation due to the I/O on the SegmentReader creation. If we add support for an Executor to be passed in to DirectoryReader.open() calls we can submit the segment reader creations into the threadpool and achieve up to an 80% decrease in DirectoryReader.open() times. The implementation would need to be fully backwards compatible - (possibly by adding an additional constructor) and users would pass their own executors into the function so they can control the threading.

Image

I have created a proof of concept for this and have been able to observe the following improvements

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

Above was run with 100 Iterations, 16 Segments, 20 threads passed into the executor service with a cleared page cache

I believe this enhancement will provide significant value to the Lucene community and would love feedback on this approach.

Thanks

Metadata

Metadata

Assignees

No one assigned

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions