LUCENE-8340: Recency boosting.

This commit is contained in:
Adrien Grand 2018-09-04 11:48:25 +02:00
parent 34a85014d5
commit a9acdfdb54
15 changed files with 831 additions and 11 deletions

View File

@ -106,6 +106,12 @@ Changes in Runtime Behavior
total hit counts accurately up to 1,000 in order to enable top-hits
optimizations such as block-max WAND (LUCENE-8135). (Adrien Grand)
New Features
* LUCENE-8340: LongPoint#newDistanceQuery may be used to boost scores based on
how close a value of a long field is from an configurable origin. This is
typically useful to boost by recency. (Adrien Grand)
Improvements
* LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

View File

@ -0,0 +1,438 @@
/*
* 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.document;
import java.io.IOException;
import java.util.Objects;
import java.util.Set;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.ScorerSupplier;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.DocIdSetBuilder;
import org.apache.lucene.util.FutureArrays;
final class LongDistanceFeatureQuery extends Query {
private final String field;
private final long origin;
private final long pivotDistance;
LongDistanceFeatureQuery(String field, long origin, long pivotDistance) {
this.field = Objects.requireNonNull(field);
this.origin = origin;
if (pivotDistance <= 0) {
throw new IllegalArgumentException("pivotDistance must be > 0, got " + pivotDistance);
}
this.pivotDistance = pivotDistance;
}
@Override
public final boolean equals(Object o) {
return sameClassAs(o) &&
equalsTo(getClass().cast(o));
}
private boolean equalsTo(LongDistanceFeatureQuery other) {
return Objects.equals(field, other.field) &&
origin == other.origin &&
pivotDistance == other.pivotDistance;
}
@Override
public int hashCode() {
int h = classHash();
h = 31 * h + field.hashCode();
h = 31 * h + Long.hashCode(origin);
h = 31 * h + Long.hashCode(pivotDistance);
return h;
}
@Override
public String toString(String field) {
return getClass().getSimpleName() + "(field=" + field + ",origin=" + origin + ",pivotDistance=" + pivotDistance + ")";
}
@Override
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
return new Weight(this) {
@Override
public boolean isCacheable(LeafReaderContext ctx) {
return false;
}
@Override
public void extractTerms(Set<Term> terms) {}
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
SortedNumericDocValues multiDocValues = DocValues.getSortedNumeric(context.reader(), field);
if (multiDocValues.advanceExact(doc) == false) {
return Explanation.noMatch("Document " + doc + " doesn't have a value for field " + field);
}
long value = selectValue(multiDocValues);
long distance = Math.max(value, origin) - Math.min(value, origin);
if (distance < 0) {
// underflow, treat as MAX_VALUE
distance = Long.MAX_VALUE;
}
float score = (float) (boost * (pivotDistance / (pivotDistance + (double) distance)));
return Explanation.match(score, "Distance score, computed as weight * pivotDistance / (pivotDistance + abs(value - origin)) from:",
Explanation.match(boost, "weight"),
Explanation.match(pivotDistance, "pivotDistance"),
Explanation.match(origin, "origin"),
Explanation.match(value, "current value"));
}
private long selectValue(SortedNumericDocValues multiDocValues) throws IOException {
int count = multiDocValues.docValueCount();
long next = multiDocValues.nextValue();
if (count == 1 || next >= origin) {
return next;
}
long previous = next;
for (int i = 1; i < count; ++i) {
next = multiDocValues.nextValue();
if (next >= origin) {
// Unsigned comparison because of underflows
if (Long.compareUnsigned(origin - previous, next - origin) < 0) {
return previous;
} else {
return next;
}
}
previous = next;
}
assert next < origin;
return next;
}
private NumericDocValues selectValues(SortedNumericDocValues multiDocValues) {
final NumericDocValues singleton = DocValues.unwrapSingleton(multiDocValues);
if (singleton != null) {
return singleton;
}
return new NumericDocValues() {
long value;
@Override
public long longValue() throws IOException {
return value;
}
@Override
public boolean advanceExact(int target) throws IOException {
if (multiDocValues.advanceExact(target)) {
value = selectValue(multiDocValues);
return true;
} else {
return false;
}
}
@Override
public int docID() {
return multiDocValues.docID();
}
@Override
public int nextDoc() throws IOException {
return multiDocValues.nextDoc();
}
@Override
public int advance(int target) throws IOException {
return multiDocValues.advance(target);
}
@Override
public long cost() {
return multiDocValues.cost();
}
};
}
@Override
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
PointValues pointValues = context.reader().getPointValues(field);
if (pointValues == null) {
// No data on this segment
return null;
}
final SortedNumericDocValues multiDocValues = DocValues.getSortedNumeric(context.reader(), field);
final NumericDocValues docValues = selectValues(multiDocValues);
final Weight weight = this;
return new ScorerSupplier() {
@Override
public Scorer get(long leadCost) throws IOException {
return new DistanceScorer(weight, context.reader().maxDoc(), leadCost, boost, pointValues, docValues);
}
@Override
public long cost() {
return docValues.cost();
}
};
}
@Override
public Scorer scorer(LeafReaderContext context) throws IOException {
ScorerSupplier scorerSupplier = scorerSupplier(context);
if (scorerSupplier == null) {
return null;
}
return scorerSupplier.get(Long.MAX_VALUE);
}
};
}
private class DistanceScorer extends Scorer {
private final int maxDoc;
private DocIdSetIterator it;
private int doc = -1;
private final long leadCost;
private final float boost;
private final PointValues pointValues;
private final NumericDocValues docValues;
private long maxDistance = Long.MAX_VALUE;
protected DistanceScorer(Weight weight, int maxDoc, long leadCost, float boost,
PointValues pointValues, NumericDocValues docValues) {
super(weight);
this.maxDoc = maxDoc;
this.leadCost = leadCost;
this.boost = boost;
this.pointValues = pointValues;
this.docValues = docValues;
// initially use doc values in order to iterate all documents that have
// a value for this field
this.it = docValues;
}
@Override
public int docID() {
return doc;
}
private float score(double distance) {
return (float) (boost * (pivotDistance / (pivotDistance + distance)));
}
/**
* Inverting the score computation is very hard due to all potential
* rounding errors, so we binary search the maximum distance.
*/
private long computeMaxDistance(float minScore, long previousMaxDistance) {
assert score(0) >= minScore;
if (score(previousMaxDistance) >= minScore) {
// minScore did not decrease enough to require an update to the max distance
return previousMaxDistance;
}
assert score(previousMaxDistance) < minScore;
long min = 0, max = previousMaxDistance;
// invariant: score(min) >= minScore && score(max) < minScore
while (max - min > 1) {
long mid = (min + max) >>> 1;
float score = score(mid);
if (score >= minScore) {
min = mid;
} else {
max = mid;
}
}
assert score(min) >= minScore;
assert min == Long.MAX_VALUE || score(min + 1) < minScore;
return min;
}
@Override
public float score() throws IOException {
if (docValues.advanceExact(docID()) == false) {
return 0;
}
long v = docValues.longValue();
// note: distance is unsigned
long distance = Math.max(v, origin) - Math.min(v, origin);
if (distance < 0) {
// underflow
// treat distances that are greater than MAX_VALUE as MAX_VALUE
distance = Long.MAX_VALUE;
}
return score(distance);
}
@Override
public DocIdSetIterator iterator() {
// add indirection so that if 'it' is updated then it will
// be taken into account
return new DocIdSetIterator() {
@Override
public int nextDoc() throws IOException {
return doc = it.nextDoc();
}
@Override
public int docID() {
return doc;
}
@Override
public long cost() {
return it.cost();
}
@Override
public int advance(int target) throws IOException {
return doc = it.advance(target);
}
};
}
@Override
public float getMaxScore(int upTo) {
return boost;
}
private int setMinCompetitiveScoreCounter = 0;
@Override
public void setMinCompetitiveScore(float minScore) throws IOException {
if (minScore > boost) {
it = DocIdSetIterator.empty();
return;
}
// Start sampling if we get called too much
setMinCompetitiveScoreCounter++;
if (setMinCompetitiveScoreCounter > 256 && (setMinCompetitiveScoreCounter & 0x1f) != 0x1f) {
return;
}
long previousMaxDistance = maxDistance;
maxDistance = computeMaxDistance(minScore, maxDistance);
if (maxDistance == previousMaxDistance) {
// nothing to update
return;
}
long minValue = origin - maxDistance;
if (minValue > origin) {
// underflow
minValue = Long.MIN_VALUE;
}
long maxValue = origin + maxDistance;
if (maxValue < origin) {
// overflow
maxValue = Long.MAX_VALUE;
}
final byte[] minValueAsBytes = new byte[Long.BYTES];
LongPoint.encodeDimension(minValue, minValueAsBytes, 0);
final byte[] maxValueAsBytes = new byte[Long.BYTES];
LongPoint.encodeDimension(maxValue, maxValueAsBytes, 0);
DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
final int doc = docID();
IntersectVisitor visitor = new IntersectVisitor() {
DocIdSetBuilder.BulkAdder adder;
@Override
public void grow(int count) {
adder = result.grow(count);
}
@Override
public void visit(int docID) {
if (docID <= doc) {
// Already visited or skipped
return;
}
adder.add(docID);
}
@Override
public void visit(int docID, byte[] packedValue) {
if (docID <= doc) {
// Already visited or skipped
return;
}
if (FutureArrays.compareUnsigned(packedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0) {
// Doc's value is too low, in this dimension
return;
}
if (FutureArrays.compareUnsigned(packedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0) {
// Doc's value is too high, in this dimension
return;
}
// Doc is in-bounds
adder.add(docID);
}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0 ||
FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0) {
return Relation.CELL_OUTSIDE_QUERY;
}
if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0 ||
FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0) {
return Relation.CELL_CROSSES_QUERY;
}
return Relation.CELL_INSIDE_QUERY;
}
};
final long currentQueryCost = Math.min(leadCost, it.cost());
final long threshold = currentQueryCost >>> 3;
long estimatedNumberOfMatches = pointValues.estimatePointCount(visitor); // runs in O(log(numPoints))
// TODO: what is the right factor compared to the current disi? Is 8 optimal?
if (estimatedNumberOfMatches >= threshold) {
// the new range is not selective enough to be worth materializing
return;
}
pointValues.intersect(visitor);
it = result.build().iterator();
}
}
}

