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 buffers = new ArrayList<>(); private int totalAllocated; // accumulated size of the allocated buffers @@ -166,8 +273,57 @@ 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 +340,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 +428,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; } @@ -279,20 +445,34 @@ 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; - this.adder = new FixedBitSetAdder(bitSet); + // 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); + } } /** Build a {@link DocIdSet} from the accumulated doc IDs. */ @@ -301,7 +481,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(); 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..bf767a7ca7f3 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/OffsetBitDocIdSet.java @@ -0,0 +1,62 @@ +/* + * 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 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(); + } +} 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); + } } }