From af1e18caea5df1ec8f9785925ae84b578edb4126 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 15 May 2025 21:39:42 +0200 Subject: [PATCH 1/6] Speed up exhaustive evaluation. This change helps speed up exhaustive evaluation of term queries, ie. calling `DocIdSetIterator#nextDoc()` then `Scorer#score()` in a loop. It helps in two ways: - Iteration of matching doc IDs gets a bit more efficient, especially in the case when a block of postings is encoded as a bit set. - Computation of scores now gets (auto-)vectorized. While this change doesn't help much when dynamic pruning kicks in, I'm hopeful that we can improve this in the future. --- .../lucene103/Lucene103PostingsReader.java | 90 +++++++++++++++++++ .../org/apache/lucene/index/PostingsEnum.java | 41 +++++++++ .../apache/lucene/search/BooleanScorer.java | 16 ++-- .../lucene/search/DocAndFreqBuffer.java | 62 +++++++++++++ .../lucene/search/DocAndScoreBuffer.java | 50 +++++++++++ .../lucene/search/MaxScoreBulkScorer.java | 24 +++-- .../java/org/apache/lucene/search/Scorer.java | 51 +++++++++++ .../org/apache/lucene/search/TermScorer.java | 51 +++++++++++ .../search/similarities/Similarity.java | 19 ++++ .../search/TestSimpleSearchEquivalence.java | 1 + .../tests/index/AssertingLeafReader.java | 16 ++++ .../lucene/tests/search/AssertingScorer.java | 18 ++++ 12 files changed, 426 insertions(+), 13 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java create mode 100644 lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java index 3ecaddef6174..73241880ca46 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java @@ -46,6 +46,7 @@ import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.internal.vectorization.VectorizationProvider; +import org.apache.lucene.search.DocAndFreqBuffer; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -1034,6 +1035,95 @@ public void intoBitSet(int upTo, FixedBitSet bitSet, int offset) throws IOExcept } } + @Override + public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + if (needsFreq == false) { + return super.nextPostings(upTo, reuse); + } + assert needsRefilling == false; + + if (doc >= upTo) { + reuse.size = 0; + return reuse; + } + + // Only return docs from the current block + reuse.grow(BLOCK_SIZE); + upTo = (int) Math.min(upTo, level0LastDocID + 1L); + + // Frequencies are decoded lazily, calling freq() makes sure that the freq block is decoded + freq(); + + int start, size; + + switch (encoding) { + case PACKED: + start = docBufferUpto - 1; + int end = computeBufferEndBoundary(upTo); + size = end - start; + System.arraycopy(docBuffer, start, reuse.docs, 0, size); + break; + case UNARY: + start = docBufferUpto - 1; + if (upTo > level0LastDocID) { + assert upTo == level0LastDocID + 1; + end = BLOCK_SIZE; + } else { + int numBits = upTo - docBitSetBase; + int lastWordIndex = numBits >> 6; + end = + docCumulativeWordPopCounts[lastWordIndex] + - Long.bitCount(docBitSet.getBits()[lastWordIndex] >>> numBits); + } + size = end - start; + + int firstWordIndex = (doc - docBitSetBase) >> 6; + int lastWordIndex = (upTo - 1 - docBitSetBase) >> 6; + + int size2 = + enumerateSetBits( + docBitSet.getBits()[firstWordIndex], firstWordIndex << 6, reuse.docs, 0); + // Remove docs from the first word that are before the current doc + int numDocsBeforeCurrentDoc = size2; + for (int i = 0; i < size2; ++i) { + if (reuse.docs[i] >= doc - docBitSetBase) { + numDocsBeforeCurrentDoc = i; + break; + } + } + size2 -= numDocsBeforeCurrentDoc; + System.arraycopy(reuse.docs, numDocsBeforeCurrentDoc, reuse.docs, 0, size2); + + for (int i = firstWordIndex + 1; i <= lastWordIndex; ++i) { + size2 = enumerateSetBits(docBitSet.getBits()[i], i << 6, reuse.docs, size2); + } + assert size2 >= size : size2 + " < " + size; + for (int i = 0; i < size; ++i) { + reuse.docs[i] += docBitSetBase; + } + break; + default: + throw new AssertionError(); + } + + assert size > 0; + System.arraycopy(freqBuffer, start, reuse.freqs, 0, size); + reuse.size = size; + + advance(upTo); + + return reuse; + } + + private static int enumerateSetBits(long word, int base, int[] dest, int offset) { + while (word != 0L) { + int ntz = Long.numberOfTrailingZeros(word); + dest[offset++] = base + ntz; + word ^= 1L << ntz; + } + return offset; + } + private int computeBufferEndBoundary(int upTo) { if (docBufferSize != 0 && docBuffer[docBufferSize - 1] < upTo) { // All docs in the buffer are under upTo diff --git a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java index 2cb0092aaa49..fedfeef5923a 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java @@ -17,6 +17,7 @@ package org.apache.lucene.index; import java.io.IOException; +import org.apache.lucene.search.DocAndFreqBuffer; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.BytesRef; @@ -97,4 +98,44 @@ protected PostingsEnum() {} * anything (neither members of the returned BytesRef nor bytes in the byte[]). */ public abstract BytesRef getPayload() throws IOException; + + /** + * Return a new batch of doc IDs and frequencies, starting at the current doc ID, and ending + * before {@code upTo}. + * + *

An empty return value indicates that there are no postings left between the current doc ID + * and {@code upTo}. + * + *

This method behaves as if implemented as below, which is the default implementation: + * + *

+   * int batchSize = 16;
+   * reuse.grow(batchSize);
+   * int size = 0;
+   * for (int doc = docID(); doc < upTo && size < batchSize; doc = nextDoc()) {
+   *   reuse.docs[size] = doc;
+   *   reuse.freqs[size] = freq();
+   *   ++size;
+   * }
+   * reuse.size = size;
+   * return reuse;
+   * 
+ * + *