View File

@ -20,9 +20,12 @@ import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.PointInSetQuery;
import org.apache.lucene.search.PointRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
@ -256,4 +259,27 @@ public final class LongPoint extends Field {
}
return newSetQuery(field, unboxed);
}
/**
* Given a field that indexes the same long values into a {@link LongPoint}
* and doc values (either {@link NumericDocValuesField} or
* {@link SortedNumericDocValuesField}), this returns a query that scores
* documents based on their distance to {@code origin}:
* {@code score = weight * pivotDistance / (pivotDistance + distance)}, ie.
* score is in the {@code [0, weight]} range, is equal to {@code weight} when
* the document's value is equal to {@code origin} and is equal to
* {@code weight/2} when the document's value is distant of
* {@code pivotDistance} from {@code origin}.
* In case of multi-valued fields, only the closest point to {@code origin}
* will be considered.
* This query is typically useful to boost results based on recency by adding
* this query to a {@link Occur#SHOULD} clause of a {@link BooleanQuery}.
*/
public static Query newDistanceFeatureQuery(String field, float weight, long origin, long pivotDistance) {
Query query = new LongDistanceFeatureQuery(field, origin, pivotDistance);
if (weight != 1f) {
query = new BoostQuery(query, weight);
}
return query;
}
}

View File

@ -244,7 +244,7 @@ final class BlockMaxConjunctionScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float score) {
public void setMinCompetitiveScore(float score) throws IOException {
minScore = score;
maxScorePropagator.setMinCompetitiveScore(score);
}

View File

@ -83,7 +83,7 @@ class ConjunctionScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
// This scorer is only used for TOP_SCORES when there is a single scoring clause
if (scorers.length == 1) {
scorers[0].setMinCompetitiveScore(minScore);

View File

@ -110,7 +110,7 @@ final class MaxScoreSumPropagator {
return scoreSumUpperBound(maxScore);
}
void setMinCompetitiveScore(float minScore) {
void setMinCompetitiveScore(float minScore) throws IOException {
if (minScore == 0) {
return ;
}

View File

@ -87,7 +87,7 @@ class ReqExclScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float score) {
public void setMinCompetitiveScore(float score) throws IOException {
// The score of this scorer is the same as the score of 'reqScorer'.
reqScorer.setMinCompetitiveScore(score);
}

View File

@ -290,7 +290,7 @@ class ReqOptSumScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
this.minScore = minScore;
// Potentially move to a conjunction
if (reqMaxScore < minScore) {

View File

@ -45,7 +45,7 @@ public abstract class Scorable {
* {@link ScoreMode#TOP_SCORES}, and successive calls may only set increasing
* values of {@code minScore}.
*/
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
// no-op by default
}

View File

@ -55,7 +55,7 @@ public final class ScoreCachingWrappingScorer extends Scorable {
}
@Override
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
in.setMinCompetitiveScore(minScore);
}

View File

@ -62,7 +62,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
final int docBase = context.docBase;
return new ScorerLeafCollector() {
private void updateMinCompetitiveScore() {
private void updateMinCompetitiveScore() throws IOException {
// since we tie-break on doc id and collect in doc id order, we can require
// the next float
scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score));

View File

@ -186,7 +186,7 @@ final class WANDScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
// Let this disjunction know about the new min score so that it can skip
// over clauses that produce low scores.
assert minScore >= 0;

View File

@ -0,0 +1,350 @@
/*
* 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.document;
import java.io.IOException;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.CheckHits;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryUtils;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TestLongDistanceFeatureQuery extends LuceneTestCase {
public void testEqualsAndHashcode() {
Query q1 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
Query q2 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
QueryUtils.checkEqual(q1, q2);
Query q3 = LongPoint.newDistanceFeatureQuery("bar", 3, 10, 5);
QueryUtils.checkUnequal(q1, q3);
Query q4 = LongPoint.newDistanceFeatureQuery("foo", 4, 10, 5);
QueryUtils.checkUnequal(q1, q4);
Query q5 = LongPoint.newDistanceFeatureQuery("foo", 3, 9, 5);
QueryUtils.checkUnequal(q1, q5);
Query q6 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 6);
QueryUtils.checkUnequal(q1, q6);
}
public void testBasics() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
.setMergePolicy(newLogMergePolicy(random().nextBoolean())));
Document doc = new Document();
LongPoint point = new LongPoint("foo", 0L);
doc.add(point);
NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
doc.add(docValue);
point.setLongValue(3);
docValue.setLongValue(3);
w.addDocument(doc);
point.setLongValue(12);
docValue.setLongValue(12);
w.addDocument(doc);
point.setLongValue(8);
docValue.setLongValue(8);
w.addDocument(doc);
point.setLongValue(-1);
docValue.setLongValue(-1);
w.addDocument(doc);
point.setLongValue(7);
docValue.setLongValue(7);
w.addDocument(doc);
DirectoryReader reader = w.getReader();
IndexSearcher searcher = newSearcher(reader);
Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
TopDocs topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(1, (float) (3f * (5. / (5. + 2.)))),
new ScoreDoc(2, (float) (3f * (5. / (5. + 2.))))
},
topHits.scoreDocs);
q = LongPoint.newDistanceFeatureQuery("foo", 3, 7, 5);
collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkExplanations(q, "", searcher);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(4, (float) (3f * (5. / (5. + 0.)))),
new ScoreDoc(2, (float) (3f * (5. / (5. + 1.))))
},
topHits.scoreDocs);
reader.close();
w.close();
dir.close();
}
public void testOverUnderFlow() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
.setMergePolicy(newLogMergePolicy(random().nextBoolean())));
Document doc = new Document();
LongPoint point = new LongPoint("foo", 0L);
doc.add(point);
NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
doc.add(docValue);
point.setLongValue(3);
docValue.setLongValue(3);
w.addDocument(doc);
point.setLongValue(12);
docValue.setLongValue(12);
w.addDocument(doc);
point.setLongValue(-10);
docValue.setLongValue(-10);
w.addDocument(doc);
point.setLongValue(Long.MAX_VALUE);
docValue.setLongValue(Long.MAX_VALUE);
w.addDocument(doc);
point.setLongValue(Long.MIN_VALUE);
docValue.setLongValue(Long.MIN_VALUE);
w.addDocument(doc);
DirectoryReader reader = w.getReader();
IndexSearcher searcher = newSearcher(reader);
Query q = LongPoint.newDistanceFeatureQuery("foo", 3, Long.MAX_VALUE - 1, 100);
TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
TopDocs topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(3, (float) (3f * (100. / (100. + 1.)))),
new ScoreDoc(0, (float) (3f * (100. / (100. + Long.MAX_VALUE)))) // rounding makes the distance treated as if it was MAX_VALUE
},
topHits.scoreDocs);
q = LongPoint.newDistanceFeatureQuery("foo", 3, Long.MIN_VALUE + 1, 100);
collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkExplanations(q, "", searcher);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(4, (float) (3f * (100. / (100. + 1.)))),
new ScoreDoc(0, (float) (3f * (100. / (100. + Long.MAX_VALUE)))) // rounding makes the distance treated as if it was MAX_VALUE
},
topHits.scoreDocs);
reader.close();
w.close();
dir.close();
}
public void testMissingField() throws IOException {
IndexReader reader = new MultiReader();
IndexSearcher searcher = newSearcher(reader);
Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
TopDocs topHits = searcher.search(q, 2);
assertEquals(0, topHits.totalHits.value);
}
public void testMissingValue() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
.setMergePolicy(newLogMergePolicy(random().nextBoolean())));
Document doc = new Document();
LongPoint point = new LongPoint("foo", 0L);
doc.add(point);
NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
doc.add(docValue);
point.setLongValue(3);
docValue.setLongValue(3);
w.addDocument(doc);
w.addDocument(new Document());
point.setLongValue(7);
docValue.setLongValue(7);
w.addDocument(doc);
DirectoryReader reader = w.getReader();
IndexSearcher searcher = newSearcher(reader);
Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
TopScoreDocCollector collector = TopScoreDocCollector.create(3, null, 1);
searcher.search(q, collector);
TopDocs topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(2, (float) (3f * (5. / (5. + 3.)))),
new ScoreDoc(0, (float) (3f * (5. / (5. + 7.))))
},
topHits.scoreDocs);
CheckHits.checkExplanations(q, "", searcher);
reader.close();
w.close();
dir.close();
}
public void testMultiValued() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
.setMergePolicy(newLogMergePolicy(random().nextBoolean())));
Document doc = new Document();
for (long v : new long[] {3, 1000, Long.MAX_VALUE}) {
doc.add(new LongPoint("foo", v));
doc.add(new SortedNumericDocValuesField("foo", v));
}
w.addDocument(doc);
doc = new Document();
for (long v : new long[] {-100, 12, 999}) {
doc.add(new LongPoint("foo", v));
doc.add(new SortedNumericDocValuesField("foo", v));
}
w.addDocument(doc);
doc = new Document();
for (long v : new long[] {Long.MIN_VALUE, -1000, 8}) {
doc.add(new LongPoint("foo", v));
doc.add(new SortedNumericDocValuesField("foo", v));
}
w.addDocument(doc);
doc = new Document();
for (long v : new long[] { -1 }) {
doc.add(new LongPoint("foo", v));
doc.add(new SortedNumericDocValuesField("foo", v));
}
w.addDocument(doc);
doc = new Document();
for (long v : new long[] {Long.MIN_VALUE, 7}) {
doc.add(new LongPoint("foo", v));
doc.add(new SortedNumericDocValuesField("foo", v));
}
w.addDocument(doc);
DirectoryReader reader = w.getReader();
IndexSearcher searcher = newSearcher(reader);
Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
TopDocs topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(1, (float) (3f * (5. / (5. + 2.)))),
new ScoreDoc(2, (float) (3f * (5. / (5. + 2.))))
},
topHits.scoreDocs);
q = LongPoint.newDistanceFeatureQuery("foo", 3, 7, 5);
collector = TopScoreDocCollector.create(2, null, 1);
searcher.search(q, collector);
topHits = collector.topDocs();
assertEquals(2, topHits.scoreDocs.length);
CheckHits.checkExplanations(q, "", searcher);
CheckHits.checkEqual(q,
new ScoreDoc[] {
new ScoreDoc(4, (float) (3f * (5. / (5. + 0.)))),
new ScoreDoc(2, (float) (3f * (5. / (5. + 1.))))
},
topHits.scoreDocs);
reader.close();
w.close();
dir.close();
}
public void testRandom() throws IOException {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()
.setMergePolicy(newLogMergePolicy(random().nextBoolean())));
Document doc = new Document();
LongPoint point = new LongPoint("foo", 0L);
doc.add(point);
NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
doc.add(docValue);
int numDocs = atLeast(10000);
for (int i = 0; i < numDocs; ++i) {
long v = random().nextLong();
point.setLongValue(v);
docValue.setLongValue(v);
w.addDocument(doc);
}
IndexReader reader = DirectoryReader.open(w);
IndexSearcher searcher = newSearcher(reader);
for (int iter = 0; iter < 10; ++iter) {
long origin = random().nextLong();
long pivotDistance;
do {
pivotDistance = random().nextLong();
} while (pivotDistance <= 0);
float boost = (1 + random().nextInt(10)) / 3f;
Query q = LongPoint.newDistanceFeatureQuery("foo", boost, origin, pivotDistance);
CheckHits.checkTopScores(random(), q, searcher);
}
reader.close();
w.close();
dir.close();
}
}

View File

@ -39,7 +39,7 @@ public class AssertingScorable extends FilterScorable {
}
@Override
public void setMinCompetitiveScore(float minScore) {
public void setMinCompetitiveScore(float minScore) throws IOException {
in.setMinCompetitiveScore(minScore);
}

View File

@ -66,7 +66,7 @@ public class AssertingScorer extends Scorer {
}
@Override
public void setMinCompetitiveScore(float score) {
public void setMinCompetitiveScore(float score) throws IOException {
assert scoreMode == ScoreMode.TOP_SCORES;
assert Float.isNaN(score) == false;
assert score >= minCompetitiveScore;