From f0b64ccfe1c3bda0167c5ae38f4e40cfdaf1904c Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 29 Oct 2025 08:40:37 -0700 Subject: [PATCH 01/17] Initial commit DocIdSetBuilder Signed-off-by: Prudhvi Godithi --- .../apache/lucene/search/IndexSearcher.java | 11 +- .../apache/lucene/search/PointRangeQuery.java | 17 +- .../java/org/apache/lucene/search/Weight.java | 25 +++ .../apache/lucene/util/DocIdSetBuilder.java | 175 +++++++++++++++++- 4 files changed, 220 insertions(+), 8 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index d1079b69089a..258e51b55e9e 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -828,7 +828,16 @@ protected void searchLeaf( // continue with the following leaf return; } - ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx); + + // Create partition object to pass to weight + final LeafReaderContextPartition partition; + if (minDocId == 0 && maxDocId == DocIdSetIterator.NO_MORE_DOCS) { + partition = LeafReaderContextPartition.createForEntireSegment(ctx); + } else { + partition = LeafReaderContextPartition.createFromAndTo(ctx, minDocId, maxDocId); + } + + ScorerSupplier scorerSupplier = weight.scorerSupplier(partition); if (scorerSupplier != null) { scorerSupplier.setTopLevelScoringClause(); BulkScorer scorer = scorerSupplier.bulkScorer(); diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java index c198fecb4b35..9de298b99e06 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java @@ -249,8 +249,9 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { } @Override - public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { - LeafReader reader = context.reader(); + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + LeafReader reader = partition.ctx.reader(); PointValues values = reader.getPointValues(field); if (checkValidPointValues(values) == false) { @@ -298,7 +299,11 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti } else { return new ConstantScoreScorerSupplier(score(), scoreMode, reader.maxDoc()) { - final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values); + // Create partition-aware DocIdSetBuilder that filters docs and uses partition-sized + // threshold + final DocIdSetBuilder result = + new DocIdSetBuilder( + reader.maxDoc(), values, partition.minDocId, partition.maxDocId); final IntersectVisitor visitor = getIntersectVisitor(result); long cost = -1; @@ -336,6 +341,12 @@ public long cost() { } } + @Override + public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { + // Delegate to partition-aware version for entire segment + return scorerSupplier(IndexSearcher.LeafReaderContextPartition.createForEntireSegment(context)); + } + @Override public int count(LeafReaderContext context) throws IOException { LeafReader reader = context.reader(); diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index 341dd3cadf6a..f1cce197d7b4 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -149,6 +149,31 @@ public final Scorer scorer(LeafReaderContext context) throws IOException { */ public abstract ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException; + /** + * Returns a {@link ScorerSupplier}, which can then be used to get a {@link Scorer} for a + * partition of a leaf reader context. + * + *

This method allows queries to optimize for intra-segment concurrency by knowing the specific + * doc ID range being searched within the segment. The default implementation delegates to {@link + * #scorerSupplier(LeafReaderContext)} ignoring the partition bounds. Queries that can benefit + * from partition awareness (e.g., by creating smaller data structures scoped to the partition) + * should override this method. + * + *

A scorer supplier for the same {@link LeafReaderContext} instance may be requested multiple + * times as part of a single search call, potentially from different threads searching different + * doc ID ranges concurrently. + * + * @param partition the leaf reader context partition containing the context and doc ID range + * @return a {@link ScorerSupplier} providing the scorer, or null if scorer is null + * @throws IOException if an IOException occurs + * @see IndexSearcher.LeafReaderContextPartition + * @since 10.1 + */ + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } + /** * Helper method that delegates to {@link #scorerSupplier(LeafReaderContext)}. It is implemented * as diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index 3ac9293681dc..2034731ea327 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -41,7 +41,11 @@ public final class DocIdSetBuilder { * * @see DocIdSetBuilder#grow */ - public sealed interface BulkAdder permits FixedBitSetAdder, BufferAdder { + public sealed interface BulkAdder + permits FixedBitSetAdder, + BufferAdder, + PartitionAwareFixedBitSetAdder, + PartitionAwareBufferAdder { void add(int doc); void add(IntsRef docs); @@ -82,6 +86,53 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { } } + /** + * Partition-aware FixedBitSetAdder that filters docs to only include those within the specified + * range. + */ + private record PartitionAwareFixedBitSetAdder(FixedBitSet bitSet, int minDocId, int maxDocId) + implements BulkAdder { + + @Override + public void add(int doc) { + if (doc >= minDocId && doc < maxDocId) { + bitSet.set(doc); + } + } + + @Override + public void add(IntsRef docs) { + for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { + int doc = docs.ints[i]; + if (doc >= minDocId && doc < maxDocId) { + bitSet.set(doc); + } + } + } + + @Override + public void add(DocIdSetIterator iterator) throws IOException { + // Advance iterator to minDocId first, then collect docs up to maxDocId + int doc = iterator.nextDoc(); + if (doc < minDocId) { + doc = iterator.advance(minDocId); + } + if (doc < maxDocId) { + iterator.intoBitSet(maxDocId, bitSet, minDocId); + } + } + + @Override + public void add(IntsRef docs, int docLowerBoundInclusive) { + for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { + int doc = docs.ints[i]; + if (doc >= Math.max(docLowerBoundInclusive, minDocId) && doc < maxDocId) { + bitSet.set(doc); + } + } + } + } + private static class Buffer { int[] array; int length; @@ -131,12 +182,63 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { } } + /** + * Partition-aware BufferAdder that filters docs to only include those within the specified + * range. + */ + private record PartitionAwareBufferAdder(Buffer buffer, int minDocId, int maxDocId) + implements BulkAdder { + + @Override + public void add(int doc) { + if (doc >= minDocId && doc < maxDocId) { + buffer.array[buffer.length++] = doc; + } + } + + @Override + public void add(IntsRef docs) { + int index = buffer.length; + for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { + int doc = docs.ints[i]; + if (doc >= minDocId && doc < maxDocId) { + buffer.array[index++] = doc; + } + } + buffer.length = index; + } + + @Override + public void add(DocIdSetIterator iterator) throws IOException { + int docID; + while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { + add(docID); + } + } + + @Override + public void add(IntsRef docs, int docLowerBoundInclusive) { + int index = buffer.length; + for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { + int doc = docs.ints[i]; + if (doc >= Math.max(docLowerBoundInclusive, minDocId) && doc < maxDocId) { + buffer.array[index++] = doc; + } + } + buffer.length = index; + } + } + private final int maxDoc; private final int threshold; // pkg-private for testing final boolean multivalued; final double numValuesPerDoc; + // Partition filtering support - filters docs to only include those within [minDocId, maxDocId) + private final int minDocId; // inclusive + private final int maxDocId; // exclusive + private List buffers = new ArrayList<>(); private int totalAllocated; // accumulated size of the allocated buffers @@ -166,8 +268,58 @@ public DocIdSetBuilder(int maxDoc, PointValues values) throws IOException { this(maxDoc, values.getDocCount(), values.size()); } + /** + * Create a partition-aware {@link DocIdSetBuilder} instance that only accepts doc IDs within the + * specified range. This is useful for intra-segment concurrency where each partition only needs + * to collect docs within its assigned range. + * + * @param maxDoc the maximum doc ID in the segment + * @param minDocId the minimum doc ID (inclusive) to accept + * @param maxDocId the maximum doc ID (exclusive) to accept + * @param docCount estimated document count + * @param valueCount estimated value count + */ + public DocIdSetBuilder(int maxDoc, int minDocId, int maxDocId, int docCount, long valueCount) { + this(maxDoc, docCount, valueCount, minDocId, maxDocId); + } + + /** + * Create a partition-aware {@link DocIdSetBuilder} for {@link PointValues} that only accepts doc + * IDs within the specified range. + * + * @param maxDoc the maximum doc ID in the segment + * @param values the point values + * @param minDocId the minimum doc ID (inclusive) to accept + * @param maxDocId the maximum doc ID (exclusive) to accept + */ + public DocIdSetBuilder(int maxDoc, PointValues values, int minDocId, int maxDocId) + throws IOException { + this(maxDoc, values.getDocCount(), values.size(), minDocId, maxDocId); + } + + /** + * Create a partition-aware {@link DocIdSetBuilder} for {@link Terms} that only accepts doc IDs + * within the specified range. + * + * @param maxDoc the maximum doc ID in the segment + * @param terms the terms + * @param minDocId the minimum doc ID (inclusive) to accept + * @param maxDocId the maximum doc ID (exclusive) to accept + */ + public DocIdSetBuilder(int maxDoc, Terms terms, int minDocId, int maxDocId) throws IOException { + this(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), minDocId, maxDocId); + } + DocIdSetBuilder(int maxDoc, int docCount, long valueCount) { + this(maxDoc, docCount, valueCount, 0, maxDoc); + } + + private DocIdSetBuilder( + int maxDoc, int docCount, long valueCount, int minDocId, int maxDocId) { this.maxDoc = maxDoc; + this.minDocId = minDocId; + this.maxDocId = maxDocId; + this.multivalued = docCount < 0 || docCount != valueCount; if (docCount <= 0 || valueCount < 0) { // assume one value per doc, this means the cost will be overestimated @@ -184,7 +336,12 @@ public DocIdSetBuilder(int maxDoc, PointValues values) throws IOException { // maxDoc >>> 7 is a good value if you want to save memory, lower values // such as maxDoc >>> 11 should provide faster building but at the expense // of using a full bitset even for quite sparse data - this.threshold = maxDoc >>> 7; + // + // When filtering to a partition (minDocId > 0 or maxDocId < maxDoc), use the partition size + // for threshold calculation to ensure the threshold scales correctly with the partition size + boolean isPartition = (minDocId > 0 || maxDocId < maxDoc); + int effectiveMaxDoc = isPartition ? (maxDocId - minDocId) : maxDoc; + this.threshold = effectiveMaxDoc >>> 7; this.bitSet = null; } @@ -267,7 +424,12 @@ private int additionalCapacity(int numDocs) { private Buffer addBuffer(int len) { Buffer buffer = new Buffer(len); buffers.add(buffer); - adder = new BufferAdder(buffer); + // Use partition-aware adder if filtering to a specific doc ID range + if (minDocId > 0 || maxDocId < maxDoc) { + adder = new PartitionAwareBufferAdder(buffer, minDocId, maxDocId); + } else { + adder = new BufferAdder(buffer); + } totalAllocated += buffer.array.length; return buffer; } @@ -292,7 +454,12 @@ private void upgradeToBitSet() { this.bitSet = bitSet; this.counter = counter; this.buffers = null; - this.adder = new FixedBitSetAdder(bitSet); + // Use partition-aware adder if filtering to a specific doc ID range + if (minDocId > 0 || maxDocId < maxDoc) { + this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId); + } else { + this.adder = new FixedBitSetAdder(bitSet); + } } /** Build a {@link DocIdSet} from the accumulated doc IDs. */ From b7701f0a51e174b5a755637489f82b9a851bb5de Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Thu, 30 Oct 2025 08:18:23 -0700 Subject: [PATCH 02/17] DocIdSetBuilder with partition aware Signed-off-by: Prudhvi Godithi --- .../apache/lucene/search/FilterWeight.java | 6 + .../java/org/apache/lucene/search/Weight.java | 5 +- .../apache/lucene/util/DocIdSetBuilder.java | 115 ++++++++++++++++-- 3 files changed, 111 insertions(+), 15 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java index 16bb75ef4062..efbd9badd8a2 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java @@ -67,4 +67,10 @@ public Matches matches(LeafReaderContext context, int doc) throws IOException { public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { return in.scorerSupplier(context); } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return in.scorerSupplier(partition); + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index f1cce197d7b4..cb6f0224aa89 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.IndexSearcher.LeafReaderContextPartition; import org.apache.lucene.util.Bits; /** @@ -166,10 +167,10 @@ public final Scorer scorer(LeafReaderContext context) throws IOException { * @param partition the leaf reader context partition containing the context and doc ID range * @return a {@link ScorerSupplier} providing the scorer, or null if scorer is null * @throws IOException if an IOException occurs - * @see IndexSearcher.LeafReaderContextPartition + * @see LeafReaderContextPartition * @since 10.1 */ - public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + public ScorerSupplier scorerSupplier(LeafReaderContextPartition partition) throws IOException { return scorerSupplier(partition.ctx); } diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index 2034731ea327..449250284ee5 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -88,15 +88,20 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { /** * Partition-aware FixedBitSetAdder that filters docs to only include those within the specified - * range. + * range. Stores docs using partition-relative indices (doc - offset) to save memory. + * + * @param bitSet the partition-sized bitset to store relative doc indices + * @param minDocId minimum doc ID (inclusive) to accept + * @param maxDocId maximum doc ID (exclusive) to accept + * @param offset the value to subtract from absolute doc IDs (typically minDocId) */ - private record PartitionAwareFixedBitSetAdder(FixedBitSet bitSet, int minDocId, int maxDocId) - implements BulkAdder { + private record PartitionAwareFixedBitSetAdder( + FixedBitSet bitSet, int minDocId, int maxDocId, int offset) implements BulkAdder { @Override public void add(int doc) { if (doc >= minDocId && doc < maxDocId) { - bitSet.set(doc); + bitSet.set(doc - offset); } } @@ -105,20 +110,21 @@ public void add(IntsRef docs) { for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { int doc = docs.ints[i]; if (doc >= minDocId && doc < maxDocId) { - bitSet.set(doc); + bitSet.set(doc - offset); } } } @Override public void add(DocIdSetIterator iterator) throws IOException { - // Advance iterator to minDocId first, then collect docs up to maxDocId + // Advance iterator to minDocId first int doc = iterator.nextDoc(); if (doc < minDocId) { doc = iterator.advance(minDocId); } + // Use optimized intoBitSet with partition boundaries and offset if (doc < maxDocId) { - iterator.intoBitSet(maxDocId, bitSet, minDocId); + iterator.intoBitSet(maxDocId, bitSet, offset); } } @@ -127,7 +133,7 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { int doc = docs.ints[i]; if (doc >= Math.max(docLowerBoundInclusive, minDocId) && doc < maxDocId) { - bitSet.set(doc); + bitSet.set(doc - offset); } } } @@ -441,22 +447,31 @@ private void growBuffer(Buffer buffer, int additionalCapacity) { private void upgradeToBitSet() { assert bitSet == null; - FixedBitSet bitSet = new FixedBitSet(maxDoc); + + // For partitions, create a smaller bitset sized to the partition range only + // This saves memory by not allocating bits outside [minDocId, maxDocId) + boolean isPartition = (minDocId > 0 || maxDocId < maxDoc); + int bitSetSize = isPartition ? (maxDocId - minDocId) : maxDoc; + + FixedBitSet bitSet = new FixedBitSet(bitSetSize); long counter = 0; for (Buffer buffer : buffers) { int[] array = buffer.array; int length = buffer.length; counter += length; for (int i = 0; i < length; ++i) { - bitSet.set(array[i]); + // For partitions, convert absolute doc ID to partition-relative index + int docId = array[i]; + int bitIndex = isPartition ? (docId - minDocId) : docId; + bitSet.set(bitIndex); } } this.bitSet = bitSet; this.counter = counter; this.buffers = null; // Use partition-aware adder if filtering to a specific doc ID range - if (minDocId > 0 || maxDocId < maxDoc) { - this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId); + if (isPartition) { + this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId, minDocId); } else { this.adder = new FixedBitSetAdder(bitSet); } @@ -468,7 +483,14 @@ public DocIdSet build() { if (bitSet != null) { assert counter >= 0; final long cost = Math.round(counter / numValuesPerDoc); - return new BitDocIdSet(bitSet, cost); + + // For partition-relative bitsets, wrap with offset to return absolute doc IDs + boolean isPartition = (minDocId > 0 || maxDocId < maxDoc); + if (isPartition) { + return new OffsetBitDocIdSet(bitSet, cost, minDocId); + } else { + return new BitDocIdSet(bitSet, cost); + } } else { Buffer concatenated = concat(buffers); LSBRadixSorter sorter = new LSBRadixSorter(); @@ -490,6 +512,73 @@ public DocIdSet build() { } } + /** + * Wrapper for partition-relative bitsets that offsets doc IDs back to absolute values when + * iterating. + */ + private static class OffsetBitDocIdSet extends DocIdSet { + private final BitDocIdSet delegate; + private final int offset; + + OffsetBitDocIdSet(FixedBitSet bitSet, long cost, int offset) { + this.delegate = new BitDocIdSet(bitSet, cost); + this.offset = offset; + } + + @Override + public DocIdSetIterator iterator() { + DocIdSetIterator delegateIterator = delegate.iterator(); + if (delegateIterator == null) { + return null; + } + return new OffsetDocIdSetIterator(delegateIterator, offset); + } + + @Override + public long ramBytesUsed() { + return delegate.ramBytesUsed(); + } + } + + /** + * Iterator that adds an offset to all doc IDs from the underlying iterator, converting + * partition-relative indices back to absolute doc IDs. + */ + private static class OffsetDocIdSetIterator extends DocIdSetIterator { + private final DocIdSetIterator delegate; + private final int offset; + + OffsetDocIdSetIterator(DocIdSetIterator delegate, int offset) { + this.delegate = delegate; + this.offset = offset; + } + + @Override + public int docID() { + int doc = delegate.docID(); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public int nextDoc() throws IOException { + int doc = delegate.nextDoc(); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public int advance(int target) throws IOException { + // Convert target from absolute to partition-relative, advance, then convert back + int relativeTarget = target - offset; + int doc = delegate.advance(Math.max(0, relativeTarget)); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public long cost() { + return delegate.cost(); + } + } + /** * Concatenate the buffers in any order, leaving at least one empty slot in the end NOTE: this * method might reuse one of the arrays From f4e24afdcc2b7457c92ded40437cc0b5e4474f22 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Thu, 30 Oct 2025 08:24:53 -0700 Subject: [PATCH 03/17] DocIdSetBuilder code refactor Signed-off-by: Prudhvi Godithi --- .../apache/lucene/util/DocIdSetBuilder.java | 67 ----------------- .../apache/lucene/util/OffsetBitDocIdSet.java | 63 ++++++++++++++++ .../lucene/util/OffsetDocIdSetIterator.java | 71 +++++++++++++++++++ 3 files changed, 134 insertions(+), 67 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java create mode 100644 lucene/core/src/java/org/apache/lucene/util/OffsetDocIdSetIterator.java diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index 449250284ee5..f9417130b75c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -512,73 +512,6 @@ public DocIdSet build() { } } - /** - * Wrapper for partition-relative bitsets that offsets doc IDs back to absolute values when - * iterating. - */ - private static class OffsetBitDocIdSet extends DocIdSet { - private final BitDocIdSet delegate; - private final int offset; - - OffsetBitDocIdSet(FixedBitSet bitSet, long cost, int offset) { - this.delegate = new BitDocIdSet(bitSet, cost); - this.offset = offset; - } - - @Override - public DocIdSetIterator iterator() { - DocIdSetIterator delegateIterator = delegate.iterator(); - if (delegateIterator == null) { - return null; - } - return new OffsetDocIdSetIterator(delegateIterator, offset); - } - - @Override - public long ramBytesUsed() { - return delegate.ramBytesUsed(); - } - } - - /** - * Iterator that adds an offset to all doc IDs from the underlying iterator, converting - * partition-relative indices back to absolute doc IDs. - */ - private static class OffsetDocIdSetIterator extends DocIdSetIterator { - private final DocIdSetIterator delegate; - private final int offset; - - OffsetDocIdSetIterator(DocIdSetIterator delegate, int offset) { - this.delegate = delegate; - this.offset = offset; - } - - @Override - public int docID() { - int doc = delegate.docID(); - return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; - } - - @Override - public int nextDoc() throws IOException { - int doc = delegate.nextDoc(); - return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; - } - - @Override - public int advance(int target) throws IOException { - // Convert target from absolute to partition-relative, advance, then convert back - int relativeTarget = target - offset; - int doc = delegate.advance(Math.max(0, relativeTarget)); - return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; - } - - @Override - public long cost() { - return delegate.cost(); - } - } - /** * Concatenate the buffers in any order, leaving at least one empty slot in the end NOTE: this * method might reuse one of the arrays diff --git a/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java b/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java new file mode 100644 index 000000000000..081f6b6f2bbc --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import java.io.IOException; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.DocIdSetIterator; + +/** + * Wrapper for partition-relative bitsets that offsets doc IDs back to absolute values when + * iterating. + * + *

This is used for partition-aware queries where a small bitset stores docs using + * partition-relative indices (0 to partitionSize-1), but the iterator must return absolute doc IDs + * (minDocId to maxDocId-1). + * + * @lucene.internal + */ +final class OffsetBitDocIdSet extends DocIdSet { + private final BitDocIdSet delegate; + private final int offset; + + /** + * Creates an offset wrapper around a BitDocIdSet. + * + * @param bitSet the partition-relative bitset + * @param cost the cost estimate + * @param offset the value to add to convert relative indices to absolute doc IDs (typically + * minDocId) + */ + OffsetBitDocIdSet(FixedBitSet bitSet, long cost, int offset) { + this.delegate = new BitDocIdSet(bitSet, cost); + this.offset = offset; + } + + @Override + public DocIdSetIterator iterator() { + DocIdSetIterator delegateIterator = delegate.iterator(); + if (delegateIterator == null) { + return null; + } + return new OffsetDocIdSetIterator(delegateIterator, offset); + } + + @Override + public long ramBytesUsed() { + return delegate.ramBytesUsed(); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/util/OffsetDocIdSetIterator.java b/lucene/core/src/java/org/apache/lucene/util/OffsetDocIdSetIterator.java new file mode 100644 index 000000000000..bbe59135790c --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/OffsetDocIdSetIterator.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import java.io.IOException; +import org.apache.lucene.search.DocIdSetIterator; + +/** + * Iterator that adds an offset to all doc IDs from the underlying iterator, converting + * partition-relative indices back to absolute doc IDs. + * + *

This is used for partition-aware queries where the underlying iterator returns relative + * indices (0 to partitionSize-1), but callers expect absolute doc IDs (minDocId to maxDocId-1). + * + * @lucene.internal + */ +final class OffsetDocIdSetIterator extends DocIdSetIterator { + private final DocIdSetIterator delegate; + private final int offset; + + /** + * Creates an offset wrapper around a DocIdSetIterator. + * + * @param delegate the underlying iterator returning partition-relative doc IDs + * @param offset the value to add to convert relative indices to absolute doc IDs (typically + * minDocId) + */ + OffsetDocIdSetIterator(DocIdSetIterator delegate, int offset) { + this.delegate = delegate; + this.offset = offset; + } + + @Override + public int docID() { + int doc = delegate.docID(); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public int nextDoc() throws IOException { + int doc = delegate.nextDoc(); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public int advance(int target) throws IOException { + // Convert target from absolute to partition-relative, advance, then convert back + int relativeTarget = target - offset; + int doc = delegate.advance(Math.max(0, relativeTarget)); + return doc == NO_MORE_DOCS ? NO_MORE_DOCS : doc + offset; + } + + @Override + public long cost() { + return delegate.cost(); + } +} From 91de456552ec13cf15189495c41ace0ef0541036 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 04:08:16 -0800 Subject: [PATCH 04/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../src/java/org/apache/lucene/util/OffsetBitDocIdSet.java | 1 - .../org/apache/lucene/tests/search/ScorerIndexSearcher.java | 6 ++++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java b/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java index 081f6b6f2bbc..bf767a7ca7f3 100644 --- a/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java @@ -16,7 +16,6 @@ */ package org.apache.lucene.util; -import java.io.IOException; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java index d2e70e777b38..23c0ea466151 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java @@ -81,6 +81,12 @@ public long cost() { } }; } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } }; super.searchLeaf(ctx, minDocId, maxDocId, filterWeight, collector); From a0b284925417c6c06ef32ae739f18b92a2f0166e Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 05:04:43 -0800 Subject: [PATCH 05/17] Fix the tidy tests Signed-off-by: Prudhvi Godithi --- .../src/java/org/apache/lucene/search/PointRangeQuery.java | 3 ++- lucene/core/src/java/org/apache/lucene/search/Weight.java | 3 +-- .../src/java/org/apache/lucene/util/DocIdSetBuilder.java | 6 ++---- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java index 9de298b99e06..fa0abcfeadad 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java @@ -344,7 +344,8 @@ public long cost() { @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { // Delegate to partition-aware version for entire segment - return scorerSupplier(IndexSearcher.LeafReaderContextPartition.createForEntireSegment(context)); + return scorerSupplier( + IndexSearcher.LeafReaderContextPartition.createForEntireSegment(context)); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index cb6f0224aa89..c903c217bf9c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -170,8 +170,7 @@ public final Scorer scorer(LeafReaderContext context) throws IOException { * @see LeafReaderContextPartition * @since 10.1 */ - public ScorerSupplier scorerSupplier(LeafReaderContextPartition partition) - throws IOException { + public ScorerSupplier scorerSupplier(LeafReaderContextPartition partition) throws IOException { return scorerSupplier(partition.ctx); } diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index f9417130b75c..ce17e9b18aec 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -189,8 +189,7 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { } /** - * Partition-aware BufferAdder that filters docs to only include those within the specified - * range. + * Partition-aware BufferAdder that filters docs to only include those within the specified range. */ private record PartitionAwareBufferAdder(Buffer buffer, int minDocId, int maxDocId) implements BulkAdder { @@ -320,8 +319,7 @@ public DocIdSetBuilder(int maxDoc, Terms terms, int minDocId, int maxDocId) thro this(maxDoc, docCount, valueCount, 0, maxDoc); } - private DocIdSetBuilder( - int maxDoc, int docCount, long valueCount, int minDocId, int maxDocId) { + private DocIdSetBuilder(int maxDoc, int docCount, long valueCount, int minDocId, int maxDocId) { this.maxDoc = maxDoc; this.minDocId = minDocId; this.maxDocId = maxDocId; From 5b013e01bcb98e7b12c05c61cc6d241ff064afe1 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 06:24:48 -0800 Subject: [PATCH 06/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../apache/lucene/sandbox/search/QueryProfilerWeight.java | 8 ++++++++ .../lucene/sandbox/search/TestQueryProfilerWeight.java | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java index 6b5ed7defd8c..46fe612607f8 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java @@ -21,6 +21,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.BulkScorer; import org.apache.lucene.search.FilterWeight; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.ScorerSupplier; import org.apache.lucene.search.Weight; @@ -103,6 +104,13 @@ public void setTopLevelScoringClause() { }; } + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + // Delegate to the context method to ensure profiling logic is applied + return scorerSupplier(partition.ctx); + } + @Override public boolean isCacheable(LeafReaderContext ctx) { return false; diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerWeight.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerWeight.java index 41dc054a0756..5bb3191804a3 100644 --- a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerWeight.java +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerWeight.java @@ -177,7 +177,7 @@ public void testPropagateTopLevelScoringClause() throws IOException { Weight fakeWeight = new FakeWeight(query); QueryProfilerBreakdown profile = new QueryProfilerBreakdown(); QueryProfilerWeight profileWeight = new QueryProfilerWeight(fakeWeight, profile); - ScorerSupplier scorerSupplier = profileWeight.scorerSupplier(null); + ScorerSupplier scorerSupplier = profileWeight.scorerSupplier((LeafReaderContext) null); scorerSupplier.setTopLevelScoringClause(); assertEquals(42, scorerSupplier.cost()); } From ad778bb1ba4f360dd62479e3c625b7b28fee041b Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 06:49:12 -0800 Subject: [PATCH 07/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../test/org/apache/lucene/search/TestLRUQueryCache.java | 6 ++++++ .../lucene/search/join/GlobalOrdinalsWithScoreQuery.java | 6 ++++++ .../apache/lucene/search/join/ToChildBlockJoinQuery.java | 6 ++++++ .../apache/lucene/search/join/ToParentBlockJoinQuery.java | 6 ++++++ .../org/apache/lucene/tests/search/AssertingWeight.java | 7 +++++++ .../lucene/tests/search/RandomApproximationQuery.java | 6 ++++++ .../lucene/tests/search/TestBaseExplanationTestCase.java | 6 ++++++ 7 files changed, 43 insertions(+) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java index fb577202d12e..89c48fecd722 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java @@ -1445,6 +1445,12 @@ public long cost() { } }; } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } } public void testPropagateBulkScorer() throws IOException { diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java index 9b118d3cf2e6..67bd0ab4a365 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java @@ -233,6 +233,12 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti return new DefaultScorerSupplier(scorer); } + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } + @Override public boolean isCacheable(LeafReaderContext ctx) { // disable caching because this query relies on a top reader context diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java index 6fd0dc99e6d1..22b58d9289af 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java @@ -122,6 +122,12 @@ public ScorerSupplier scorerSupplier(LeafReaderContext readerContext) throws IOE return new DefaultScorerSupplier(scorer); } + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { ToChildBlockJoinScorer scorer = (ToChildBlockJoinScorer) scorer(context); diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java index 25b705b4efcd..9a553a84ce7b 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java @@ -188,6 +188,12 @@ public void setTopLevelScoringClause() { }; } + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { BlockJoinScorer scorer = (BlockJoinScorer) scorer(context); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java index f46158fb2b1a..866eefbb1c1e 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java @@ -23,6 +23,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.BulkScorer; import org.apache.lucene.search.FilterWeight; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Matches; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; @@ -113,4 +114,10 @@ public void setTopLevelScoringClause() { } }; } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java index 92f247e3754c..0865b5495d5b 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java @@ -100,6 +100,12 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti } return new DefaultScorerSupplier(scorer); } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } } private static class RandomApproximationScorer extends Scorer { diff --git a/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java b/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java index 6d184fbbe3fd..1215e136a0c8 100644 --- a/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java +++ b/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java @@ -128,5 +128,11 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti } return new DefaultScorerSupplier(scorer); } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } } } From 14f176188f34cecdf554d949d7b1b07fc21d21f3 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 06:49:28 -0800 Subject: [PATCH 08/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../org/apache/lucene/sandbox/search/QueryProfilerWeight.java | 1 - 1 file changed, 1 deletion(-) diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java index 46fe612607f8..b2ec75a127a9 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java @@ -107,7 +107,6 @@ public void setTopLevelScoringClause() { @Override public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) throws IOException { - // Delegate to the context method to ensure profiling logic is applied return scorerSupplier(partition.ctx); } From 6c59ef752a67161966d0b1efd81cad48134f54af Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 07:22:38 -0800 Subject: [PATCH 09/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../org/apache/lucene/tests/search/ScorerIndexSearcher.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java index 23c0ea466151..d2e70e777b38 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java @@ -81,12 +81,6 @@ public long cost() { } }; } - - @Override - public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) - throws IOException { - return scorerSupplier(partition.ctx); - } }; super.searchLeaf(ctx, minDocId, maxDocId, filterWeight, collector); From 476d16baee71e1b28f600235a31c15a9e9bb10b3 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 07:25:25 -0800 Subject: [PATCH 10/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../org/apache/lucene/tests/search/ScorerIndexSearcher.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java index d2e70e777b38..f743d16d39d0 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java @@ -81,6 +81,12 @@ public long cost() { } }; } + + @Override + public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) + throws IOException { + return scorerSupplier(partition.ctx); + } }; super.searchLeaf(ctx, minDocId, maxDocId, filterWeight, collector); From e24f9da98ec29b3fc631c08015abe27215caa9b0 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 3 Nov 2025 07:48:50 -0800 Subject: [PATCH 11/17] Fix the tests Signed-off-by: Prudhvi Godithi --- .../org/apache/lucene/tests/search/ScorerIndexSearcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java index f743d16d39d0..23c0ea466151 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java @@ -81,7 +81,7 @@ public long cost() { } }; } - + @Override public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition) throws IOException { From 785246f86bdc676bf8383d1488c5f3ae6cb0ed8d Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 10 Nov 2025 10:00:29 -0800 Subject: [PATCH 12/17] Test with partition defaults Signed-off-by: Prudhvi Godithi --- .../apache/lucene/search/IndexSearcher.java | 2 +- .../apache/lucene/util/DocIdSetBuilder.java | 91 ++----------------- 2 files changed, 11 insertions(+), 82 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 258e51b55e9e..daaa668d40d3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -321,7 +321,7 @@ public QueryCachingPolicy getQueryCachingPolicy() { * href="https://github.com/apache/lucene/issues/13745">the corresponding github issue. */ protected LeafSlice[] slices(List leaves) { - return slices(leaves, MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE, false); + return slices(leaves, MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE, true); } /** diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index ce17e9b18aec..fe175b333ed6 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -42,10 +42,7 @@ public final class DocIdSetBuilder { * @see DocIdSetBuilder#grow */ public sealed interface BulkAdder - permits FixedBitSetAdder, - BufferAdder, - PartitionAwareFixedBitSetAdder, - PartitionAwareBufferAdder { + permits PartitionAwareFixedBitSetAdder, PartitionAwareBufferAdder { void add(int doc); void add(IntsRef docs); @@ -55,37 +52,6 @@ public sealed interface BulkAdder void add(IntsRef docs, int docLowerBoundInclusive); } - private record FixedBitSetAdder(FixedBitSet bitSet) implements BulkAdder { - - @Override - public void add(int doc) { - bitSet.set(doc); - } - - @Override - public void add(IntsRef docs) { - for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { - bitSet.set(docs.ints[i]); - } - } - - @Override - public void add(DocIdSetIterator iterator) throws IOException { - iterator.nextDoc(); - iterator.intoBitSet(DocIdSetIterator.NO_MORE_DOCS, bitSet, 0); - } - - @Override - public void add(IntsRef docs, int docLowerBoundInclusive) { - for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { - int doc = docs.ints[i]; - if (doc >= docLowerBoundInclusive) { - bitSet.set(doc); - } - } - } - } - /** * Partition-aware FixedBitSetAdder that filters docs to only include those within the specified * range. Stores docs using partition-relative indices (doc - offset) to save memory. @@ -154,40 +120,6 @@ private static class Buffer { } } - private record BufferAdder(Buffer buffer) implements BulkAdder { - - @Override - public void add(int doc) { - buffer.array[buffer.length++] = doc; - } - - @Override - public void add(IntsRef docs) { - System.arraycopy(docs.ints, docs.offset, buffer.array, buffer.length, docs.length); - buffer.length += docs.length; - } - - @Override - public void add(DocIdSetIterator iterator) throws IOException { - int docID; - while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { - add(docID); - } - } - - @Override - public void add(IntsRef docs, int docLowerBoundInclusive) { - int index = buffer.length; - for (int i = docs.offset, to = docs.offset + docs.length; i < to; i++) { - int doc = docs.ints[i]; - if (doc >= docLowerBoundInclusive) { - buffer.array[index++] = doc; - } - } - buffer.length = index; - } - } - /** * Partition-aware BufferAdder that filters docs to only include those within the specified range. */ @@ -428,12 +360,10 @@ private int additionalCapacity(int numDocs) { private Buffer addBuffer(int len) { Buffer buffer = new Buffer(len); buffers.add(buffer); - // Use partition-aware adder if filtering to a specific doc ID range - if (minDocId > 0 || maxDocId < maxDoc) { - adder = new PartitionAwareBufferAdder(buffer, minDocId, maxDocId); - } else { - adder = new BufferAdder(buffer); - } + // Always use partition-aware adder to avoid megamorphic call sites + // For non-partitioned case (minDocId=0, maxDocId=maxDoc), the bounds check + // becomes a predictable branch that the JIT optimizes away + adder = new PartitionAwareBufferAdder(buffer, minDocId, maxDocId); totalAllocated += buffer.array.length; return buffer; } @@ -467,12 +397,11 @@ private void upgradeToBitSet() { this.bitSet = bitSet; this.counter = counter; this.buffers = null; - // Use partition-aware adder if filtering to a specific doc ID range - if (isPartition) { - this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId, minDocId); - } else { - this.adder = new FixedBitSetAdder(bitSet); - } + // Always use partition-aware adder to avoid megamorphic call sites + // For non-partitioned case, use offset=0 and bounds=[0, maxDoc) + // The JIT will optimize away the redundant checks for non-partitioned case + int offset = isPartition ? minDocId : 0; + this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId, offset); } /** Build a {@link DocIdSet} from the accumulated doc IDs. */ From a18b2cfcc490e96da78e47e4f2f9bc165c6646a1 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 12 Nov 2025 14:52:00 -0800 Subject: [PATCH 13/17] Add tests related to partition aware DocIdSetBuilder Signed-off-by: Prudhvi Godithi --- .../apache/lucene/util/DocIdSetBuilder.java | 30 +-- .../lucene/util/TestDocIdSetBuilder.java | 187 ++++++++++++++++++ 2 files changed, 188 insertions(+), 29 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index fe175b333ed6..b7d8e4785860 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -205,21 +205,6 @@ public DocIdSetBuilder(int maxDoc, PointValues values) throws IOException { this(maxDoc, values.getDocCount(), values.size()); } - /** - * Create a partition-aware {@link DocIdSetBuilder} instance that only accepts doc IDs within the - * specified range. This is useful for intra-segment concurrency where each partition only needs - * to collect docs within its assigned range. - * - * @param maxDoc the maximum doc ID in the segment - * @param minDocId the minimum doc ID (inclusive) to accept - * @param maxDocId the maximum doc ID (exclusive) to accept - * @param docCount estimated document count - * @param valueCount estimated value count - */ - public DocIdSetBuilder(int maxDoc, int minDocId, int maxDocId, int docCount, long valueCount) { - this(maxDoc, docCount, valueCount, minDocId, maxDocId); - } - /** * Create a partition-aware {@link DocIdSetBuilder} for {@link PointValues} that only accepts doc * IDs within the specified range. @@ -234,19 +219,6 @@ public DocIdSetBuilder(int maxDoc, PointValues values, int minDocId, int maxDocI this(maxDoc, values.getDocCount(), values.size(), minDocId, maxDocId); } - /** - * Create a partition-aware {@link DocIdSetBuilder} for {@link Terms} that only accepts doc IDs - * within the specified range. - * - * @param maxDoc the maximum doc ID in the segment - * @param terms the terms - * @param minDocId the minimum doc ID (inclusive) to accept - * @param maxDocId the maximum doc ID (exclusive) to accept - */ - public DocIdSetBuilder(int maxDoc, Terms terms, int minDocId, int maxDocId) throws IOException { - this(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), minDocId, maxDocId); - } - DocIdSetBuilder(int maxDoc, int docCount, long valueCount) { this(maxDoc, docCount, valueCount, 0, maxDoc); } @@ -259,7 +231,7 @@ private DocIdSetBuilder(int maxDoc, int docCount, long valueCount, int minDocId, this.multivalued = docCount < 0 || docCount != valueCount; if (docCount <= 0 || valueCount < 0) { // assume one value per doc, this means the cost will be overestimated - // if the docs are actually multi-valued + // if the docs are actually multivalued this.numValuesPerDoc = 1; } else { // otherwise compute from index stats diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java index 02fa1ff54cd8..b26f23d8509a 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java @@ -17,6 +17,9 @@ package org.apache.lucene.util; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import org.apache.lucene.index.PointValues; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; @@ -250,6 +253,190 @@ public void testLeverageStats() throws IOException { assertTrue(builder.multivalued); } + /** Test empty partition - no docs match */ + public void testPartitionEmpty() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSet result = builder.build(); + assertEquals(null, result); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + /** Test single doc in partition */ + public void testPartitionSingleDoc() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + builder.grow(1).add(5500); + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(5500, iter.nextDoc()); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + + /** Test that docs outside partition range are filtered */ + public void testPartitionFiltering() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(8); + adder.add(4999); + adder.add(6000); + adder.add(6001); + adder.add(100); + adder.add(9000); + adder.add(5000); + adder.add(5500); + adder.add(5999); + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(5000, iter.nextDoc()); + assertEquals(5500, iter.nextDoc()); + assertEquals(5999, iter.nextDoc()); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + /** Test advance() operation across partition boundaries */ + public void testPartitionAdvance() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(5); + adder.add(5000); + adder.add(5100); + adder.add(5500); + adder.add(5900); + adder.add(5999); + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(5500, iter.advance(5500)); + assertEquals(5900, iter.nextDoc()); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.advance(6000)); + } + + /** Test advance() before partition starts */ + public void testPartitionAdvanceBeforeStart() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(2); + adder.add(5100); + adder.add(5500); + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(5100, iter.advance(4000)); + assertEquals(5500, iter.nextDoc()); + } + + /** Test large partition with many docs crossing upgrade threshold */ + public void testPartitionUpgradeToBitSet() throws IOException { + final int maxDoc = 100000; + final int minDocId = 50000; + final int maxDocId = 60000; + PointValues values = new DummyPointValues(10000, 10000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(200); + for (int i = 0; i < 200; i++) { + adder.add(minDocId + i * 50); + } + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + for (int i = 0; i < 200; i++) { + int expected = minDocId + i * 50; + if (expected < maxDocId) { + assertEquals("Doc " + i, expected, iter.nextDoc()); + } + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + /** Test partition boundaries are respected after upgrade to bitset */ + public void testPartitionBitSetFiltering() throws IOException { + final int maxDoc = 10000; + final int minDocId = 5000; + final int maxDocId = 6000; + PointValues values = new DummyPointValues(1000, 1000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(500); + for (int i = 0; i < 500; i++) { + adder.add(4000 + i); + } + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + /** Test entire segment as partition (no filtering) */ + public void testFullSegmentPartition() throws IOException { + final int maxDoc = 1000; + final int minDocId = 0; + final int maxDocId = 1000; + PointValues values = new DummyPointValues(100, 100); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + DocIdSetBuilder.BulkAdder adder = builder.grow(3); + adder.add(0); + adder.add(500); + adder.add(999); + DocIdSet result = builder.build(); + assertNotNull(result); + DocIdSetIterator iter = result.iterator(); + assertEquals(0, iter.nextDoc()); + assertEquals(500, iter.nextDoc()); + assertEquals(999, iter.nextDoc()); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); + } + + /** Test random docs across partition boundaries */ + public void testPartitionRandom() throws IOException { + final int maxDoc = 100000; + final int minDocId = 30000; + final int maxDocId = 70000; + PointValues values = new DummyPointValues(40000, 40000); + DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc, values, minDocId, maxDocId); + int numDocs = 1000; + List docsInPartition = new ArrayList<>(); + DocIdSetBuilder.BulkAdder adder = builder.grow(numDocs); + for (int i = 0; i < numDocs; i++) { + int doc = random().nextInt(maxDoc); + adder.add(doc); + if (doc >= minDocId && doc < maxDocId) { + docsInPartition.add(doc); + } + } + Collections.sort(docsInPartition); + RoaringDocIdSet.Builder expected = new RoaringDocIdSet.Builder(maxDoc); + int prevDoc = -1; + for (int doc : docsInPartition) { + if (doc != prevDoc) { + expected.add(doc); + prevDoc = doc; + } + } + DocIdSet result = builder.build(); + DocIdSet expectedSet = expected.build(); + assertEquals(expectedSet, result); + } + public void testCostIsCorrectAfterBitsetUpgrade() throws IOException { final int maxDoc = 1000000; DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc); From 715d944592094cc0f892aad8bd43835b17506855 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 12 Nov 2025 14:59:22 -0800 Subject: [PATCH 14/17] Add tests related to partition aware DocIdSetBuilder Signed-off-by: Prudhvi Godithi --- .../src/test/org/apache/lucene/util/TestDocIdSetBuilder.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java index b26f23d8509a..bb1210044f23 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java @@ -282,7 +282,6 @@ public void testPartitionSingleDoc() throws IOException { assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); } - /** Test that docs outside partition range are filtered */ public void testPartitionFiltering() throws IOException { final int maxDoc = 10000; @@ -358,6 +357,7 @@ public void testPartitionUpgradeToBitSet() throws IOException { adder.add(minDocId + i * 50); } DocIdSet result = builder.build(); + assertTrue(result instanceof OffsetBitDocIdSet); assertNotNull(result); DocIdSetIterator iter = result.iterator(); for (int i = 0; i < 200; i++) { @@ -382,6 +382,7 @@ public void testPartitionBitSetFiltering() throws IOException { } DocIdSet result = builder.build(); assertNotNull(result); + assertTrue(result instanceof OffsetBitDocIdSet); DocIdSetIterator iter = result.iterator(); assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc()); } From fc5a728660a58608e42d8d3813169ab6094f97b5 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 12 Nov 2025 15:06:33 -0800 Subject: [PATCH 15/17] code cleanup Signed-off-by: Prudhvi Godithi --- .../apache/lucene/search/IndexSearcher.java | 2 -- .../apache/lucene/search/PointRangeQuery.java | 3 --- .../apache/lucene/util/DocIdSetBuilder.java | 18 +++--------------- 3 files changed, 3 insertions(+), 20 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index daaa668d40d3..a7f9fcee34b9 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -828,8 +828,6 @@ protected void searchLeaf( // continue with the following leaf return; } - - // Create partition object to pass to weight final LeafReaderContextPartition partition; if (minDocId == 0 && maxDocId == DocIdSetIterator.NO_MORE_DOCS) { partition = LeafReaderContextPartition.createForEntireSegment(ctx); diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java index fa0abcfeadad..92c42490fa22 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java @@ -299,8 +299,6 @@ public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition pa } else { return new ConstantScoreScorerSupplier(score(), scoreMode, reader.maxDoc()) { - // Create partition-aware DocIdSetBuilder that filters docs and uses partition-sized - // threshold final DocIdSetBuilder result = new DocIdSetBuilder( reader.maxDoc(), values, partition.minDocId, partition.maxDocId); @@ -343,7 +341,6 @@ public long cost() { @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { - // Delegate to partition-aware version for entire segment return scorerSupplier( IndexSearcher.LeafReaderContextPartition.createForEntireSegment(context)); } diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java index b7d8e4785860..7b019a7e68e2 100644 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java @@ -83,12 +83,10 @@ public void add(IntsRef docs) { @Override public void add(DocIdSetIterator iterator) throws IOException { - // Advance iterator to minDocId first int doc = iterator.nextDoc(); if (doc < minDocId) { doc = iterator.advance(minDocId); } - // Use optimized intoBitSet with partition boundaries and offset if (doc < maxDocId) { iterator.intoBitSet(maxDocId, bitSet, offset); } @@ -172,9 +170,8 @@ public void add(IntsRef docs, int docLowerBoundInclusive) { final boolean multivalued; final double numValuesPerDoc; - // Partition filtering support - filters docs to only include those within [minDocId, maxDocId) - private final int minDocId; // inclusive - private final int maxDocId; // exclusive + private final int minDocId; + private final int maxDocId; private List buffers = new ArrayList<>(); private int totalAllocated; // accumulated size of the allocated buffers @@ -231,7 +228,7 @@ private DocIdSetBuilder(int maxDoc, int docCount, long valueCount, int minDocId, this.multivalued = docCount < 0 || docCount != valueCount; if (docCount <= 0 || valueCount < 0) { // assume one value per doc, this means the cost will be overestimated - // if the docs are actually multivalued + // if the docs are actually multi-valued this.numValuesPerDoc = 1; } else { // otherwise compute from index stats @@ -244,7 +241,6 @@ private DocIdSetBuilder(int maxDoc, int docCount, long valueCount, int minDocId, // maxDoc >>> 7 is a good value if you want to save memory, lower values // such as maxDoc >>> 11 should provide faster building but at the expense // of using a full bitset even for quite sparse data - // // When filtering to a partition (minDocId > 0 or maxDocId < maxDoc), use the partition size // for threshold calculation to ensure the threshold scales correctly with the partition size boolean isPartition = (minDocId > 0 || maxDocId < maxDoc); @@ -332,9 +328,6 @@ private int additionalCapacity(int numDocs) { private Buffer addBuffer(int len) { Buffer buffer = new Buffer(len); buffers.add(buffer); - // Always use partition-aware adder to avoid megamorphic call sites - // For non-partitioned case (minDocId=0, maxDocId=maxDoc), the bounds check - // becomes a predictable branch that the JIT optimizes away adder = new PartitionAwareBufferAdder(buffer, minDocId, maxDocId); totalAllocated += buffer.array.length; return buffer; @@ -360,7 +353,6 @@ private void upgradeToBitSet() { int length = buffer.length; counter += length; for (int i = 0; i < length; ++i) { - // For partitions, convert absolute doc ID to partition-relative index int docId = array[i]; int bitIndex = isPartition ? (docId - minDocId) : docId; bitSet.set(bitIndex); @@ -369,9 +361,6 @@ private void upgradeToBitSet() { this.bitSet = bitSet; this.counter = counter; this.buffers = null; - // Always use partition-aware adder to avoid megamorphic call sites - // For non-partitioned case, use offset=0 and bounds=[0, maxDoc) - // The JIT will optimize away the redundant checks for non-partitioned case int offset = isPartition ? minDocId : 0; this.adder = new PartitionAwareFixedBitSetAdder(bitSet, minDocId, maxDocId, offset); } @@ -382,7 +371,6 @@ public DocIdSet build() { if (bitSet != null) { assert counter >= 0; final long cost = Math.round(counter / numValuesPerDoc); - // For partition-relative bitsets, wrap with offset to return absolute doc IDs boolean isPartition = (minDocId > 0 || maxDocId < maxDoc); if (isPartition) { From 88aa0b2391d38c79200cf1ac1e33840f08cd082b Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Sun, 16 Nov 2025 17:32:28 -0800 Subject: [PATCH 16/17] intra segment default to false Signed-off-by: Prudhvi Godithi --- .../core/src/java/org/apache/lucene/search/IndexSearcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index a7f9fcee34b9..4dffc3076e3c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -321,7 +321,7 @@ public QueryCachingPolicy getQueryCachingPolicy() { * href="https://github.com/apache/lucene/issues/13745">the corresponding github issue. */ protected LeafSlice[] slices(List leaves) { - return slices(leaves, MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE, true); + return slices(leaves, MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE, false); } /** From f2fa6af2f27956988fc84fb226678f8c5eb3ce22 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Mon, 17 Nov 2025 09:54:11 -0800 Subject: [PATCH 17/17] code optimize Signed-off-by: Prudhvi Godithi --- .../java/org/apache/lucene/search/IndexSearcher.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 4dffc3076e3c..aa6c62c7edef 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -828,14 +828,14 @@ protected void searchLeaf( // continue with the following leaf return; } - final LeafReaderContextPartition partition; + ScorerSupplier scorerSupplier; if (minDocId == 0 && maxDocId == DocIdSetIterator.NO_MORE_DOCS) { - partition = LeafReaderContextPartition.createForEntireSegment(ctx); + scorerSupplier = weight.scorerSupplier(ctx); } else { - partition = LeafReaderContextPartition.createFromAndTo(ctx, minDocId, maxDocId); + LeafReaderContextPartition partition = + LeafReaderContextPartition.createFromAndTo(ctx, minDocId, maxDocId); + scorerSupplier = weight.scorerSupplier(partition); } - - ScorerSupplier scorerSupplier = weight.scorerSupplier(partition); if (scorerSupplier != null) { scorerSupplier.setTopLevelScoringClause(); BulkScorer scorer = scorerSupplier.bulkScorer();