LUCENE-9280: Collectors to skip noncompetitive documents (#1351)

Similar how scorers can update their iterators to skip non-competitive
documents, collectors and comparators should also provide and update
iterators that allow them to skip non-competive documents.
This commit is contained in:
Mayya Sharipova 2020-06-23 16:04:58 -04:00 committed by GitHub
parent ea1bb9f2e9
commit b0333ab5c8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 977 additions and 63 deletions

View File

@ -107,6 +107,14 @@ Improvements
* LUCENE-9074: Introduce Slice Executor For Dynamic Runtime Execution Of Slices (Atri Sharma)
* LUCENE-9280: Add an ability for field comparators to skip non-competitive documents.
Creating a TopFieldCollector with totalHitsThreshold less than Integer.MAX_VALUE
instructs Lucene to skip non-competitive documents whenever possible. For numeric
sort fields the skipping functionality works when the same field is indexed both
with doc values and points. In this case, there is an assumption that the same data is
stored in these points and doc values (Mayya Sharipova, Jim Ferenczi, Adrien Grand)
Bug fixes
* LUCENE-8663: NRTCachingDirectory.slowFileExists may open a file while

View File

@ -287,3 +287,9 @@ TopDocsCollector shall no longer return an empty TopDocs for malformed arguments
Rather, an IllegalArgumentException shall be thrown. This is introduced for better
defence and to ensure that there is no bubbling up of errors when Lucene is
used in multi level applications
## Assumption of data consistency between different data-structures sharing the same field name
Sorting on a numeric field that is indexed with both doc values and points may use an
optimization to skip non-competitive documents. This optimization relies on the assumption
that the same data is stored in these points and doc values.

View File

@ -115,7 +115,7 @@ public final class ConstantScoreQuery extends Query {
return new ConstantScoreWeight(this, boost) {
@Override
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
if (scoreMode == ScoreMode.TOP_SCORES) {
if (scoreMode.isExhaustive() == false) {
return super.bulkScorer(context);
}
final BulkScorer innerScorer = innerWeight.bulkScorer(context);

View File

@ -165,8 +165,8 @@ public abstract class FieldComparator<T> {
* org.apache.lucene.index.LeafReader#getNumericDocValues} and sorts by ascending value */
public static class DoubleComparator extends NumericComparator<Double> {
private final double[] values;
private double bottom;
private double topValue;
protected double bottom;
protected double topValue;
/**
* Creates a new comparator based on {@link Double#compare} for {@code numHits}.
@ -225,8 +225,8 @@ public abstract class FieldComparator<T> {
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
public static class FloatComparator extends NumericComparator<Float> {
private final float[] values;
private float bottom;
private float topValue;
protected float bottom;
protected float topValue;
/**
* Creates a new comparator based on {@link Float#compare} for {@code numHits}.
@ -285,8 +285,8 @@ public abstract class FieldComparator<T> {
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
public static class IntComparator extends NumericComparator<Integer> {
private final int[] values;
private int bottom; // Value of bottom of queue
private int topValue;
protected int bottom; // Value of bottom of queue
protected int topValue;
/**
* Creates a new comparator based on {@link Integer#compare} for {@code numHits}.
@ -347,8 +347,8 @@ public abstract class FieldComparator<T> {
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
public static class LongComparator extends NumericComparator<Long> {
private final long[] values;
private long bottom;
private long topValue;
protected long bottom;
protected long topValue;
/**
* Creates a new comparator based on {@link Long#compare} for {@code numHits}.

View File

@ -58,8 +58,8 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
private final int oneReverseMul;
private final FieldComparator<?> oneComparator;
public OneComparatorFieldValueHitQueue(SortField[] fields, int size) {
super(fields, size);
public OneComparatorFieldValueHitQueue(SortField[] fields, int size, boolean filterNonCompetitiveDocs) {
super(fields, size, filterNonCompetitiveDocs);
assert fields.length == 1;
oneComparator = comparators[0];
@ -95,8 +95,8 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
*/
private static final class MultiComparatorsFieldValueHitQueue<T extends FieldValueHitQueue.Entry> extends FieldValueHitQueue<T> {
public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size) {
super(fields, size);
public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size, boolean filterNonCompetitiveDocs) {
super(fields, size, filterNonCompetitiveDocs);
}
@Override
@ -121,7 +121,7 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
}
// prevent instantiation and extension.
private FieldValueHitQueue(SortField[] fields, int size) {
private FieldValueHitQueue(SortField[] fields, int size, boolean filterNonCompetitiveDocs) {
super(size);
// When we get here, fields.length is guaranteed to be > 0, therefore no
// need to check it again.
@ -135,11 +135,17 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
reverseMul = new int[numComparators];
for (int i = 0; i < numComparators; ++i) {
SortField field = fields[i];
reverseMul[i] = field.reverse ? -1 : 1;
if (i == 0 && filterNonCompetitiveDocs) {
// try to rewrite the 1st comparator to the comparator that can skip non-competitive documents
// skipping functionality is beneficial only for the 1st comparator
comparators[i] = FilteringFieldComparator.wrapToFilteringComparator(field.getComparator(size, i),
field.reverse, numComparators == 1);
} else {
comparators[i] = field.getComparator(size, i);
}
}
}
/**
* Creates a hit queue sorted by the given list of fields.
@ -152,17 +158,20 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
* priority first); cannot be <code>null</code> or empty
* @param size
* The number of hits to retain. Must be greater than zero.
* @param filterNonCompetitiveDocs
* {@code true} If comparators should be allowed to filter non-competitive documents, {@code false} otherwise
*/
public static <T extends FieldValueHitQueue.Entry> FieldValueHitQueue<T> create(SortField[] fields, int size) {
public static <T extends FieldValueHitQueue.Entry> FieldValueHitQueue<T> create(SortField[] fields, int size,
boolean filterNonCompetitiveDocs) {
if (fields.length == 0) {
throw new IllegalArgumentException("Sort must contain at least one field");
}
if (fields.length == 1) {
return new OneComparatorFieldValueHitQueue<>(fields, size);
return new OneComparatorFieldValueHitQueue<>(fields, size, filterNonCompetitiveDocs);
} else {
return new MultiComparatorsFieldValueHitQueue<>(fields, size);
return new MultiComparatorsFieldValueHitQueue<>(fields, size, filterNonCompetitiveDocs);
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.index.LeafReaderContext;
import java.io.IOException;
/**
* A wrapper over {@code FieldComparator} that provides a leaf comparator that can filter non-competitive docs.
*/
abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
protected final FieldComparator<T> in;
protected final boolean reverse;
// singleSort is true, if sort is based on a single sort field. As there are no other sorts configured
// as tie breakers, we can filter out docs with equal values.
protected final boolean singleSort;
protected boolean hasTopValue = false;
public FilteringFieldComparator(FieldComparator<T> in, boolean reverse, boolean singleSort) {
this.in = in;
this.reverse = reverse;
this.singleSort = singleSort;
}
@Override
public abstract FilteringLeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException;
@Override
public int compare(int slot1, int slot2) {
return in.compare(slot1, slot2);
}
@Override
public T value(int slot) {
return in.value(slot);
}
@Override
public void setTopValue(T value) {
in.setTopValue(value);
hasTopValue = true;
}
@Override
public int compareValues(T first, T second) {
return in.compareValues(first, second);
}
/**
* Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
* If for the given comparator the skip functionality is not implemented, return the comparator itself.
* @param comparator comparator to wrap
* @param reverse if this sort is reverse
* @param singleSort true if this sort is based on a single field and there are no other sort fields for tie breaking
* @return comparator wrapped as a filtering comparator or the original comparator if the filtering functionality
* is not implemented for it
*/
public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse, boolean singleSort) {
Class<?> comparatorClass = comparator.getClass();
if (comparatorClass == FieldComparator.LongComparator.class){
return new FilteringNumericComparator<>((FieldComparator.LongComparator) comparator, reverse, singleSort);
}
if (comparatorClass == FieldComparator.IntComparator.class){
return new FilteringNumericComparator<>((FieldComparator.IntComparator) comparator, reverse, singleSort);
}
if (comparatorClass == FieldComparator.DoubleComparator.class){
return new FilteringNumericComparator<>((FieldComparator.DoubleComparator) comparator, reverse, singleSort);
}
if (comparatorClass == FieldComparator.FloatComparator.class){
return new FilteringNumericComparator<>((FieldComparator.FloatComparator) comparator, reverse, singleSort);
}
return comparator;
}
}

View File

@ -0,0 +1,39 @@
/*
* 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.io.IOException;
/**
* Decorates a wrapped LeafFieldComparator to add a functionality to skip over non-competitive docs.
* FilteringLeafFieldComparator provides two additional functions to a LeafFieldComparator:
* {@code competitiveIterator()} and {@code setCanUpdateIterator()}.
*/
public interface FilteringLeafFieldComparator extends LeafFieldComparator {
/**
* Returns a competitive iterator
* @return an iterator over competitive docs that are stronger than already collected docs
* or {@code null} if such an iterator is not available for the current segment.
*/
DocIdSetIterator competitiveIterator() throws IOException;
/**
* Informs this leaf comparator that it is allowed to start updating its competitive iterator.
* This method is called from a collector when queue becomes full and threshold is reached.
*/
void setCanUpdateIterator() throws IOException;
}

View File

@ -0,0 +1,52 @@
/*
* 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.index.LeafReaderContext;
import java.io.IOException;
/**
* A wrapper over {@code NumericComparator} that provides a leaf comparator that can filter non-competitive docs.
*/
class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> {
public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, boolean singleSort) {
super(in, reverse, singleSort);
}
@Override
public final FilteringLeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
LeafFieldComparator inLeafComparator = in.getLeafComparator(context);
Class<?> comparatorClass = inLeafComparator.getClass();
if (comparatorClass == FieldComparator.LongComparator.class) {
return new FilteringNumericLeafComparator.FilteringLongLeafComparator((FieldComparator.LongComparator) inLeafComparator, context,
((LongComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
} if (comparatorClass == FieldComparator.IntComparator.class) {
return new FilteringNumericLeafComparator.FilteringIntLeafComparator((FieldComparator.IntComparator) inLeafComparator, context,
((IntComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
} else if (comparatorClass == FieldComparator.DoubleComparator.class) {
return new FilteringNumericLeafComparator.FilteringDoubleLeafComparator((FieldComparator.DoubleComparator) inLeafComparator, context,
((DoubleComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
} else if (comparatorClass == FieldComparator.FloatComparator.class) {
return new FilteringNumericLeafComparator.FilteringFloatLeafComparator((FieldComparator.FloatComparator) inLeafComparator, context,
((FloatComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
} else {
throw new IllegalStateException("Unexpected numeric class of ["+ comparatorClass + "] for [FieldComparator]!");
}
}
}

View File

@ -0,0 +1,336 @@
/*
* 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.document.DoublePoint;
import org.apache.lucene.document.FloatPoint;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.util.DocIdSetBuilder;
import java.io.IOException;
import java.util.Arrays;
/**
* A {@code FilteringLeafFieldComparator} that provides a functionality to skip over non-competitive documents
* for numeric fields indexed with points.
*/
abstract class FilteringNumericLeafComparator implements FilteringLeafFieldComparator {
protected final LeafFieldComparator in;
protected final boolean reverse;
protected final boolean singleSort; //if sort is based on a single sort field as opposed to multiple sort fields
private final boolean hasTopValue;
private final PointValues pointValues;
private final int bytesCount;
private final int maxDoc;
private final byte[] minValueAsBytes;
private final byte[] maxValueAsBytes;
private long iteratorCost;
private int maxDocVisited = 0;
private int updateCounter = 0;
private boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
private DocIdSetIterator competitiveIterator;
public FilteringNumericLeafComparator(LeafFieldComparator in, LeafReaderContext context, String field,
boolean reverse, boolean singleSort, boolean hasTopValue, int bytesCount) throws IOException {
this.in = in;
this.pointValues = context.reader().getPointValues(field);
this.reverse = reverse;
this.singleSort = singleSort;
this.hasTopValue = hasTopValue;
this.maxDoc = context.reader().maxDoc();
this.bytesCount = bytesCount;
this.maxValueAsBytes = reverse == false ? new byte[bytesCount] : hasTopValue ? new byte[bytesCount] : null;
this.minValueAsBytes = reverse ? new byte[bytesCount] : hasTopValue ? new byte[bytesCount] : null;
// TODO: optimize a case when pointValues are missing only on this segment
this.competitiveIterator = pointValues == null ? null : DocIdSetIterator.all(maxDoc);
this.iteratorCost = maxDoc;
}
@Override
public void setBottom(int slot) throws IOException {
in.setBottom(slot);
updateCompetitiveIterator(); // update an iterator if we set a new bottom
}
@Override
public int compareBottom(int doc) throws IOException {
return in.compareBottom(doc);
}
@Override
public int compareTop(int doc) throws IOException {
return in.compareTop(doc);
}
@Override
public void copy(int slot, int doc) throws IOException {
in.copy(slot, doc);
maxDocVisited = doc;
}
@Override
public void setScorer(Scorable scorer) throws IOException {
in.setScorer(scorer);
if (scorer instanceof Scorer) {
iteratorCost = ((Scorer) scorer).iterator().cost(); // starting iterator cost is the scorer's cost
updateCompetitiveIterator(); // update an iterator when we have a new segment
}
}
@Override
public void setCanUpdateIterator() throws IOException {
this.canUpdateIterator = true;
updateCompetitiveIterator();
}
@Override
public DocIdSetIterator competitiveIterator() {
if (competitiveIterator == null) return null;
return new DocIdSetIterator() {
private int doc;
@Override
public int nextDoc() throws IOException {
return doc = competitiveIterator.nextDoc();
}
@Override
public int docID() {
return doc;
}
@Override
public long cost() {
return competitiveIterator.cost();
}
@Override
public int advance(int target) throws IOException {
return doc = competitiveIterator.advance(target);
}
};
}
// update its iterator to include possibly only docs that are "stronger" than the current bottom entry
private void updateCompetitiveIterator() throws IOException {
if (canUpdateIterator == false) return;
if (pointValues == null) return;
// if some documents have missing points, check that missing values prohibits optimization
if ((pointValues.getDocCount() < maxDoc) && isMissingValueCompetitive()) {
return; // we can't filter out documents, as documents with missing values are competitive
}
updateCounter++;
if (updateCounter > 256 && (updateCounter & 0x1f) != 0x1f) { // Start sampling if we get called too much
return;
}
if (reverse == false) {
encodeBottom(maxValueAsBytes);
if (hasTopValue) {
encodeTop(minValueAsBytes);
}
} else {
encodeBottom(minValueAsBytes);
if (hasTopValue) {
encodeTop(maxValueAsBytes);
}
}
DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
DocIdSetBuilder.BulkAdder adder;
@Override
public void grow(int count) {
adder = result.grow(count);
}
@Override
public void visit(int docID) {
if (docID <= maxDocVisited) {
return; // Already visited or skipped
}
adder.add(docID);
}
@Override
public void visit(int docID, byte[] packedValue) {
if (docID <= maxDocVisited) {
return; // already visited or skipped
}
if (maxValueAsBytes != null) {
int cmp = Arrays.compareUnsigned(packedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount);
// if doc's value is too high or for single sort even equal, it is not competitive and the doc can be skipped
if (cmp > 0 || (singleSort && cmp == 0)) return;
}
if (minValueAsBytes != null) {
int cmp = Arrays.compareUnsigned(packedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount);
// if doc's value is too low or for single sort even equal, it is not competitive and the doc can be skipped
if (cmp < 0 || (singleSort && cmp == 0)) return;
}
adder.add(docID); // doc is competitive
}
@Override
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
if (maxValueAsBytes != null) {
int cmp = Arrays.compareUnsigned(minPackedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount);
if (cmp > 0 || (singleSort && cmp == 0)) return PointValues.Relation.CELL_OUTSIDE_QUERY;
}
if (minValueAsBytes != null) {
int cmp = Arrays.compareUnsigned(maxPackedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount);
if (cmp < 0 || (singleSort && cmp == 0)) return PointValues.Relation.CELL_OUTSIDE_QUERY;
}
if ((maxValueAsBytes != null && Arrays.compareUnsigned(maxPackedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount) > 0) ||
(minValueAsBytes != null && Arrays.compareUnsigned(minPackedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount) < 0)) {
return PointValues.Relation.CELL_CROSSES_QUERY;
}
return PointValues.Relation.CELL_INSIDE_QUERY;
}
};
final long threshold = iteratorCost >>> 3;
long estimatedNumberOfMatches = pointValues.estimatePointCount(visitor); // runs in O(log(numPoints))
if (estimatedNumberOfMatches >= threshold) {
// the new range is not selective enough to be worth materializing, it doesn't reduce number of docs at least 8x
return;
}
pointValues.intersect(visitor);
competitiveIterator = result.build().iterator();
iteratorCost = competitiveIterator.cost();
}
protected abstract boolean isMissingValueCompetitive();
protected abstract void encodeBottom(byte[] packedValue);
protected abstract void encodeTop(byte[] packedValue);
/**
* A wrapper over double long comparator that adds a functionality to filter non-competitive docs.
*/
static class FilteringLongLeafComparator extends FilteringNumericLeafComparator {
public FilteringLongLeafComparator(FieldComparator.LongComparator in, LeafReaderContext context,
String field, boolean reverse, boolean singleSort, boolean hasTopValue) throws IOException {
super(in, context, field, reverse, singleSort, hasTopValue, Long.BYTES);
}
@Override
protected boolean isMissingValueCompetitive() {
int result = Long.compare(((FieldComparator.LongComparator) in).missingValue, ((FieldComparator.LongComparator) in).bottom);
// in reverse (desc) sort missingValue is competitive when it's greater or equal to bottom,
// in asc sort missingValue is competitive when it's smaller or equal to bottom
return reverse ? (result >= 0) : (result <= 0);
}
@Override
protected void encodeBottom(byte[] packedValue) {
LongPoint.encodeDimension(((FieldComparator.LongComparator) in).bottom, packedValue, 0);
}
@Override
protected void encodeTop(byte[] packedValue) {
LongPoint.encodeDimension(((FieldComparator.LongComparator) in).topValue, packedValue, 0);
}
}
/**
* A wrapper over integer leaf comparator that adds a functionality to filter non-competitive docs.
*/
static class FilteringIntLeafComparator extends FilteringNumericLeafComparator {
public FilteringIntLeafComparator(FieldComparator.IntComparator in, LeafReaderContext context,
String field, boolean reverse, boolean singleSort, boolean hasTopValue) throws IOException {
super(in, context, field, reverse, singleSort, hasTopValue, Integer.BYTES);
}
@Override
protected boolean isMissingValueCompetitive() {
int result = Integer.compare(((FieldComparator.IntComparator) in).missingValue, ((FieldComparator.IntComparator) in).bottom);
// in reverse (desc) sort missingValue is competitive when it's greater or equal to bottom,
// in asc sort missingValue is competitive when it's smaller or equal to bottom
return reverse ? (result >= 0) : (result <= 0);
}
@Override
protected void encodeBottom(byte[] packedValue) {
IntPoint.encodeDimension(((FieldComparator.IntComparator) in).bottom, packedValue, 0);
}
@Override
protected void encodeTop(byte[] packedValue) {
IntPoint.encodeDimension(((FieldComparator.IntComparator) in).topValue, packedValue, 0);
}
}
/**
* A wrapper over double leaf comparator that adds a functionality to filter non-competitive docs.
*/
static class FilteringDoubleLeafComparator extends FilteringNumericLeafComparator {
public FilteringDoubleLeafComparator(FieldComparator.DoubleComparator in, LeafReaderContext context,
String field, boolean reverse, boolean singleSort, boolean hasTopValue) throws IOException {
super(in, context, field, reverse, singleSort, hasTopValue, Double.BYTES);
}
@Override
protected boolean isMissingValueCompetitive() {
int result = Double.compare(((FieldComparator.DoubleComparator) in).missingValue, ((FieldComparator.DoubleComparator) in).bottom);
return reverse ? (result >= 0) : (result <= 0);
}
@Override
protected void encodeBottom(byte[] packedValue) {
DoublePoint.encodeDimension(((FieldComparator.DoubleComparator) in).bottom, packedValue, 0);
}
@Override
protected void encodeTop(byte[] packedValue) {
DoublePoint.encodeDimension(((FieldComparator.DoubleComparator) in).topValue, packedValue, 0);
}
}
/**
* A wrapper over float leaf comparator that adds a functionality to filter non-competitive docs.
*/
static class FilteringFloatLeafComparator extends FilteringNumericLeafComparator {
public FilteringFloatLeafComparator(FieldComparator.FloatComparator in, LeafReaderContext context,
String field, boolean reverse, boolean singleSort, boolean hasTopValue) throws IOException {
super(in, context, field, reverse, singleSort, hasTopValue, Float.BYTES);
}
@Override
protected boolean isMissingValueCompetitive() {
int result = Float.compare(((FieldComparator.FloatComparator) in).missingValue, ((FieldComparator.FloatComparator) in).bottom);
return reverse ? (result >= 0) : (result <= 0);
}
@Override
protected void encodeBottom(byte[] packedValue) {
FloatPoint.encodeDimension(((FieldComparator.FloatComparator) in).bottom, packedValue, 0);
}
@Override
protected void encodeTop(byte[] packedValue) {
FloatPoint.encodeDimension(((FieldComparator.FloatComparator) in).topValue, packedValue, 0);
}
}
}

View File

@ -93,4 +93,16 @@ public interface LeafCollector {
*/
void collect(int doc) throws IOException;
/**
* Optionally returns an iterator over competitive documents.
*
* Collectors should delegate this method to their comparators if
* their comparators provide the skipping functionality over non-competitive docs.
*
* The default is to return {@code null} which is interpreted as the collector provide any competitive iterator.
*/
default DocIdSetIterator competitiveIterator() throws IOException {
return null;
}
}

View File

@ -46,7 +46,7 @@ public final class MatchAllDocsQuery extends Query {
@Override
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
if (scoreMode == ScoreMode.TOP_SCORES) {
if (scoreMode.isExhaustive() == false) {
return super.bulkScorer(context);
}
final float score = score();

View File

@ -24,37 +24,53 @@ public enum ScoreMode {
/**
* Produced scorers will allow visiting all matches and get their score.
*/
COMPLETE {
@Override
public boolean needsScores() {
return true;
}
},
COMPLETE(true, true),
/**
* Produced scorers will allow visiting all matches but scores won't be
* available.
*/
COMPLETE_NO_SCORES {
@Override
public boolean needsScores() {
return false;
}
},
COMPLETE_NO_SCORES(true, false),
/**
* Produced scorers will optionally allow skipping over non-competitive
* hits using the {@link Scorer#setMinCompetitiveScore(float)} API.
*/
TOP_SCORES {
@Override
public boolean needsScores() {
return true;
TOP_SCORES(false, true),
/**
* ScoreMode for top field collectors that can provide their own iterators,
* to optionally allow to skip for non-competitive docs
*/
TOP_DOCS(false, false),
/**
* ScoreMode for top field collectors that can provide their own iterators,
* to optionally allow to skip for non-competitive docs.
* This mode is used when there is a secondary sort by _score.
*/
TOP_DOCS_WITH_SCORES(false, true);
private final boolean needsScores;
private final boolean isExhaustive;
ScoreMode(boolean isExhaustive, boolean needsScores) {
this.isExhaustive = isExhaustive;
this.needsScores = needsScores;
}
};
/**
* Whether this {@link ScoreMode} needs to compute scores.
*/
public abstract boolean needsScores();
public boolean needsScores() {
return needsScores;
}
/**
* Returns {@code true} if for this {@link ScoreMode} it is necessary to process all documents,
* or {@code false} if is enough to go through top documents only.
*/
public boolean isExhaustive() {
return isExhaustive;
}
}

View File

@ -33,6 +33,10 @@ import org.apache.lucene.util.NumericUtils;
* Stores information about how to sort documents by terms in an individual
* field. Fields must be indexed in order to sort by them.
*
* Sorting on a numeric field that is indexed with both doc values and points may use an
* optimization to skip non-competitive documents. This optimization relies on the assumption
* that the same data is stored in these points and doc values.
*
* <p>Created: Feb 11, 2004 1:25:29 PM
*
* @since lucene 1.4

View File

@ -49,10 +49,13 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
private static abstract class MultiComparatorLeafCollector implements LeafCollector {
final LeafFieldComparator comparator;
final FilteringLeafFieldComparator filteringLeafComparator;
final int reverseMul;
Scorable scorer;
MultiComparatorLeafCollector(LeafFieldComparator[] comparators, int[] reverseMul) {
this.filteringLeafComparator = comparators[0] instanceof FilteringLeafFieldComparator ?
(FilteringLeafFieldComparator) comparators[0] : null;
if (comparators.length == 1) {
this.reverseMul = reverseMul[0];
this.comparator = comparators[0];
@ -87,6 +90,14 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
if (minScoreAcc != null && (totalHits & minScoreAcc.modInterval) == 0) {
updateGlobalMinCompetitiveScore(scorer);
}
if (filteringLeafComparator != null && queueFull &&
hitsThresholdChecker.isThresholdReached() && totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// for the first time queue becomes full and hitsThreshold is reached,
// notify leaf comparator that its competitive iterator can be updated
filteringLeafComparator.setCanUpdateIterator();
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
boolean thresholdCheck(int doc) throws IOException {
@ -139,9 +150,25 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
if (minScoreAcc != null) {
updateGlobalMinCompetitiveScore(scorer);
}
if (filteringLeafComparator != null && queueFull && hitsThresholdChecker.isThresholdReached()) {
// if queue became full and hitsThreshold was reached in previous segments,
// notify this segment's leaf comparator that its competitive iterator can be updated
filteringLeafComparator.setCanUpdateIterator();
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
}
}
@Override
public DocIdSetIterator competitiveIterator() throws IOException {
if (filteringLeafComparator == null) {
return null;
} else {
return filteringLeafComparator.competitiveIterator();
}
}
}
static boolean canEarlyTerminate(Sort searchSort, Sort indexSort) {
return canEarlyTerminateOnDocId(searchSort) ||
canEarlyTerminateOnPrefix(searchSort, indexSort);
@ -246,18 +273,16 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
return;
}
}
final int topCmp = reverseMul * comparator.compareTop(doc);
if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) {
// Already collected on a previous page
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) {
// we just reached totalHitsThreshold, we can start setting the min
// competitive score now
// check if totalHitsThreshold is reached and we can update competitive score
// necessary to account for possible update to global min competitive score
updateMinCompetitiveScore(scorer);
}
return;
}
if (queueFull) {
collectCompetitiveHit(doc);
} else {
@ -274,7 +299,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final int numHits;
final HitsThresholdChecker hitsThresholdChecker;
final FieldComparator.RelevanceComparator firstComparator;
final FieldComparator.RelevanceComparator relevanceComparator;
final boolean canSetMinScore;
// an accumulator that maintains the maximum of the segment's minimum competitive scores
@ -302,18 +327,24 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
this.numHits = numHits;
this.hitsThresholdChecker = hitsThresholdChecker;
this.numComparators = pq.getComparators().length;
FieldComparator<?> fieldComparator = pq.getComparators()[0];
FieldComparator<?> firstComparator = pq.getComparators()[0];
int reverseMul = pq.reverseMul[0];
if (fieldComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
if (firstComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
&& reverseMul == 1 // if the natural sort is preserved (sort by descending relevance)
&& hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
firstComparator = (FieldComparator.RelevanceComparator) fieldComparator;
relevanceComparator = (FieldComparator.RelevanceComparator) firstComparator;
scoreMode = ScoreMode.TOP_SCORES;
canSetMinScore = true;
} else {
firstComparator = null;
scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
relevanceComparator = null;
canSetMinScore = false;
if (firstComparator instanceof FilteringFieldComparator) {
assert hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE;
scoreMode = needsScores ? ScoreMode.TOP_DOCS_WITH_SCORES : ScoreMode.TOP_DOCS;
} else {
scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
}
}
this.minScoreAcc = minScoreAcc;
}
@ -343,8 +374,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
if (canSetMinScore
&& queueFull
&& hitsThresholdChecker.isThresholdReached()) {
assert bottom != null && firstComparator != null;
float minScore = firstComparator.value(bottom.slot);
assert bottom != null && relevanceComparator != null;
float minScore = relevanceComparator.value(bottom.slot);
if (minScore > minCompetitiveScore) {
scorer.setMinCompetitiveScore(minScore);
minCompetitiveScore = minScore;
@ -356,6 +387,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
}
}
/**
* Creates a new {@link TopFieldCollector} from the given
* arguments.
@ -403,6 +435,11 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* {@code totalHitsThreshold} hits then the hit count of the result will
* be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
* count accurate, but this will also make query processing slower.
* Setting totalHitsThreshold less than {@link Integer#MAX_VALUE}
* instructs Lucene to skip non-competitive documents whenever possible. For numeric
* sort fields the skipping functionality works when the same field is indexed both
* with doc values and points. In this case, there is an assumption that the same data is
* stored in these points and doc values.
* @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria.
*/
@ -432,7 +469,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
throw new IllegalArgumentException("hitsThresholdChecker should not be null");
}
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
// here we assume that if hitsThreshold was set, we let a comparator to skip non-competitive docs
boolean filterNonCompetitiveDocs = hitsThresholdChecker.getHitsThreshold() == Integer.MAX_VALUE ? false : true;
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits, filterNonCompetitiveDocs);
if (after == null) {
return new SimpleFieldCollector(sort, queue, numHits, hitsThresholdChecker, minScoreAcc);

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReader;
@ -201,19 +202,20 @@ public abstract class Weight implements SegmentCacheable {
@Override
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
collector.setScorer(scorer);
DocIdSetIterator scorerIterator = twoPhase == null ? iterator : twoPhase.approximation();
DocIdSetIterator collectorIterator = collector.competitiveIterator();
// if possible filter scorerIterator to keep only competitive docs as defined by collector
DocIdSetIterator filteredIterator = collectorIterator == null ? scorerIterator :
ConjunctionDISI.intersectIterators(Arrays.asList(scorerIterator, collectorIterator));
if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
scoreAll(collector, iterator, twoPhase, acceptDocs);
scoreAll(collector, filteredIterator, twoPhase, acceptDocs);
return DocIdSetIterator.NO_MORE_DOCS;
} else {
int doc = scorer.docID();
if (doc < min) {
if (twoPhase == null) {
doc = iterator.advance(min);
} else {
doc = twoPhase.approximation().advance(min);
doc = scorerIterator.advance(min);
}
}
return scoreRange(collector, iterator, twoPhase, acceptDocs, doc, max);
return scoreRange(collector, filteredIterator, twoPhase, acceptDocs, doc, max);
}
}

View File

@ -0,0 +1,294 @@
/*
* 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.document.Document;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.FloatPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import java.io.IOException;
import static org.apache.lucene.search.SortField.FIELD_SCORE;
public class TestFieldSortOptimizationSkipping extends LuceneTestCase {
public void testLongSortOptimization() throws IOException {
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
final int numDocs = atLeast(10000);
for (int i = 0; i < numDocs; ++i) {
final Document doc = new Document();
doc.add(new NumericDocValuesField("my_field", i));
doc.add(new LongPoint("my_field", i));
writer.addDocument(doc);
if (i == 7000) writer.flush(); // two segments
}
final IndexReader reader = DirectoryReader.open(writer);
IndexSearcher searcher = new IndexSearcher(reader);
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
final Sort sort = new Sort(sortField);
final int numHits = 3;
final int totalHitsThreshold = 3;
{ // simple sort
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(i, ((Long) fieldDoc.fields[0]).intValue());
}
assertTrue(collector.isEarlyTerminated());
assertTrue(topDocs.totalHits.value < numDocs);
}
{ // paging sort with after
long afterValue = 2;
FieldDoc after = new FieldDoc(2, Float.NaN, new Long[] {afterValue});
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(afterValue + 1 + i, fieldDoc.fields[0]);
}
assertTrue(collector.isEarlyTerminated());
assertTrue(topDocs.totalHits.value < numDocs);
}
{ // test that if there is the secondary sort on _score, scores are filled correctly
final TopFieldCollector collector = TopFieldCollector.create(new Sort(sortField, FIELD_SCORE), numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(i, ((Long) fieldDoc.fields[0]).intValue());
float score = (float) fieldDoc.fields[1];
assertEquals(1.0, score, 0.001);
}
assertTrue(collector.isEarlyTerminated());
assertTrue(topDocs.totalHits.value < numDocs);
}
writer.close();
reader.close();
dir.close();
}
/**
* test that even if a field is not indexed with points, optimized sort still works as expected,
* although no optimization will be run
*/
public void testLongSortOptimizationOnFieldNotIndexedWithPoints() throws IOException {
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
final int numDocs = atLeast(100);
// my_field is not indexed with points
for (int i = 0; i < numDocs; ++i) {
final Document doc = new Document();
doc.add(new NumericDocValuesField("my_field", i));
writer.addDocument(doc);
}
final IndexReader reader = DirectoryReader.open(writer);
IndexSearcher searcher = new IndexSearcher(reader);
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
final Sort sort = new Sort(sortField);
final int numHits = 3;
final int totalHitsThreshold = 3;
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits); // sort still works and returns expected number of docs
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(i, ((Long) fieldDoc.fields[0]).intValue()); // returns expected values
}
assertEquals(topDocs.totalHits.value, numDocs); // assert that all documents were collected => optimization was not run
writer.close();
reader.close();
dir.close();
}
public void testSortOptimizationWithMissingValues() throws IOException {
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
final int numDocs = atLeast(10000);
for (int i = 0; i < numDocs; ++i) {
final Document doc = new Document();
if ((i % 500) != 0) { // miss values on every 500th document
doc.add(new NumericDocValuesField("my_field", i));
doc.add(new LongPoint("my_field", i));
}
writer.addDocument(doc);
if (i == 7000) writer.flush(); // two segments
}
final IndexReader reader = DirectoryReader.open(writer);
IndexSearcher searcher = new IndexSearcher(reader);
final int numHits = 3;
final int totalHitsThreshold = 3;
{ // test that optimization is not run when missing value setting of SortField is competitive
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
sortField.setMissingValue(0L); // set a competitive missing value
final Sort sort = new Sort(sortField);
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
assertEquals(topDocs.totalHits.value, numDocs); // assert that all documents were collected => optimization was not run
}
{ // test that optimization is run when missing value setting of SortField is NOT competitive
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
sortField.setMissingValue(100L); // set a NON competitive missing value
final Sort sort = new Sort(sortField);
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
assertTrue(topDocs.totalHits.value < numDocs); // assert that some docs were skipped => optimization was run
}
writer.close();
reader.close();
dir.close();
}
public void testSortOptimizationEqualValues() throws IOException {
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
final int numDocs = atLeast(10000);
for (int i = 1; i <= numDocs; ++i) {
final Document doc = new Document();
doc.add(new NumericDocValuesField("my_field1", 100)); // all docs have the same value of my_field1
doc.add(new IntPoint("my_field1", 100));
doc.add(new NumericDocValuesField("my_field2", numDocs - i)); // diff values for the field my_field2
writer.addDocument(doc);
if (i == 7000) writer.flush(); // two segments
}
final IndexReader reader = DirectoryReader.open(writer);
IndexSearcher searcher = new IndexSearcher(reader);
final int numHits = 3;
final int totalHitsThreshold = 3;
{ // test that sorting on a single field with equal values uses the optimization
final SortField sortField = new SortField("my_field1", SortField.Type.INT);
final Sort sort = new Sort(sortField);
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(100, fieldDoc.fields[0]);
}
assertTrue(topDocs.totalHits.value < numDocs); // assert that some docs were skipped => optimization was run
}
{ // test that sorting on a single field with equal values and after parameter uses the optimization
final int afterValue = 100;
final SortField sortField = new SortField("my_field1", SortField.Type.INT);
final Sort sort = new Sort(sortField);
FieldDoc after = new FieldDoc(10, Float.NaN, new Integer[] {afterValue});
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(100, fieldDoc.fields[0]);
}
assertTrue(topDocs.totalHits.value < numDocs); // assert that some docs were skipped => optimization was run
}
{ // test that sorting on main field with equal values + another field for tie breaks doesn't use optimization
final SortField sortField1 = new SortField("my_field1", SortField.Type.INT);
final SortField sortField2 = new SortField("my_field2", SortField.Type.INT);
final Sort sort = new Sort(sortField1, sortField2);
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(100, fieldDoc.fields[0]); // sort on 1st field as expected
assertEquals(i, fieldDoc.fields[1]); // sort on 2nd field as expected
}
assertEquals(topDocs.scoreDocs.length, numHits);
assertEquals(topDocs.totalHits.value, numDocs); // assert that all documents were collected => optimization was not run
}
writer.close();
reader.close();
dir.close();
}
public void testFloatSortOptimization() throws IOException {
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
final int numDocs = atLeast(10000);
for (int i = 0; i < numDocs; ++i) {
final Document doc = new Document();
float f = 1f * i;
doc.add(new FloatDocValuesField("my_field", f));
doc.add(new FloatPoint("my_field", i));
writer.addDocument(doc);
}
final IndexReader reader = DirectoryReader.open(writer);
IndexSearcher searcher = new IndexSearcher(reader);
final SortField sortField = new SortField("my_field", SortField.Type.FLOAT);
final Sort sort = new Sort(sortField);
final int numHits = 3;
final int totalHitsThreshold = 3;
{ // simple sort
final TopFieldCollector collector = TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
searcher.search(new MatchAllDocsQuery(), collector);
TopDocs topDocs = collector.topDocs();
assertEquals(topDocs.scoreDocs.length, numHits);
for (int i = 0; i < numHits; i++) {
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
assertEquals(1f * i, fieldDoc.fields[0]);
}
assertTrue(collector.isEarlyTerminated());
assertTrue(topDocs.totalHits.value < numDocs);
}
writer.close();
reader.close();
dir.close();
}
}

View File

@ -50,5 +50,9 @@ class AssertingLeafCollector extends FilterLeafCollector {
lastCollected = doc;
}
}
@Override
public DocIdSetIterator competitiveIterator() throws IOException {
return in.competitiveIterator();
}
}