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/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..fa0abcfeadad 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,13 @@ 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..c903c217bf9c 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; /** @@ -149,6 +150,30 @@ 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 LeafReaderContextPartition
+ * @since 10.1
+ */
+ public ScorerSupplier scorerSupplier(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..ce17e9b18aec 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,59 @@ public void add(IntsRef docs, int docLowerBoundInclusive) {
}
}
+ /**
+ * 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.
+ *
+ * @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, int offset) implements BulkAdder {
+
+ @Override
+ public void add(int doc) {
+ if (doc >= minDocId && doc < maxDocId) {
+ bitSet.set(doc - offset);
+ }
+ }
+
+ @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 - offset);
+ }
+ }
+ }
+
+ @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);
+ }
+ }
+
+ @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 - offset);
+ }
+ }
+ }
+ }
+
private static class Buffer {
int[] array;
int length;
@@ -131,12 +188,62 @@ 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 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();
+ }
+}
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/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
index 6b5ed7defd8c..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
@@ -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,12 @@ public void setTopLevelScoringClause() {
};
}
+ @Override
+ public ScorerSupplier scorerSupplier(IndexSearcher.LeafReaderContextPartition partition)
+ throws IOException {
+ 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());
}
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/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);
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);
+ }
}
}