mirror of https://github.com/apache/lucene.git
LUCENE-5702: Move comparators to a per-leaf API.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1649818 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
75f6dd52e4
commit
cdb651976e
|
@ -225,6 +225,9 @@ API Changes
|
|||
* LUCENE-5527: The Collector API has been refactored to use a dedicated Collector
|
||||
per leaf. (Shikhar Bhushan, Adrien Grand)
|
||||
|
||||
* LUCENE-5702: The FieldComparator API has been refactor to a per-leaf API, just
|
||||
like Collectors. (Adrien Grand)
|
||||
|
||||
* LUCENE-4246: IndexWriter.close now always closes, even if it throws
|
||||
an exception. The new IndexWriterConfig.setCommitOnClose (default
|
||||
true) determines whether close() should commit before closing.
|
||||
|
|
|
@ -17,11 +17,12 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -38,46 +39,24 @@ import org.apache.lucene.util.BytesRefBuilder;
|
|||
* sorting, by exposing a tight interaction with {@link
|
||||
* FieldValueHitQueue} as it visits hits. Whenever a hit is
|
||||
* competitive, it's enrolled into a virtual slot, which is
|
||||
* an int ranging from 0 to numHits-1. The {@link
|
||||
* FieldComparator} is made aware of segment transitions
|
||||
* during searching in case any internal state it's tracking
|
||||
* needs to be recomputed during these transitions.</p>
|
||||
*
|
||||
* <p>A comparator must define these functions:</p>
|
||||
*
|
||||
* an int ranging from 0 to numHits-1. Segment transitions are
|
||||
* handled by creating a dedicated per-segment
|
||||
* {@link LeafFieldComparator} which also needs to interact
|
||||
* with the {@link FieldValueHitQueue} but can optimize based
|
||||
* on the segment to collect.</p>
|
||||
*
|
||||
* <p>The following functions need to be implemented</p>
|
||||
* <ul>
|
||||
*
|
||||
* <li> {@link #compare} Compare a hit at 'slot a'
|
||||
* with hit 'slot b'.
|
||||
*
|
||||
* <li> {@link #setBottom} This method is called by
|
||||
* {@link FieldValueHitQueue} to notify the
|
||||
* FieldComparator of the current weakest ("bottom")
|
||||
* slot. Note that this slot may not hold the weakest
|
||||
* value according to your comparator, in cases where
|
||||
* your comparator is not the primary one (ie, is only
|
||||
* used to break ties from the comparators before it).
|
||||
*
|
||||
* <li> {@link #compareBottom} Compare a new hit (docID)
|
||||
* against the "weakest" (bottom) entry in the queue.
|
||||
*
|
||||
*
|
||||
* <li> {@link #setTopValue} This method is called by
|
||||
* {@link TopFieldCollector} to notify the
|
||||
* FieldComparator of the top most value, which is
|
||||
* used by future calls to {@link #compareTop}.
|
||||
*
|
||||
* <li> {@link #compareBottom} Compare a new hit (docID)
|
||||
* against the "weakest" (bottom) entry in the queue.
|
||||
*
|
||||
* <li> {@link #compareTop} Compare a new hit (docID)
|
||||
* against the top value previously set by a call to
|
||||
* {@link #setTopValue}.
|
||||
*
|
||||
* <li> {@link #copy} Installs a new hit into the
|
||||
* priority queue. The {@link FieldValueHitQueue}
|
||||
* calls this method when a new hit is competitive.
|
||||
*
|
||||
* <li> {@link #setNextReader(org.apache.lucene.index.LeafReaderContext)} Invoked
|
||||
* used by future calls to
|
||||
* {@link LeafFieldComparator#compareTop}.
|
||||
*
|
||||
* <li> {@link #getLeafComparator(org.apache.lucene.index.LeafReaderContext)} Invoked
|
||||
* when the search is switching to the next segment.
|
||||
* You may need to update internal state of the
|
||||
* comparator, for example retrieving new values from
|
||||
|
@ -89,6 +68,7 @@ import org.apache.lucene.util.BytesRefBuilder;
|
|||
* FieldDoc#fields} when returning the top results.
|
||||
* </ul>
|
||||
*
|
||||
* @see LeafFieldComparator
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class FieldComparator<T> {
|
||||
|
@ -104,93 +84,14 @@ public abstract class FieldComparator<T> {
|
|||
*/
|
||||
public abstract int compare(int slot1, int slot2);
|
||||
|
||||
/**
|
||||
* Set the bottom slot, ie the "weakest" (sorted last)
|
||||
* entry in the queue. When {@link #compareBottom} is
|
||||
* called, you should compare against this slot. This
|
||||
* will always be called before {@link #compareBottom}.
|
||||
*
|
||||
* @param slot the currently weakest (sorted last) slot in the queue
|
||||
*/
|
||||
public abstract void setBottom(final int slot);
|
||||
|
||||
/**
|
||||
* Record the top value, for future calls to {@link
|
||||
* #compareTop}. This is only called for searches that
|
||||
* LeafFieldComparator#compareTop}. This is only called for searches that
|
||||
* use searchAfter (deep paging), and is called before any
|
||||
* calls to {@link #setNextReader}.
|
||||
* calls to {@link #getLeafComparator(LeafReaderContext)}.
|
||||
*/
|
||||
public abstract void setTopValue(T value);
|
||||
|
||||
/**
|
||||
* Compare the bottom of the queue with this doc. This will
|
||||
* only invoked after setBottom has been called. This
|
||||
* should return the same result as {@link
|
||||
* #compare(int,int)}} as if bottom were slot1 and the new
|
||||
* document were slot 2.
|
||||
*
|
||||
* <p>For a search that hits many results, this method
|
||||
* will be the hotspot (invoked by far the most
|
||||
* frequently).</p>
|
||||
*
|
||||
* @param doc that was hit
|
||||
* @return any {@code N < 0} if the doc's value is sorted after
|
||||
* the bottom entry (not competitive), any {@code N > 0} if the
|
||||
* doc's value is sorted before the bottom entry and {@code 0} if
|
||||
* they are equal.
|
||||
*/
|
||||
public abstract int compareBottom(int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* Compare the top value with this doc. This will
|
||||
* only invoked after setTopValue has been called. This
|
||||
* should return the same result as {@link
|
||||
* #compare(int,int)}} as if topValue were slot1 and the new
|
||||
* document were slot 2. This is only called for searches that
|
||||
* use searchAfter (deep paging).
|
||||
*
|
||||
* @param doc that was hit
|
||||
* @return any {@code N < 0} if the doc's value is sorted after
|
||||
* the bottom entry (not competitive), any {@code N > 0} if the
|
||||
* doc's value is sorted before the bottom entry and {@code 0} if
|
||||
* they are equal.
|
||||
*/
|
||||
public abstract int compareTop(int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* This method is called when a new hit is competitive.
|
||||
* You should copy any state associated with this document
|
||||
* that will be required for future comparisons, into the
|
||||
* specified slot.
|
||||
*
|
||||
* @param slot which slot to copy the hit to
|
||||
* @param doc docID relative to current reader
|
||||
*/
|
||||
public abstract void copy(int slot, int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* Set a new {@link org.apache.lucene.index.LeafReaderContext}. All subsequent docIDs are relative to
|
||||
* the current reader (you must add docBase if you need to
|
||||
* map it to a top-level docID).
|
||||
*
|
||||
* @param context current reader context
|
||||
* @return the comparator to use for this segment; most
|
||||
* comparators can just return "this" to reuse the same
|
||||
* comparator across segments
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public abstract FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException;
|
||||
|
||||
/** Sets the Scorer to use in case a document's score is
|
||||
* needed.
|
||||
*
|
||||
* @param scorer Scorer instance that you should use to
|
||||
* obtain the current hit's score, if necessary. */
|
||||
public void setScorer(Scorer scorer) {
|
||||
// Empty implementation since most comparators don't need the score. This
|
||||
// can be overridden by those that need it.
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the actual value in the slot.
|
||||
*
|
||||
|
@ -199,7 +100,20 @@ public abstract class FieldComparator<T> {
|
|||
*/
|
||||
public abstract T value(int slot);
|
||||
|
||||
/** Returns -1 if first is less than second. Default
|
||||
/**
|
||||
* Get a per-segment {@link LeafFieldComparator} to collect the given
|
||||
* {@link org.apache.lucene.index.LeafReaderContext}. All docIDs supplied to
|
||||
* this {@link LeafFieldComparator} are relative to the current reader (you
|
||||
* must add docBase if you need to map it to a top-level docID).
|
||||
*
|
||||
* @param context current reader context
|
||||
* @return the comparator to use for this segment
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public abstract LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException;
|
||||
|
||||
/** Returns a negative integer if first is less than second,
|
||||
* 0 if they are equal and a positive integer otherwise. Default
|
||||
* impl to assume the type implements Comparable and
|
||||
* invoke .compareTo; be sure to override this method if
|
||||
* your FieldComparator's type isn't a Comparable or
|
||||
|
@ -219,10 +133,11 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Base FieldComparator class for numeric types
|
||||
*/
|
||||
public static abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
|
||||
public static abstract class NumericComparator<T extends Number> extends SimpleFieldComparator<T> {
|
||||
protected final T missingValue;
|
||||
protected final String field;
|
||||
protected Bits docsWithField;
|
||||
|
@ -234,7 +149,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
currentReaderValues = getNumericDocValues(context, field);
|
||||
if (missingValue != null) {
|
||||
docsWithField = DocValues.getDocsWithField(context.reader(), field);
|
||||
|
@ -245,7 +160,6 @@ public abstract class FieldComparator<T> {
|
|||
} else {
|
||||
docsWithField = null;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/** Retrieves the NumericDocValues for the field in this segment */
|
||||
|
@ -551,7 +465,7 @@ public abstract class FieldComparator<T> {
|
|||
* using {@link TopScoreDocCollector} directly (which {@link
|
||||
* IndexSearcher#search} uses when no {@link Sort} is
|
||||
* specified). */
|
||||
public static final class RelevanceComparator extends FieldComparator<Float> {
|
||||
public static final class RelevanceComparator extends FieldComparator<Float> implements LeafFieldComparator {
|
||||
private final float[] scores;
|
||||
private float bottom;
|
||||
private Scorer scorer;
|
||||
|
@ -581,7 +495,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Float> setNextReader(LeafReaderContext context) {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) {
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -629,7 +543,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Sorts by ascending docID */
|
||||
public static final class DocComparator extends FieldComparator<Integer> {
|
||||
public static final class DocComparator extends FieldComparator<Integer> implements LeafFieldComparator {
|
||||
private final int[] docIDs;
|
||||
private int docBase;
|
||||
private int bottom;
|
||||
|
@ -658,7 +572,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Integer> setNextReader(LeafReaderContext context) {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) {
|
||||
// TODO: can we "map" our docIDs to the current
|
||||
// reader? saves having to then subtract on every
|
||||
// compare call
|
||||
|
@ -686,6 +600,9 @@ public abstract class FieldComparator<T> {
|
|||
int docValue = docBase + doc;
|
||||
return Integer.compare(topValue, docValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
}
|
||||
|
||||
/** Sorts by field's natural Term sort order, using
|
||||
|
@ -697,7 +614,7 @@ public abstract class FieldComparator<T> {
|
|||
* to large results, this comparator will be much faster
|
||||
* than {@link org.apache.lucene.search.FieldComparator.TermValComparator}. For very small
|
||||
* result sets it may be slower. */
|
||||
public static class TermOrdValComparator extends FieldComparator<BytesRef> {
|
||||
public static class TermOrdValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
|
||||
/* Ords for each slot.
|
||||
@lucene.internal */
|
||||
final int[] ords;
|
||||
|
@ -841,7 +758,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
termsIndex = getSortedDocValues(context, field);
|
||||
currentReaderGen++;
|
||||
|
||||
|
@ -859,7 +776,7 @@ public abstract class FieldComparator<T> {
|
|||
topOrd = missingOrd;
|
||||
topSameReader = true;
|
||||
}
|
||||
//System.out.println(" setNextReader topOrd=" + topOrd + " topSameReader=" + topSameReader);
|
||||
//System.out.println(" getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
|
||||
|
||||
if (bottomSlot != -1) {
|
||||
// Recompute bottomOrd/SameReader
|
||||
|
@ -947,13 +864,16 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
return val1.compareTo(val2);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
}
|
||||
|
||||
/** Sorts by field's natural Term sort order. All
|
||||
* comparisons are done using BytesRef.compareTo, which is
|
||||
* slow for medium to large result sets but possibly
|
||||
* very fast for very small results sets. */
|
||||
public static class TermValComparator extends FieldComparator<BytesRef> {
|
||||
public static class TermValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
|
||||
|
||||
private final BytesRef[] values;
|
||||
private final BytesRefBuilder[] tempBRs;
|
||||
|
@ -1019,7 +939,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
docTerms = getBinaryDocValues(context, field);
|
||||
docsWithField = getDocsWithField(context, field);
|
||||
if (docsWithField instanceof Bits.MatchAllBits) {
|
||||
|
@ -1075,5 +995,8 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/**
|
||||
|
@ -53,17 +54,17 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
* there is just one comparator.
|
||||
*/
|
||||
private static final class OneComparatorFieldValueHitQueue<T extends FieldValueHitQueue.Entry> extends FieldValueHitQueue<T> {
|
||||
|
||||
private final int oneReverseMul;
|
||||
private final FieldComparator<?> oneComparator;
|
||||
|
||||
public OneComparatorFieldValueHitQueue(SortField[] fields, int size)
|
||||
throws IOException {
|
||||
super(fields, size);
|
||||
|
||||
SortField field = fields[0];
|
||||
setComparator(0,field.getComparator(size, 0));
|
||||
oneReverseMul = field.reverse ? -1 : 1;
|
||||
|
||||
reverseMul[0] = oneReverseMul;
|
||||
assert fields.length == 1;
|
||||
oneComparator = comparators[0];
|
||||
oneReverseMul = reverseMul[0];
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -78,7 +79,7 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
assert hitA != hitB;
|
||||
assert hitA.slot != hitB.slot;
|
||||
|
||||
final int c = oneReverseMul * firstComparator.compare(hitA.slot, hitB.slot);
|
||||
final int c = oneReverseMul * oneComparator.compare(hitA.slot, hitB.slot);
|
||||
if (c != 0) {
|
||||
return c > 0;
|
||||
}
|
||||
|
@ -98,14 +99,6 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size)
|
||||
throws IOException {
|
||||
super(fields, size);
|
||||
|
||||
int numComparators = comparators.length;
|
||||
for (int i = 0; i < numComparators; ++i) {
|
||||
SortField field = fields[i];
|
||||
|
||||
reverseMul[i] = field.reverse ? -1 : 1;
|
||||
setComparator(i, field.getComparator(size, i));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -130,8 +123,7 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
}
|
||||
|
||||
// prevent instantiation and extension.
|
||||
@SuppressWarnings({"rawtypes","unchecked"})
|
||||
private FieldValueHitQueue(SortField[] fields, int size) {
|
||||
private FieldValueHitQueue(SortField[] fields, int size) throws IOException {
|
||||
super(size);
|
||||
// When we get here, fields.length is guaranteed to be > 0, therefore no
|
||||
// need to check it again.
|
||||
|
@ -141,8 +133,14 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
// anyway.
|
||||
this.fields = fields;
|
||||
int numComparators = fields.length;
|
||||
comparators = new FieldComparator[numComparators];
|
||||
comparators = new FieldComparator<?>[numComparators];
|
||||
reverseMul = new int[numComparators];
|
||||
for (int i = 0; i < numComparators; ++i) {
|
||||
SortField field = fields[i];
|
||||
|
||||
reverseMul[i] = field.reverse ? -1 : 1;
|
||||
comparators[i] = field.getComparator(size, i);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -179,15 +177,17 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
|
|||
return reverseMul;
|
||||
}
|
||||
|
||||
public void setComparator(int pos, FieldComparator<?> comparator) {
|
||||
if (pos==0) firstComparator = comparator;
|
||||
comparators[pos] = comparator;
|
||||
public LeafFieldComparator[] getComparators(LeafReaderContext context) throws IOException {
|
||||
LeafFieldComparator[] comparators = new LeafFieldComparator[this.comparators.length];
|
||||
for (int i = 0; i < comparators.length; ++i) {
|
||||
comparators[i] = this.comparators[i].getLeafComparator(context);
|
||||
}
|
||||
return comparators;
|
||||
}
|
||||
|
||||
/** Stores the sort criteria being used. */
|
||||
protected final SortField[] fields;
|
||||
protected final FieldComparator<?>[] comparators; // use setComparator to change this array
|
||||
protected FieldComparator<?> firstComparator; // this must always be equal to comparators[0]
|
||||
protected final FieldComparator<?>[] comparators;
|
||||
protected final int[] reverseMul;
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Expert: comparator that gets instantiated on each leaf
|
||||
* from a top-level {@link FieldComparator} instance.
|
||||
*
|
||||
* <p>A leaf comparator must define these functions:</p>
|
||||
*
|
||||
* <ul>
|
||||
*
|
||||
* <li> {@link #setBottom} This method is called by
|
||||
* {@link FieldValueHitQueue} to notify the
|
||||
* FieldComparator of the current weakest ("bottom")
|
||||
* slot. Note that this slot may not hold the weakest
|
||||
* value according to your comparator, in cases where
|
||||
* your comparator is not the primary one (ie, is only
|
||||
* used to break ties from the comparators before it).
|
||||
*
|
||||
* <li> {@link #compareBottom} Compare a new hit (docID)
|
||||
* against the "weakest" (bottom) entry in the queue.
|
||||
*
|
||||
* <li> {@link #compareBottom} Compare a new hit (docID)
|
||||
* against the "weakest" (bottom) entry in the queue.
|
||||
*
|
||||
* <li> {@link #compareTop} Compare a new hit (docID)
|
||||
* against the top value previously set by a call to
|
||||
* {@link FieldComparator#setTopValue}.
|
||||
*
|
||||
* <li> {@link #copy} Installs a new hit into the
|
||||
* priority queue. The {@link FieldValueHitQueue}
|
||||
* calls this method when a new hit is competitive.
|
||||
*
|
||||
* </ul>
|
||||
*
|
||||
* @see FieldComparator
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public interface LeafFieldComparator {
|
||||
|
||||
/**
|
||||
* Set the bottom slot, ie the "weakest" (sorted last)
|
||||
* entry in the queue. When {@link #compareBottom} is
|
||||
* called, you should compare against this slot. This
|
||||
* will always be called before {@link #compareBottom}.
|
||||
*
|
||||
* @param slot the currently weakest (sorted last) slot in the queue
|
||||
*/
|
||||
void setBottom(final int slot);
|
||||
|
||||
/**
|
||||
* Compare the bottom of the queue with this doc. This will
|
||||
* only invoked after setBottom has been called. This
|
||||
* should return the same result as {@link
|
||||
* FieldComparator#compare(int,int)}} as if bottom were slot1 and the new
|
||||
* document were slot 2.
|
||||
*
|
||||
* <p>For a search that hits many results, this method
|
||||
* will be the hotspot (invoked by far the most
|
||||
* frequently).</p>
|
||||
*
|
||||
* @param doc that was hit
|
||||
* @return any {@code N < 0} if the doc's value is sorted after
|
||||
* the bottom entry (not competitive), any {@code N > 0} if the
|
||||
* doc's value is sorted before the bottom entry and {@code 0} if
|
||||
* they are equal.
|
||||
*/
|
||||
int compareBottom(int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* Compare the top value with this doc. This will
|
||||
* only invoked after setTopValue has been called. This
|
||||
* should return the same result as {@link
|
||||
* FieldComparator#compare(int,int)}} as if topValue were slot1 and the new
|
||||
* document were slot 2. This is only called for searches that
|
||||
* use searchAfter (deep paging).
|
||||
*
|
||||
* @param doc that was hit
|
||||
* @return any {@code N < 0} if the doc's value is sorted after
|
||||
* the bottom entry (not competitive), any {@code N > 0} if the
|
||||
* doc's value is sorted before the bottom entry and {@code 0} if
|
||||
* they are equal.
|
||||
*/
|
||||
int compareTop(int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* This method is called when a new hit is competitive.
|
||||
* You should copy any state associated with this document
|
||||
* that will be required for future comparisons, into the
|
||||
* specified slot.
|
||||
*
|
||||
* @param slot which slot to copy the hit to
|
||||
* @param doc docID relative to current reader
|
||||
*/
|
||||
void copy(int slot, int doc) throws IOException;
|
||||
|
||||
/** Sets the Scorer to use in case a document's score is
|
||||
* needed.
|
||||
*
|
||||
* @param scorer Scorer instance that you should use to
|
||||
* obtain the current hit's score, if necessary. */
|
||||
void setScorer(Scorer scorer);
|
||||
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* Base {@link FieldComparator} implementation that is used for all contexts.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class SimpleFieldComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
|
||||
|
||||
/** This method is called before collecting <code>context</code>. */
|
||||
protected abstract void doSetNextReader(LeafReaderContext context) throws IOException;
|
||||
|
||||
@Override
|
||||
public final LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
doSetNextReader(context);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
}
|
|
@ -61,6 +61,7 @@ public class SortRescorer extends Rescorer {
|
|||
int endDoc = 0;
|
||||
int docBase = 0;
|
||||
|
||||
LeafCollector leafCollector = null;
|
||||
FakeScorer fakeScorer = new FakeScorer();
|
||||
|
||||
while (hitUpto < hits.length) {
|
||||
|
@ -75,15 +76,15 @@ public class SortRescorer extends Rescorer {
|
|||
|
||||
if (readerContext != null) {
|
||||
// We advanced to another segment:
|
||||
collector.getLeafCollector(readerContext);
|
||||
collector.setScorer(fakeScorer);
|
||||
leafCollector = collector.getLeafCollector(readerContext);
|
||||
leafCollector.setScorer(fakeScorer);
|
||||
docBase = readerContext.docBase;
|
||||
}
|
||||
|
||||
fakeScorer.score = hit.score;
|
||||
fakeScorer.doc = docID - docBase;
|
||||
|
||||
collector.collect(fakeScorer.doc);
|
||||
leafCollector.collect(fakeScorer.doc);
|
||||
|
||||
hitUpto++;
|
||||
}
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.lucene.util.PriorityQueue;
|
|||
* however, you might want to consider overriding all methods, in order to avoid
|
||||
* a NullPointerException.
|
||||
*/
|
||||
public abstract class TopDocsCollector<T extends ScoreDoc> extends SimpleCollector {
|
||||
public abstract class TopDocsCollector<T extends ScoreDoc> implements Collector {
|
||||
|
||||
/** This is used in case topDocs() is called with illegal parameters, or there
|
||||
* simply aren't (enough) results. */
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -36,198 +36,189 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
*/
|
||||
public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
||||
|
||||
// Assumes docs are scored in order.
|
||||
private static class InOrderTopScoreDocCollector extends TopScoreDocCollector {
|
||||
private InOrderTopScoreDocCollector(int numHits) {
|
||||
super(numHits);
|
||||
private abstract static class ScorerLeafCollector implements LeafCollector {
|
||||
|
||||
final boolean scoreDocsInOrder;
|
||||
|
||||
ScorerLeafCollector(boolean scoreDocsInOrder) {
|
||||
this.scoreDocsInOrder = scoreDocsInOrder;
|
||||
}
|
||||
|
||||
|
||||
Scorer scorer;
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle these scores:
|
||||
assert score != Float.NEGATIVE_INFINITY;
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score <= pqTop.score) {
|
||||
// Since docs are returned in-order (i.e., increasing doc Id), a document
|
||||
// with equal score to pqTop.score cannot compete since HitQueue favors
|
||||
// documents with lower doc Ids. Therefore reject those docs too.
|
||||
return;
|
||||
}
|
||||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
return scoreDocsInOrder == false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Assumes docs are scored in order.
|
||||
private static class InOrderPagingScoreDocCollector extends TopScoreDocCollector {
|
||||
|
||||
private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
|
||||
|
||||
private final boolean scoreDocsInOrder;
|
||||
|
||||
SimpleTopScoreDocCollector(int numHits, boolean scoreDocsInOrder) {
|
||||
super(numHits);
|
||||
this.scoreDocsInOrder = scoreDocsInOrder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context)
|
||||
throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
if (scoreDocsInOrder) {
|
||||
return new ScorerLeafCollector(scoreDocsInOrder) {
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle these scores:
|
||||
assert score != Float.NEGATIVE_INFINITY;
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score <= pqTop.score) {
|
||||
// Since docs are returned in-order (i.e., increasing doc Id), a document
|
||||
// with equal score to pqTop.score cannot compete since HitQueue favors
|
||||
// documents with lower doc Ids. Therefore reject those docs too.
|
||||
return;
|
||||
}
|
||||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
|
||||
};
|
||||
} else {
|
||||
return new ScorerLeafCollector(scoreDocsInOrder) {
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle NaN
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score < pqTop.score) {
|
||||
// Doesn't compete w/ bottom entry in queue
|
||||
return;
|
||||
}
|
||||
doc += docBase;
|
||||
if (score == pqTop.score && doc > pqTop.doc) {
|
||||
// Break tie in score by doc ID:
|
||||
return;
|
||||
}
|
||||
pqTop.doc = doc;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
|
||||
|
||||
private final boolean scoreDocsInOrder;
|
||||
private final ScoreDoc after;
|
||||
// this is always after.doc - docBase, to save an add when score == after.score
|
||||
private int afterDoc;
|
||||
private int collectedHits;
|
||||
|
||||
private InOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
|
||||
PagingTopScoreDocCollector(int numHits, boolean scoreDocsInOrder, ScoreDoc after) {
|
||||
super(numHits);
|
||||
this.scoreDocsInOrder = scoreDocsInOrder;
|
||||
this.after = after;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle these scores:
|
||||
assert score != Float.NEGATIVE_INFINITY;
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
|
||||
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||
// hit was collected on a previous page
|
||||
return;
|
||||
}
|
||||
|
||||
if (score <= pqTop.score) {
|
||||
// Since docs are returned in-order (i.e., increasing doc Id), a document
|
||||
// with equal score to pqTop.score cannot compete since HitQueue favors
|
||||
// documents with lower doc Ids. Therefore reject those docs too.
|
||||
return;
|
||||
}
|
||||
collectedHits++;
|
||||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
afterDoc = after.doc - context.docBase;
|
||||
this.collectedHits = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int topDocsSize() {
|
||||
return collectedHits < pq.size() ? collectedHits : pq.size();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
|
||||
return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
|
||||
}
|
||||
}
|
||||
|
||||
// Assumes docs are scored out of order.
|
||||
private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
|
||||
private OutOfOrderTopScoreDocCollector(int numHits) {
|
||||
super(numHits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
final int afterDoc = after.doc - context.docBase;
|
||||
if (scoreDocsInOrder) {
|
||||
return new ScorerLeafCollector(scoreDocsInOrder) {
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle NaN
|
||||
assert !Float.isNaN(score);
|
||||
// This collector cannot handle these scores:
|
||||
assert score != Float.NEGATIVE_INFINITY;
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score < pqTop.score) {
|
||||
// Doesn't compete w/ bottom entry in queue
|
||||
return;
|
||||
totalHits++;
|
||||
|
||||
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||
// hit was collected on a previous page
|
||||
return;
|
||||
}
|
||||
|
||||
if (score <= pqTop.score) {
|
||||
// Since docs are returned in-order (i.e., increasing doc Id), a document
|
||||
// with equal score to pqTop.score cannot compete since HitQueue favors
|
||||
// documents with lower doc Ids. Therefore reject those docs too.
|
||||
return;
|
||||
}
|
||||
collectedHits++;
|
||||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ScorerLeafCollector(scoreDocsInOrder) {
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle NaN
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||
// hit was collected on a previous page
|
||||
return;
|
||||
}
|
||||
if (score < pqTop.score) {
|
||||
// Doesn't compete w/ bottom entry in queue
|
||||
return;
|
||||
}
|
||||
doc += docBase;
|
||||
if (score == pqTop.score && doc > pqTop.doc) {
|
||||
// Break tie in score by doc ID:
|
||||
return;
|
||||
}
|
||||
collectedHits++;
|
||||
pqTop.doc = doc;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
};
|
||||
}
|
||||
doc += docBase;
|
||||
if (score == pqTop.score && doc > pqTop.doc) {
|
||||
// Break tie in score by doc ID:
|
||||
return;
|
||||
}
|
||||
pqTop.doc = doc;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// Assumes docs are scored out of order.
|
||||
private static class OutOfOrderPagingScoreDocCollector extends TopScoreDocCollector {
|
||||
private final ScoreDoc after;
|
||||
// this is always after.doc - docBase, to save an add when score == after.score
|
||||
private int afterDoc;
|
||||
private int collectedHits;
|
||||
|
||||
private OutOfOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
|
||||
super(numHits);
|
||||
this.after = after;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
float score = scorer.score();
|
||||
|
||||
// This collector cannot handle NaN
|
||||
assert !Float.isNaN(score);
|
||||
|
||||
totalHits++;
|
||||
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||
// hit was collected on a previous page
|
||||
return;
|
||||
}
|
||||
if (score < pqTop.score) {
|
||||
// Doesn't compete w/ bottom entry in queue
|
||||
return;
|
||||
}
|
||||
doc += docBase;
|
||||
if (score == pqTop.score && doc > pqTop.doc) {
|
||||
// Break tie in score by doc ID:
|
||||
return;
|
||||
}
|
||||
collectedHits++;
|
||||
pqTop.doc = doc;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
afterDoc = after.doc - context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int topDocsSize() {
|
||||
return collectedHits < pq.size() ? collectedHits : pq.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
|
||||
return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link TopScoreDocCollector} given the number of hits to
|
||||
* collect and whether documents are scored in order by the input
|
||||
* {@link Scorer} to {@link #setScorer(Scorer)}.
|
||||
* {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
|
||||
*
|
||||
* <p><b>NOTE</b>: The instances returned by this method
|
||||
* pre-allocate a full array of length
|
||||
|
@ -237,11 +228,11 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
|
||||
return create(numHits, null, docsScoredInOrder);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new {@link TopScoreDocCollector} given the number of hits to
|
||||
* collect, the bottom of the previous page, and whether documents are scored in order by the input
|
||||
* {@link Scorer} to {@link #setScorer(Scorer)}.
|
||||
* {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
|
||||
*
|
||||
* <p><b>NOTE</b>: The instances returned by this method
|
||||
* pre-allocate a full array of length
|
||||
|
@ -249,27 +240,20 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
* objects.
|
||||
*/
|
||||
public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) {
|
||||
|
||||
|
||||
if (numHits <= 0) {
|
||||
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
|
||||
}
|
||||
|
||||
if (docsScoredInOrder) {
|
||||
return after == null
|
||||
? new InOrderTopScoreDocCollector(numHits)
|
||||
: new InOrderPagingScoreDocCollector(after, numHits);
|
||||
|
||||
if (after == null) {
|
||||
return new SimpleTopScoreDocCollector(numHits, docsScoredInOrder);
|
||||
} else {
|
||||
return after == null
|
||||
? new OutOfOrderTopScoreDocCollector(numHits)
|
||||
: new OutOfOrderPagingScoreDocCollector(after, numHits);
|
||||
return new PagingTopScoreDocCollector(numHits, docsScoredInOrder, after);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
ScoreDoc pqTop;
|
||||
int docBase = 0;
|
||||
Scorer scorer;
|
||||
|
||||
|
||||
// prevents instantiation
|
||||
private TopScoreDocCollector(int numHits) {
|
||||
super(new HitQueue(numHits, true));
|
||||
|
@ -283,7 +267,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
if (results == null) {
|
||||
return EMPTY_TOPDOCS;
|
||||
}
|
||||
|
||||
|
||||
// We need to compute maxScore in order to set it in TopDocs. If start == 0,
|
||||
// it means the largest element is already in results, use its score as
|
||||
// maxScore. Otherwise pop everything else, until the largest element is
|
||||
|
@ -295,17 +279,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
for (int i = pq.size(); i > 1; i--) { pq.pop(); }
|
||||
maxScore = pq.pop().score;
|
||||
}
|
||||
|
||||
|
||||
return new TopDocs(totalHits, results, maxScore);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -98,43 +98,23 @@ final class JustCompileSearch {
|
|||
|
||||
static final class JustCompileFieldComparator extends FieldComparator<Object> {
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTopValue(Object value) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Object> setNextReader(LeafReaderContext context) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object value(int slot) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTop(int doc) {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
}
|
||||
|
@ -260,23 +240,8 @@ final class JustCompileSearch {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.search.similarities.DefaultSimilarity;
|
|||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
@ -146,61 +147,67 @@ class ElevationComparatorSource extends FieldComparatorSource {
|
|||
public FieldComparator<Integer> newComparator(final String fieldname, final int numHits, int sortPos, boolean reversed) throws IOException {
|
||||
return new FieldComparator<Integer>() {
|
||||
|
||||
SortedDocValues idIndex;
|
||||
private final int[] values = new int[numHits];
|
||||
int bottomVal;
|
||||
|
||||
@Override
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context)
|
||||
throws IOException {
|
||||
final SortedDocValues idIndex = DocValues.getSorted(context.reader(), fieldname);
|
||||
return new LeafFieldComparator() {
|
||||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
bottomVal = values[slot];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTop(int doc) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private int docVal(int doc) {
|
||||
int ord = idIndex.getOrd(doc);
|
||||
if (ord == -1) {
|
||||
return 0;
|
||||
} else {
|
||||
final BytesRef term = idIndex.lookupOrd(ord);
|
||||
Integer prio = priority.get(term);
|
||||
return prio == null ? 0 : prio.intValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
return docVal(doc) - bottomVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
values[slot] = docVal(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
return values[slot2] - values[slot1]; // values will be small enough that there is no overflow concern
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
bottomVal = values[slot];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTopValue(Integer value) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private int docVal(int doc) {
|
||||
int ord = idIndex.getOrd(doc);
|
||||
if (ord == -1) {
|
||||
return 0;
|
||||
} else {
|
||||
final BytesRef term = idIndex.lookupOrd(ord);
|
||||
Integer prio = priority.get(term);
|
||||
return prio == null ? 0 : prio.intValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
return docVal(doc) - bottomVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
values[slot] = docVal(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Integer> setNextReader(LeafReaderContext context) throws IOException {
|
||||
idIndex = DocValues.getSorted(context.reader(), fieldname);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer value(int slot) {
|
||||
return Integer.valueOf(values[slot]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTop(int doc) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -785,4 +785,63 @@ public class TestSort extends LuceneTestCase {
|
|||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on multiple sort fields */
|
||||
public void testMultiSort() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new SortedDocValuesField("value1", new BytesRef("foo")));
|
||||
doc.add(new NumericDocValuesField("value2", 0));
|
||||
doc.add(newStringField("value1", "foo", Field.Store.YES));
|
||||
doc.add(newStringField("value2", "0", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new SortedDocValuesField("value1", new BytesRef("bar")));
|
||||
doc.add(new NumericDocValuesField("value2", 1));
|
||||
doc.add(newStringField("value1", "bar", Field.Store.YES));
|
||||
doc.add(newStringField("value2", "1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new SortedDocValuesField("value1", new BytesRef("bar")));
|
||||
doc.add(new NumericDocValuesField("value2", 0));
|
||||
doc.add(newStringField("value1", "bar", Field.Store.YES));
|
||||
doc.add(newStringField("value2", "0", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new SortedDocValuesField("value1", new BytesRef("foo")));
|
||||
doc.add(new NumericDocValuesField("value2", 1));
|
||||
doc.add(newStringField("value1", "foo", Field.Store.YES));
|
||||
doc.add(newStringField("value2", "1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Sort sort = new Sort(
|
||||
new SortField("value1", SortField.Type.STRING),
|
||||
new SortField("value2", SortField.Type.LONG));
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(4, td.totalHits);
|
||||
// 'bar' comes before 'foo'
|
||||
assertEquals("bar", searcher.doc(td.scoreDocs[0].doc).get("value1"));
|
||||
assertEquals("bar", searcher.doc(td.scoreDocs[1].doc).get("value1"));
|
||||
assertEquals("foo", searcher.doc(td.scoreDocs[2].doc).get("value1"));
|
||||
assertEquals("foo", searcher.doc(td.scoreDocs[3].doc).get("value1"));
|
||||
// 0 comes before 1
|
||||
assertEquals("0", searcher.doc(td.scoreDocs[0].doc).get("value2"));
|
||||
assertEquals("1", searcher.doc(td.scoreDocs[1].doc).get("value2"));
|
||||
assertEquals("0", searcher.doc(td.scoreDocs[2].doc).get("value2"));
|
||||
assertEquals("1", searcher.doc(td.scoreDocs[3].doc).get("value2"));
|
||||
|
||||
// Now with overflow
|
||||
td = searcher.search(new MatchAllDocsQuery(), 1, sort);
|
||||
assertEquals(4, td.totalHits);
|
||||
assertEquals("bar", searcher.doc(td.scoreDocs[0].doc).get("value1"));
|
||||
assertEquals("0", searcher.doc(td.scoreDocs[0].doc).get("value2"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
private static final class MyTopsDocCollector extends TopDocsCollector<ScoreDoc> {
|
||||
|
||||
private int idx = 0;
|
||||
private int base = 0;
|
||||
|
||||
public MyTopsDocCollector(int size) {
|
||||
super(new HitQueue(size, false));
|
||||
|
@ -55,24 +54,26 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) {
|
||||
++totalHits;
|
||||
pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++]));
|
||||
}
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int base = context.docBase;
|
||||
return new LeafCollector() {
|
||||
|
||||
@Override
|
||||
public void collect(int doc) {
|
||||
++totalHits;
|
||||
pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++]));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
// Don't do anything. Assign scores in random
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
// Don't do anything. Assign scores in random
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -171,14 +171,14 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
new boolean[] { true, true, true },
|
||||
};
|
||||
String[] actualTFCClasses = new String[] {
|
||||
"OutOfOrderOneComparatorNonScoringCollector",
|
||||
"OutOfOrderOneComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderOneComparatorScoringNoMaxScoreCollector",
|
||||
"OutOfOrderOneComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderOneComparatorNonScoringCollector",
|
||||
"OutOfOrderOneComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderOneComparatorScoringNoMaxScoreCollector",
|
||||
"OutOfOrderOneComparatorScoringMaxScoreCollector"
|
||||
"OutOfOrderNonScoringCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderScoringNoMaxScoreCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderNonScoringCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderScoringNoMaxScoreCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector"
|
||||
};
|
||||
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
|
@ -220,14 +220,14 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
new boolean[] { true, true, true },
|
||||
};
|
||||
String[] actualTFCClasses = new String[] {
|
||||
"OutOfOrderMultiComparatorNonScoringCollector",
|
||||
"OutOfOrderMultiComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderMultiComparatorScoringNoMaxScoreCollector",
|
||||
"OutOfOrderMultiComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderMultiComparatorNonScoringCollector",
|
||||
"OutOfOrderMultiComparatorScoringMaxScoreCollector",
|
||||
"OutOfOrderMultiComparatorScoringNoMaxScoreCollector",
|
||||
"OutOfOrderMultiComparatorScoringMaxScoreCollector"
|
||||
"OutOfOrderNonScoringCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderScoringNoMaxScoreCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderNonScoringCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector",
|
||||
"OutOfOrderScoringNoMaxScoreCollector",
|
||||
"OutOfOrderScoringMaxScoreCollector"
|
||||
};
|
||||
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
|
|
|
@ -34,10 +34,6 @@ public class TestTopScoreDocCollector extends LuceneTestCase {
|
|||
}
|
||||
|
||||
boolean[] inOrder = new boolean[] { false, true };
|
||||
String[] actualTSDCClass = new String[] {
|
||||
"OutOfOrderTopScoreDocCollector",
|
||||
"InOrderTopScoreDocCollector"
|
||||
};
|
||||
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
// Add a Query with SHOULD, since bw.scorer() returns BooleanScorer2
|
||||
|
@ -50,7 +46,8 @@ public class TestTopScoreDocCollector extends LuceneTestCase {
|
|||
IndexSearcher searcher = newSearcher(reader);
|
||||
for (int i = 0; i < inOrder.length; i++) {
|
||||
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(3, inOrder[i]);
|
||||
assertEquals("org.apache.lucene.search.TopScoreDocCollector$" + actualTSDCClass[i], tdc.getClass().getName());
|
||||
LeafCollector leafCollector = tdc.getLeafCollector(reader.leaves().get(0));
|
||||
assertEquals(!inOrder[i], leafCollector.acceptsDocsOutOfOrder());
|
||||
|
||||
searcher.search(new MatchAllDocsQuery(), tdc);
|
||||
|
||||
|
|
|
@ -24,10 +24,11 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
||||
/** A custom comparator for sorting documents by an expression */
|
||||
class ExpressionComparator extends FieldComparator<Double> {
|
||||
class ExpressionComparator extends FieldComparator<Double> implements LeafFieldComparator {
|
||||
private final double[] values;
|
||||
private double bottom;
|
||||
private double topValue;
|
||||
|
@ -44,7 +45,6 @@ class ExpressionComparator extends FieldComparator<Double> {
|
|||
// TODO: change FieldComparator.setScorer to throw IOException and remove this try-catch
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
super.setScorer(scorer);
|
||||
// TODO: might be cleaner to lazy-init 'source' and set scorer after?
|
||||
assert readerContext != null;
|
||||
try {
|
||||
|
@ -83,7 +83,7 @@ class ExpressionComparator extends FieldComparator<Double> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Double> setNextReader(LeafReaderContext context) throws IOException {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
this.readerContext = context;
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
|
||||
private final Sort groupSort;
|
||||
private final FieldComparator<?>[] comparators;
|
||||
private final LeafFieldComparator[] leafComparators;
|
||||
private final int[] reversed;
|
||||
private final int topNGroups;
|
||||
private final HashMap<GROUP_VALUE_TYPE, CollectedSearchGroup<GROUP_VALUE_TYPE>> groupMap;
|
||||
|
@ -60,7 +61,6 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
* @param topNGroups How many top groups to keep.
|
||||
* @throws IOException If I/O related errors occur
|
||||
*/
|
||||
@SuppressWarnings({"unchecked","rawtypes"})
|
||||
public AbstractFirstPassGroupingCollector(Sort groupSort, int topNGroups) throws IOException {
|
||||
if (topNGroups < 1) {
|
||||
throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
|
||||
|
@ -74,6 +74,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
|
||||
final SortField[] sortFields = groupSort.getSort();
|
||||
comparators = new FieldComparator[sortFields.length];
|
||||
leafComparators = new LeafFieldComparator[sortFields.length];
|
||||
compIDXEnd = comparators.length - 1;
|
||||
reversed = new int[sortFields.length];
|
||||
for (int i = 0; i < sortFields.length; i++) {
|
||||
|
@ -137,7 +138,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
for (FieldComparator<?> comparator : comparators) {
|
||||
for (LeafFieldComparator comparator : leafComparators) {
|
||||
comparator.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
@ -157,7 +158,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
// wasted effort as we will most likely be updating an existing group.
|
||||
if (orderedGroups != null) {
|
||||
for (int compIDX = 0;; compIDX++) {
|
||||
final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc);
|
||||
final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
|
||||
if (c < 0) {
|
||||
// Definitely not competitive. So don't even bother to continue
|
||||
return;
|
||||
|
@ -197,7 +198,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
sg.groupValue = copyDocGroupValue(groupValue, null);
|
||||
sg.comparatorSlot = groupMap.size();
|
||||
sg.topDoc = docBase + doc;
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.copy(sg.comparatorSlot, doc);
|
||||
}
|
||||
groupMap.put(sg.groupValue, sg);
|
||||
|
@ -223,7 +224,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
|
||||
bottomGroup.topDoc = docBase + doc;
|
||||
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.copy(bottomGroup.comparatorSlot, doc);
|
||||
}
|
||||
|
||||
|
@ -232,7 +233,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
assert orderedGroups.size() == topNGroups;
|
||||
|
||||
final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.setBottom(lastComparatorSlot);
|
||||
}
|
||||
|
||||
|
@ -241,17 +242,16 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
|
||||
// Update existing group:
|
||||
for (int compIDX = 0;; compIDX++) {
|
||||
final FieldComparator<?> fc = comparators[compIDX];
|
||||
fc.copy(spareSlot, doc);
|
||||
leafComparators[compIDX].copy(spareSlot, doc);
|
||||
|
||||
final int c = reversed[compIDX] * fc.compare(group.comparatorSlot, spareSlot);
|
||||
final int c = reversed[compIDX] * comparators[compIDX].compare(group.comparatorSlot, spareSlot);
|
||||
if (c < 0) {
|
||||
// Definitely not competitive.
|
||||
return;
|
||||
} else if (c > 0) {
|
||||
// Definitely competitive; set remaining comparators:
|
||||
for (int compIDX2=compIDX+1; compIDX2<comparators.length; compIDX2++) {
|
||||
comparators[compIDX2].copy(spareSlot, doc);
|
||||
leafComparators[compIDX2].copy(spareSlot, doc);
|
||||
}
|
||||
break;
|
||||
} else if (compIDX == compIDXEnd) {
|
||||
|
@ -288,7 +288,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
final CollectedSearchGroup<?> newLast = orderedGroups.last();
|
||||
// If we changed the value of the last group, or changed which group was last, then update bottom:
|
||||
if (group == newLast || prevLast != newLast) {
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.setBottom(newLast.comparatorSlot);
|
||||
}
|
||||
}
|
||||
|
@ -315,7 +315,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
orderedGroups.addAll(groupMap.values());
|
||||
assert orderedGroups.size() > 0;
|
||||
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.setBottom(orderedGroups.last().comparatorSlot);
|
||||
}
|
||||
}
|
||||
|
@ -329,7 +329,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
|
|||
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
|
||||
docBase = readerContext.docBase;
|
||||
for (int i=0; i<comparators.length; i++) {
|
||||
comparators[i] = comparators[i].setNextReader(readerContext);
|
||||
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -83,7 +83,7 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
|
|||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
|
||||
group.collector.setScorer(scorer);
|
||||
group.leafCollector.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,7 +93,7 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
|
|||
SearchGroupDocs<GROUP_VALUE_TYPE> group = retrieveGroup(doc);
|
||||
if (group != null) {
|
||||
totalGroupedHitCount++;
|
||||
group.collector.collect(doc);
|
||||
group.leafCollector.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -110,7 +110,7 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
|
|||
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
|
||||
//System.out.println("SP.setNextReader");
|
||||
for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
|
||||
group.collector.getLeafCollector(readerContext);
|
||||
group.leafCollector = group.collector.getLeafCollector(readerContext);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -151,6 +151,7 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
|
|||
|
||||
public final GROUP_VALUE_TYPE groupValue;
|
||||
public final TopDocsCollector<?> collector;
|
||||
public LeafCollector leafCollector;
|
||||
|
||||
public SearchGroupDocs(GROUP_VALUE_TYPE groupValue, TopDocsCollector<?> collector) {
|
||||
this.groupValue = groupValue;
|
||||
|
|
|
@ -69,6 +69,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
private final boolean needsScores;
|
||||
|
||||
private final FieldComparator<?>[] comparators;
|
||||
private final LeafFieldComparator[] leafComparators;
|
||||
private final int[] reversed;
|
||||
private final int compIDXEnd;
|
||||
private int bottomSlot;
|
||||
|
@ -202,7 +203,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
bottomSlot = bottomGroup.comparatorSlot;
|
||||
//System.out.println(" set bottom=" + bottomSlot);
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].setBottom(bottomSlot);
|
||||
leafComparators[i].setBottom(bottomSlot);
|
||||
}
|
||||
//System.out.println(" QUEUE FULL");
|
||||
} else {
|
||||
|
@ -231,7 +232,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
|
||||
//System.out.println(" set bottom=" + bottomSlot);
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].setBottom(bottomSlot);
|
||||
leafComparators[i].setBottom(bottomSlot);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -278,6 +279,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
|
||||
final SortField[] sortFields = groupSort.getSort();
|
||||
comparators = new FieldComparator<?>[sortFields.length];
|
||||
leafComparators = new LeafFieldComparator[sortFields.length];
|
||||
compIDXEnd = comparators.length - 1;
|
||||
reversed = new int[sortFields.length];
|
||||
for (int i = 0; i < sortFields.length; i++) {
|
||||
|
@ -349,15 +351,15 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores, true);
|
||||
}
|
||||
|
||||
collector.setScorer(fakeScorer);
|
||||
collector.getLeafCollector(og.readerContext);
|
||||
LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);
|
||||
leafCollector.setScorer(fakeScorer);
|
||||
for(int docIDX=0;docIDX<og.count;docIDX++) {
|
||||
final int doc = og.docs[docIDX];
|
||||
fakeScorer.doc = doc;
|
||||
if (needsScores) {
|
||||
fakeScorer.score = og.scores[docIDX];
|
||||
}
|
||||
collector.collect(doc);
|
||||
leafCollector.collect(doc);
|
||||
}
|
||||
totalGroupedHitCount += og.count;
|
||||
|
||||
|
@ -402,7 +404,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
for (FieldComparator<?> comparator : comparators) {
|
||||
for (LeafFieldComparator comparator : leafComparators) {
|
||||
comparator.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
@ -443,7 +445,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
assert !queueFull;
|
||||
|
||||
//System.out.println(" init copy to bottomSlot=" + bottomSlot);
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.copy(bottomSlot, doc);
|
||||
fc.setBottom(bottomSlot);
|
||||
}
|
||||
|
@ -451,7 +453,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
} else {
|
||||
// Compare to bottomSlot
|
||||
for (int compIDX = 0;; compIDX++) {
|
||||
final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc);
|
||||
final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
|
||||
if (c < 0) {
|
||||
// Definitely not competitive -- done
|
||||
return;
|
||||
|
@ -468,7 +470,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
|
||||
//System.out.println(" best w/in group!");
|
||||
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.copy(bottomSlot, doc);
|
||||
// Necessary because some comparators cache
|
||||
// details of bottom slot; this forces them to
|
||||
|
@ -481,7 +483,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
// We're not sure this group will make it into the
|
||||
// queue yet
|
||||
for (int compIDX = 0;; compIDX++) {
|
||||
final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc);
|
||||
final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
|
||||
if (c < 0) {
|
||||
// Definitely not competitive -- done
|
||||
//System.out.println(" doc doesn't compete w/ top groups");
|
||||
|
@ -498,7 +500,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
groupCompetes = true;
|
||||
for (FieldComparator<?> fc : comparators) {
|
||||
for (LeafFieldComparator fc : leafComparators) {
|
||||
fc.copy(bottomSlot, doc);
|
||||
// Necessary because some comparators cache
|
||||
// details of bottom slot; this forces them to
|
||||
|
@ -528,7 +530,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
|
||||
currentReaderContext = readerContext;
|
||||
for (int i=0; i<comparators.length; i++) {
|
||||
comparators[i] = comparators[i].setNextReader(readerContext);
|
||||
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.grouping;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
|
@ -158,7 +159,7 @@ public class SearchGroup<GROUP_VALUE_TYPE> {
|
|||
|
||||
public GroupComparator(Sort groupSort) throws IOException {
|
||||
final SortField[] sortFields = groupSort.getSort();
|
||||
comparators = new FieldComparator<?>[sortFields.length];
|
||||
comparators = new FieldComparator[sortFields.length];
|
||||
reversed = new int[sortFields.length];
|
||||
for (int compIDX = 0; compIDX < sortFields.length; compIDX++) {
|
||||
final SortField sortField = sortFields[compIDX];
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -94,7 +95,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
for (GroupHead groupHead : groups.values()) {
|
||||
for (FieldComparator<?> comparator : groupHead.comparators) {
|
||||
for (LeafFieldComparator comparator : groupHead.leafComparators) {
|
||||
comparator.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
@ -109,7 +110,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
|
||||
for (GroupHead groupHead : groups.values()) {
|
||||
for (int i = 0; i < groupHead.comparators.length; i++) {
|
||||
groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
|
||||
groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -120,28 +121,31 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
public class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<MutableValue> {
|
||||
|
||||
final FieldComparator<?>[] comparators;
|
||||
final LeafFieldComparator[] leafComparators;
|
||||
|
||||
@SuppressWarnings({"unchecked","rawtypes"})
|
||||
private GroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
|
||||
super(groupValue, doc + readerContext.docBase);
|
||||
final SortField[] sortFields = sort.getSort();
|
||||
comparators = new FieldComparator[sortFields.length];
|
||||
leafComparators = new LeafFieldComparator[sortFields.length];
|
||||
for (int i = 0; i < sortFields.length; i++) {
|
||||
comparators[i] = sortFields[i].getComparator(1, i).setNextReader(readerContext);
|
||||
comparators[i].setScorer(scorer);
|
||||
comparators[i].copy(0, doc);
|
||||
comparators[i].setBottom(0);
|
||||
comparators[i] = sortFields[i].getComparator(1, i);
|
||||
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
|
||||
leafComparators[i].setScorer(scorer);
|
||||
leafComparators[i].copy(0, doc);
|
||||
leafComparators[i].setBottom(0);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int compIDX, int doc) throws IOException {
|
||||
return comparators[compIDX].compareBottom(doc);
|
||||
return leafComparators[compIDX].compareBottom(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateDocHead(int doc) throws IOException {
|
||||
for (FieldComparator<?> comparator : comparators) {
|
||||
for (LeafFieldComparator comparator : leafComparators) {
|
||||
comparator.copy(0, doc);
|
||||
comparator.setBottom(0);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -164,7 +165,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
|
||||
for (GroupHead groupHead : groups.values()) {
|
||||
for (int i = 0; i < groupHead.comparators.length; i++) {
|
||||
groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
|
||||
groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -173,7 +174,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
for (GroupHead groupHead : groups.values()) {
|
||||
for (FieldComparator<?> comparator : groupHead.comparators) {
|
||||
for (LeafFieldComparator comparator : groupHead.leafComparators) {
|
||||
comparator.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
@ -181,28 +182,31 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
|
||||
class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
|
||||
|
||||
final FieldComparator<?>[] comparators;
|
||||
final FieldComparator[] comparators;
|
||||
final LeafFieldComparator[] leafComparators;
|
||||
|
||||
private GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
|
||||
super(groupValue, doc + readerContext.docBase);
|
||||
final SortField[] sortFields = sort.getSort();
|
||||
comparators = new FieldComparator[sortFields.length];
|
||||
leafComparators = new LeafFieldComparator[sortFields.length];
|
||||
for (int i = 0; i < sortFields.length; i++) {
|
||||
comparators[i] = sortFields[i].getComparator(1, i).setNextReader(readerContext);
|
||||
comparators[i].setScorer(scorer);
|
||||
comparators[i].copy(0, doc);
|
||||
comparators[i].setBottom(0);
|
||||
comparators[i] = sortFields[i].getComparator(1, i);
|
||||
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
|
||||
leafComparators[i].setScorer(scorer);
|
||||
leafComparators[i].copy(0, doc);
|
||||
leafComparators[i].setBottom(0);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int compIDX, int doc) throws IOException {
|
||||
return comparators[compIDX].compareBottom(doc);
|
||||
return leafComparators[compIDX].compareBottom(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateDocHead(int doc) throws IOException {
|
||||
for (FieldComparator<?> comparator : comparators) {
|
||||
for (LeafFieldComparator comparator : leafComparators) {
|
||||
comparator.copy(0, doc);
|
||||
comparator.setBottom(0);
|
||||
}
|
||||
|
|
|
@ -74,7 +74,7 @@ import java.util.*;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class ToParentBlockJoinCollector extends SimpleCollector {
|
||||
public class ToParentBlockJoinCollector implements Collector {
|
||||
|
||||
private final Sort sort;
|
||||
|
||||
|
@ -83,16 +83,11 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
private final Map<Query,Integer> joinQueryID = new HashMap<>();
|
||||
private final int numParentHits;
|
||||
private final FieldValueHitQueue<OneGroup> queue;
|
||||
private final FieldComparator[] comparators;
|
||||
private final int[] reverseMul;
|
||||
private final int compEnd;
|
||||
private final FieldComparator<?>[] comparators;
|
||||
private final boolean trackMaxScore;
|
||||
private final boolean trackScores;
|
||||
|
||||
private int docBase;
|
||||
private ToParentBlockJoinQuery.BlockJoinScorer[] joinScorers = new ToParentBlockJoinQuery.BlockJoinScorer[0];
|
||||
private LeafReaderContext currentReaderContext;
|
||||
private Scorer scorer;
|
||||
private boolean queueFull;
|
||||
|
||||
private OneGroup bottom;
|
||||
|
@ -116,8 +111,6 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
this.numParentHits = numParentHits;
|
||||
queue = FieldValueHitQueue.create(sort.getSort(), numParentHits);
|
||||
comparators = queue.getComparators();
|
||||
reverseMul = queue.getReverseMul();
|
||||
compEnd = comparators.length - 1;
|
||||
}
|
||||
|
||||
private static final class OneGroup extends FieldValueHitQueue.Entry {
|
||||
|
@ -143,143 +136,172 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void collect(int parentDoc) throws IOException {
|
||||
//System.out.println("\nC parentDoc=" + parentDoc);
|
||||
totalHitCount++;
|
||||
public LeafCollector getLeafCollector(final LeafReaderContext context)
|
||||
throws IOException {
|
||||
final LeafFieldComparator[] comparators = queue.getComparators(context);
|
||||
final int[] reverseMul = queue.getReverseMul();
|
||||
final int docBase = context.docBase;
|
||||
return new LeafCollector() {
|
||||
|
||||
float score = Float.NaN;
|
||||
private Scorer scorer;
|
||||
|
||||
if (trackMaxScore) {
|
||||
score = scorer.score();
|
||||
maxScore = Math.max(maxScore, score);
|
||||
}
|
||||
|
||||
// TODO: we could sweep all joinScorers here and
|
||||
// aggregate total child hit count, so we can fill this
|
||||
// in getTopGroups (we wire it to 0 now)
|
||||
|
||||
if (queueFull) {
|
||||
//System.out.println(" queueFull");
|
||||
// Fastmatch: return if this hit is not competitive
|
||||
for (int i = 0;; i++) {
|
||||
final int c = reverseMul[i] * comparators[i].compareBottom(parentDoc);
|
||||
if (c < 0) {
|
||||
// Definitely not competitive.
|
||||
//System.out.println(" skip");
|
||||
return;
|
||||
} else if (c > 0) {
|
||||
// Definitely competitive.
|
||||
break;
|
||||
} else if (i == compEnd) {
|
||||
// Here c=0. If we're at the last comparator, this doc is not
|
||||
// competitive, since docs are visited in doc Id order, which means
|
||||
// this doc cannot compete with any other document in the queue.
|
||||
//System.out.println(" skip");
|
||||
return;
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
//System.out.println("C.setScorer scorer=" + scorer);
|
||||
// Since we invoke .score(), and the comparators likely
|
||||
// do as well, cache it so it's only "really" computed
|
||||
// once:
|
||||
if (scorer instanceof ScoreCachingWrappingScorer == false) {
|
||||
scorer = new ScoreCachingWrappingScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
||||
//System.out.println(" competes! doc=" + (docBase + parentDoc));
|
||||
|
||||
// This hit is competitive - replace bottom element in queue & adjustTop
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].copy(bottom.slot, parentDoc);
|
||||
}
|
||||
if (!trackMaxScore && trackScores) {
|
||||
score = scorer.score();
|
||||
}
|
||||
bottom.doc = docBase + parentDoc;
|
||||
bottom.readerContext = currentReaderContext;
|
||||
bottom.score = score;
|
||||
copyGroups(bottom);
|
||||
bottom = queue.updateTop();
|
||||
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].setBottom(bottom.slot);
|
||||
}
|
||||
} else {
|
||||
// Startup transient: queue is not yet full:
|
||||
final int comparatorSlot = totalHitCount - 1;
|
||||
|
||||
// Copy hit into queue
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].copy(comparatorSlot, parentDoc);
|
||||
}
|
||||
//System.out.println(" startup: new OG doc=" + (docBase+parentDoc));
|
||||
if (!trackMaxScore && trackScores) {
|
||||
score = scorer.score();
|
||||
}
|
||||
final OneGroup og = new OneGroup(comparatorSlot, docBase+parentDoc, score, joinScorers.length, trackScores);
|
||||
og.readerContext = currentReaderContext;
|
||||
copyGroups(og);
|
||||
bottom = queue.add(og);
|
||||
queueFull = totalHitCount == numParentHits;
|
||||
if (queueFull) {
|
||||
// End of startup transient: queue just filled up:
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
comparators[i].setBottom(bottom.slot);
|
||||
this.scorer = scorer;
|
||||
for (LeafFieldComparator comparator : comparators) {
|
||||
comparator.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Arrays.fill(joinScorers, null);
|
||||
|
||||
// Pulls out child doc and scores for all join queries:
|
||||
private void copyGroups(OneGroup og) {
|
||||
// While rare, it's possible top arrays could be too
|
||||
// short if join query had null scorer on first
|
||||
// segment(s) but then became non-null on later segments
|
||||
final int numSubScorers = joinScorers.length;
|
||||
if (og.docs.length < numSubScorers) {
|
||||
// While rare, this could happen if join query had
|
||||
// null scorer on first segment(s) but then became
|
||||
// non-null on later segments
|
||||
og.docs = ArrayUtil.grow(og.docs);
|
||||
}
|
||||
if (og.counts.length < numSubScorers) {
|
||||
og.counts = ArrayUtil.grow(og.counts);
|
||||
}
|
||||
if (trackScores && og.scores.length < numSubScorers) {
|
||||
og.scores = ArrayUtil.grow(og.scores);
|
||||
}
|
||||
|
||||
//System.out.println("\ncopyGroups parentDoc=" + og.doc);
|
||||
for(int scorerIDX = 0;scorerIDX < numSubScorers;scorerIDX++) {
|
||||
final ToParentBlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
|
||||
//System.out.println(" scorer=" + joinScorer);
|
||||
if (joinScorer != null && docBase + joinScorer.getParentDoc() == og.doc) {
|
||||
og.counts[scorerIDX] = joinScorer.getChildCount();
|
||||
//System.out.println(" count=" + og.counts[scorerIDX]);
|
||||
og.docs[scorerIDX] = joinScorer.swapChildDocs(og.docs[scorerIDX]);
|
||||
assert og.docs[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.docs[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
|
||||
//System.out.println(" len=" + og.docs[scorerIDX].length);
|
||||
/*
|
||||
for(int idx=0;idx<og.counts[scorerIDX];idx++) {
|
||||
System.out.println(" docs[" + idx + "]=" + og.docs[scorerIDX][idx]);
|
||||
Queue<Scorer> queue = new LinkedList<>();
|
||||
//System.out.println("\nqueue: add top scorer=" + scorer);
|
||||
queue.add(scorer);
|
||||
while ((scorer = queue.poll()) != null) {
|
||||
//System.out.println(" poll: " + scorer + "; " + scorer.getWeight().getQuery());
|
||||
if (scorer instanceof ToParentBlockJoinQuery.BlockJoinScorer) {
|
||||
enroll((ToParentBlockJoinQuery) scorer.getWeight().getQuery(), (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
|
||||
}
|
||||
|
||||
for (ChildScorer sub : scorer.getChildren()) {
|
||||
//System.out.println(" add sub: " + sub.child + "; " + sub.child.getWeight().getQuery());
|
||||
queue.add(sub.child);
|
||||
}
|
||||
*/
|
||||
if (trackScores) {
|
||||
//System.out.println(" copy scores");
|
||||
og.scores[scorerIDX] = joinScorer.swapChildScores(og.scores[scorerIDX]);
|
||||
assert og.scores[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.scores[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
|
||||
}
|
||||
} else {
|
||||
og.counts[scorerIDX] = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int parentDoc) throws IOException {
|
||||
//System.out.println("\nC parentDoc=" + parentDoc);
|
||||
totalHitCount++;
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
currentReaderContext = context;
|
||||
docBase = context.docBase;
|
||||
for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
|
||||
queue.setComparator(compIDX, comparators[compIDX].setNextReader(context));
|
||||
}
|
||||
}
|
||||
float score = Float.NaN;
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
if (trackMaxScore) {
|
||||
score = scorer.score();
|
||||
maxScore = Math.max(maxScore, score);
|
||||
}
|
||||
|
||||
// TODO: we could sweep all joinScorers here and
|
||||
// aggregate total child hit count, so we can fill this
|
||||
// in getTopGroups (we wire it to 0 now)
|
||||
|
||||
if (queueFull) {
|
||||
//System.out.println(" queueFull");
|
||||
// Fastmatch: return if this hit is not competitive
|
||||
int c = 0;
|
||||
for (int i = 0; i < comparators.length; ++i) {
|
||||
c = reverseMul[i] * comparators[i].compareBottom(parentDoc);
|
||||
if (c != 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (c <= 0) { // in case of equality, this hit is not competitive as docs are visited in order
|
||||
// Definitely not competitive.
|
||||
//System.out.println(" skip");
|
||||
return;
|
||||
}
|
||||
|
||||
//System.out.println(" competes! doc=" + (docBase + parentDoc));
|
||||
|
||||
// This hit is competitive - replace bottom element in queue & adjustTop
|
||||
for (LeafFieldComparator comparator : comparators) {
|
||||
comparator.copy(bottom.slot, parentDoc);
|
||||
}
|
||||
if (!trackMaxScore && trackScores) {
|
||||
score = scorer.score();
|
||||
}
|
||||
bottom.doc = docBase + parentDoc;
|
||||
bottom.readerContext = context;
|
||||
bottom.score = score;
|
||||
copyGroups(bottom);
|
||||
bottom = queue.updateTop();
|
||||
|
||||
for (LeafFieldComparator comparator : comparators) {
|
||||
comparator.setBottom(bottom.slot);
|
||||
}
|
||||
} else {
|
||||
// Startup transient: queue is not yet full:
|
||||
final int comparatorSlot = totalHitCount - 1;
|
||||
|
||||
// Copy hit into queue
|
||||
for (LeafFieldComparator comparator : comparators) {
|
||||
comparator.copy(comparatorSlot, parentDoc);
|
||||
}
|
||||
//System.out.println(" startup: new OG doc=" + (docBase+parentDoc));
|
||||
if (!trackMaxScore && trackScores) {
|
||||
score = scorer.score();
|
||||
}
|
||||
final OneGroup og = new OneGroup(comparatorSlot, docBase+parentDoc, score, joinScorers.length, trackScores);
|
||||
og.readerContext = context;
|
||||
copyGroups(og);
|
||||
bottom = queue.add(og);
|
||||
queueFull = totalHitCount == numParentHits;
|
||||
if (queueFull) {
|
||||
// End of startup transient: queue just filled up:
|
||||
for (LeafFieldComparator comparator : comparators) {
|
||||
comparator.setBottom(bottom.slot);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Pulls out child doc and scores for all join queries:
|
||||
private void copyGroups(OneGroup og) {
|
||||
// While rare, it's possible top arrays could be too
|
||||
// short if join query had null scorer on first
|
||||
// segment(s) but then became non-null on later segments
|
||||
final int numSubScorers = joinScorers.length;
|
||||
if (og.docs.length < numSubScorers) {
|
||||
// While rare, this could happen if join query had
|
||||
// null scorer on first segment(s) but then became
|
||||
// non-null on later segments
|
||||
og.docs = ArrayUtil.grow(og.docs);
|
||||
}
|
||||
if (og.counts.length < numSubScorers) {
|
||||
og.counts = ArrayUtil.grow(og.counts);
|
||||
}
|
||||
if (trackScores && og.scores.length < numSubScorers) {
|
||||
og.scores = ArrayUtil.grow(og.scores);
|
||||
}
|
||||
|
||||
//System.out.println("\ncopyGroups parentDoc=" + og.doc);
|
||||
for(int scorerIDX = 0;scorerIDX < numSubScorers;scorerIDX++) {
|
||||
final ToParentBlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
|
||||
//System.out.println(" scorer=" + joinScorer);
|
||||
if (joinScorer != null && docBase + joinScorer.getParentDoc() == og.doc) {
|
||||
og.counts[scorerIDX] = joinScorer.getChildCount();
|
||||
//System.out.println(" count=" + og.counts[scorerIDX]);
|
||||
og.docs[scorerIDX] = joinScorer.swapChildDocs(og.docs[scorerIDX]);
|
||||
assert og.docs[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.docs[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
|
||||
//System.out.println(" len=" + og.docs[scorerIDX].length);
|
||||
/*
|
||||
for(int idx=0;idx<og.counts[scorerIDX];idx++) {
|
||||
System.out.println(" docs[" + idx + "]=" + og.docs[scorerIDX][idx]);
|
||||
}
|
||||
*/
|
||||
if (trackScores) {
|
||||
//System.out.println(" copy scores");
|
||||
og.scores[scorerIDX] = joinScorer.swapChildScores(og.scores[scorerIDX]);
|
||||
assert og.scores[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.scores[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
|
||||
}
|
||||
} else {
|
||||
og.counts[scorerIDX] = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private void enroll(ToParentBlockJoinQuery query, ToParentBlockJoinQuery.BlockJoinScorer scorer) {
|
||||
|
@ -296,34 +318,6 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
joinScorers[slot] = scorer;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
//System.out.println("C.setScorer scorer=" + scorer);
|
||||
// Since we invoke .score(), and the comparators likely
|
||||
// do as well, cache it so it's only "really" computed
|
||||
// once:
|
||||
this.scorer = new ScoreCachingWrappingScorer(scorer);
|
||||
for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
|
||||
comparators[compIDX].setScorer(this.scorer);
|
||||
}
|
||||
Arrays.fill(joinScorers, null);
|
||||
|
||||
Queue<Scorer> queue = new LinkedList<>();
|
||||
//System.out.println("\nqueue: add top scorer=" + scorer);
|
||||
queue.add(scorer);
|
||||
while ((scorer = queue.poll()) != null) {
|
||||
//System.out.println(" poll: " + scorer + "; " + scorer.getWeight().getQuery());
|
||||
if (scorer instanceof ToParentBlockJoinQuery.BlockJoinScorer) {
|
||||
enroll((ToParentBlockJoinQuery) scorer.getWeight().getQuery(), (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
|
||||
}
|
||||
|
||||
for (ChildScorer sub : scorer.getChildren()) {
|
||||
//System.out.println(" add sub: " + sub.child + "; " + sub.child.getWeight().getQuery());
|
||||
queue.add(sub.child);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private OneGroup[] sortedGroups;
|
||||
|
||||
|
@ -420,8 +414,8 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
collector = TopFieldCollector.create(withinGroupSort, numDocsInGroup, fillSortFields, trackScores, trackMaxScore, true);
|
||||
}
|
||||
|
||||
collector.setScorer(fakeScorer);
|
||||
collector.getLeafCollector(og.readerContext);
|
||||
LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);
|
||||
leafCollector.setScorer(fakeScorer);
|
||||
for(int docIDX=0;docIDX<numChildDocs;docIDX++) {
|
||||
//System.out.println("docIDX=" + docIDX + " vs " + og.docs[slot].length);
|
||||
final int doc = og.docs[slot][docIDX];
|
||||
|
@ -429,7 +423,7 @@ public class ToParentBlockJoinCollector extends SimpleCollector {
|
|||
if (trackScores) {
|
||||
fakeScorer.score = og.scores[slot][docIDX];
|
||||
}
|
||||
collector.collect(doc);
|
||||
leafCollector.collect(doc);
|
||||
}
|
||||
totalGroupedHitCount += numChildDocs;
|
||||
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.SimpleFieldComparator;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
|
||||
|
@ -31,13 +33,14 @@ import org.apache.lucene.util.BitSet;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<Object> {
|
||||
public abstract class ToParentBlockJoinFieldComparator extends SimpleFieldComparator<Object> implements LeafFieldComparator { // repeat LeafFieldComparator for javadocs
|
||||
|
||||
private final BitDocIdSetFilter parentFilter;
|
||||
private final BitDocIdSetFilter childFilter;
|
||||
final int spareSlot;
|
||||
|
||||
FieldComparator<Object> wrappedComparator;
|
||||
LeafFieldComparator wrappedLeafComparator;
|
||||
BitSet parentDocuments;
|
||||
BitSet childDocuments;
|
||||
|
||||
|
@ -55,7 +58,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
wrappedComparator.setBottom(slot);
|
||||
wrappedLeafComparator.setBottom(slot);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -64,7 +67,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Object> setNextReader(LeafReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
BitDocIdSet children = childFilter.getDocIdSet(context);
|
||||
if (children == null) {
|
||||
childDocuments = null;
|
||||
|
@ -77,8 +80,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
} else {
|
||||
parentDocuments = parents.bits();
|
||||
}
|
||||
wrappedComparator = wrappedComparator.setNextReader(context);
|
||||
return this;
|
||||
wrappedLeafComparator = wrappedComparator.getLeafComparator(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -90,12 +92,12 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
* Concrete implementation of {@link ToParentBlockJoinSortField} to sorts the parent docs with the lowest values
|
||||
* in the child / nested docs first.
|
||||
*/
|
||||
public static final class Lowest extends ToParentBlockJoinFieldComparator {
|
||||
public static final class Lowest extends ToParentBlockJoinFieldComparator implements LeafFieldComparator {
|
||||
|
||||
/**
|
||||
* Create ToParentBlockJoinFieldComparator.Lowest
|
||||
*
|
||||
* @param wrappedComparator The {@link FieldComparator} on the child / nested level.
|
||||
* @param wrappedComparator The {@link LeafFieldComparator} on the child / nested level.
|
||||
* @param parentFilter Filter that identifies the parent documents.
|
||||
* @param childFilter Filter that defines which child / nested documents participates in sorting.
|
||||
* @param spareSlot The extra slot inside the wrapped comparator that is used to compare which nested document
|
||||
|
@ -119,7 +121,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
}
|
||||
|
||||
// We only need to emit a single cmp value for any matching child doc
|
||||
int cmp = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp > 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
@ -129,7 +131,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return cmp;
|
||||
}
|
||||
int cmp1 = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp1 = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp1 > 0) {
|
||||
return cmp1;
|
||||
} else {
|
||||
|
@ -152,23 +154,22 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return;
|
||||
}
|
||||
wrappedComparator.copy(spareSlot, childDoc);
|
||||
wrappedComparator.copy(slot, childDoc);
|
||||
wrappedLeafComparator.copy(spareSlot, childDoc);
|
||||
wrappedLeafComparator.copy(slot, childDoc);
|
||||
|
||||
while (true) {
|
||||
childDoc = childDocuments.nextSetBit(childDoc + 1);
|
||||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return;
|
||||
}
|
||||
wrappedComparator.copy(spareSlot, childDoc);
|
||||
wrappedLeafComparator.copy(spareSlot, childDoc);
|
||||
if (wrappedComparator.compare(spareSlot, slot) < 0) {
|
||||
wrappedComparator.copy(slot, childDoc);
|
||||
wrappedLeafComparator.copy(slot, childDoc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public int compareTop(int parentDoc) throws IOException {
|
||||
if (parentDoc == 0 || parentDocuments == null || childDocuments == null) {
|
||||
return 0;
|
||||
|
@ -182,7 +183,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
}
|
||||
|
||||
// We only need to emit a single cmp value for any matching child doc
|
||||
int cmp = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp > 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
@ -192,7 +193,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return cmp;
|
||||
}
|
||||
int cmp1 = wrappedComparator.compareTop(childDoc);
|
||||
int cmp1 = wrappedLeafComparator.compareTop(childDoc);
|
||||
if (cmp1 > 0) {
|
||||
return cmp1;
|
||||
} else {
|
||||
|
@ -209,12 +210,12 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
* Concrete implementation of {@link ToParentBlockJoinSortField} to sorts the parent docs with the highest values
|
||||
* in the child / nested docs first.
|
||||
*/
|
||||
public static final class Highest extends ToParentBlockJoinFieldComparator {
|
||||
public static final class Highest extends ToParentBlockJoinFieldComparator implements LeafFieldComparator {
|
||||
|
||||
/**
|
||||
* Create ToParentBlockJoinFieldComparator.Highest
|
||||
*
|
||||
* @param wrappedComparator The {@link FieldComparator} on the child / nested level.
|
||||
* @param wrappedComparator The {@link LeafFieldComparator} on the child / nested level.
|
||||
* @param parentFilter Filter that identifies the parent documents.
|
||||
* @param childFilter Filter that defines which child / nested documents participates in sorting.
|
||||
* @param spareSlot The extra slot inside the wrapped comparator that is used to compare which nested document
|
||||
|
@ -236,7 +237,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
return 0;
|
||||
}
|
||||
|
||||
int cmp = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp < 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
@ -246,7 +247,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return cmp;
|
||||
}
|
||||
int cmp1 = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp1 = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp1 < 0) {
|
||||
return cmp1;
|
||||
} else {
|
||||
|
@ -268,23 +269,22 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return;
|
||||
}
|
||||
wrappedComparator.copy(spareSlot, childDoc);
|
||||
wrappedComparator.copy(slot, childDoc);
|
||||
wrappedLeafComparator.copy(spareSlot, childDoc);
|
||||
wrappedLeafComparator.copy(slot, childDoc);
|
||||
|
||||
while (true) {
|
||||
childDoc = childDocuments.nextSetBit(childDoc + 1);
|
||||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return;
|
||||
}
|
||||
wrappedComparator.copy(spareSlot, childDoc);
|
||||
wrappedLeafComparator.copy(spareSlot, childDoc);
|
||||
if (wrappedComparator.compare(spareSlot, slot) > 0) {
|
||||
wrappedComparator.copy(slot, childDoc);
|
||||
wrappedLeafComparator.copy(slot, childDoc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public int compareTop(int parentDoc) throws IOException {
|
||||
if (parentDoc == 0 || parentDocuments == null || childDocuments == null) {
|
||||
return 0;
|
||||
|
@ -296,7 +296,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
return 0;
|
||||
}
|
||||
|
||||
int cmp = wrappedComparator.compareBottom(childDoc);
|
||||
int cmp = wrappedLeafComparator.compareBottom(childDoc);
|
||||
if (cmp < 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
@ -306,7 +306,7 @@ public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<O
|
|||
if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return cmp;
|
||||
}
|
||||
int cmp1 = wrappedComparator.compareTop(childDoc);
|
||||
int cmp1 = wrappedLeafComparator.compareTop(childDoc);
|
||||
if (cmp1 < 0) {
|
||||
return cmp1;
|
||||
} else {
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.join;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
import java.io.IOException;
|
||||
|
|
|
@ -52,9 +52,12 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.FilterLeafCollector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
|
@ -482,30 +485,25 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
// Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector...
|
||||
final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
|
||||
final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10, false);
|
||||
indexSearcher.search(joinQuery, new SimpleCollector() {
|
||||
|
||||
int docBase;
|
||||
indexSearcher.search(joinQuery, new Collector() {
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
actualResult.set(doc + docBase);
|
||||
topScoreDocCollector.collect(doc);
|
||||
}
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
final LeafCollector in = topScoreDocCollector.getLeafCollector(context);
|
||||
return new FilterLeafCollector(in) {
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
topScoreDocCollector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
topScoreDocCollector.setScorer(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return scoreDocsInOrder;
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
super.collect(doc);
|
||||
actualResult.set(doc + docBase);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return scoreDocsInOrder;
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
// Asserting bit set...
|
||||
|
|
|
@ -20,9 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -214,12 +212,11 @@ final class Sorter {
|
|||
DocMap sort(LeafReader reader) throws IOException {
|
||||
SortField fields[] = sort.getSort();
|
||||
final int reverseMul[] = new int[fields.length];
|
||||
final FieldComparator<?> comparators[] = new FieldComparator[fields.length];
|
||||
final LeafFieldComparator comparators[] = new LeafFieldComparator[fields.length];
|
||||
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
reverseMul[i] = fields[i].getReverse() ? -1 : 1;
|
||||
comparators[i] = fields[i].getComparator(1, i);
|
||||
comparators[i].setNextReader(reader.getContext());
|
||||
comparators[i] = fields[i].getComparator(1, i).getLeafComparator(reader.getContext());
|
||||
comparators[i].setScorer(FAKESCORER);
|
||||
}
|
||||
final DocComparator comparator = new DocComparator() {
|
||||
|
|
|
@ -21,18 +21,8 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.SortingMergePolicy;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.FieldComparatorSource;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.IndexSearcher; // javadocs
|
||||
import org.apache.lucene.search.Query; // javadocs
|
||||
import org.apache.lucene.search.ScoreDoc; // javadocs
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
|
||||
/**
|
||||
* Helper class to sort readers that contain blocks of documents.
|
||||
|
@ -51,22 +41,22 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
final Filter parentsFilter;
|
||||
final Sort parentSort;
|
||||
final Sort childSort;
|
||||
|
||||
/**
|
||||
|
||||
/**
|
||||
* Create a new BlockJoinComparatorSource, sorting only blocks of documents
|
||||
* with {@code parentSort} and not reordering children with a block.
|
||||
*
|
||||
*
|
||||
* @param parentsFilter Filter identifying parent documents
|
||||
* @param parentSort Sort for parent documents
|
||||
*/
|
||||
public BlockJoinComparatorSource(Filter parentsFilter, Sort parentSort) {
|
||||
this(parentsFilter, parentSort, new Sort(SortField.FIELD_DOC));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
/**
|
||||
* Create a new BlockJoinComparatorSource, specifying the sort order for both
|
||||
* blocks of documents and children within a block.
|
||||
*
|
||||
*
|
||||
* @param parentsFilter Filter identifying parent documents
|
||||
* @param parentSort Sort for parent documents
|
||||
* @param childSort Sort for child documents in the same block
|
||||
|
@ -82,7 +72,7 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
// we keep parallel slots: the parent ids and the child ids
|
||||
final int parentSlots[] = new int[numHits];
|
||||
final int childSlots[] = new int[numHits];
|
||||
|
||||
|
||||
SortField parentFields[] = parentSort.getSort();
|
||||
final int parentReverseMul[] = new int[parentFields.length];
|
||||
final FieldComparator<?> parentComparators[] = new FieldComparator[parentFields.length];
|
||||
|
@ -90,7 +80,7 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
parentReverseMul[i] = parentFields[i].getReverse() ? -1 : 1;
|
||||
parentComparators[i] = parentFields[i].getComparator(1, i);
|
||||
}
|
||||
|
||||
|
||||
SortField childFields[] = childSort.getSort();
|
||||
final int childReverseMul[] = new int[childFields.length];
|
||||
final FieldComparator<?> childComparators[] = new FieldComparator[childFields.length];
|
||||
|
@ -98,14 +88,16 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
childReverseMul[i] = childFields[i].getReverse() ? -1 : 1;
|
||||
childComparators[i] = childFields[i].getComparator(1, i);
|
||||
}
|
||||
|
||||
|
||||
// NOTE: we could return parent ID as value but really our sort "value" is more complex...
|
||||
// So we throw UOE for now. At the moment you really should only use this at indexing time.
|
||||
return new FieldComparator<Integer>() {
|
||||
int bottomParent;
|
||||
int bottomChild;
|
||||
FixedBitSet parentBits;
|
||||
|
||||
BitSet parentBits;
|
||||
LeafFieldComparator[] parentLeafComparators;
|
||||
LeafFieldComparator[] childLeafComparators;
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
try {
|
||||
|
@ -115,12 +107,6 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
bottomParent = parentSlots[slot];
|
||||
bottomChild = childSlots[slot];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTopValue(Integer value) {
|
||||
// we dont have enough information (the docid is needed)
|
||||
|
@ -128,39 +114,63 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) throws IOException {
|
||||
return compare(bottomChild, bottomParent, doc, parent(doc));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTop(int doc) throws IOException {
|
||||
// we dont have enough information (the docid is needed)
|
||||
throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) throws IOException {
|
||||
childSlots[slot] = doc;
|
||||
parentSlots[slot] = parent(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Integer> setNextReader(LeafReaderContext context) throws IOException {
|
||||
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
|
||||
if (parentBits != null) {
|
||||
throw new IllegalStateException("This comparator can only be used on a single segment");
|
||||
}
|
||||
final DocIdSet parents = parentsFilter.getDocIdSet(context, null);
|
||||
if (parents == null) {
|
||||
throw new IllegalStateException("LeafReader " + context.reader() + " contains no parents!");
|
||||
}
|
||||
if (!(parents instanceof BitDocIdSet)) {
|
||||
throw new IllegalStateException("parentFilter must return FixedBitSet; got " + parents);
|
||||
if (parents instanceof BitDocIdSet == false) {
|
||||
throw new IllegalStateException("parentFilter must return BitSet; got " + parents);
|
||||
}
|
||||
parentBits = (FixedBitSet) parents.bits();
|
||||
parentBits = (BitSet) parents.bits();
|
||||
parentLeafComparators = new LeafFieldComparator[parentComparators.length];
|
||||
for (int i = 0; i < parentComparators.length; i++) {
|
||||
parentComparators[i] = parentComparators[i].setNextReader(context);
|
||||
parentLeafComparators[i] = parentComparators[i].getLeafComparator(context);
|
||||
}
|
||||
childLeafComparators = new LeafFieldComparator[childComparators.length];
|
||||
for (int i = 0; i < childComparators.length; i++) {
|
||||
childComparators[i] = childComparators[i].setNextReader(context);
|
||||
childLeafComparators[i] = childComparators[i].getLeafComparator(context);
|
||||
}
|
||||
return this;
|
||||
|
||||
return new LeafFieldComparator() {
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) throws IOException {
|
||||
return compare(bottomChild, bottomParent, doc, parent(doc));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTop(int doc) throws IOException {
|
||||
// we dont have enough information (the docid is needed)
|
||||
throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) throws IOException {
|
||||
childSlots[slot] = doc;
|
||||
parentSlots[slot] = parent(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(int slot) {
|
||||
bottomParent = parentSlots[slot];
|
||||
bottomChild = childSlots[slot];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
for (LeafFieldComparator comp : parentLeafComparators) {
|
||||
comp.setScorer(scorer);
|
||||
}
|
||||
for (LeafFieldComparator comp : childLeafComparators) {
|
||||
comp.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -168,32 +178,21 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
// really our sort "value" is more complex...
|
||||
throw new UnsupportedOperationException("filling sort field values is not yet supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {
|
||||
super.setScorer(scorer);
|
||||
for (FieldComparator<?> comp : parentComparators) {
|
||||
comp.setScorer(scorer);
|
||||
}
|
||||
for (FieldComparator<?> comp : childComparators) {
|
||||
comp.setScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
||||
int parent(int doc) {
|
||||
return parentBits.nextSetBit(doc);
|
||||
}
|
||||
|
||||
|
||||
int compare(int docID1, int parent1, int docID2, int parent2) throws IOException {
|
||||
if (parent1 == parent2) { // both are in the same block
|
||||
if (docID1 == parent1 || docID2 == parent2) {
|
||||
// keep parents at the end of blocks
|
||||
return docID1 - docID2;
|
||||
} else {
|
||||
return compare(docID1, docID2, childComparators, childReverseMul);
|
||||
return compare(docID1, docID2, childLeafComparators, childReverseMul);
|
||||
}
|
||||
} else {
|
||||
int cmp = compare(parent1, parent2, parentComparators, parentReverseMul);
|
||||
int cmp = compare(parent1, parent2, parentLeafComparators, parentReverseMul);
|
||||
if (cmp == 0) {
|
||||
return parent1 - parent2;
|
||||
} else {
|
||||
|
@ -201,8 +200,8 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
int compare(int docID1, int docID2, FieldComparator<?> comparators[], int reverseMul[]) throws IOException {
|
||||
|
||||
int compare(int docID1, int docID2, LeafFieldComparator comparators[], int reverseMul[]) throws IOException {
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
// TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
|
||||
// the segments are always the same here...
|
||||
|
@ -217,7 +216,7 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "blockJoin(parentSort=" + parentSort + ",childSort=" + childSort + ")";
|
||||
|
|
|
@ -17,16 +17,17 @@ package org.apache.lucene.queries.function;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.FieldComparatorSource;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.SimpleFieldComparator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Instantiates {@link FunctionValues} for a particular reader.
|
||||
* <br>
|
||||
|
@ -126,7 +127,7 @@ public abstract class ValueSource {
|
|||
* off of the {@link FunctionValues} for a ValueSource
|
||||
* instead of the normal Lucene FieldComparator that works off of a FieldCache.
|
||||
*/
|
||||
class ValueSourceComparator extends FieldComparator<Double> {
|
||||
class ValueSourceComparator extends SimpleFieldComparator<Double> {
|
||||
private final double[] values;
|
||||
private FunctionValues docVals;
|
||||
private double bottom;
|
||||
|
@ -154,9 +155,8 @@ public abstract class ValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(LeafReaderContext context) throws IOException {
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docVals = getValues(fcontext, context);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.text.Collator;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.SimpleFieldComparator;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -36,7 +36,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* This class will be removed in Lucene 5.0
|
||||
*/
|
||||
@Deprecated
|
||||
public final class SlowCollatedStringComparator extends FieldComparator<String> {
|
||||
public final class SlowCollatedStringComparator extends SimpleFieldComparator<String> {
|
||||
|
||||
private final String[] values;
|
||||
private BinaryDocValues currentDocTerms;
|
||||
|
@ -93,10 +93,9 @@ public final class SlowCollatedStringComparator extends FieldComparator<String>
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<String> setNextReader(LeafReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
currentDocTerms = DocValues.getBinary(context.reader(), field);
|
||||
docsWithField = DocValues.getDocsWithField(context.reader(), field);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.index.ReaderUtil;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
@ -598,7 +599,8 @@ public class QueryComponent extends SearchComponent
|
|||
// :TODO: would be simpler to always serialize every position of SortField[]
|
||||
if (type==SortField.Type.SCORE || type==SortField.Type.DOC) continue;
|
||||
|
||||
FieldComparator comparator = null;
|
||||
FieldComparator<?> comparator = null;
|
||||
LeafFieldComparator leafComparator = null;
|
||||
Object[] vals = new Object[nDocs];
|
||||
|
||||
int lastIdx = -1;
|
||||
|
@ -621,12 +623,12 @@ public class QueryComponent extends SearchComponent
|
|||
|
||||
if (comparator == null) {
|
||||
comparator = sortField.getComparator(1,0);
|
||||
comparator = comparator.setNextReader(currentLeaf);
|
||||
leafComparator = comparator.getLeafComparator(currentLeaf);
|
||||
}
|
||||
|
||||
doc -= currentLeaf.docBase; // adjust for what segment this is in
|
||||
comparator.setScorer(new FakeScorer(doc, score));
|
||||
comparator.copy(0, doc);
|
||||
leafComparator.setScorer(new FakeScorer(doc, score));
|
||||
leafComparator.copy(0, doc);
|
||||
Object val = comparator.value(0);
|
||||
if (null != ft) val = ft.marshalSortValue(val);
|
||||
vals[position] = val;
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.search.DocIdSetIterator;
|
|||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.FieldComparatorSource;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.SimpleFieldComparator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
@ -631,7 +632,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
|
|||
|
||||
@Override
|
||||
public FieldComparator<Integer> newComparator(String fieldname, final int numHits, int sortPos, boolean reversed) throws IOException {
|
||||
return new FieldComparator<Integer>() {
|
||||
return new SimpleFieldComparator<Integer>() {
|
||||
private final int[] values = new int[numHits];
|
||||
private int bottomVal;
|
||||
private int topVal;
|
||||
|
@ -677,13 +678,13 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(LeafReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
//convert the ids to Lucene doc ids, the ordSet and termValues needs to be the same size as the number of elevation docs we have
|
||||
ordSet.clear();
|
||||
Fields fields = context.reader().fields();
|
||||
if (fields == null) return this;
|
||||
if (fields == null) return;
|
||||
Terms terms = fields.terms(idField);
|
||||
if (terms == null) return this;
|
||||
if (terms == null) return;
|
||||
termsEnum = terms.iterator(termsEnum);
|
||||
BytesRefBuilder term = new BytesRefBuilder();
|
||||
Bits liveDocs = context.reader().getLiveDocs();
|
||||
|
@ -701,7 +702,6 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
|
|||
}
|
||||
}
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -109,7 +109,7 @@ public class RandomSortField extends FieldType {
|
|||
private static FieldComparatorSource randomComparatorSource = new FieldComparatorSource() {
|
||||
@Override
|
||||
public FieldComparator<Integer> newComparator(final String fieldname, final int numHits, int sortPos, boolean reversed) {
|
||||
return new FieldComparator<Integer>() {
|
||||
return new SimpleFieldComparator<Integer>() {
|
||||
int seed;
|
||||
private final int[] values = new int[numHits];
|
||||
int bottomVal;
|
||||
|
@ -141,9 +141,8 @@ public class RandomSortField extends FieldType {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(LeafReaderContext context) {
|
||||
protected void doSetNextReader(LeafReaderContext context) {
|
||||
seed = getSeed(fieldname, context);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.solr.common.util.NamedList;
|
|||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -129,22 +128,32 @@ public class ExportQParserPlugin extends QParserPlugin {
|
|||
private class ExportCollector extends TopDocsCollector {
|
||||
|
||||
private FixedBitSet[] sets;
|
||||
private FixedBitSet set;
|
||||
|
||||
public ExportCollector(FixedBitSet[] sets) {
|
||||
super(null);
|
||||
this.sets = sets;
|
||||
}
|
||||
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
this.set = new FixedBitSet(context.reader().maxDoc());
|
||||
this.sets[context.ord] = set;
|
||||
|
||||
}
|
||||
|
||||
public void collect(int docId) throws IOException{
|
||||
++totalHits;
|
||||
set.set(docId);
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final FixedBitSet set = new FixedBitSet(context.reader().maxDoc());
|
||||
this.sets[context.ord] = set;
|
||||
return new LeafCollector() {
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void collect(int docId) throws IOException{
|
||||
++totalHits;
|
||||
set.set(docId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private ScoreDoc[] getScoreDocs(int howMany) {
|
||||
|
@ -170,12 +179,5 @@ public class ExportQParserPlugin extends QParserPlugin {
|
|||
return new TopDocs(totalHits, scoreDocs, 0.0f);
|
||||
}
|
||||
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.solr.search;
|
||||
|
||||
import com.carrotsearch.hppc.IntIntOpenHashMap;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
|
@ -33,7 +34,7 @@ import org.apache.solr.common.util.NamedList;
|
|||
import org.apache.solr.handler.component.MergeStrategy;
|
||||
import org.apache.solr.handler.component.QueryElevationComponent;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -43,6 +44,7 @@ import org.apache.lucene.search.Sort;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
|
||||
import com.carrotsearch.hppc.IntFloatOpenHashMap;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -255,26 +257,15 @@ public class ReRankQParserPlugin extends QParserPlugin {
|
|||
this.reRankWeight = reRankWeight;
|
||||
}
|
||||
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public void collect(int doc) throws IOException {
|
||||
mainCollector.collect(doc);
|
||||
}
|
||||
|
||||
public void setScorer(Scorer scorer) throws IOException{
|
||||
mainCollector.setScorer(scorer);
|
||||
}
|
||||
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException{
|
||||
mainCollector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
public int getTotalHits() {
|
||||
return mainCollector.getTotalHits();
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
return mainCollector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
public TopDocs topDocs(int start, int howMany) {
|
||||
|
||||
try {
|
||||
|
@ -387,6 +378,7 @@ public class ReRankQParserPlugin extends QParserPlugin {
|
|||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public class BoostedComp implements Comparator {
|
||||
|
|
|
@ -2055,16 +2055,17 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
|||
int end=0;
|
||||
int readerIndex = 0;
|
||||
|
||||
LeafCollector leafCollector = null;
|
||||
while (iter.hasNext()) {
|
||||
int doc = iter.nextDoc();
|
||||
while (doc>=end) {
|
||||
LeafReaderContext leaf = leafContexts.get(readerIndex++);
|
||||
base = leaf.docBase;
|
||||
end = base + leaf.reader().maxDoc();
|
||||
topCollector.getLeafCollector(leaf);
|
||||
leafCollector = topCollector.getLeafCollector(leaf);
|
||||
// we should never need to set the scorer given the settings for the collector
|
||||
}
|
||||
topCollector.collect(doc-base);
|
||||
leafCollector.collect(doc-base);
|
||||
}
|
||||
|
||||
TopDocs topDocs = topCollector.topDocs(0, nDocs);
|
||||
|
|
|
@ -23,6 +23,8 @@ import org.apache.lucene.index.IndexReaderContext;
|
|||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -51,7 +53,6 @@ import org.apache.solr.schema.FieldType;
|
|||
import org.apache.solr.schema.IndexSchema;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
|
||||
import org.junit.Ignore;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -410,7 +411,8 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
// :TODO: would be simpler to always serialize every position of SortField[]
|
||||
if (type==SortField.Type.SCORE || type==SortField.Type.DOC) continue;
|
||||
|
||||
FieldComparator comparator = null;
|
||||
FieldComparator<?> comparator = null;
|
||||
LeafFieldComparator leafComparator = null;
|
||||
Object[] vals = new Object[nDocs];
|
||||
|
||||
int lastIdx = -1;
|
||||
|
@ -433,12 +435,12 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
|
||||
if (comparator == null) {
|
||||
comparator = sortField.getComparator(1,0);
|
||||
comparator = comparator.setNextReader(currentLeaf);
|
||||
leafComparator = comparator.getLeafComparator(currentLeaf);
|
||||
}
|
||||
|
||||
doc -= currentLeaf.docBase; // adjust for what segment this is in
|
||||
comparator.setScorer(new FakeScorer(doc, score));
|
||||
comparator.copy(0, doc);
|
||||
leafComparator.setScorer(new FakeScorer(doc, score));
|
||||
leafComparator.copy(0, doc);
|
||||
Object val = comparator.value(0);
|
||||
if (null != ft) val = ft.marshalSortValue(val);
|
||||
vals[position] = val;
|
||||
|
@ -705,24 +707,28 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
class TestCollector extends TopDocsCollector {
|
||||
|
||||
private List<ScoreDoc> list = new ArrayList();
|
||||
private NumericDocValues values;
|
||||
private int base;
|
||||
|
||||
public TestCollector(PriorityQueue pq) {
|
||||
super(pq);
|
||||
}
|
||||
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int base = context.docBase;
|
||||
final NumericDocValues values = DocValues.getNumeric(context.reader(), "sort_i");
|
||||
return new LeafCollector() {
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {}
|
||||
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
values = DocValues.getNumeric(context.reader(), "sort_i");
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
public void collect(int doc) {
|
||||
list.add(new ScoreDoc(doc+base, (float)values.get(doc)));
|
||||
public void collect(int doc) {
|
||||
list.add(new ScoreDoc(doc+base, (float)values.get(doc)));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public int topDocsSize() {
|
||||
|
@ -759,27 +765,32 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
class TestCollector1 extends TopDocsCollector {
|
||||
|
||||
private List<ScoreDoc> list = new ArrayList();
|
||||
private int base;
|
||||
private Scorer scorer;
|
||||
|
||||
public TestCollector1(PriorityQueue pq) {
|
||||
super(pq);
|
||||
}
|
||||
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
public void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
public void collect(int doc) throws IOException {
|
||||
list.add(new ScoreDoc(doc+base, scorer.score()));
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int base = context.docBase;
|
||||
return new LeafCollector() {
|
||||
|
||||
Scorer scorer;
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
public void collect(int doc) throws IOException {
|
||||
list.add(new ScoreDoc(doc+base, scorer.score()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public int topDocsSize() {
|
||||
|
@ -813,7 +824,4 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue