check the global maximum minimum score every 1024 collected docs

This commit is contained in:
jimczi 2019-10-07 09:56:06 +02:00
parent 5895797fae
commit 47f9bf6f31
8 changed files with 350 additions and 193 deletions

View File

@ -1,52 +0,0 @@
/*
* 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;
/**
* Maintains the bottom value across multiple collectors
*/
abstract class BottomValueChecker {
/** Maintains global bottom score as the maximum of all bottom scores */
private static class MaximumBottomScoreChecker extends BottomValueChecker {
private volatile float maxMinScore;
@Override
public void updateThreadLocalBottomValue(float value) {
if (value <= maxMinScore) {
return;
}
synchronized (this) {
if (value > maxMinScore) {
maxMinScore = value;
}
}
}
@Override
public float getBottomValue() {
return maxMinScore;
}
}
public static BottomValueChecker createMaxBottomScoreChecker() {
return new MaximumBottomScoreChecker();
}
public abstract void updateThreadLocalBottomValue(float value);
public abstract float getBottomValue();
}

View File

@ -479,11 +479,11 @@ public class IndexSearcher {
private final HitsThresholdChecker hitsThresholdChecker = (executor == null || leafSlices.length <= 1) ? HitsThresholdChecker.create(TOTAL_HITS_THRESHOLD) :
HitsThresholdChecker.createShared(TOTAL_HITS_THRESHOLD);
private final BottomValueChecker bottomValueChecker = (executor == null || leafSlices.length <= 1) ? null : BottomValueChecker.createMaxBottomScoreChecker();
private final MaxScoreAccumulator minScoreAcc = (executor == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
@Override
public TopScoreDocCollector newCollector() throws IOException {
return TopScoreDocCollector.create(cappedNumHits, after, hitsThresholdChecker, bottomValueChecker);
return TopScoreDocCollector.create(cappedNumHits, after, hitsThresholdChecker, minScoreAcc);
}
@Override
@ -612,12 +612,12 @@ public class IndexSearcher {
private final HitsThresholdChecker hitsThresholdChecker = (executor == null || leafSlices.length <= 1) ? HitsThresholdChecker.create(TOTAL_HITS_THRESHOLD) :
HitsThresholdChecker.createShared(TOTAL_HITS_THRESHOLD);
private final BottomValueChecker bottomValueChecker = (executor == null || leafSlices.length <= 1) ? null : BottomValueChecker.createMaxBottomScoreChecker();
private final MaxScoreAccumulator minScoreAcc = (executor == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
@Override
public TopFieldCollector newCollector() throws IOException {
// TODO: don't pay the price for accurate hit counts by default
return TopFieldCollector.create(rewrittenSort, cappedNumHits, after, hitsThresholdChecker, bottomValueChecker);
return TopFieldCollector.create(rewrittenSort, cappedNumHits, after, hitsThresholdChecker, minScoreAcc);
}
@Override

View File

@ -0,0 +1,90 @@
/*
* 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 java.util.Objects;
import java.util.concurrent.atomic.LongAccumulator;
/**
* Maintains the maximum score and its corresponding document id concurrently
*/
final class MaxScoreAccumulator {
static final int DEFAULT_INTERVAL = 1 << 10;
// scores are always positive
final LongAccumulator acc = new LongAccumulator(Long::max, Long.MIN_VALUE);
// non-final and visible for tests
long modInterval;
MaxScoreAccumulator() {
this.modInterval = DEFAULT_INTERVAL;
}
void accumulate(int docID, float score) {
assert docID >= 0 && score >= 0;
long encode = (((long) Float.floatToIntBits(score)) << 32) | (docID & 0xffffffffL);
acc.accumulate(encode);
}
Result get() {
long value = acc.get();
if (value == Long.MIN_VALUE) {
return null;
}
float score = Float.intBitsToFloat((int) (value >> 32));
int docID = (int) value;
return new Result(docID, score);
}
static class Result implements Comparable<Result> {
final int docID;
final float score;
Result(int docID, float score) {
this.docID = docID;
this.score = score;
}
@Override
public int compareTo(Result o) {
int cmp = Float.compare(score, o.score);
if (cmp == 0) {
return Integer.compare(docID, o.docID);
}
return cmp;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Result result = (Result) o;
return docID == result.docID &&
Float.compare(result.score, score) == 0;
}
@Override
public String toString() {
return "Max{" +
"docID=" + docID +
", score=" + score +
'}';
}
}
}

View File

@ -102,8 +102,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
public SimpleFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits,
HitsThresholdChecker hitsThresholdChecker,
BottomValueChecker bottomValueChecker) {
super(queue, numHits, hitsThresholdChecker, sort.needsScores(), bottomValueChecker);
MaxScoreAccumulator minScoreAcc) {
super(queue, numHits, hitsThresholdChecker, sort.needsScores(), minScoreAcc);
this.sort = sort;
this.queue = queue;
}
@ -124,13 +124,22 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
@Override
public void setScorer(Scorable scorer) throws IOException {
super.setScorer(scorer);
minCompetitiveScore = 0f;
updateMinCompetitiveScore(scorer);
if (minScoreAcc != null) {
updateGlobalMinCompetitiveScore(scorer);
}
}
@Override
public void collect(int doc) throws IOException {
++totalHits;
hitsThresholdChecker.incrementHitCount();
if (minScoreAcc != null && totalHits % minScoreAcc.modInterval == 0) {
updateGlobalMinCompetitiveScore(scorer);
}
if (queueFull) {
if (collectedAllCompetitiveHits || reverseMul * comparator.compareBottom(doc) <= 0) {
// since docs are visited in doc Id order, if compare is 0, it means
@ -186,8 +195,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final FieldDoc after;
public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits,
HitsThresholdChecker hitsThresholdChecker, BottomValueChecker bottomValueChecker) {
super(queue, numHits, hitsThresholdChecker, sort.needsScores(), bottomValueChecker);
HitsThresholdChecker hitsThresholdChecker, MaxScoreAccumulator minScoreAcc) {
super(queue, numHits, hitsThresholdChecker, sort.needsScores(), minScoreAcc);
this.sort = sort;
this.queue = queue;
this.after = after;
@ -216,6 +225,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
super.setScorer(scorer);
minCompetitiveScore = 0f;
updateMinCompetitiveScore(scorer);
if (minScoreAcc != null) {
updateGlobalMinCompetitiveScore(scorer);
}
}
@Override
@ -225,6 +237,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
totalHits++;
hitsThresholdChecker.incrementHitCount();
if (minScoreAcc != null && totalHits % minScoreAcc.modInterval == 0) {
updateGlobalMinCompetitiveScore(scorer);
}
if (queueFull) {
// Fastmatch: return if this hit is no better than
// the worst hit currently in the queue:
@ -240,6 +256,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
collectedAllCompetitiveHits = true;
}
} else if (totalHitsRelation == Relation.EQUAL_TO) {
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
updateMinCompetitiveScore(scorer);
}
return;
@ -250,8 +268,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) {
// Already collected on a previous page
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// we can start setting the min competitive score if we just
// reached totalHitsThreshold
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
updateMinCompetitiveScore(scorer);
}
return;
@ -291,12 +309,14 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final int numHits;
final HitsThresholdChecker hitsThresholdChecker;
final BottomValueChecker bottomValueChecker;
final FieldComparator.RelevanceComparator firstComparator;
final boolean canSetMinScore;
// the minimum competitive score (if canSetMinScore is true) that is currently
// used by the underlying scorer (see setScorer)
// an accumulator that maintains the maximum of the segment's minimum competitive scores
final MaxScoreAccumulator minScoreAcc;
// the current local minimum competitive score already propagated to the underlying scorer
float minCompetitiveScore;
final int numComparators;
FieldValueHitQueue.Entry bottom = null;
boolean queueFull;
@ -311,7 +331,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// we want.
private TopFieldCollector(FieldValueHitQueue<Entry> pq, int numHits,
HitsThresholdChecker hitsThresholdChecker, boolean needsScores,
BottomValueChecker bottomValueChecker) {
MaxScoreAccumulator minScoreAcc) {
super(pq);
this.needsScores = needsScores;
this.numHits = numHits;
@ -330,7 +350,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
canSetMinScore = false;
}
this.bottomValueChecker = bottomValueChecker;
this.minScoreAcc = minScoreAcc;
}
@Override
@ -338,29 +358,33 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
return scoreMode;
}
protected void updateGlobalMinCompetitiveScore(Scorable scorer) throws IOException {
assert minScoreAcc != null;
if (canSetMinScore
&& hitsThresholdChecker.isThresholdReached()) {
MaxScoreAccumulator.Result maxMinScore = minScoreAcc.get();
if (maxMinScore != null && maxMinScore.score > minCompetitiveScore) {
scorer.setMinCompetitiveScore(maxMinScore.score);
minCompetitiveScore = maxMinScore.score;
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
}
protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
if (canSetMinScore
&& hitsThresholdChecker.isThresholdReached()
&& ((bottomValueChecker != null && bottomValueChecker.getBottomValue() > 0) || queueFull)) {
assert firstComparator != null;
float bottomScore = 0f;
if (queueFull) {
bottomScore = firstComparator.value(bottom.slot);
if (bottomValueChecker != null) {
bottomValueChecker.updateThreadLocalBottomValue(bottomScore);
&& queueFull) {
assert bottom != null && firstComparator != null;
float minScore = firstComparator.value(bottom.slot);
if (minScore > minCompetitiveScore) {
scorer.setMinCompetitiveScore(minScore);
minCompetitiveScore = minScore;
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
if (minScoreAcc != null) {
minScoreAcc.accumulate(bottom.doc, minScore);
}
}
// Global bottom can only be greater than or equal to the local bottom score
// The updating of global bottom score for this hit before getting here should
// ensure that
if (bottomValueChecker != null && bottomValueChecker.getBottomValue() > bottomScore) {
bottomScore = bottomValueChecker.getBottomValue();
}
scorer.setMinCompetitiveScore(bottomScore);
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
@ -423,10 +447,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
}
/**
* Same as above with additional parameters to allow passing in the threshold checker and the bottom value checker.
* Same as above with additional parameters to allow passing in the threshold checker and the max score accumulator.
*/
static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
HitsThresholdChecker hitsThresholdChecker, BottomValueChecker bottomValueChecker) {
HitsThresholdChecker hitsThresholdChecker, MaxScoreAccumulator minScoreAcc) {
if (sort.fields.length == 0) {
throw new IllegalArgumentException("Sort must contain at least one field");
@ -443,7 +467,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
if (after == null) {
return new SimpleFieldCollector(sort, queue, numHits, hitsThresholdChecker, bottomValueChecker);
return new SimpleFieldCollector(sort, queue, numHits, hitsThresholdChecker, minScoreAcc);
} else {
if (after.fields == null) {
throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search");
@ -453,13 +477,13 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length);
}
return new PagingFieldCollector(sort, queue, after, numHits, hitsThresholdChecker, bottomValueChecker);
return new PagingFieldCollector(sort, queue, after, numHits, hitsThresholdChecker, minScoreAcc);
}
}
/**
* Create a CollectorManager which uses a shared hit counter to maintain number of hits
* and a shared bottom value checker to propagate the minimum score accross segments if
* and a shared {@link MaxScoreAccumulator} to propagate the minimum score accross segments if
* the primary sort is by relevancy.
*/
public static CollectorManager<TopFieldCollector, TopFieldDocs> createSharedManager(Sort sort, int numHits, FieldDoc after,
@ -467,11 +491,11 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
return new CollectorManager<>() {
private final HitsThresholdChecker hitsThresholdChecker = HitsThresholdChecker.createShared(totalHitsThreshold);
private final BottomValueChecker bottomValueChecker = BottomValueChecker.createMaxBottomScoreChecker();
private final MaxScoreAccumulator minScoreAcc = new MaxScoreAccumulator();
@Override
public TopFieldCollector newCollector() throws IOException {
return create(sort, numHits, after, hitsThresholdChecker, bottomValueChecker);
return create(sort, numHits, after, hitsThresholdChecker, minScoreAcc);
}
@Override

View File

@ -50,19 +50,24 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
SimpleTopScoreDocCollector(int numHits, HitsThresholdChecker hitsThresholdChecker,
BottomValueChecker bottomValueChecker) {
super(numHits, hitsThresholdChecker, bottomValueChecker);
MaxScoreAccumulator minScoreAcc) {
super(numHits, hitsThresholdChecker, minScoreAcc);
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final int docBase = context.docBase;
// reset the minimum competitive score
docBase = context.docBase;
return new ScorerLeafCollector() {
@Override
public void setScorer(Scorable scorer) throws IOException {
super.setScorer(scorer);
minCompetitiveScore = 0f;
updateMinCompetitiveScore(scorer);
if (minScoreAcc != null) {
updateGlobalMinCompetitiveScore(scorer);
}
}
@Override
@ -75,8 +80,12 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
totalHits++;
hitsThresholdChecker.incrementHitCount();
if (minScoreAcc != null && totalHits % minScoreAcc.modInterval == 0) {
updateGlobalMinCompetitiveScore(scorer);
}
if (score <= pqTop.score) {
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO && hitsThresholdChecker.isThresholdReached()) {
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
updateMinCompetitiveScore(scorer);
@ -102,8 +111,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
private int collectedHits;
PagingTopScoreDocCollector(int numHits, ScoreDoc after, HitsThresholdChecker hitsThresholdChecker,
BottomValueChecker bottomValueChecker) {
super(numHits, hitsThresholdChecker, bottomValueChecker);
MaxScoreAccumulator minScoreAcc) {
super(numHits, hitsThresholdChecker, minScoreAcc);
this.after = after;
this.collectedHits = 0;
}
@ -123,7 +132,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final int docBase = context.docBase;
docBase = context.docBase;
final int afterDoc = after.doc - context.docBase;
return new ScorerLeafCollector() {
@ -137,9 +146,13 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
totalHits++;
hitsThresholdChecker.incrementHitCount();
if (minScoreAcc != null && totalHits % minScoreAcc.modInterval == 0) {
updateGlobalMinCompetitiveScore(scorer);
}
if (score > after.score || (score == after.score && doc <= afterDoc)) {
// hit was collected on a previous page
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO && hitsThresholdChecker.isThresholdReached()) {
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
updateMinCompetitiveScore(scorer);
@ -148,7 +161,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
if (score <= pqTop.score) {
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO && hitsThresholdChecker.isThresholdReached()) {
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
updateMinCompetitiveScore(scorer);
@ -207,7 +220,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
static TopScoreDocCollector create(int numHits, ScoreDoc after, HitsThresholdChecker hitsThresholdChecker,
BottomValueChecker bottomValueChecker) {
MaxScoreAccumulator minScoreAcc) {
if (numHits <= 0) {
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
@ -218,25 +231,26 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
if (after == null) {
return new SimpleTopScoreDocCollector(numHits, hitsThresholdChecker, bottomValueChecker);
return new SimpleTopScoreDocCollector(numHits, hitsThresholdChecker, minScoreAcc);
} else {
return new PagingTopScoreDocCollector(numHits, after, hitsThresholdChecker, bottomValueChecker);
return new PagingTopScoreDocCollector(numHits, after, hitsThresholdChecker, minScoreAcc);
}
}
/**
* Create a CollectorManager which uses a shared hit counter to maintain number of hits
* and a shared {@link MaxScoreAccumulator} to propagate the minimum score accross segments
*/
public static CollectorManager<TopScoreDocCollector, TopDocs> createSharedManager(int numHits, FieldDoc after,
int totalHitsThreshold) {
int totalHitsThreshold, int maxDocs) {
return new CollectorManager<>() {
private final HitsThresholdChecker hitsThresholdChecker = HitsThresholdChecker.createShared(totalHitsThreshold);
private final BottomValueChecker bottomValueChecker = BottomValueChecker.createMaxBottomScoreChecker();
private final MaxScoreAccumulator minScoreAcc = new MaxScoreAccumulator();
@Override
public TopScoreDocCollector newCollector() throws IOException {
return TopScoreDocCollector.create(numHits, after, hitsThresholdChecker, bottomValueChecker);
return TopScoreDocCollector.create(numHits, after, hitsThresholdChecker, minScoreAcc);
}
@Override
@ -252,13 +266,15 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
};
}
int docBase;
ScoreDoc pqTop;
final HitsThresholdChecker hitsThresholdChecker;
final BottomValueChecker bottomValueChecker;
final MaxScoreAccumulator minScoreAcc;
float minCompetitiveScore;
// prevents instantiation
TopScoreDocCollector(int numHits, HitsThresholdChecker hitsThresholdChecker,
BottomValueChecker bottomValueChecker) {
MaxScoreAccumulator minScoreAcc) {
super(new HitQueue(numHits, true));
assert hitsThresholdChecker != null;
@ -266,7 +282,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
// that at this point top() is already initialized.
pqTop = pq.top();
this.hitsThresholdChecker = hitsThresholdChecker;
this.bottomValueChecker = bottomValueChecker;
this.minScoreAcc = minScoreAcc;
}
@Override
@ -283,31 +299,41 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
return hitsThresholdChecker.scoreMode();
}
protected void updateGlobalMinCompetitiveScore(Scorable scorer) throws IOException {
assert minScoreAcc != null;
MaxScoreAccumulator.Result maxMinScore = minScoreAcc.get();
if (maxMinScore != null) {
// since we tie-break on doc id and collect in doc id order we can require
// the next float if the global minimum score is set on a document that is
// greater than the ids in the current leaf
float score = maxMinScore.docID > docBase ? Math.nextUp(maxMinScore.score) : maxMinScore.score;
if (score > minCompetitiveScore) {
assert hitsThresholdChecker.isThresholdReached();
scorer.setMinCompetitiveScore(score);
minCompetitiveScore = score;
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
}
protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
if (hitsThresholdChecker.isThresholdReached()
&& ((bottomValueChecker != null && bottomValueChecker.getBottomValue() > 0)
|| (pqTop != null && pqTop.score != Float.NEGATIVE_INFINITY))) { // -Infinity is the score of sentinels
float bottomScore = 0f;
if (pqTop != null && pqTop.score != Float.NEGATIVE_INFINITY) {
// since we tie-break on doc id and collect in doc id order, we can require
// the next float
bottomScore = Math.nextUp(pqTop.score);
if (bottomValueChecker != null) {
bottomValueChecker.updateThreadLocalBottomValue(pqTop.score);
&& pqTop != null
&& pqTop.score != Float.NEGATIVE_INFINITY) { // -Infinity is the score of sentinels
// since we tie-break on doc id and collect in doc id order, we can require
// the next float
float localMinScore = Math.nextUp(pqTop.score);
if (localMinScore > minCompetitiveScore) {
scorer.setMinCompetitiveScore(localMinScore);
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
minCompetitiveScore = localMinScore;
if (minScoreAcc!= null) {
// we don't use the next float but we register the document
// id so that other leaves can require it if they are after
// the current maximum
minScoreAcc.accumulate(pqTop.doc, pqTop.score);
}
}
// Global bottom can only be greater than or equal to the local bottom score
// The updating of global bottom score for this hit before getting here should
// ensure that
if (bottomValueChecker != null && bottomValueChecker.getBottomValue() > bottomScore) {
bottomScore = bottomValueChecker.getBottomValue();
}
scorer.setMinCompetitiveScore(bottomScore);
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.LuceneTestCase;
public class TestMaxScoreAccumulator extends LuceneTestCase {
public void testSimple() {
MaxScoreAccumulator acc = new MaxScoreAccumulator();
acc.accumulate(0, 0f);
acc.accumulate(10, 0f);
assertEquals(0f, acc.get().score, 0);
assertEquals(10, acc.get().docID, 0);
acc.accumulate(100, 1000f);
assertEquals(1000f, acc.get().score, 0);
assertEquals(100, acc.get().docID, 0);
acc.accumulate(1000, 5f);
assertEquals(1000f, acc.get().score, 0);
assertEquals(100, acc.get().docID, 0);
acc.accumulate(99, 1000f);
assertEquals(1000f, acc.get().score, 0);
assertEquals(100, acc.get().docID, 0);
acc.accumulate(0, 1001f);
assertEquals(1001f, acc.get().score, 0);
assertEquals(0, acc.get().docID, 0);
}
public void testRandom() {
MaxScoreAccumulator acc = new MaxScoreAccumulator();
int numDocs = atLeast(100);
int maxDocs = atLeast(10000);
MaxScoreAccumulator.Result max = new MaxScoreAccumulator.Result(-1, -1);
for (int i = 0; i < numDocs; i++) {
MaxScoreAccumulator.Result res = new MaxScoreAccumulator.Result(random().nextInt(maxDocs), random().nextFloat());
acc.accumulate(res.docID, res.score);
if (res.compareTo(max) > 0) {
max = res;
}
}
assertEquals(max, acc.get());
}
}

View File

@ -129,7 +129,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
IndexSearcher searcher = new IndexSearcher(indexReader, service);
CollectorManager collectorManager = TopScoreDocCollector.createSharedManager(numResults,
null, threshold);
null, threshold, indexReader.maxDoc());
return (TopDocs) searcher.search(q, collectorManager);
} finally {
@ -417,11 +417,13 @@ public class TestTopDocsCollector extends LuceneTestCase {
w.close();
CollectorManager<TopScoreDocCollector, TopDocs> manager =
TopScoreDocCollector.createSharedManager(2, null, 0);
TopScoreDocCollector.createSharedManager(2, null, 0, reader.maxDoc());
TopScoreDocCollector collector = manager.newCollector();
TopScoreDocCollector collector2 = manager.newCollector();
assertTrue(collector.bottomValueChecker == collector2.bottomValueChecker);
BottomValueChecker minValueChecker = collector.bottomValueChecker;
assertTrue(collector.minScoreAcc == collector2.minScoreAcc);
MaxScoreAccumulator minValueChecker = collector.minScoreAcc;
// force the check of the global minimum score on every round
minValueChecker.modInterval = 1;
ScoreAndDoc scorer = new ScoreAndDoc();
ScoreAndDoc scorer2 = new ScoreAndDoc();
@ -434,80 +436,84 @@ public class TestTopDocsCollector extends LuceneTestCase {
scorer.doc = 0;
scorer.score = 3;
leafCollector.collect(0);
assertNull(minValueChecker.get());
assertNull(scorer.minCompetitiveScore);
scorer2.doc = 0;
scorer2.score = 6;
leafCollector2.collect(0);
assertNull(minValueChecker.get());
assertNull(scorer2.minCompetitiveScore);
scorer.doc = 1;
scorer.score = 2;
leafCollector.collect(1);
assertEquals(minValueChecker.getBottomValue(), 2f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(2f), 0f);
assertEquals(2f, minValueChecker.get().score, 0f);
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
assertNull(scorer2.minCompetitiveScore);
scorer2.doc = 1;
scorer2.score = 9;
leafCollector2.collect(1);
assertEquals(minValueChecker.getBottomValue(), 6f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(2f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(6f), 0f);
assertEquals(6f, minValueChecker.get().score, 0f);
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(6f), scorer2.minCompetitiveScore, 0f);
scorer2.doc = 2;
scorer2.score = 7;
leafCollector2.collect(2);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(2f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(minValueChecker.get().score, 7f, 0f);
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
scorer2.doc = 3;
scorer2.score = 1;
leafCollector2.collect(3);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(2f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(minValueChecker.get().score, 7f, 0f);
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
scorer.doc = 2;
scorer.score = 10;
leafCollector.collect(2);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, 7f, 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(minValueChecker.get().score, 7f, 0f);
assertEquals(Math.nextUp(7f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
scorer.doc = 3;
scorer.score = 11;
leafCollector.collect(3);
assertEquals(minValueChecker.getBottomValue(), 10, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(10f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(minValueChecker.get().score, 10, 0f);
assertEquals(Math.nextUp(10f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
TopScoreDocCollector collector3 = manager.newCollector();
LeafCollector leafCollector3 = collector3.getLeafCollector(reader.leaves().get(2));
ScoreAndDoc scorer3 = new ScoreAndDoc();
leafCollector3.setScorer(scorer3);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer3.doc = 0;
scorer3.score = 1f;
leafCollector3.collect(0);
assertEquals(minValueChecker.getBottomValue(), 10f, 0f);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(10f, minValueChecker.get().score, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer.doc = 4;
scorer.score = 11;
leafCollector.collect(4);
assertEquals(minValueChecker.getBottomValue(), 11f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(11f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(11f, minValueChecker.get().score, 0f);
assertEquals(Math.nextUp(11f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer3.doc = 1;
scorer3.score = 2f;
leafCollector3.collect(1);
assertEquals(minValueChecker.getBottomValue(), 11f, 0f);
assertEquals(scorer.minCompetitiveScore, Math.nextUp(11f), 0f);
assertEquals(scorer2.minCompetitiveScore, Math.nextUp(7f), 0f);
assertEquals(scorer3.minCompetitiveScore, 11f, 0f);
assertEquals(minValueChecker.get().score, 11f, 0f);
assertEquals(Math.nextUp(11f), scorer.minCompetitiveScore, 0f);
assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f);
assertEquals(11f, scorer3.minCompetitiveScore, 0f);
TopDocs topDocs = manager.reduce(Arrays.asList(collector, collector2, collector3));

View File

@ -540,8 +540,10 @@ public class TestTopFieldCollector extends LuceneTestCase {
TopFieldCollector.createSharedManager(sort, 2, null, 0);
TopFieldCollector collector = manager.newCollector();
TopFieldCollector collector2 = manager.newCollector();
assertTrue(collector.bottomValueChecker == collector2.bottomValueChecker);
BottomValueChecker minValueChecker = collector.bottomValueChecker;
assertTrue(collector.minScoreAcc == collector2.minScoreAcc);
MaxScoreAccumulator minValueChecker = collector.minScoreAcc;
// force the check of the global minimum score on every round
minValueChecker.modInterval = 1;
ScoreAndDoc scorer = new ScoreAndDoc();
ScoreAndDoc scorer2 = new ScoreAndDoc();
@ -554,80 +556,84 @@ public class TestTopFieldCollector extends LuceneTestCase {
scorer.doc = 0;
scorer.score = 3;
leafCollector.collect(0);
assertNull(minValueChecker.get());
assertNull(scorer.minCompetitiveScore);
scorer2.doc = 0;
scorer2.score = 6;
leafCollector2.collect(0);
assertNull(minValueChecker.get());
assertNull(scorer2.minCompetitiveScore);
scorer.doc = 1;
scorer.score = 2;
leafCollector.collect(1);
assertEquals(minValueChecker.getBottomValue(), 2f, 0f);
assertEquals(scorer.minCompetitiveScore, 2f, 0f);
assertEquals(2f, minValueChecker.get().score, 0f);
assertEquals(2f, scorer.minCompetitiveScore, 0f);
assertNull(scorer2.minCompetitiveScore);
scorer2.doc = 1;
scorer2.score = 9;
leafCollector2.collect(1);
assertEquals(minValueChecker.getBottomValue(), 6f, 0f);
assertEquals(scorer.minCompetitiveScore, 2f, 0f);
assertEquals(scorer2.minCompetitiveScore, 6f, 0f);
assertEquals(6f, minValueChecker.get().score, 0f);
assertEquals(2f, scorer.minCompetitiveScore, 0f);
assertEquals(6f, scorer2.minCompetitiveScore, 0f);
scorer2.doc = 2;
scorer2.score = 7;
leafCollector2.collect(2);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, 2f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(7f, minValueChecker.get().score, 0f);
assertEquals(2f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
scorer2.doc = 3;
scorer2.score = 1;
leafCollector2.collect(3);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, 2f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(7f, minValueChecker.get().score, 0f);
assertEquals(2f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
scorer.doc = 2;
scorer.score = 10;
leafCollector.collect(2);
assertEquals(minValueChecker.getBottomValue(), 7f, 0f);
assertEquals(scorer.minCompetitiveScore, 7f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(7f, minValueChecker.get().score, 0f);
assertEquals(7f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
scorer.doc = 3;
scorer.score = 11;
leafCollector.collect(3);
assertEquals(minValueChecker.getBottomValue(), 10, 0f);
assertEquals(scorer.minCompetitiveScore, 10f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(10f, minValueChecker.get().score, 0f);
assertEquals(10f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
TopFieldCollector collector3 = manager.newCollector();
LeafCollector leafCollector3 = collector3.getLeafCollector(reader.leaves().get(2));
ScoreAndDoc scorer3 = new ScoreAndDoc();
leafCollector3.setScorer(scorer3);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer3.doc = 0;
scorer3.score = 1f;
leafCollector3.collect(0);
assertEquals(minValueChecker.getBottomValue(), 10f, 0f);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(10f, minValueChecker.get().score, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer.doc = 4;
scorer.score = 11;
leafCollector.collect(4);
assertEquals(minValueChecker.getBottomValue(), 11f, 0f);
assertEquals(scorer.minCompetitiveScore, 11f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(scorer3.minCompetitiveScore, 10f, 0f);
assertEquals(11f, minValueChecker.get().score, 0f);
assertEquals(11f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
assertEquals(10f, scorer3.minCompetitiveScore, 0f);
scorer3.doc = 1;
scorer3.score = 2f;
leafCollector3.collect(1);
assertEquals(minValueChecker.getBottomValue(), 11f, 0f);
assertEquals(scorer.minCompetitiveScore, 11f, 0f);
assertEquals(scorer2.minCompetitiveScore, 7f, 0f);
assertEquals(scorer3.minCompetitiveScore, 11f, 0f);
assertEquals(11f, minValueChecker.get().score, 0f);
assertEquals(11f, scorer.minCompetitiveScore, 0f);
assertEquals(7f, scorer2.minCompetitiveScore, 0f);
assertEquals(11f, scorer3.minCompetitiveScore, 0f);
TopFieldDocs topDocs = manager.reduce(Arrays.asList(collector, collector2, collector3));