NOTE: The returned {@link DocAndFreqBuffer} should not hold references to internal + * data structures. + * + * @lucene.internal + */ + public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + int batchSize = 16; + reuse.grow(batchSize); + int size = 0; + for (int doc = docID(); doc < upTo && size < batchSize; doc = nextDoc()) { + reuse.docs[size] = doc; + reuse.freqs[size] = freq(); + ++size; + } + reuse.size = size; + return reuse; + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index 0be42a2a3e7d..bf9c74a1eb82 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -81,6 +81,7 @@ public DisiWrapper get(int i) { final int minShouldMatch; final long cost; final boolean needsScores; + private final DocAndScoreBuffer docAndScoreBuffer = new DocAndScoreBuffer(); BooleanScorer(Collection scorers, int minShouldMatch, boolean needsScores) { if (minShouldMatch < 1 || minShouldMatch > scorers.size()) { @@ -135,22 +136,25 @@ private void scoreWindowIntoBitSetAndReplay( assert w.doc < max; DocIdSetIterator it = w.iterator; - int doc = w.doc; - if (doc < min) { - doc = it.advance(min); + if (w.doc < min) { + it.advance(min); } if (buckets == null) { // This doesn't apply live docs, so we'll need to apply them later it.intoBitSet(max, matching, base); } else { - for (; doc < max; doc = it.nextDoc()) { - if (acceptDocs == null || acceptDocs.get(doc)) { + for (DocAndScoreBuffer buffer = w.scorer.nextScores(max, acceptDocs, docAndScoreBuffer); + buffer.size > 0; + buffer = w.scorer.nextScores(max, acceptDocs, docAndScoreBuffer)) { + for (int index = 0; index < buffer.size; ++index) { + final int doc = buffer.docs[index]; + final float score = buffer.scores[index]; final int d = doc & MASK; matching.set(d); final Bucket bucket = buckets[d]; bucket.freq++; if (needsScores) { - bucket.score += w.scorable.score(); + bucket.score += score; } } } diff --git a/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java b/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java new file mode 100644 index 000000000000..d39a5bdea6a8 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.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.search; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IntsRef; + +/** + * Wrapper around parallel arrays storing doc IDs and their corresponding frequencies. + * + * @lucene.internal + */ +public final class DocAndFreqBuffer { + + /** Doc IDs */ + public int[] docs = IntsRef.EMPTY_INTS; + + /** Frequencies */ + public int[] freqs = IntsRef.EMPTY_INTS; + + /** Number of valid entries in the doc ID and frequency arrays. */ + public int size; + + /** Sole constructor. */ + public DocAndFreqBuffer() {} + + /** Grow both arrays to ensure that they can store at least the given number of entries. */ + public void grow(int minSize) { + if (docs.length < minSize) { + docs = ArrayUtil.grow(docs, minSize); + freqs = ArrayUtil.growExact(freqs, docs.length); + } + } + + /** Remove entries from this buffer if their bit is unset in the given {@link Bits}. */ + public void apply(Bits liveDocs) { + int newSize = 0; + for (int i = 0; i < size; ++i) { + if (liveDocs.get(docs[i])) { + docs[newSize] = docs[i]; + freqs[newSize] = freqs[i]; + newSize++; + } + } + this.size = newSize; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java new file mode 100644 index 000000000000..ccc2f0e4a2a2 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java @@ -0,0 +1,50 @@ +/* + * 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.search; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.IntsRef; + +/** + * Wrapper around parallel arrays storing doc IDs and their corresponding scores. + * + * @lucene.internal + */ +public final class DocAndScoreBuffer { + + private static final float[] EMPTY_FLOATS = new float[0]; + + /** Doc IDs */ + public int[] docs = IntsRef.EMPTY_INTS; + + /** Scores */ + public float[] scores = EMPTY_FLOATS; + + /** Number of valid entries in the doc ID and score arrays. */ + public int size; + + /** Sole constructor. */ + public DocAndScoreBuffer() {} + + /** Grow both arrays to ensure that they can store at least the given number of entries. */ + public void grow(int minSize) { + if (docs.length < minSize) { + docs = ArrayUtil.grow(docs, minSize); + scores = ArrayUtil.growExact(scores, docs.length); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index 40e44b7c4286..04d40da94dd7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -51,6 +51,8 @@ final class MaxScoreBulkScorer extends BulkScorer { private final long[] windowMatches = new long[FixedBitSet.bits2words(INNER_WINDOW_SIZE)]; private final double[] windowScores = new double[INNER_WINDOW_SIZE]; + private final DocAndScoreBuffer docAndScoreBuffer = new DocAndScoreBuffer(); + MaxScoreBulkScorer(int maxDoc, List scorers, Scorer filter) throws IOException { this.maxDoc = maxDoc; this.filter = filter == null ? null : new DisiWrapper(filter, false); @@ -218,12 +220,14 @@ private void scoreInnerWindowSingleEssentialClause( // single essential clause in this window, we can iterate it directly and skip the bitset. // this is a common case for 2-clauses queries - for (int doc = top.doc; doc < upTo; doc = top.iterator.nextDoc()) { - if (acceptDocs != null && acceptDocs.get(doc) == false) { - continue; + for (DocAndScoreBuffer buffer = top.scorer.nextScores(upTo, acceptDocs, docAndScoreBuffer); + buffer.size > 0; + buffer = top.scorer.nextScores(upTo, acceptDocs, docAndScoreBuffer)) { + for (int i = 0; i < buffer.size; ++i) { + scoreNonEssentialClauses(collector, buffer.docs[i], buffer.scores[i], firstEssentialScorer); } - scoreNonEssentialClauses(collector, doc, top.scorable.score(), firstEssentialScorer); } + top.doc = top.iterator.docID(); essentialQueue.updateTop(); } @@ -304,13 +308,19 @@ private void scoreInnerWindowMultipleEssentialClauses( // Collect matches of essential clauses into a bitset do { - for (int doc = top.doc; doc < innerWindowMax; doc = top.iterator.nextDoc()) { - if (acceptDocs == null || acceptDocs.get(doc)) { + for (DocAndScoreBuffer buffer = + top.scorer.nextScores(innerWindowMax, acceptDocs, docAndScoreBuffer); + buffer.size > 0; + buffer = top.scorer.nextScores(innerWindowMax, acceptDocs, docAndScoreBuffer)) { + for (int index = 0; index < buffer.size; ++index) { + final int doc = buffer.docs[index]; + final float score = buffer.scores[index]; final int i = doc - innerWindowMin; windowMatches[i >>> 6] |= 1L << i; - windowScores[i] += top.scorable.score(); + windowScores[i] += score; } } + top.doc = top.iterator.docID(); top = essentialQueue.updateTop(); } while (top.doc < innerWindowMax); diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 7da2420207ec..1f37b5dd1c8b 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -17,6 +17,7 @@ package org.apache.lucene.search; import java.io.IOException; +import org.apache.lucene.util.Bits; /** * Expert: Common scoring functionality for different types of queries. @@ -76,4 +77,54 @@ public int advanceShallow(int target) throws IOException { * {@link #advanceShallow(int) shallow-advanced} to included and {@code upTo} included. */ public abstract float getMaxScore(int upTo) throws IOException; + + /** + * Return a new batch of doc IDs and scores, starting at the current doc ID, and ending before + * {@code upTo}. + * + *

An empty return value indicates that there are no postings left between the current doc ID + * and {@code upTo}. + * + *

This method behaves as if implemented as below, which is the default implementation: + * + *

+   * int batchSize = 16;
+   * reuse.grow(batchSize);
+   * int size = 0;
+   * DocIdSetIterator iterator = iterator();
+   * for (int doc = docID(); doc < upTo && size < batchSize; doc = iterator.nextDoc()) {
+   *   if (liveDocs == null || liveDocs.get(doc)) {
+   *     reuse.docs[size] = doc;
+   *     reuse.scores[size] = score();
+   *     ++size;
+   *   }
+   * }
+   * reuse.size = size;
+   * return reuse;
+   * 
+ * + *

NOTE: The returned {@link DocAndScoreBuffer} should not hold references to internal + * data structures. + * + *

NOTE: In case this {@link Scorer} exposes a {@link #twoPhaseIterator() + * TwoPhaseIterator}, it should be positioned on a matching document before this method is called. + * + * @lucene.internal + */ + public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + throws IOException { + int batchSize = 16; + reuse.grow(batchSize); + int size = 0; + DocIdSetIterator iterator = iterator(); + for (int doc = docID(); doc < upTo && size < batchSize; doc = iterator.nextDoc()) { + if (liveDocs == null || liveDocs.get(doc)) { + reuse.docs[size] = doc; + reuse.scores[size] = score(); + ++size; + } + } + reuse.size = size; + return reuse; + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java index 4b53788f233e..a39ee6dba4bf 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java @@ -17,11 +17,15 @@ package org.apache.lucene.search; import java.io.IOException; +import java.util.Arrays; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SlowImpactsEnum; import org.apache.lucene.search.similarities.Similarity.SimScorer; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.LongsRef; /** * Expert: A Scorer for documents matching a Term. @@ -35,6 +39,8 @@ public final class TermScorer extends Scorer { private final NumericDocValues norms; private final ImpactsDISI impactsDisi; private final MaxScoreCache maxScoreCache; + private DocAndFreqBuffer docAndFreqBuffer; + private long[] normValues = LongsRef.EMPTY_LONGS; /** Construct a {@link TermScorer} that will iterate all documents. */ public TermScorer(PostingsEnum postingsEnum, SimScorer scorer, NumericDocValues norms) { @@ -120,4 +126,49 @@ public void setMinCompetitiveScore(float minScore) { impactsDisi.setMinCompetitiveScore(minScore); } } + + @Override + public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + throws IOException { + if (docAndFreqBuffer == null) { + docAndFreqBuffer = new DocAndFreqBuffer(); + } + + DocAndFreqBuffer docAndFreqBuffer; + for (; ; ) { + docAndFreqBuffer = postingsEnum.nextPostings(upTo, this.docAndFreqBuffer); + if (liveDocs != null && docAndFreqBuffer.size != 0) { + // An empty return value indicates that there are no more docs before upTo. We may be + // unlucky, and there are docs left, but all docs from the current batch happen to be marked + // as deleted. So we need to iterate until we find a batch that has at least one non-deleted + // doc. + docAndFreqBuffer.apply(liveDocs); + if (docAndFreqBuffer.size == 0) { + continue; + } + } + break; + } + + int size = docAndFreqBuffer.size; + normValues = ArrayUtil.grow(normValues, size); + if (norms == null) { + Arrays.fill(normValues, 0, size, 1L); + } else { + for (int i = 0; i < size; ++i) { + if (norms.advanceExact(docAndFreqBuffer.docs[i])) { + normValues[i] = norms.longValue(); + } else { + normValues[i] = 1L; + } + } + } + + reuse.grow(size); + reuse.size = size; + System.arraycopy(docAndFreqBuffer.docs, 0, reuse.docs, 0, size); + scorer.score(size, docAndFreqBuffer.freqs, normValues, reuse.scores); + + return reuse; + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java index 83582e44e25f..d0c1c0f99d78 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java @@ -208,6 +208,25 @@ protected SimScorer() {} */ public abstract float score(float freq, long norm); + /** + * Batch-score documents. This method scores {@code size} documents at once. The default + * implementation can be found below: + * + *

+     * for (int i = 0; i < size; ++i) {
+     *   scores[i] = score(freqs[i], norms[i]);
+     * }
+     * 
+ * + * @see #score(float, long) + * @lucene.internal + */ + public void score(int size, int[] freqs, long[] norms, float[] scores) { + for (int i = 0; i < size; ++i) { + scores[i] = score(freqs[i], norms[i]); + } + } + /** * Explain the score for a single document * diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java b/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java index 30680d0cf005..515b353c636b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java @@ -37,6 +37,7 @@ public void testTermVersusBooleanOr() throws Exception { BooleanQuery.Builder q2 = new BooleanQuery.Builder(); q2.add(new TermQuery(t1), Occur.SHOULD); q2.add(new TermQuery(t2), Occur.SHOULD); + assertSubsetOf(q1, q2.build()); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java index c56f880c0893..6a8c60edcb1a 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java @@ -50,6 +50,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.internal.tests.IndexPackageAccess; import org.apache.lucene.internal.tests.TestSecrets; +import org.apache.lucene.search.DocAndFreqBuffer; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -578,6 +579,21 @@ public BytesRef getPayload() throws IOException { return payload; } + @Override + public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + assert state != DocsEnumState.START : "nextPostings() called before nextDoc()/advance()"; + DocAndFreqBuffer result = in.nextPostings(upTo, reuse); + doc = in.docID(); + if (doc == DocIdSetIterator.NO_MORE_DOCS) { + state = DocsEnumState.FINISHED; + positionMax = 0; + } else { + state = DocsEnumState.ITERATING; + positionMax = super.freq(); + } + return result; + } + void reset() { state = DocsEnumState.START; doc = in.docID(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java index 0e9d8bfa9ba9..08241b75273c 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java @@ -20,11 +20,13 @@ import java.util.Collection; import java.util.Collections; import java.util.Random; +import org.apache.lucene.search.DocAndScoreBuffer; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FilterDocIdSetIterator; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TwoPhaseIterator; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.FixedBitSet; /** Wraps a Scorer with additional checks */ @@ -278,4 +280,20 @@ public String toString() { } }; } + + @Override + public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + throws IOException { + assert doc != -1; + DocAndScoreBuffer result = in.nextScores(upTo, liveDocs, reuse); + if (doc != in.iterator().docID()) { + doc = in.iterator().docID(); + if (doc == DocIdSetIterator.NO_MORE_DOCS) { + state = IteratorState.FINISHED; + } else { + state = IteratorState.ITERATING; + } + } + return result; + } } From 5b2bb4d5fac34efd6b65fa4146bc2cfe76880920 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 20 May 2025 21:54:41 +0200 Subject: [PATCH 2/6] Simplify. --- .../lucene103/Lucene103PostingsReader.java | 83 +++++-------------- .../apache/lucene/index/CompositeReader.java | 2 +- .../org/apache/lucene/index/IndexReader.java | 2 +- .../org/apache/lucene/index/LeafReader.java | 2 +- .../org/apache/lucene/index/PostingsEnum.java | 40 ++++----- .../apache/lucene/search/BooleanScorer.java | 29 ++++--- .../lucene/search/DocAndFreqBuffer.java | 6 +- .../lucene/search/DocAndScoreBuffer.java | 6 +- .../lucene/search/MaxScoreBulkScorer.java | 27 +++--- .../java/org/apache/lucene/search/Scorer.java | 30 +++---- .../org/apache/lucene/search/TermScorer.java | 19 +++-- .../search/similarities/Similarity.java | 19 ----- .../tests/index/AssertingLeafReader.java | 5 +- .../lucene/tests/search/AssertingScorer.java | 5 +- 14 files changed, 113 insertions(+), 162 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java index 73241880ca46..fac90c25da32 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene103/Lucene103PostingsReader.java @@ -1036,92 +1036,47 @@ public void intoBitSet(int upTo, FixedBitSet bitSet, int offset) throws IOExcept } @Override - public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + public void nextPostings(int upTo, DocAndFreqBuffer buffer) throws IOException { + assert needsRefilling == false; + if (needsFreq == false) { - return super.nextPostings(upTo, reuse); + super.nextPostings(upTo, buffer); + return; } - assert needsRefilling == false; + buffer.size = 0; if (doc >= upTo) { - reuse.size = 0; - return reuse; + return; } // Only return docs from the current block - reuse.grow(BLOCK_SIZE); + buffer.growNoCopy(BLOCK_SIZE); upTo = (int) Math.min(upTo, level0LastDocID + 1L); // Frequencies are decoded lazily, calling freq() makes sure that the freq block is decoded freq(); - int start, size; - + int start = docBufferUpto - 1; + buffer.size = 0; switch (encoding) { case PACKED: - start = docBufferUpto - 1; int end = computeBufferEndBoundary(upTo); - size = end - start; - System.arraycopy(docBuffer, start, reuse.docs, 0, size); + buffer.size = end - start; + System.arraycopy(docBuffer, start, buffer.docs, 0, buffer.size); break; case UNARY: - start = docBufferUpto - 1; - if (upTo > level0LastDocID) { - assert upTo == level0LastDocID + 1; - end = BLOCK_SIZE; - } else { - int numBits = upTo - docBitSetBase; - int lastWordIndex = numBits >> 6; - end = - docCumulativeWordPopCounts[lastWordIndex] - - Long.bitCount(docBitSet.getBits()[lastWordIndex] >>> numBits); - } - size = end - start; - - int firstWordIndex = (doc - docBitSetBase) >> 6; - int lastWordIndex = (upTo - 1 - docBitSetBase) >> 6; - - int size2 = - enumerateSetBits( - docBitSet.getBits()[firstWordIndex], firstWordIndex << 6, reuse.docs, 0); - // Remove docs from the first word that are before the current doc - int numDocsBeforeCurrentDoc = size2; - for (int i = 0; i < size2; ++i) { - if (reuse.docs[i] >= doc - docBitSetBase) { - numDocsBeforeCurrentDoc = i; - break; - } - } - size2 -= numDocsBeforeCurrentDoc; - System.arraycopy(reuse.docs, numDocsBeforeCurrentDoc, reuse.docs, 0, size2); - - for (int i = firstWordIndex + 1; i <= lastWordIndex; ++i) { - size2 = enumerateSetBits(docBitSet.getBits()[i], i << 6, reuse.docs, size2); - } - assert size2 >= size : size2 + " < " + size; - for (int i = 0; i < size; ++i) { - reuse.docs[i] += docBitSetBase; - } + docBitSet.forEach( + doc - docBitSetBase, + upTo - docBitSetBase, + docBitSetBase, + d -> buffer.docs[buffer.size++] = d); break; - default: - throw new AssertionError(); } - assert size > 0; - System.arraycopy(freqBuffer, start, reuse.freqs, 0, size); - reuse.size = size; + assert buffer.size > 0; + System.arraycopy(freqBuffer, start, buffer.freqs, 0, buffer.size); advance(upTo); - - return reuse; - } - - private static int enumerateSetBits(long word, int base, int[] dest, int offset) { - while (word != 0L) { - int ntz = Long.numberOfTrailingZeros(word); - dest[offset++] = base + ntz; - word ^= 1L << ntz; - } - return offset; } private int computeBufferEndBoundary(int upTo) { diff --git a/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java b/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java index 4b35ba6cd7c9..ccc454c59944 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java @@ -44,7 +44,7 @@ * synchronization, you should not synchronize on the IndexReader instance; use * your own (non-Lucene) objects instead. */ -public abstract non-sealed class CompositeReader extends IndexReader { +public abstract class CompositeReader extends IndexReader { private volatile CompositeReaderContext readerContext = null; // lazy init diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java index 8e965ee8099c..e23efc23ece2 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java @@ -63,7 +63,7 @@ * synchronization, you should not synchronize on the IndexReader instance; use * your own (non-Lucene) objects instead. */ -public abstract sealed class IndexReader implements Closeable permits CompositeReader, LeafReader { +public abstract class IndexReader implements Closeable { private boolean closed = false; private boolean closedByChild = false; diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java index 0f39d1ae1e8d..eea6c317b9a7 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java @@ -44,7 +44,7 @@ * synchronization, you should not synchronize on the IndexReader instance; use * your own (non-Lucene) objects instead. */ -public abstract non-sealed class LeafReader extends IndexReader { +public abstract class LeafReader extends IndexReader { private final LeafReaderContext readerContext = new LeafReaderContext(this); diff --git a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java index fedfeef5923a..d87e1a51b5ee 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java @@ -100,42 +100,44 @@ protected PostingsEnum() {} public abstract BytesRef getPayload() throws IOException; /** - * Return a new batch of doc IDs and frequencies, starting at the current doc ID, and ending - * before {@code upTo}. + * Fill a buffer of doc IDs and frequencies with some number of doc IDs and their corresponding + * frequencies, starting at the current doc ID, and ending before {@code upTo}. * - *

An empty return value indicates that there are no postings left between the current doc ID - * and {@code upTo}. + *

An empty buffer after this method returns indicates that there are no postings left between + * the current doc ID and {@code upTo}. * - *

This method behaves as if implemented as below, which is the default implementation: + *

Implementations should ideally fill the buffer with a number of entries comprised between 8 + * and a couple hundreds, to keep heap requirements contained, while still being large enough to + * enable operations on the buffer to auto-vectorize efficiently. + * + *

The default implementation is provided below: * *

-   * int batchSize = 16;
-   * reuse.grow(batchSize);
+   * int batchSize = 16; // arbitrary
+   * buffer.growNoCopy(batchSize);
    * int size = 0;
    * for (int doc = docID(); doc < upTo && size < batchSize; doc = nextDoc()) {
-   *   reuse.docs[size] = doc;
-   *   reuse.freqs[size] = freq();
+   *   buffer.docs[size] = doc;
+   *   buffer.freqs[size] = freq();
    *   ++size;
    * }
-   * reuse.size = size;
-   * return reuse;
+   * buffer.size = size;
    * 
* - *

NOTE: The returned {@link DocAndFreqBuffer} should not hold references to internal + *

NOTE: The provided {@link DocAndFreqBuffer} should not hold references to internal * data structures. * * @lucene.internal */ - public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { - int batchSize = 16; - reuse.grow(batchSize); + public void nextPostings(int upTo, DocAndFreqBuffer buffer) throws IOException { + int batchSize = 16; // arbitrary + buffer.growNoCopy(batchSize); int size = 0; for (int doc = docID(); doc < upTo && size < batchSize; doc = nextDoc()) { - reuse.docs[size] = doc; - reuse.freqs[size] = freq(); + buffer.docs[size] = doc; + buffer.freqs[size] = freq(); ++size; } - reuse.size = size; - return reuse; + buffer.size = size; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index bf9c74a1eb82..1f4dd4f80ce1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -139,23 +139,32 @@ private void scoreWindowIntoBitSetAndReplay( if (w.doc < min) { it.advance(min); } - if (buckets == null) { + if (buckets == null) { // means minShouldMatch=1 and scores are not needed // This doesn't apply live docs, so we'll need to apply them later it.intoBitSet(max, matching, base); + } else if (needsScores) { + for (w.scorer.nextDocsAndScores(max, acceptDocs, docAndScoreBuffer); + docAndScoreBuffer.size > 0; + w.scorer.nextDocsAndScores(max, acceptDocs, docAndScoreBuffer)) { + for (int index = 0; index < docAndScoreBuffer.size; ++index) { + final int doc = docAndScoreBuffer.docs[index]; + final float score = docAndScoreBuffer.scores[index]; + final int d = doc & MASK; + matching.set(d); + final Bucket bucket = buckets[d]; + bucket.freq++; + bucket.score += score; + } + } } else { - for (DocAndScoreBuffer buffer = w.scorer.nextScores(max, acceptDocs, docAndScoreBuffer); - buffer.size > 0; - buffer = w.scorer.nextScores(max, acceptDocs, docAndScoreBuffer)) { - for (int index = 0; index < buffer.size; ++index) { - final int doc = buffer.docs[index]; - final float score = buffer.scores[index]; + // Scores are not needed but we need to keep track of freqs to know which hits match + assert minShouldMatch > 1; + for (int doc = it.docID(); doc < max; doc = it.nextDoc()) { + if (acceptDocs == null || acceptDocs.get(doc)) { final int d = doc & MASK; matching.set(d); final Bucket bucket = buckets[d]; bucket.freq++; - if (needsScores) { - bucket.score += score; - } } } } diff --git a/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java b/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java index d39a5bdea6a8..3e45d3cf429f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java +++ b/lucene/core/src/java/org/apache/lucene/search/DocAndFreqBuffer.java @@ -40,10 +40,10 @@ public final class DocAndFreqBuffer { public DocAndFreqBuffer() {} /** Grow both arrays to ensure that they can store at least the given number of entries. */ - public void grow(int minSize) { + public void growNoCopy(int minSize) { if (docs.length < minSize) { - docs = ArrayUtil.grow(docs, minSize); - freqs = ArrayUtil.growExact(freqs, docs.length); + docs = ArrayUtil.growNoCopy(docs, minSize); + freqs = new int[docs.length]; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java index ccc2f0e4a2a2..5daef38c0ff6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java +++ b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreBuffer.java @@ -41,10 +41,10 @@ public final class DocAndScoreBuffer { public DocAndScoreBuffer() {} /** Grow both arrays to ensure that they can store at least the given number of entries. */ - public void grow(int minSize) { + public void growNoCopy(int minSize) { if (docs.length < minSize) { - docs = ArrayUtil.grow(docs, minSize); - scores = ArrayUtil.growExact(scores, docs.length); + docs = ArrayUtil.growNoCopy(docs, minSize); + scores = new float[docs.length]; } } } diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index 04d40da94dd7..847fdb010a96 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -220,11 +220,15 @@ private void scoreInnerWindowSingleEssentialClause( // single essential clause in this window, we can iterate it directly and skip the bitset. // this is a common case for 2-clauses queries - for (DocAndScoreBuffer buffer = top.scorer.nextScores(upTo, acceptDocs, docAndScoreBuffer); - buffer.size > 0; - buffer = top.scorer.nextScores(upTo, acceptDocs, docAndScoreBuffer)) { - for (int i = 0; i < buffer.size; ++i) { - scoreNonEssentialClauses(collector, buffer.docs[i], buffer.scores[i], firstEssentialScorer); + for (top.scorer.nextDocsAndScores(upTo, acceptDocs, docAndScoreBuffer); + docAndScoreBuffer.size > 0; + top.scorer.nextDocsAndScores(upTo, acceptDocs, docAndScoreBuffer)) { + for (int i = 0; i < docAndScoreBuffer.size; ++i) { + scoreNonEssentialClauses( + collector, + docAndScoreBuffer.docs[i], + docAndScoreBuffer.scores[i], + firstEssentialScorer); } } @@ -308,13 +312,12 @@ private void scoreInnerWindowMultipleEssentialClauses( // Collect matches of essential clauses into a bitset do { - for (DocAndScoreBuffer buffer = - top.scorer.nextScores(innerWindowMax, acceptDocs, docAndScoreBuffer); - buffer.size > 0; - buffer = top.scorer.nextScores(innerWindowMax, acceptDocs, docAndScoreBuffer)) { - for (int index = 0; index < buffer.size; ++index) { - final int doc = buffer.docs[index]; - final float score = buffer.scores[index]; + for (top.scorer.nextDocsAndScores(innerWindowMax, acceptDocs, docAndScoreBuffer); + docAndScoreBuffer.size > 0; + top.scorer.nextDocsAndScores(innerWindowMax, acceptDocs, docAndScoreBuffer)) { + for (int index = 0; index < docAndScoreBuffer.size; ++index) { + final int doc = docAndScoreBuffer.docs[index]; + final float score = docAndScoreBuffer.scores[index]; final int i = doc - innerWindowMin; windowMatches[i >>> 6] |= 1L << i; windowScores[i] += score; diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 1f37b5dd1c8b..1c4c8794e8cd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -85,25 +85,28 @@ public int advanceShallow(int target) throws IOException { *

An empty return value indicates that there are no postings left between the current doc ID * and {@code upTo}. * - *

This method behaves as if implemented as below, which is the default implementation: + *

Implementations should ideally fill the buffer with a number of entries comprised between 8 + * and a couple hundreds, to keep heap requirements contained, while still being large enough to + * enable operations on the buffer to auto-vectorize efficiently. + * + *

The default implementation is provided below: * *

-   * int batchSize = 16;
-   * reuse.grow(batchSize);
+   * int batchSize = 16; // arbitrary
+   * buffer.growNoCopy(batchSize);
    * int size = 0;
    * DocIdSetIterator iterator = iterator();
    * for (int doc = docID(); doc < upTo && size < batchSize; doc = iterator.nextDoc()) {
    *   if (liveDocs == null || liveDocs.get(doc)) {
-   *     reuse.docs[size] = doc;
-   *     reuse.scores[size] = score();
+   *     buffer.docs[size] = doc;
+   *     buffer.scores[size] = score();
    *     ++size;
    *   }
    * }
    * reuse.size = size;
-   * return reuse;
    * 
* - *

NOTE: The returned {@link DocAndScoreBuffer} should not hold references to internal + *

NOTE: The provided {@link DocAndScoreBuffer} should not hold references to internal * data structures. * *

NOTE: In case this {@link Scorer} exposes a {@link #twoPhaseIterator() @@ -111,20 +114,19 @@ public int advanceShallow(int target) throws IOException { * * @lucene.internal */ - public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) throws IOException { - int batchSize = 16; - reuse.grow(batchSize); + int batchSize = 16; // arbitrary + buffer.growNoCopy(batchSize); int size = 0; DocIdSetIterator iterator = iterator(); for (int doc = docID(); doc < upTo && size < batchSize; doc = iterator.nextDoc()) { if (liveDocs == null || liveDocs.get(doc)) { - reuse.docs[size] = doc; - reuse.scores[size] = score(); + buffer.docs[size] = doc; + buffer.scores[size] = score(); ++size; } } - reuse.size = size; - return reuse; + buffer.size = size; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java index a39ee6dba4bf..8668b8ec63e8 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java @@ -128,15 +128,14 @@ public void setMinCompetitiveScore(float minScore) { } @Override - public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) throws IOException { if (docAndFreqBuffer == null) { docAndFreqBuffer = new DocAndFreqBuffer(); } - DocAndFreqBuffer docAndFreqBuffer; for (; ; ) { - docAndFreqBuffer = postingsEnum.nextPostings(upTo, this.docAndFreqBuffer); + postingsEnum.nextPostings(upTo, docAndFreqBuffer); if (liveDocs != null && docAndFreqBuffer.size != 0) { // An empty return value indicates that there are no more docs before upTo. We may be // unlucky, and there are docs left, but all docs from the current batch happen to be marked @@ -164,11 +163,13 @@ public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer r } } - reuse.grow(size); - reuse.size = size; - System.arraycopy(docAndFreqBuffer.docs, 0, reuse.docs, 0, size); - scorer.score(size, docAndFreqBuffer.freqs, normValues, reuse.scores); - - return reuse; + buffer.growNoCopy(size); + buffer.size = size; + System.arraycopy(docAndFreqBuffer.docs, 0, buffer.docs, 0, size); + for (int i = 0; i < size; ++i) { + // Unless SimScorer#score is megamorphic, SimScorer#score should inline and (part of) score + // computations should auto-vectorize. + buffer.scores[i] = scorer.score(docAndFreqBuffer.freqs[i], normValues[i]); + } } } diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java index d0c1c0f99d78..83582e44e25f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java @@ -208,25 +208,6 @@ protected SimScorer() {} */ public abstract float score(float freq, long norm); - /** - * Batch-score documents. This method scores {@code size} documents at once. The default - * implementation can be found below: - * - *

-     * for (int i = 0; i < size; ++i) {
-     *   scores[i] = score(freqs[i], norms[i]);
-     * }
-     * 
- * - * @see #score(float, long) - * @lucene.internal - */ - public void score(int size, int[] freqs, long[] norms, float[] scores) { - for (int i = 0; i < size; ++i) { - scores[i] = score(freqs[i], norms[i]); - } - } - /** * Explain the score for a single document * diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java index 6a8c60edcb1a..f252df0dd02b 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java @@ -580,9 +580,9 @@ public BytesRef getPayload() throws IOException { } @Override - public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + public void nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { assert state != DocsEnumState.START : "nextPostings() called before nextDoc()/advance()"; - DocAndFreqBuffer result = in.nextPostings(upTo, reuse); + in.nextPostings(upTo, reuse); doc = in.docID(); if (doc == DocIdSetIterator.NO_MORE_DOCS) { state = DocsEnumState.FINISHED; @@ -591,7 +591,6 @@ public DocAndFreqBuffer nextPostings(int upTo, DocAndFreqBuffer reuse) throws IO state = DocsEnumState.ITERATING; positionMax = super.freq(); } - return result; } void reset() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java index 08241b75273c..90b775e72f07 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java @@ -282,10 +282,10 @@ public String toString() { } @Override - public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer reuse) + public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) throws IOException { assert doc != -1; - DocAndScoreBuffer result = in.nextScores(upTo, liveDocs, reuse); + in.nextDocsAndScores(upTo, liveDocs, buffer); if (doc != in.iterator().docID()) { doc = in.iterator().docID(); if (doc == DocIdSetIterator.NO_MORE_DOCS) { @@ -294,6 +294,5 @@ public DocAndScoreBuffer nextScores(int upTo, Bits liveDocs, DocAndScoreBuffer r state = IteratorState.ITERATING; } } - return result; } } From aa24297d188ae57af94043ba0049e41a4dcdd14d Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 20 May 2025 21:55:57 +0200 Subject: [PATCH 3/6] CHANGES --- lucene/CHANGES.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f9f036eb1040..7ad0c916172a 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -111,6 +111,9 @@ Optimizations * GITHUB#14609: Optimizes PointRangeQuery by rewriting to MatchAllDocsQuery/FieldExistsQuery/MatchNoDocsQuery if all docs in index are contained or excluded (Elliott Bradshaw) +* GITHUB#14679: Optimize exhaustive evaluation of disjunctive queries. + (Adrien Grand) + Bug Fixes --------------------- (No changes) From fd97a02273c57bfc9e72cee49a54e818820d2b93 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 20 May 2025 21:57:15 +0200 Subject: [PATCH 4/6] Fix name. --- lucene/core/src/java/org/apache/lucene/search/Scorer.java | 2 +- .../org/apache/lucene/tests/index/AssertingLeafReader.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 1c4c8794e8cd..35d587b526ff 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -103,7 +103,7 @@ public int advanceShallow(int target) throws IOException { * ++size; * } * } - * reuse.size = size; + * buffer.size = size; * * *

NOTE: The provided {@link DocAndScoreBuffer} should not hold references to internal diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java index f252df0dd02b..cb9d83e8ac39 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java @@ -580,9 +580,9 @@ public BytesRef getPayload() throws IOException { } @Override - public void nextPostings(int upTo, DocAndFreqBuffer reuse) throws IOException { + public void nextPostings(int upTo, DocAndFreqBuffer buffer) throws IOException { assert state != DocsEnumState.START : "nextPostings() called before nextDoc()/advance()"; - in.nextPostings(upTo, reuse); + in.nextPostings(upTo, buffer); doc = in.docID(); if (doc == DocIdSetIterator.NO_MORE_DOCS) { state = DocsEnumState.FINISHED; From 38338045a0695e3e72be1e9d081acf0376ea3025 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 20 May 2025 22:01:22 +0200 Subject: [PATCH 5/6] Improve docs. --- .../core/src/java/org/apache/lucene/index/PostingsEnum.java | 4 +++- lucene/core/src/java/org/apache/lucene/search/Scorer.java | 3 ++- .../org/apache/lucene/search/TestSimpleSearchEquivalence.java | 1 - 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java index d87e1a51b5ee..8e82e0a6d696 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java @@ -101,7 +101,9 @@ protected PostingsEnum() {} /** * Fill a buffer of doc IDs and frequencies with some number of doc IDs and their corresponding - * frequencies, starting at the current doc ID, and ending before {@code upTo}. + * frequencies, starting at the current doc ID, and ending before {@code upTo}. Because it starts + * on the current doc ID, it is illegal to call this method if the {@link #docID() current doc ID} + * is {@code -1}. * *

An empty buffer after this method returns indicates that there are no postings left between * the current doc ID and {@code upTo}. diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 35d587b526ff..64db31ba9fdd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -80,7 +80,8 @@ public int advanceShallow(int target) throws IOException { /** * Return a new batch of doc IDs and scores, starting at the current doc ID, and ending before - * {@code upTo}. + * {@code upTo}. Because it starts on the current doc ID, it is illegal to call this method if the + * {@link #docID() current doc ID} is {@code -1}. * *

An empty return value indicates that there are no postings left between the current doc ID * and {@code upTo}. diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java b/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java index 515b353c636b..30680d0cf005 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSimpleSearchEquivalence.java @@ -37,7 +37,6 @@ public void testTermVersusBooleanOr() throws Exception { BooleanQuery.Builder q2 = new BooleanQuery.Builder(); q2.add(new TermQuery(t1), Occur.SHOULD); q2.add(new TermQuery(t2), Occur.SHOULD); - assertSubsetOf(q1, q2.build()); } From 36d4efaefa2f992917e323d49645c6e2597b0771 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 20 May 2025 14:41:49 +0200 Subject: [PATCH 6/6] Speed up conjunctive queries that need scores. Calls to `DocIdSetIterator#nextDoc`, `DocIdSetIterator#advance` and `SimScorer#score` are currently interleaved and include lots of conditionals. This builds up on #14679 and refactors the code a bit to make it eligible to auto-vectorization and better pipelining. This effectively speeds up conjunctive queries (e.g. `AndHighHigh`) but also disjunctive queries that run as conjunctive queries in practice (e.g. `OrHighHigh`). --- .../search/BlockMaxConjunctionBulkScorer.java | 121 +++++------------- .../lucene/search/ConstantScoreScorer.java | 24 ++++ .../lucene/search/DocAndScoreAccBuffer.java | 60 +++++++++ .../lucene/search/MaxScoreBulkScorer.java | 68 +++------- .../java/org/apache/lucene/search/Scorer.java | 23 ++++ .../org/apache/lucene/search/ScorerUtil.java | 19 +++ .../org/apache/lucene/search/TermScorer.java | 33 +++++ .../lucene/tests/search/AssertingScorer.java | 14 ++ 8 files changed, 224 insertions(+), 138 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/search/DocAndScoreAccBuffer.java diff --git a/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionBulkScorer.java index fd89153ffe6d..09910c752ed3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionBulkScorer.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.lucene.search.Weight.DefaultBulkScorer; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.MathUtil; /** * BulkScorer implementation of {@link BlockMaxConjunctionScorer} that focuses on top-level @@ -36,13 +35,13 @@ final class BlockMaxConjunctionBulkScorer extends BulkScorer { private final Scorer[] scorers; - private final Scorable[] scorables; private final DocIdSetIterator[] iterators; - private final DocIdSetIterator lead1, lead2; - private final Scorable scorer1, scorer2; + private final DocIdSetIterator lead1; private final DocAndScore scorable = new DocAndScore(); private final double[] sumOfOtherClauses; private final int maxDoc; + private final DocAndScoreBuffer docAndScoreBuffer = new DocAndScoreBuffer(); + private final DocAndScoreAccBuffer docAndScoreAccBuffer = new DocAndScoreAccBuffer(); BlockMaxConjunctionBulkScorer(int maxDoc, List scorers) throws IOException { if (scorers.size() <= 1) { @@ -50,14 +49,9 @@ final class BlockMaxConjunctionBulkScorer extends BulkScorer { } this.scorers = scorers.toArray(Scorer[]::new); Arrays.sort(this.scorers, Comparator.comparingLong(scorer -> scorer.iterator().cost())); - this.scorables = - Arrays.stream(this.scorers).map(ScorerUtil::likelyTermScorer).toArray(Scorable[]::new); this.iterators = Arrays.stream(this.scorers).map(Scorer::iterator).toArray(DocIdSetIterator[]::new); lead1 = ScorerUtil.likelyImpactsEnum(iterators[0]); - lead2 = ScorerUtil.likelyImpactsEnum(iterators[1]); - scorer1 = this.scorables[0]; - scorer2 = this.scorables[1]; this.sumOfOtherClauses = new double[this.scorers.length]; for (int i = 0; i < sumOfOtherClauses.length; i++) { sumOfOtherClauses[i] = Double.POSITIVE_INFINITY; @@ -118,98 +112,45 @@ private void scoreWindow( return; } - Scorable scorer1 = this.scorer1; - if (scorers[0].getMaxScore(max - 1) == 0f) { - // Null out scorer1 if it may only produce 0 scores over this window. In practice, this is - // mostly useful because FILTER clauses are pushed as constant-scoring MUST clauses with a - // 0 score to this scorer. Setting it to null instead of using a different impl helps - // reduce polymorphism of calls to Scorable#score and skip the check of whether the leading - // clause produced a high-enough score for the doc to be competitive. - scorer1 = null; - } - final double sumOfOtherMaxScoresAt1 = sumOfOtherClauses[1]; - advanceHead: - for (int doc = lead1.docID(); doc < max; ) { - if (acceptDocs != null && acceptDocs.get(doc) == false) { - doc = lead1.nextDoc(); - continue; - } + for (scorers[0].nextDocsAndScores(max, acceptDocs, docAndScoreBuffer); + docAndScoreBuffer.size > 0; + scorers[0].nextDocsAndScores(max, acceptDocs, docAndScoreBuffer)) { - // Compute the score as we find more matching clauses, in order to skip advancing other - // clauses if the total score has no chance of being competitive. This works well because - // computing a score is usually cheaper than decoding a full block of postings and - // frequencies. - final boolean hasMinCompetitiveScore = scorable.minCompetitiveScore > 0; - double currentScore; - if (scorer1 != null && hasMinCompetitiveScore) { - currentScore = scorer1.score(); - - // This is the same logic as in the below for loop, specialized for the 2nd least costly - // clause. This seems to help the JVM. - - // First check if we have a chance of having a match based on max scores - if ((float) MathUtil.sumUpperBound(currentScore + sumOfOtherMaxScoresAt1, scorers.length) - < scorable.minCompetitiveScore) { - doc = lead1.nextDoc(); - continue advanceHead; - } - } else { - currentScore = 0; - } + docAndScoreAccBuffer.copyFrom(docAndScoreBuffer); - // NOTE: lead2 may be on `doc` already if we `continue`d on the previous loop iteration. - if (lead2.docID() < doc) { - int next = lead2.advance(doc); - if (next != doc) { - doc = lead1.advance(next); - continue advanceHead; - } - } - assert lead2.docID() == doc; - if (hasMinCompetitiveScore) { - currentScore += scorer2.score(); + if (scorable.minCompetitiveScore > 0) { + ScorerUtil.filterCompetitiveHits( + docAndScoreAccBuffer, + sumOfOtherMaxScoresAt1, + scorable.minCompetitiveScore, + scorers.length); } - for (int i = 2; i < iterators.length; ++i) { - // First check if we have a chance of having a match based on max scores - if (hasMinCompetitiveScore - && (float) MathUtil.sumUpperBound(currentScore + sumOfOtherClauses[i], scorers.length) - < scorable.minCompetitiveScore) { - doc = lead1.nextDoc(); - continue advanceHead; - } - - // NOTE: these iterators may be on `doc` already if we called `continue advanceHead` on the - // previous loop iteration. - if (iterators[i].docID() < doc) { - int next = iterators[i].advance(doc); - if (next != doc) { - doc = lead1.advance(next); - continue advanceHead; - } - } - assert iterators[i].docID() == doc; - if (hasMinCompetitiveScore) { - currentScore += scorables[i].score(); + for (int i = 1; i < scorers.length; ++i) { + if (scorable.minCompetitiveScore > 0) { + ScorerUtil.filterCompetitiveHits( + docAndScoreAccBuffer, + sumOfOtherClauses[i], + scorable.minCompetitiveScore, + scorers.length); } + scorers[i].applyAsRequiredClause(docAndScoreAccBuffer); } - if (hasMinCompetitiveScore == false) { - for (Scorable scorer : scorables) { - currentScore += scorer.score(); - } - } - scorable.score = (float) currentScore; - collector.collect(doc); - // The collect() call may have updated the minimum competitive score. - if (maxWindowScore < scorable.minCompetitiveScore) { - // no more hits are competitive - return; + for (int i = 0; i < docAndScoreAccBuffer.size; ++i) { + scorable.score = (float) docAndScoreAccBuffer.scores[i]; + collector.collect(docAndScoreAccBuffer.docs[i]); } + } - doc = lead1.nextDoc(); + int maxOtherDoc = -1; + for (int i = 0; i < iterators.length; ++i) { + maxOtherDoc = Math.max(iterators[i].docID(), maxOtherDoc); + } + if (lead1.docID() < maxOtherDoc) { + lead1.advance(maxOtherDoc); } } diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java index 7407edaaf548..d81332bcaeb4 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java @@ -135,6 +135,30 @@ public TwoPhaseIterator twoPhaseIterator() { return twoPhaseIterator; } + @Override + public void applyAsRequiredClause(DocAndScoreAccBuffer buffer) throws IOException { + int intersectionSize = 0; + int curDoc = disi.docID(); + for (int i = 0; i < buffer.size; ++i) { + int targetDoc = buffer.docs[i]; + if (curDoc < targetDoc) { + curDoc = disi.advance(targetDoc); + } + if (curDoc == targetDoc) { + buffer.docs[intersectionSize] = targetDoc; + buffer.scores[intersectionSize] = buffer.scores[i]; + intersectionSize++; + } + } + + buffer.size = intersectionSize; + if (score != 0) { + for (int i = 0; i < intersectionSize; ++i) { + buffer.scores[i] += score; + } + } + } + @Override public int docID() { return disi.docID(); diff --git a/lucene/core/src/java/org/apache/lucene/search/DocAndScoreAccBuffer.java b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreAccBuffer.java new file mode 100644 index 000000000000..a9e0a56c2cf1 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/DocAndScoreAccBuffer.java @@ -0,0 +1,60 @@ +/* + * 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.search; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.IntsRef; + +/** + * Wrapper around parallel arrays storing doc IDs and their corresponding score accumulators. + * + * @lucene.internal + */ +public final class DocAndScoreAccBuffer { + + private static final double[] EMPTY_DOUBLES = new double[0]; + + /** Doc IDs */ + public int[] docs = IntsRef.EMPTY_INTS; + + /** Scores */ + public double[] scores = EMPTY_DOUBLES; + + /** Number of valid entries in the doc ID and score arrays. */ + public int size; + + /** Sole constructor. */ + public DocAndScoreAccBuffer() {} + + /** Grow both arrays to ensure that they can store at least the given number of entries. */ + public void growNoCopy(int minSize) { + if (docs.length < minSize) { + docs = ArrayUtil.growNoCopy(docs, minSize); + scores = new double[docs.length]; + } + } + + /** Copy content from the given {@link DocAndScoreBuffer}, expanding float scores to doubles. */ + public void copyFrom(DocAndScoreBuffer buffer) { + growNoCopy(buffer.size); + System.arraycopy(buffer.docs, 0, docs, 0, buffer.size); + for (int i = 0; i < buffer.size; ++i) { + scores[i] = buffer.scores[i]; + } + this.size = buffer.size; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index 847fdb010a96..d1d2edac6849 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -52,6 +52,7 @@ final class MaxScoreBulkScorer extends BulkScorer { private final double[] windowScores = new double[INNER_WINDOW_SIZE]; private final DocAndScoreBuffer docAndScoreBuffer = new DocAndScoreBuffer(); + private final DocAndScoreAccBuffer docAndScoreAccBuffer = new DocAndScoreAccBuffer(); MaxScoreBulkScorer(int maxDoc, List scorers, Scorer filter) throws IOException { this.maxDoc = maxDoc; @@ -243,63 +244,34 @@ private void scoreInnerWindowAsConjunction(LeafCollector collector, Bits acceptD DisiWrapper lead1 = allScorers[allScorers.length - 1]; assert essentialQueue.size() == 1; assert lead1 == essentialQueue.top(); - DisiWrapper lead2 = allScorers[allScorers.length - 2]; - if (lead1.doc < lead2.doc) { - lead1.doc = lead1.iterator.advance(Math.min(lead2.doc, max)); - } - // maximum score contribution of all scorers but the lead - double maxScoreSumAtLead2 = maxScoreSums[allScorers.length - 2]; - outer: - while (lead1.doc < max) { + for (lead1.scorer.nextDocsAndScores(max, acceptDocs, docAndScoreBuffer); + docAndScoreBuffer.size > 0; + lead1.scorer.nextDocsAndScores(max, acceptDocs, docAndScoreBuffer)) { - if (acceptDocs != null && acceptDocs.get(lead1.doc) == false) { - lead1.doc = lead1.iterator.nextDoc(); - continue; - } + docAndScoreAccBuffer.copyFrom(docAndScoreBuffer); - double score = lead1.scorable.score(); + for (int i = allScorers.length - 2; i >= firstRequiredScorer; --i) { - // We specialize handling the second best scorer, which seems to help a bit with performance. - // But this is the exact same logic as in the below for loop. - if ((float) MathUtil.sumUpperBound(score + maxScoreSumAtLead2, allScorers.length) - < minCompetitiveScore) { - // a competitive match is not possible according to max scores, skip to the next candidate - lead1.doc = lead1.iterator.nextDoc(); - continue; - } + if (minCompetitiveScore > 0) { + ScorerUtil.filterCompetitiveHits( + docAndScoreAccBuffer, maxScoreSums[i], minCompetitiveScore, allScorers.length); + } - if (lead2.doc < lead1.doc) { - lead2.doc = lead2.iterator.advance(lead1.doc); - } - if (lead2.doc != lead1.doc) { - lead1.doc = lead1.iterator.advance(Math.min(lead2.doc, max)); - continue; + allScorers[i].scorer.applyAsRequiredClause(docAndScoreAccBuffer); } - score += lead2.scorable.score(); - - for (int i = allScorers.length - 3; i >= firstRequiredScorer; --i) { - if ((float) MathUtil.sumUpperBound(score + maxScoreSums[i], allScorers.length) - < minCompetitiveScore) { - // a competitive match is not possible according to max scores, skip to the next candidate - lead1.doc = lead1.iterator.nextDoc(); - continue outer; - } - - DisiWrapper w = allScorers[i]; - if (w.doc < lead1.doc) { - w.doc = w.iterator.advance(lead1.doc); - } - if (w.doc != lead1.doc) { - lead1.doc = lead1.iterator.advance(Math.min(w.doc, max)); - continue outer; - } - score += w.scorable.score(); + for (int i = 0; i < docAndScoreAccBuffer.size; ++i) { + scoreNonEssentialClauses( + collector, + docAndScoreAccBuffer.docs[i], + docAndScoreAccBuffer.scores[i], + firstRequiredScorer); } + } - scoreNonEssentialClauses(collector, lead1.doc, score, firstRequiredScorer); - lead1.doc = lead1.iterator.nextDoc(); + for (int i = allScorers.length - 1; i >= firstRequiredScorer; --i) { + allScorers[i].doc = allScorers[i].iterator.docID(); } } diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 64db31ba9fdd..d63c5ef79df8 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -130,4 +130,27 @@ public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) } buffer.size = size; } + + /** + * Apply this {@link Scorer} as a required clause on the given {@link DocAndScoreAccBuffer}. This + * filters out documents from the buffer that do not match this scorer, and adds the scores of + * this {@link Scorer} to the scores. + */ + public void applyAsRequiredClause(DocAndScoreAccBuffer buffer) throws IOException { + DocIdSetIterator iterator = iterator(); + int intersectionSize = 0; + int curDoc = iterator.docID(); + for (int i = 0; i < buffer.size; ++i) { + int targetDoc = buffer.docs[i]; + if (curDoc < targetDoc) { + curDoc = iterator.advance(targetDoc); + } + if (curDoc == targetDoc) { + buffer.docs[intersectionSize] = targetDoc; + buffer.scores[intersectionSize] = buffer.scores[i] + score(); + intersectionSize++; + } + } + buffer.size = intersectionSize; + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/ScorerUtil.java b/lucene/core/src/java/org/apache/lucene/search/ScorerUtil.java index a08b36413a58..881ad4ed6609 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ScorerUtil.java +++ b/lucene/core/src/java/org/apache/lucene/search/ScorerUtil.java @@ -22,6 +22,7 @@ import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.util.Bits; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.MathUtil; import org.apache.lucene.util.PriorityQueue; /** Util class for Scorer related methods */ @@ -117,4 +118,22 @@ public int length() { return in.length(); } } + + static void filterCompetitiveHits( + DocAndScoreAccBuffer buffer, + double maxRemainingScore, + float minCompetitiveScore, + int numScorers) { + int newSize = 0; + for (int i = 0; i < buffer.size; ++i) { + float maxPossibleScore = + (float) MathUtil.sumUpperBound(buffer.scores[i] + maxRemainingScore, numScorers); + if (maxPossibleScore >= minCompetitiveScore) { + buffer.docs[newSize] = buffer.docs[i]; + buffer.scores[newSize] = buffer.scores[i]; + newSize++; + } + } + buffer.size = newSize; + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java index 8668b8ec63e8..47d9eae80b6c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.LongsRef; /** @@ -40,6 +41,7 @@ public final class TermScorer extends Scorer { private final ImpactsDISI impactsDisi; private final MaxScoreCache maxScoreCache; private DocAndFreqBuffer docAndFreqBuffer; + private int[] freqs = IntsRef.EMPTY_INTS; private long[] normValues = LongsRef.EMPTY_LONGS; /** Construct a {@link TermScorer} that will iterate all documents. */ @@ -172,4 +174,35 @@ public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) buffer.scores[i] = scorer.score(docAndFreqBuffer.freqs[i], normValues[i]); } } + + @Override + public void applyAsRequiredClause(DocAndScoreAccBuffer buffer) throws IOException { + freqs = ArrayUtil.growNoCopy(freqs, buffer.size); + normValues = ArrayUtil.growNoCopy(normValues, buffer.size); + + int intersectionSize = 0; + int curDoc = iterator.docID(); + for (int i = 0; i < buffer.size; ++i) { + int targetDoc = buffer.docs[i]; + if (curDoc < targetDoc) { + curDoc = iterator.advance(targetDoc); + } + if (curDoc == targetDoc) { + buffer.docs[intersectionSize] = targetDoc; + buffer.scores[intersectionSize] = buffer.scores[i]; + freqs[intersectionSize] = postingsEnum.freq(); + if (norms == null || norms.advanceExact(targetDoc) == false) { + normValues[intersectionSize] = 1L; + } else { + normValues[intersectionSize] = norms.longValue(); + } + intersectionSize++; + } + } + + buffer.size = intersectionSize; + for (int i = 0; i < intersectionSize; ++i) { + buffer.scores[i] += scorer.score(freqs[i], normValues[i]); + } + } } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java index 90b775e72f07..e0819a069c86 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorer.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Random; +import org.apache.lucene.search.DocAndScoreAccBuffer; import org.apache.lucene.search.DocAndScoreBuffer; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FilterDocIdSetIterator; @@ -295,4 +296,17 @@ public void nextDocsAndScores(int upTo, Bits liveDocs, DocAndScoreBuffer buffer) } } } + + @Override + public void applyAsRequiredClause(DocAndScoreAccBuffer buffer) throws IOException { + in.applyAsRequiredClause(buffer); + if (doc != in.iterator().docID()) { + doc = in.iterator().docID(); + if (doc == DocIdSetIterator.NO_MORE_DOCS) { + state = IteratorState.FINISHED; + } else { + state = IteratorState.ITERATING; + } + } + } }