diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index c470290..dc59d91 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -18,11 +18,9 @@ package org.apache.lucene.search; */ import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.List; -import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.BooleanQuery.BooleanWeight; /* Description from Doug Cutting (excerpted from @@ -60,12 +58,12 @@ import org.apache.lucene.search.BooleanQuery.BooleanWeight; final class BooleanScorer extends Scorer { - private static final class BooleanScorerCollector extends Collector { + private static final class BooleanScorerSubCollector implements SubCollector { private BucketTable bucketTable; private int mask; private Scorer scorer; - public BooleanScorerCollector(int mask, BucketTable bucketTable) { + public BooleanScorerSubCollector(int mask, BucketTable bucketTable) { this.mask = mask; this.bucketTable = bucketTable; } @@ -75,7 +73,6 @@ final class BooleanScorer extends Scorer { final BucketTable table = bucketTable; final int i = doc & BucketTable.MASK; final Bucket bucket = table.buckets[i]; - if (bucket.doc != doc) { // invalid bucket bucket.doc = doc; // set doc bucket.score = scorer.score(); // initialize score @@ -90,22 +87,21 @@ final class BooleanScorer extends Scorer { bucket.coord++; // increment coord } } - - @Override - public void setNextReader(AtomicReaderContext context) { - // not needed by this implementation - } - + @Override - public void setScorer(Scorer scorer) { - this.scorer = scorer; + public void done() throws IOException { } - + @Override public boolean acceptsDocsOutOfOrder() { return true; } + @Override + public void setScorer(Scorer scorer) { + this.scorer = scorer; + } + } // An internal class which is used in score(Collector, int) for setting the @@ -167,8 +163,8 @@ final class BooleanScorer extends Scorer { } } - public Collector newCollector(int mask) { - return new BooleanScorerCollector(mask, this); + public SubCollector newSubCollector(int mask) { + return new BooleanScorerSubCollector(mask, this); } public int size() { return SIZE; } @@ -179,11 +175,11 @@ final class BooleanScorer extends Scorer { // TODO: re-enable this if BQ ever sends us required clauses //public boolean required = false; public boolean prohibited; - public Collector collector; + public SubCollector collector; public SubScorer next; public SubScorer(Scorer scorer, boolean required, boolean prohibited, - Collector collector, SubScorer next) { + SubCollector collector, SubScorer next) { if (required) { throw new IllegalArgumentException("this scorer cannot handle required=true"); } @@ -215,7 +211,7 @@ final class BooleanScorer extends Scorer { if (optionalScorers != null && optionalScorers.size() > 0) { for (Scorer scorer : optionalScorers) { if (scorer.nextDoc() != NO_MORE_DOCS) { - scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers); + scorers = new SubScorer(scorer, false, false, bucketTable.newSubCollector(0), scorers); } } } @@ -223,7 +219,7 @@ final class BooleanScorer extends Scorer { if (prohibitedScorers != null && prohibitedScorers.size() > 0) { for (Scorer scorer : prohibitedScorers) { if (scorer.nextDoc() != NO_MORE_DOCS) { - scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers); + scorers = new SubScorer(scorer, false, true, bucketTable.newSubCollector(PROHIBITED_MASK), scorers); } } } @@ -236,7 +232,7 @@ final class BooleanScorer extends Scorer { // firstDocID is ignored since nextDoc() initializes 'current' @Override - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector collector, int max, int firstDocID) throws IOException { // Make sure it's only BooleanScorer that calls us: assert firstDocID == -1; boolean more; @@ -335,7 +331,7 @@ final class BooleanScorer extends Scorer { } @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { score(collector, Integer.MAX_VALUE, -1); } diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java index 85fa403..7f46cf3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java @@ -283,7 +283,7 @@ class BooleanScorer2 extends Scorer { * @param collector The collector to which all matching documents are passed through. */ @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { collector.setScorer(this); while ((doc = countingSumScorer.nextDoc()) != NO_MORE_DOCS) { collector.collect(doc); @@ -291,7 +291,7 @@ class BooleanScorer2 extends Scorer { } @Override - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector collector, int max, int firstDocID) throws IOException { doc = firstDocID; collector.setScorer(this); while (doc < max) { diff --git a/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java b/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java index 554da52..735b73d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java @@ -47,7 +47,7 @@ import java.util.List; * * @lucene.experimental */ -public abstract class CachingCollector extends Collector { +public abstract class CachingCollector extends SerialCollector { // Max out at 512K arrays private static final int MAX_ARRAY_SIZE = 512 * 1024; @@ -57,10 +57,12 @@ public abstract class CachingCollector extends Collector { private static class SegStart { public final AtomicReaderContext readerContext; public final int end; + public final boolean acceptsDocsOutOfOrder; - public SegStart(AtomicReaderContext readerContext, int end) { + public SegStart(AtomicReaderContext readerContext, int end, boolean acceptsDocsOutOfOrder) { this.readerContext = readerContext; this.end = end; + this.acceptsDocsOutOfOrder = acceptsDocsOutOfOrder; } } @@ -128,7 +130,7 @@ public abstract class CachingCollector extends Collector { // Cache was too large cachedScorer.score = scorer.score(); cachedScorer.doc = doc; - other.collect(doc); + otherSub.collect(doc); return; } @@ -154,7 +156,7 @@ public abstract class CachingCollector extends Collector { cachedScores.clear(); cachedScorer.score = scorer.score(); cachedScorer.doc = doc; - other.collect(doc); + otherSub.collect(doc); return; } } @@ -170,20 +172,22 @@ public abstract class CachingCollector extends Collector { cachedScorer.score = curScores[upto] = scorer.score(); upto++; cachedScorer.doc = doc; - other.collect(doc); + otherSub.collect(doc); } @Override - public void replay(Collector other) throws IOException { - replayInit(other); - + public void replay(Collector replayTargetCollector) throws IOException { + checkCached(); + int curUpto = 0; int curBase = 0; int chunkUpto = 0; curDocs = EMPTY_INT_ARRAY; + for (SegStart seg : cachedSegs) { - other.setNextReader(seg.readerContext); - other.setScorer(cachedScorer); + final SubCollector replayTargetSub = replayTargetCollector.subCollector(seg.readerContext); + checkOrdering(replayTargetSub, seg); + replayTargetSub.setScorer(cachedScorer); while (curBase + curUpto < seg.end) { if (curUpto == curDocs.length) { curBase += curDocs.length; @@ -194,15 +198,17 @@ public abstract class CachingCollector extends Collector { } cachedScorer.score = curScores[curUpto]; cachedScorer.doc = curDocs[curUpto]; - other.collect(curDocs[curUpto++]); + replayTargetSub.collect(curDocs[curUpto++]); } + + replayTargetSub.done(); } } @Override public void setScorer(Scorer scorer) throws IOException { this.scorer = scorer; - other.setScorer(cachedScorer); + otherSub.setScorer(cachedScorer); } @Override @@ -232,7 +238,7 @@ public abstract class CachingCollector extends Collector { if (curDocs == null) { // Cache was too large - other.collect(doc); + otherSub.collect(doc); return; } @@ -254,7 +260,7 @@ public abstract class CachingCollector extends Collector { curDocs = null; cachedSegs.clear(); cachedDocs.clear(); - other.collect(doc); + otherSub.collect(doc); return; } } @@ -266,19 +272,21 @@ public abstract class CachingCollector extends Collector { curDocs[upto] = doc; upto++; - other.collect(doc); + otherSub.collect(doc); } @Override - public void replay(Collector other) throws IOException { - replayInit(other); - + public void replay(Collector replayTargetCollector) throws IOException { + checkCached(); + int curUpto = 0; int curbase = 0; int chunkUpto = 0; curDocs = EMPTY_INT_ARRAY; + for (SegStart seg : cachedSegs) { - other.setNextReader(seg.readerContext); + final SubCollector replayTargetSub = replayTargetCollector.subCollector(seg.readerContext); + checkOrdering(replayTargetSub, seg); while (curbase + curUpto < seg.end) { if (curUpto == curDocs.length) { curbase += curDocs.length; @@ -286,14 +294,16 @@ public abstract class CachingCollector extends Collector { chunkUpto++; curUpto = 0; } - other.collect(curDocs[curUpto++]); + replayTargetSub.collect(curDocs[curUpto++]); } + + replayTargetSub.done(); } } @Override public void setScorer(Scorer scorer) throws IOException { - other.setScorer(scorer); + otherSub.setScorer(scorer); } @Override @@ -313,18 +323,18 @@ public abstract class CachingCollector extends Collector { // version -- if the wrapped Collector does not need // scores, it can avoid cachedScorer entirely. protected final Collector other; - + protected SubCollector otherSub; + protected final int maxDocsToCache; protected final List cachedSegs = new ArrayList(); protected final List cachedDocs; private AtomicReaderContext lastReaderContext; - + protected int[] curDocs; protected int upto; protected int base; - protected int lastDocBase; - + /** * Creates a {@link CachingCollector} which does not wrap another collector. * The cached documents and scores can later be {@link #replay(Collector) @@ -336,19 +346,35 @@ public abstract class CachingCollector extends Collector { public static CachingCollector create(final boolean acceptDocsOutOfOrder, boolean cacheScores, double maxRAMMB) { Collector other = new Collector() { @Override - public boolean acceptsDocsOutOfOrder() { - return acceptDocsOutOfOrder; + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + return new SubCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public void collect(int doc) throws IOException { + } + + @Override + public void done() throws IOException { + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return acceptDocsOutOfOrder; + } + }; } - - @Override - public void setScorer(Scorer scorer) {} @Override - public void collect(int doc) {} + public void setParallelized() { + } @Override - public void setNextReader(AtomicReaderContext context) {} - + public boolean isParallelizable() { + return true; + } }; return create(other, cacheScores, maxRAMMB); } @@ -415,7 +441,7 @@ public abstract class CachingCollector extends Collector { @Override public boolean acceptsDocsOutOfOrder() { - return other.acceptsDocsOutOfOrder(); + return otherSub.acceptsDocsOutOfOrder(); } public boolean isCached() { @@ -424,31 +450,31 @@ public abstract class CachingCollector extends Collector { @Override public void setNextReader(AtomicReaderContext context) throws IOException { - other.setNextReader(context); - if (lastReaderContext != null) { - cachedSegs.add(new SegStart(lastReaderContext, base+upto)); - } lastReaderContext = context; + otherSub = other.subCollector(context); } - /** Reused by the specialized inner classes. */ - void replayInit(Collector other) { + @Override + public void done() throws IOException { + otherSub.done(); + cachedSegs.add(new SegStart(lastReaderContext, base+upto, otherSub.acceptsDocsOutOfOrder())); + otherSub = null; + lastReaderContext = null; + } + + void checkCached() { if (!isCached()) { throw new IllegalStateException("cannot replay: cache was cleared because too much RAM was required"); } - - if (!other.acceptsDocsOutOfOrder() && this.other.acceptsDocsOutOfOrder()) { + } + + void checkOrdering(SubCollector sub, SegStart seg) throws IOException { + if (!sub.acceptsDocsOutOfOrder() && seg.acceptsDocsOutOfOrder) { throw new IllegalArgumentException( "cannot replay: given collector does not support " + "out-of-order collection, while the wrapped collector does. " + "Therefore cached documents may be out-of-order."); } - - //System.out.println("CC: replay totHits=" + (upto + base)); - if (lastReaderContext != null) { - cachedSegs.add(new SegStart(lastReaderContext, base+upto)); - lastReaderContext = null; - } } /** diff --git a/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java b/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java index 9caadfa..2982ed6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java +++ b/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java @@ -17,7 +17,7 @@ package org.apache.lucene.search; * limitations under the License. */ -/** Throw this exception in {@link Collector#collect(int)} to prematurely +/** Throw this exception in {@link SubCollector#collect(int)} to prematurely * terminate collection of the current leaf. *

Note: IndexSearcher swallows this exception and never re-throws it. * As a consequence, you should not catch it when calling diff --git a/lucene/core/src/java/org/apache/lucene/search/Collector.java b/lucene/core/src/java/org/apache/lucene/search/Collector.java index 312f507..4480039 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Collector.java +++ b/lucene/core/src/java/org/apache/lucene/search/Collector.java @@ -17,163 +17,67 @@ package org.apache.lucene.search; * limitations under the License. */ -import java.io.IOException; - import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.index.IndexReaderContext; + +import java.io.IOException; /** - *

Expert: Collectors are primarily meant to be used to - * gather raw results from a search, and implement sorting - * or custom result filtering, collation, etc.

- * - *

Lucene's core collectors are derived from Collector. - * Likely your application can use one of these classes, or - * subclass {@link TopDocsCollector}, instead of - * implementing Collector directly: - * + *

Expert: Collectors are primarily meant to be used to gather raw results from a search, and implement sorting or + * custom result filtering, collation, etc.

+ *

Lucene's core collectors are derived from Collector. Likely your application can use one of these classes, or + * subclass {@link TopDocsCollector}, instead of implementing Collector directly: *

    - * - *
  • {@link TopDocsCollector} is an abstract base class - * that assumes you will retrieve the top N docs, - * according to some criteria, after collection is - * done.
  • - * - *
  • {@link TopScoreDocCollector} is a concrete subclass - * {@link TopDocsCollector} and sorts according to score + - * docID. This is used internally by the {@link - * IndexSearcher} search methods that do not take an - * explicit {@link Sort}. It is likely the most frequently - * used collector.
  • - * - *
  • {@link TopFieldCollector} subclasses {@link - * TopDocsCollector} and sorts according to a specified - * {@link Sort} object (sort by field). This is used - * internally by the {@link IndexSearcher} search methods - * that take an explicit {@link Sort}. - * - *
  • {@link TimeLimitingCollector}, which wraps any other - * Collector and aborts the search if it's taken too much - * time.
  • - * - *
  • {@link PositiveScoresOnlyCollector} wraps any other - * Collector and prevents collection of hits whose score - * is <= 0.0
  • - * + *
  • {@link TopDocsCollector} is an abstract base class that assumes you will retrieve the top N docs, according to + * some criteria, after collection is done.
  • + *
  • {@link TopScoreDocCollector} is a concrete subclass {@link TopDocsCollector} and sorts according to score + + * docID. This is used internally by the {@link IndexSearcher} search methods that do not take an explicit {@link + * Sort}. It is likely the most frequently used collector.
  • + *
  • {@link TopFieldCollector} subclasses {@link TopDocsCollector} and sorts according to a specified {@link Sort} + * object (sort by field). This is used internally by the {@link IndexSearcher} search methods that take an explicit + * {@link Sort}. + *
  • {@link TimeLimitingCollector}, which wraps any other Collector and aborts the search if it's taken too much + * time.
  • + *
  • {@link PositiveScoresOnlyCollector} wraps any other Collector and prevents collection of hits whose score is + * <= 0.0
  • *
- * - *

Collector decouples the score from the collected doc: - * the score computation is skipped entirely if it's not - * needed. Collectors that do need the score should - * implement the {@link #setScorer} method, to hold onto the - * passed {@link Scorer} instance, and call {@link - * Scorer#score()} within the collect method to compute the - * current hit's score. If your collector may request the - * score for a single hit multiple times, you should use - * {@link ScoreCachingWrappingScorer}.

- * - *

NOTE: The doc that is passed to the collect - * method is relative to the current reader. If your - * collector needs to resolve this to the docID space of the - * Multi*Reader, you must re-base it by recording the - * docBase from the most recent setNextReader call. Here's - * a simple example showing how to collect docIDs into a - * BitSet:

- * - *
- * IndexSearcher searcher = new IndexSearcher(indexReader);
- * final BitSet bits = new BitSet(indexReader.maxDoc());
- * searcher.search(query, new Collector() {
- *   private int docBase;
- * 
- *   // ignore scorer
- *   public void setScorer(Scorer scorer) {
- *   }
- *
- *   // accept docs out of order (for a BitSet it doesn't matter)
- *   public boolean acceptsDocsOutOfOrder() {
- *     return true;
- *   }
- * 
- *   public void collect(int doc) {
- *     bits.set(doc + docBase);
- *   }
- * 
- *   public void setNextReader(AtomicReaderContext context) {
- *     this.docBase = context.docBase;
- *   }
- * });
- * 
- * - *

Not all collectors will need to rebase the docID. For - * example, a collector that simply counts the total number - * of hits would skip it.

- * - *

NOTE: Prior to 2.9, Lucene silently filtered - * out hits with score <= 0. As of 2.9, the core Collectors - * no longer do that. It's very unusual to have such hits - * (a negative query boost, or function query returning - * negative custom scores, could cause it to happen). If - * you need that behavior, use {@link - * PositiveScoresOnlyCollector}.

+ *

NOTE: Prior to 2.9, Lucene silently filtered out hits with score <= 0. As of 2.9, the core Collectors no + * longer do that. It's very unusual to have such hits (a negative query boost, or function query returning negative + * custom scores, could cause it to happen). If you need that behavior, use {@link PositiveScoresOnlyCollector}.

* * @lucene.experimental - * - * @since 2.9 */ -public abstract class Collector { - - /** - * Called before successive calls to {@link #collect(int)}. Implementations - * that need the score of the current document (passed-in to - * {@link #collect(int)}), should save the passed-in Scorer and call - * scorer.score() when needed. - */ - public abstract void setScorer(Scorer scorer) throws IOException; - +public interface Collector { + /** - * Called once for every document matching a query, with the unbased document - * number. - *

Note: The collection of the current segment can be terminated by throwing - * a {@link CollectionTerminatedException}. In this case, the last docs of the - * current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher} - * will swallow the exception and continue collection with the next leaf. - *

- * Note: This is called in an inner search loop. For good search performance, - * implementations of this method should not call {@link IndexSearcher#doc(int)} or - * {@link org.apache.lucene.index.IndexReader#document(int)} on every hit. - * Doing so can slow searches by an order of magnitude or more. + *

Called in order to obtain the {@link SubCollector} that will be used for collecting from a {@link + * AtomicReaderContext}.

+ *

Add {@link AtomicReaderContext#docBase} to re-base document ID's in {@link SubCollector#collect(int)}.

+ * + * @param context + * atomic reader context for this sub-collection + * @return the SubCollector that will be used for the supplied context + * @throws IOException */ - public abstract void collect(int doc) throws IOException; + SubCollector subCollector(AtomicReaderContext context) throws IOException; /** - * Called before collecting from each {@link AtomicReaderContext}. All doc ids in - * {@link #collect(int)} will correspond to {@link IndexReaderContext#reader}. - * - * Add {@link AtomicReaderContext#docBase} to the current {@link IndexReaderContext#reader}'s - * internal document id to re-base ids in {@link #collect(int)}. - * - * @param context - * next atomic reader context + *

Advise this collector about whether parallel collection will be performed.

+ *

This method can only be called if {@link #isParallelizable()} returns {@code true}, and will only be called + * before any call to {@link #subCollector(org.apache.lucene.index.AtomicReaderContext)}.

+ *

This is intended as a way for implementations to adjust strategy in case the optimal solution is very different + * for single-threaded vs parallel collection, e.g. it may be desirable to accumulate shared state rather than + * divide-and-conquer.

*/ - public abstract void setNextReader(AtomicReaderContext context) throws IOException; + void setParallelized(); /** - * Return true if this collector does not - * require the matching docIDs to be delivered in int sort - * order (smallest to largest) to {@link #collect}. + * Parallel collection implies that besides a primary collection thread which invokes the methods on this + * {@link Collector} implementation, another thread may be responsible for invocation of all {@link SubCollector} + * methods except for the {@link SubCollector#done()} method which can be safely assumed to always be invoked from + * the primary collection thread. * - *

Most Lucene Query implementations will visit - * matching docIDs in order. However, some queries - * (currently limited to certain cases of {@link - * BooleanQuery}) can achieve faster searching if the - * Collector allows them to deliver the - * docIDs out of order.

- * - *

Many collectors don't mind getting docIDs out of - * order, so it's important to return true - * here. + * @return whether parallel collection is supported */ - public abstract boolean acceptsDocsOutOfOrder(); - + boolean isParallelizable(); + } diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java index ea4b809..3fccc7d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java @@ -222,36 +222,36 @@ public class ConstantScoreQuery extends Query { return docIdSetIterator.cost(); } - private Collector wrapCollector(final Collector collector) { - return new Collector() { + private SubCollector wrapSubCollector(final SubCollector subCollector) { + return new SubCollector() { @Override public void setScorer(Scorer scorer) throws IOException { // we must wrap again here, but using the scorer passed in as parameter: - collector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight, ConstantScorer.this.theScore)); + subCollector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight, ConstantScorer.this.theScore)); } @Override public void collect(int doc) throws IOException { - collector.collect(doc); + subCollector.collect(doc); } - + @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - collector.setNextReader(context); + public void done() throws IOException { + subCollector.done(); } - + @Override public boolean acceptsDocsOutOfOrder() { - return collector.acceptsDocsOutOfOrder(); + return subCollector.acceptsDocsOutOfOrder(); } }; } // this optimization allows out of order scoring as top scorer! @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { if (query != null) { - ((Scorer) docIdSetIterator).score(wrapCollector(collector)); + ((Scorer) docIdSetIterator).score(wrapSubCollector(collector)); } else { super.score(collector); } @@ -259,9 +259,9 @@ public class ConstantScoreQuery extends Query { // this optimization allows out of order scoring as top scorer, @Override - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector collector, int max, int firstDocID) throws IOException { if (query != null) { - return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID); + return ((Scorer) docIdSetIterator).score(wrapSubCollector(collector), max, firstDocID); } else { return super.score(collector, max, firstDocID); } diff --git a/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java b/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java index bf2fdde..ffccfb3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java @@ -155,7 +155,7 @@ public class FilteredQuery extends Query { // optimization: we are topScorer and collect directly @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { // the normalization trick already applies the boost of this query, // so we can use the wrapped scorer directly: collector.setScorer(scorer); @@ -239,7 +239,7 @@ public class FilteredQuery extends Query { // optimization: we are topScorer and collect directly using short-circuited algo @Override - public final void score(Collector collector) throws IOException { + public final void score(SubCollector collector) throws IOException { // the normalization trick already applies the boost of this query, // so we can use the wrapped scorer directly: collector.setScorer(scorer); @@ -488,7 +488,7 @@ public class FilteredQuery extends Query { * and/or {@link Scorer#advance(int)}, therefore it is recommended to * request an in-order scorer if use of these methods is required. * @param topScorer - * if true, {@link Scorer#score(Collector)} will be called; if false, + * if true, {@link Scorer#score(SubCollector)} will be called; if false, * {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will * be called. * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer. diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 90310e8..42d573d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -18,19 +18,15 @@ package org.apache.lucene.search; */ import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.List; -import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.DirectoryReader; // javadocs @@ -46,7 +42,6 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.search.similarities.DefaultSimilarity; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.NIOFSDirectory; // javadoc -import org.apache.lucene.util.ThreadInterruptedException; import org.apache.lucene.index.IndexWriter; // javadocs /** Implements search over a single IndexReader. @@ -82,11 +77,9 @@ public class IndexSearcher { // in the next release protected final IndexReaderContext readerContext; protected final List leafContexts; - /** used with executor - each slice holds a set of leafs executed within one thread */ - protected final LeafSlice[] leafSlices; - // These are only used for multi-threaded search - private final ExecutorService executor; + // This is used for parallel search + protected final ExecutorService executor; // the default Similarity private static final Similarity defaultSimilarity = new DefaultSimilarity(); @@ -147,7 +140,6 @@ public class IndexSearcher { this.executor = executor; this.readerContext = context; leafContexts = context.leaves(); - this.leafSlices = executor == null ? null : slices(leafContexts); } /** @@ -160,21 +152,7 @@ public class IndexSearcher { public IndexSearcher(IndexReaderContext context) { this(context, null); } - - /** - * Expert: Creates an array of leaf slices each holding a subset of the given leaves. - * Each {@link LeafSlice} is executed in a single thread. By default there - * will be one {@link LeafSlice} per leaf ({@link AtomicReaderContext}). - */ - protected LeafSlice[] slices(List leaves) { - LeafSlice[] slices = new LeafSlice[leaves.size()]; - for (int i = 0; i < slices.length; i++) { - slices[i] = new LeafSlice(leaves.get(i)); - } - return slices; - } - /** Return the {@link IndexReader} this searches. */ public IndexReader getIndexReader() { return reader; @@ -275,7 +253,7 @@ public class IndexSearcher { /** Lower-level search API. * - *

{@link Collector#collect(int)} is called for every matching + *

{@link SubCollector#collect(int)} is called for every matching * document. * * @param query to match documents @@ -291,7 +269,7 @@ public class IndexSearcher { /** Lower-level search API. * - *

{@link Collector#collect(int)} is called for every matching document. + *

{@link SubCollector#collect(int)} is called for every matching document. * * @throws BooleanQuery.TooManyClauses If a query would exceed * {@link BooleanQuery#getMaxClauseCount()} clauses. @@ -434,34 +412,7 @@ public class IndexSearcher { throw new IllegalArgumentException("after.doc exceeds the number of documents in that reader: after.doc=" + after.doc + " limit=" + limit); } - nDocs = Math.min(nDocs, limit); - - if (executor == null) { - return search(leafContexts, weight, after, nDocs); - } else { - final HitQueue hq = new HitQueue(nDocs, false); - final Lock lock = new ReentrantLock(); - final ExecutionHelper runner = new ExecutionHelper(executor); - - for (int i = 0; i < leafSlices.length; i++) { // search each sub - runner.submit(new SearcherCallableNoSort(lock, this, leafSlices[i], weight, after, nDocs, hq)); - } - - int totalHits = 0; - float maxScore = Float.NEGATIVE_INFINITY; - for (final TopDocs topDocs : runner) { - if(topDocs.totalHits != 0) { - totalHits += topDocs.totalHits; - maxScore = Math.max(maxScore, topDocs.getMaxScore()); - } - } - - final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()]; - for (int i = hq.size() - 1; i >= 0; i--) // put docs in array - scoreDocs[i] = hq.pop(); - - return new TopDocs(totalHits, scoreDocs, maxScore); - } + return search(leafContexts, weight, after, Math.min(nDocs, limit)); } /** Expert: Low-level search implementation. Finds the top n @@ -518,38 +469,7 @@ public class IndexSearcher { if (limit == 0) { limit = 1; } - nDocs = Math.min(nDocs, limit); - - if (executor == null) { - // use all leaves here! - return search(leafContexts, weight, after, nDocs, sort, fillFields, doDocScores, doMaxScore); - } else { - final TopFieldCollector topCollector = TopFieldCollector.create(sort, nDocs, - after, - fillFields, - doDocScores, - doMaxScore, - false); - - final Lock lock = new ReentrantLock(); - final ExecutionHelper runner = new ExecutionHelper(executor); - for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice - runner.submit( - new SearcherCallableWithSort(lock, this, leafSlices[i], weight, after, nDocs, topCollector, sort, doDocScores, doMaxScore)); - } - int totalHits = 0; - float maxScore = Float.NEGATIVE_INFINITY; - for (final TopFieldDocs topFieldDocs : runner) { - if (topFieldDocs.totalHits != 0) { - totalHits += topFieldDocs.totalHits; - maxScore = Math.max(maxScore, topFieldDocs.getMaxScore()); - } - } - - final TopFieldDocs topDocs = (TopFieldDocs) topCollector.topDocs(); - - return new TopFieldDocs(totalHits, topDocs.scoreDocs, topDocs.fields, topDocs.getMaxScore()); - } + return search(leafContexts, weight, after, Math.min(nDocs, limit), sort, fillFields, doDocScores, doMaxScore); } @@ -578,7 +498,7 @@ public class IndexSearcher { * Lower-level search API. * *

- * {@link Collector#collect(int)} is called for every document.
+ * {@link SubCollector#collect(int)} is called for every document.
* *

* NOTE: this method executes the searches on all given leaves exclusively. @@ -593,29 +513,83 @@ public class IndexSearcher { * @throws BooleanQuery.TooManyClauses If a query would exceed * {@link BooleanQuery#getMaxClauseCount()} clauses. */ - protected void search(List leaves, Weight weight, Collector collector) + protected void search(List leaves, final Weight weight, final Collector collector) throws IOException { + if (executor != null && collector.isParallelizable() && leaves.size() > 1) { + searchParallel(leaves, weight, collector); + } else { + searchSerial(leaves, weight, collector); + } + } - // TODO: should we make this - // threaded...? the Collector could be sync'd? - // always use single thread: - for (AtomicReaderContext ctx : leaves) { // search each subreader + protected void searchSerial(List leaves, final Weight weight, final Collector collector) + throws IOException { + for (final AtomicReaderContext ctx : leaves) { // search each subreader + final SubCollector sub; try { - collector.setNextReader(ctx); + sub = collector.subCollector(ctx); } catch (CollectionTerminatedException e) { - // there is no doc of interest in this reader context - // continue with the following leaf - continue; + continue; // there is no doc of interest in this reader context } - Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs()); - if (scorer != null) { - try { - scorer.score(collector); - } catch (CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf + collect(ctx, weight, sub); + sub.done(); + } + } + + protected void searchParallel(List leaves, final Weight weight, final Collector collector) + throws IOException { + collector.setParallelized(); + + final List> futures = new ArrayList>(leaves.size()); + + for (final AtomicReaderContext ctx : leaves) { // search each subreader + final SubCollector sub; + try { + sub = collector.subCollector(ctx); + } catch (CollectionTerminatedException e) { + continue; // there is no doc of interest in this reader context + } + futures.add(executor.submit(new Callable() { + @Override + public SubCollector call() throws IOException { + collect(ctx, weight, sub); + return sub; + } + })); + } + + for (Future f: futures) { + if (f instanceof FutureTask) { + ((FutureTask) f).run(); // help out if it hasn't begun executing, rather than blocking idly + } + } + + for (Future f: futures) { + final SubCollector sub; + try { + sub = f.get(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + final Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException) cause; + } else { + throw new RuntimeException(cause); } } + sub.done(); // SubCollector.done() is guaranteed to execute in the primary thread + } + } + + private static void collect(AtomicReaderContext ctx, Weight weight, SubCollector sub) throws IOException { + final Scorer scorer = weight.scorer(ctx, !sub.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs()); + if (scorer != null) { + try { + scorer.score(sub); + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + } } } @@ -692,217 +666,6 @@ public class IndexSearcher { return readerContext; } - /** - * A thread subclass for searching a single searchable - */ - private static final class SearcherCallableNoSort implements Callable { - - private final Lock lock; - private final IndexSearcher searcher; - private final Weight weight; - private final ScoreDoc after; - private final int nDocs; - private final HitQueue hq; - private final LeafSlice slice; - - public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight, - ScoreDoc after, int nDocs, HitQueue hq) { - this.lock = lock; - this.searcher = searcher; - this.weight = weight; - this.after = after; - this.nDocs = nDocs; - this.hq = hq; - this.slice = slice; - } - - @Override - public TopDocs call() throws IOException { - final TopDocs docs = searcher.search(Arrays.asList(slice.leaves), weight, after, nDocs); - final ScoreDoc[] scoreDocs = docs.scoreDocs; - //it would be so nice if we had a thread-safe insert - lock.lock(); - try { - for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq - final ScoreDoc scoreDoc = scoreDocs[j]; - if (scoreDoc == hq.insertWithOverflow(scoreDoc)) { - break; - } - } - } finally { - lock.unlock(); - } - return docs; - } - } - - - /** - * A thread subclass for searching a single searchable - */ - private static final class SearcherCallableWithSort implements Callable { - - private final Lock lock; - private final IndexSearcher searcher; - private final Weight weight; - private final int nDocs; - private final TopFieldCollector hq; - private final Sort sort; - private final LeafSlice slice; - private final FieldDoc after; - private final boolean doDocScores; - private final boolean doMaxScore; - - public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight, - FieldDoc after, int nDocs, TopFieldCollector hq, Sort sort, - boolean doDocScores, boolean doMaxScore) { - this.lock = lock; - this.searcher = searcher; - this.weight = weight; - this.nDocs = nDocs; - this.hq = hq; - this.sort = sort; - this.slice = slice; - this.after = after; - this.doDocScores = doDocScores; - this.doMaxScore = doMaxScore; - } - - private final class FakeScorer extends Scorer { - float score; - int doc; - - public FakeScorer() { - super(null); - } - - @Override - public int advance(int target) { - throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)"); - } - - @Override - public int docID() { - return doc; - } - - @Override - public int freq() { - throw new UnsupportedOperationException("FakeScorer doesn't support freq()"); - } - - @Override - public int nextDoc() { - throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()"); - } - - @Override - public float score() { - return score; - } - - @Override - public long cost() { - return 1; - } - } - - private final FakeScorer fakeScorer = new FakeScorer(); - - @Override - public TopFieldDocs call() throws IOException { - assert slice.leaves.length == 1; - final TopFieldDocs docs = searcher.search(Arrays.asList(slice.leaves), - weight, after, nDocs, sort, true, doDocScores || sort.needsScores(), doMaxScore); - lock.lock(); - try { - final AtomicReaderContext ctx = slice.leaves[0]; - final int base = ctx.docBase; - hq.setNextReader(ctx); - hq.setScorer(fakeScorer); - for(ScoreDoc scoreDoc : docs.scoreDocs) { - fakeScorer.doc = scoreDoc.doc - base; - fakeScorer.score = scoreDoc.score; - hq.collect(scoreDoc.doc-base); - } - - // Carry over maxScore from sub: - if (doMaxScore && docs.getMaxScore() > hq.maxScore) { - hq.maxScore = docs.getMaxScore(); - } - } finally { - lock.unlock(); - } - return docs; - } - } - - /** - * A helper class that wraps a {@link CompletionService} and provides an - * iterable interface to the completed {@link Callable} instances. - * - * @param - * the type of the {@link Callable} return value - */ - private static final class ExecutionHelper implements Iterator, Iterable { - private final CompletionService service; - private int numTasks; - - ExecutionHelper(final Executor executor) { - this.service = new ExecutorCompletionService(executor); - } - - @Override - public boolean hasNext() { - return numTasks > 0; - } - - public void submit(Callable task) { - this.service.submit(task); - ++numTasks; - } - - @Override - public T next() { - if(!this.hasNext()) - throw new NoSuchElementException("next() is called but hasNext() returned false"); - try { - return service.take().get(); - } catch (InterruptedException e) { - throw new ThreadInterruptedException(e); - } catch (ExecutionException e) { - throw new RuntimeException(e); - } finally { - --numTasks; - } - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator iterator() { - // use the shortcut here - this is only used in a private context - return this; - } - } - - /** - * A class holding a subset of the {@link IndexSearcher}s leaf contexts to be - * executed within a single thread. - * - * @lucene.experimental - */ - public static class LeafSlice { - final AtomicReaderContext[] leaves; - - public LeafSlice(AtomicReaderContext... leaves) { - this.leaves = leaves; - } - } - @Override public String toString() { return "IndexSearcher(" + reader + "; executor=" + executor + ")"; diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java index 40c0838..457df08 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java @@ -20,8 +20,6 @@ package org.apache.lucene.search; import java.io.IOException; import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; /** * A {@link Collector} which allows running a search with several @@ -29,7 +27,7 @@ import org.apache.lucene.search.Scorer; * list of collectors and wraps them with {@link MultiCollector}, while * filtering out the null null ones. */ -public class MultiCollector extends Collector { +public class MultiCollector implements Collector { /** * Wraps a list of {@link Collector}s with a {@link MultiCollector}. This @@ -90,35 +88,72 @@ public class MultiCollector extends Collector { this.collectors = collectors; } - @Override - public boolean acceptsDocsOutOfOrder() { - for (Collector c : collectors) { - if (!c.acceptsDocsOutOfOrder()) { - return false; + private static final class MultiSubCollector implements SubCollector { + + private final SubCollector[] subCollectors; + + private MultiSubCollector(SubCollector[] subCollectors) { + this.subCollectors = subCollectors; + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + for (SubCollector s: subCollectors) { + s.setScorer(scorer); } } - return true; + + @Override + public void collect(int doc) throws IOException { + for (SubCollector s: subCollectors) { + s.collect(doc); + } + } + + @Override + public void done() throws IOException { + for (SubCollector s: subCollectors) { + s.done(); + } + } + + + @Override + public boolean acceptsDocsOutOfOrder() { + for (SubCollector s: subCollectors) { + if (!s.acceptsDocsOutOfOrder()) { + return false; + } + } + return true; + } + } @Override - public void collect(int doc) throws IOException { - for (Collector c : collectors) { - c.collect(doc); + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + SubCollector[] subCollectors = new SubCollector[collectors.length]; + for (int i = 0; i < collectors.length; i++) { + subCollectors[i] = collectors[i].subCollector(context); } + return new MultiSubCollector(subCollectors); } @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - for (Collector c : collectors) { - c.setNextReader(context); + public void setParallelized() { + for (Collector c: collectors) { + c.setParallelized(); } } @Override - public void setScorer(Scorer s) throws IOException { - for (Collector c : collectors) { - c.setScorer(s); + public boolean isParallelizable() { + for (Collector c: collectors) { + if (!c.isParallelizable()) { + return false; + } } + return true; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java b/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java index d35a755..4877bd6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java @@ -26,38 +26,31 @@ import org.apache.lucene.index.AtomicReaderContext; * {@link Collector} and makes sure only documents with * scores > 0 are collected. */ -public class PositiveScoresOnlyCollector extends Collector { +public class PositiveScoresOnlyCollector extends WrappingCollector { - final private Collector c; - private Scorer scorer; - public PositiveScoresOnlyCollector(Collector c) { - this.c = c; - } - - @Override - public void collect(int doc) throws IOException { - if (scorer.score() > 0) { - c.collect(doc); - } - } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - c.setNextReader(context); - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - // Set a ScoreCachingWrappingScorer in case the wrapped Collector will call - // score() also. - this.scorer = new ScoreCachingWrappingScorer(scorer); - c.setScorer(this.scorer); + super(c); } @Override - public boolean acceptsDocsOutOfOrder() { - return c.acceptsDocsOutOfOrder(); + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + return new WrappingSubCollector(delegate.subCollector(context)) { + Scorer scorer; + + @Override + public void setScorer(Scorer scorer) throws IOException { + // Set a ScoreCachingWrappingScorer in case the wrapped Collector will call score() also. + delegate.setScorer(this.scorer = new ScoreCachingWrappingScorer(scorer)); + } + + @Override + public void collect(int doc) throws IOException { + if (scorer.score() > 0) { + delegate.collect(doc); + } + } + + }; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java index f850f53..ab80bcb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java @@ -45,7 +45,7 @@ public class ScoreCachingWrappingScorer extends Scorer { } @Override - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector collector, int max, int firstDocID) throws IOException { return scorer.score(collector, max, firstDocID); } @@ -76,7 +76,7 @@ public class ScoreCachingWrappingScorer extends Scorer { } @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { scorer.score(collector); } diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java index 47fef12..e248cc3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java @@ -55,14 +55,14 @@ public abstract class Scorer extends DocsEnum { } /** Scores and collects all matching documents. - * @param collector The collector to which all matching documents are passed. + * @param subCollector The collector to which all matching documents are passed. */ - public void score(Collector collector) throws IOException { + public void score(SubCollector subCollector) throws IOException { assert docID() == -1; // not started - collector.setScorer(this); + subCollector.setScorer(this); int doc; while ((doc = nextDoc()) != NO_MORE_DOCS) { - collector.collect(doc); + subCollector.collect(doc); } } @@ -71,7 +71,7 @@ public abstract class Scorer extends DocsEnum { * Note, firstDocID is added to ensure that {@link #nextDoc()} * was called before this method. * - * @param collector + * @param subCollector * The collector to which all matching documents are passed. * @param max * Do not score documents past this. @@ -80,12 +80,12 @@ public abstract class Scorer extends DocsEnum { * this method. * @return true if more matching documents may remain. */ - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector subCollector, int max, int firstDocID) throws IOException { assert docID() == firstDocID; - collector.setScorer(this); + subCollector.setScorer(this); int doc; for (doc = firstDocID; doc < max; doc = nextDoc()) { - collector.collect(doc); + subCollector.collect(doc); } return doc != NO_MORE_DOCS; } @@ -93,7 +93,7 @@ public abstract class Scorer extends DocsEnum { /** Returns the score of the current document matching the query. * Initially invalid, until {@link #nextDoc()} or {@link #advance(int)} * is called the first time, or when called from within - * {@link Collector#collect}. + * {@link SubCollector#collect}. */ public abstract float score() throws IOException; diff --git a/lucene/core/src/java/org/apache/lucene/search/SerialCollector.java b/lucene/core/src/java/org/apache/lucene/search/SerialCollector.java new file mode 100644 index 0000000..f720104 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/SerialCollector.java @@ -0,0 +1,63 @@ +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 org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.index.IndexReaderContext; // javadoc + +import java.io.IOException; + +/** + * SerialCollector is intended as a drop-in replacement for the old Collector API which did not rely on + * per-reader-context {@link SubCollector} instances. + */ +public abstract class SerialCollector implements Collector, SubCollector { + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + setNextReader(context); + return this; + } + + @Override + public void done() throws IOException { + } + + @Override + public void setParallelized() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isParallelizable() { + return false; + } + + /** + * Called before collecting from each {@link AtomicReaderContext}. All doc ids in + * {@link #collect(int)} will correspond to {@link IndexReaderContext#reader}. + *

+ * Add {@link AtomicReaderContext#docBase} to the current {@link IndexReaderContext#reader}'s + * internal document id to re-base ids in {@link #collect(int)}. + * + * @param context + * next atomic reader context + */ + protected abstract void setNextReader(AtomicReaderContext context) throws IOException; + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/SubCollector.java b/lucene/core/src/java/org/apache/lucene/search/SubCollector.java new file mode 100644 index 0000000..5178f95 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/SubCollector.java @@ -0,0 +1,76 @@ +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; + +/** + *

A SubCollector is the target of successive calls to {@link #collect(int)} for each document that matched a + * query.

+ *

SubCollector decouples the score from the collected doc: the score computation can be skipped entirely if it's + * not needed. SubCollectors that do need the score should implement the {@link #setScorer(Scorer)} method.

+ *

It is strongly recommended that {@link Collector#isParallelizable() parallelizable implementations} not achieve + * the parallelizability by way of blocking constructs like synchronization.

+ */ +public interface SubCollector { + + /** + *

Called before successive calls to {@link #collect(int)}. Implementations that need the score of the current + * document (passed-in to {@link #collect(int)}), should save the passed-in {@link Scorer} and call {@code + * scorer.score()} when needed.

+ *

If the score for a hit may be requested multiple times, you should use {@link ScoreCachingWrappingScorer}.

+ * + * @param scorer + */ + void setScorer(Scorer scorer) throws IOException; + + /** + *

Called once for every document matching a query, with the unbased document number.

+ *

Note: The collection of the current segment can be terminated by throwing a {@link + * CollectionTerminatedException}. In this case, the last docs of the current {@link + * org.apache.lucene.index.AtomicReader reader} will be skipped and {@link IndexSearcher} will swallow the exception + * and continue collection with the next leaf. However {@link #done()} will still be invoked.

+ *

Note: This is called in an inner search loop. For good search performance, implementations of this method + * should not call {@link IndexSearcher#doc(int)} or {@link org.apache.lucene.index.IndexReader#document(int)} on + * every hit. Doing so can slow searches by an order of magnitude or more.

+ * + * @param doc + * unbased document ID + */ + void collect(int doc) throws IOException; + + /** + * Advise that no further calls to this sub-collector will be made. + * + * @throws IOException + */ + void done() throws IOException; + + /** + *

Return {@code true} if this collector does not require the matching docIDs to be delivered in int sort + * order (smallest to largest) to {@link #collect}.

+ *

Most Lucene Query implementations will visit matching docIDs in order. However, some queries (currently limited + * to certain cases of {@link BooleanQuery}) can achieve faster searching if the {@code SubCollector} allows them to + * deliver the docIDs out of order.

+ *

Many collectors don't mind getting docIDs out of order, so it's important to return {@code true} here. + * + * @return whether calls to {@link #collect(int)} must be in int sort order of docID. + */ + boolean acceptsDocsOutOfOrder(); + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java index 2d2eb0e..0c258a7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java @@ -29,8 +29,7 @@ import java.io.IOException; * exceeded, the search thread is stopped by throwing a * {@link TimeExceededException}. */ -public class TimeLimitingCollector extends Collector { - +public class TimeLimitingCollector implements Collector { /** Thrown when elapsed search time exceeds allowed search time. */ @SuppressWarnings("serial") @@ -64,7 +63,6 @@ public class TimeLimitingCollector extends Collector { private final Counter clock; private final long ticksAllowed; private boolean greedy = false; - private int docBase; /** * Create a TimeLimitedCollector wrapper over another {@link Collector} with a specified timeout. @@ -78,10 +76,10 @@ public class TimeLimitingCollector extends Collector { this.clock = clock; this.ticksAllowed = ticksAllowed; } - + /** - * Sets the baseline for this collector. By default the collectors baseline is - * initialized once the first reader is passed to the collector. + * Sets the baseline for this collector. By default the collectors baseline is + * initialized once the first reader is passed to the collector. * To include operations executed in prior to the actual document collection * set the baseline through this method in your prelude. *

@@ -95,13 +93,13 @@ public class TimeLimitingCollector extends Collector { * indexSearcher.search(query, collector); * *

- * @see #setBaseline() + * @see #setBaseline() */ public void setBaseline(long clockTime) { t0 = clockTime; timeout = t0 + ticksAllowed; } - + /** * Syntactic sugar for {@link #setBaseline(long)} using {@link Counter#get()} * on the clock passed to the constructor. @@ -109,7 +107,7 @@ public class TimeLimitingCollector extends Collector { public void setBaseline() { setBaseline(clock.get()); } - + /** * Checks if this time limited collector is greedy in collecting the last hit. * A non greedy collector, upon a timeout, would throw a {@link TimeExceededException} @@ -130,48 +128,65 @@ public class TimeLimitingCollector extends Collector { public void setGreedy(boolean greedy) { this.greedy = greedy; } - - /** - * Calls {@link Collector#collect(int)} on the decorated {@link Collector} - * unless the allowed time has passed, in which case it throws an exception. - * - * @throws TimeExceededException - * if the time allowed has exceeded. - */ - @Override - public void collect(final int doc) throws IOException { - final long time = clock.get(); - if (timeout < time) { - if (greedy) { - //System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0)); - collector.collect(doc); - } - //System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0)); - throw new TimeExceededException( timeout-t0, time-t0, docBase + doc ); - } - //System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0)); - collector.collect(doc); - } - + @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - collector.setNextReader(context); - this.docBase = context.docBase; + public SubCollector subCollector(final AtomicReaderContext context) throws IOException { if (Long.MIN_VALUE == t0) { setBaseline(); } + final int docBase = context.docBase; + final SubCollector sub = collector.subCollector(context); + return new SubCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + sub.setScorer(scorer); + } + + /** + * Calls {@link org.apache.lucene.search.SubCollector#collect(int)} on the decorated + * {@link org.apache.lucene.search.SubCollector} unless the allowed time has passed, in which case it throws an + * exception. + * + * @throws org.apache.lucene.search.TimeLimitingCollector.TimeExceededException + * if the time allowed has exceeded. + */ + @Override + public void collect(int doc) throws IOException { + final long time = clock.get(); + if (timeout < time) { + if (greedy) { + //System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0)); + sub.collect(doc); + } + //System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0)); + throw new TimeExceededException( timeout-t0, time-t0, docBase + doc ); + } + //System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0)); + sub.collect(doc); + } + + @Override + public void done() throws IOException { + sub.done(); + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return sub.acceptsDocsOutOfOrder(); + } + }; } - + @Override - public void setScorer(Scorer scorer) throws IOException { - collector.setScorer(scorer); + public void setParallelized() { + collector.setParallelized(); } @Override - public boolean acceptsDocsOutOfOrder() { - return collector.acceptsDocsOutOfOrder(); + public boolean isParallelizable() { + return collector.isParallelizable(); } - + /** * This is so the same timer can be used with a multi-phase search process such as grouping. * We don't want to create a new TimeLimitingCollector for each phase because that would diff --git a/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java index 5e7dd50..76b0ea1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java @@ -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 extends Collector { +public abstract class TopDocsCollector implements Collector { /** This is used in case topDocs() is called with illegal parameters, or there * simply aren't (enough) results. */ diff --git a/lucene/core/src/java/org/apache/lucene/search/TopDocsSerialCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopDocsSerialCollector.java new file mode 100644 index 0000000..c102564 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/TopDocsSerialCollector.java @@ -0,0 +1,52 @@ +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 org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.util.PriorityQueue; + +import java.io.IOException; + +public abstract class TopDocsSerialCollector extends TopDocsCollector implements SubCollector { + + protected TopDocsSerialCollector(PriorityQueue pq) { + super(pq); + } + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + setNextReader(context); + return this; + } + + @Override + public void done() throws IOException { + } + + @Override + public void setParallelized() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isParallelizable() { + return false; + } + + protected abstract void setNextReader(AtomicReaderContext context) throws IOException; +} diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index 9842aeb..66ce61c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.FieldValueHitQueue.Entry; -import org.apache.lucene.util.PriorityQueue; /** * A {@link Collector} that sorts by {@link SortField} using @@ -33,7 +32,7 @@ import org.apache.lucene.util.PriorityQueue; * @lucene.experimental */ public abstract class TopFieldCollector extends TopDocsCollector { - + // TODO: one optimization we could do is to pre-fill // the queue with sentinel value that guaranteed to // always compare lower than a real hit; this would @@ -43,25 +42,21 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over one SortField criteria, without * tracking document scores and maxScore. */ - private static class OneComparatorNonScoringCollector extends - TopFieldCollector { + private class OneComparatorNonScoringSubCollector extends TopFieldSubCollector { - FieldComparator comparator; + final FieldComparator comparator; final int reverseMul; - final FieldValueHitQueue queue; - - public OneComparatorNonScoringCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); - this.queue = queue; - comparator = queue.getComparators()[0]; - reverseMul = queue.getReverseMul()[0]; + + OneComparatorNonScoringSubCollector(AtomicReaderContext context) throws IOException { + super(context); + comparator = queue.firstComparator; + reverseMul = queue.reverseMul[0]; } - + final void updateBottom(int doc) { // bottom.score is already set to Float.NaN in add(). bottom.doc = docBase + doc; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -74,7 +69,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { // therefore not competitive. return; } - + // This hit is competitive - replace bottom element in queue & adjustTop comparator.copy(bottom.slot, doc); updateBottom(doc); @@ -90,19 +85,12 @@ public abstract class TopFieldCollector extends TopDocsCollector { } } } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.docBase = context.docBase; - queue.setComparator(0, comparator.setNextReader(context)); - comparator = queue.firstComparator; - } - + @Override public void setScorer(Scorer scorer) throws IOException { comparator.setScorer(scorer); } - + } /* @@ -110,12 +98,11 @@ public abstract class TopFieldCollector extends TopDocsCollector { * tracking document scores and maxScore, and assumes out of orderness in doc * Ids collection. */ - private static class OutOfOrderOneComparatorNonScoringCollector extends - OneComparatorNonScoringCollector { + private class OutOfOrderOneComparatorNonScoringSubCollector extends OneComparatorNonScoringSubCollector { - public OutOfOrderOneComparatorNonScoringCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderOneComparatorNonScoringSubCollector(AtomicReaderContext context) + throws IOException { + super(context); } @Override @@ -155,20 +142,18 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over one SortField criteria, while tracking * document scores but no maxScore. */ - private static class OneComparatorScoringNoMaxScoreCollector extends - OneComparatorNonScoringCollector { + private class OneComparatorScoringNoMaxScoreSubCollector extends OneComparatorNonScoringSubCollector { Scorer scorer; - public OneComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OneComparatorScoringNoMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } final void updateBottom(int doc, float score) { bottom.doc = docBase + doc; bottom.score = score; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -217,12 +202,10 @@ public abstract class TopFieldCollector extends TopDocsCollector { * document scores but no maxScore, and assumes out of orderness in doc Ids * collection. */ - private static class OutOfOrderOneComparatorScoringNoMaxScoreCollector extends - OneComparatorScoringNoMaxScoreCollector { + private class OutOfOrderOneComparatorScoringNoMaxScoreSubCollector extends OneComparatorScoringNoMaxScoreSubCollector { - public OutOfOrderOneComparatorScoringNoMaxScoreCollector( - FieldValueHitQueue queue, int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderOneComparatorScoringNoMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } @Override @@ -268,22 +251,19 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over one SortField criteria, with tracking * document scores and maxScore. */ - private static class OneComparatorScoringMaxScoreCollector extends - OneComparatorNonScoringCollector { + private class OneComparatorScoringMaxScoreSubCollector extends OneComparatorNonScoringSubCollector { Scorer scorer; - public OneComparatorScoringMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); - // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN. - maxScore = Float.NEGATIVE_INFINITY; + public OneComparatorScoringMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); + initMaxScore(); } final void updateBottom(int doc, float score) { bottom.doc = docBase + doc; bottom.score = score; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -330,12 +310,10 @@ public abstract class TopFieldCollector extends TopDocsCollector { * document scores and maxScore, and assumes out of orderness in doc Ids * collection. */ - private static class OutOfOrderOneComparatorScoringMaxScoreCollector extends - OneComparatorScoringMaxScoreCollector { + private class OutOfOrderOneComparatorScoringMaxScoreSubCollector extends OneComparatorScoringMaxScoreSubCollector { - public OutOfOrderOneComparatorScoringMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderOneComparatorScoringMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } @Override @@ -379,23 +357,22 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over multiple SortField criteria, without * tracking document scores and maxScore. */ - private static class MultiComparatorNonScoringCollector extends TopFieldCollector { + private class MultiComparatorNonScoringSubCollector extends TopFieldSubCollector { final FieldComparator[] comparators; final int[] reverseMul; - final FieldValueHitQueue queue; - public MultiComparatorNonScoringCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); - this.queue = queue; - comparators = queue.getComparators(); - reverseMul = queue.getReverseMul(); + + public MultiComparatorNonScoringSubCollector(AtomicReaderContext context) + throws IOException { + super(context); + comparators = queue.comparators; + reverseMul = queue.reverseMul; } final void updateBottom(int doc) { // bottom.score is already set to Float.NaN in add(). bottom.doc = docBase + doc; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -446,14 +423,6 @@ public abstract class TopFieldCollector extends TopDocsCollector { } @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - docBase = context.docBase; - for (int i = 0; i < comparators.length; i++) { - queue.setComparator(i, comparators[i].setNextReader(context)); - } - } - - @Override public void setScorer(Scorer scorer) throws IOException { // set the scorer on all comparators for (int i = 0; i < comparators.length; i++) { @@ -467,12 +436,10 @@ public abstract class TopFieldCollector extends TopDocsCollector { * tracking document scores and maxScore, and assumes out of orderness in doc * Ids collection. */ - private static class OutOfOrderMultiComparatorNonScoringCollector extends - MultiComparatorNonScoringCollector { + private class OutOfOrderMultiComparatorNonScoringSubCollector extends MultiComparatorNonScoringSubCollector { - public OutOfOrderMultiComparatorNonScoringCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderMultiComparatorNonScoringSubCollector(AtomicReaderContext context) throws IOException { + super(context); } @Override @@ -535,21 +502,19 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over multiple SortField criteria, with * tracking document scores and maxScore. */ - private static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector { + private class MultiComparatorScoringMaxScoreSubCollector extends MultiComparatorNonScoringSubCollector { Scorer scorer; - public MultiComparatorScoringMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); - // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN. - maxScore = Float.NEGATIVE_INFINITY; + public MultiComparatorScoringMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); + initMaxScore(); } final void updateBottom(int doc, float score) { bottom.doc = docBase + doc; bottom.score = score; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -615,12 +580,11 @@ public abstract class TopFieldCollector extends TopDocsCollector { * tracking document scores and maxScore, and assumes out of orderness in doc * Ids collection. */ - private final static class OutOfOrderMultiComparatorScoringMaxScoreCollector - extends MultiComparatorScoringMaxScoreCollector { + private final class OutOfOrderMultiComparatorScoringMaxScoreSubCollector + extends MultiComparatorScoringMaxScoreSubCollector { - public OutOfOrderMultiComparatorScoringMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderMultiComparatorScoringMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } @Override @@ -687,19 +651,18 @@ public abstract class TopFieldCollector extends TopDocsCollector { * Implements a TopFieldCollector over multiple SortField criteria, with * tracking document scores and maxScore. */ - private static class MultiComparatorScoringNoMaxScoreCollector extends MultiComparatorNonScoringCollector { + private class MultiComparatorScoringNoMaxScoreSubCollector extends MultiComparatorNonScoringSubCollector { Scorer scorer; - public MultiComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue, - int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public MultiComparatorScoringNoMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } final void updateBottom(int doc, float score) { bottom.doc = docBase + doc; bottom.score = score; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @Override @@ -766,12 +729,11 @@ public abstract class TopFieldCollector extends TopDocsCollector { * tracking document scores and maxScore, and assumes out of orderness in doc * Ids collection. */ - private final static class OutOfOrderMultiComparatorScoringNoMaxScoreCollector - extends MultiComparatorScoringNoMaxScoreCollector { + private final class OutOfOrderMultiComparatorScoringNoMaxScoreSubCollector + extends MultiComparatorScoringNoMaxScoreSubCollector { - public OutOfOrderMultiComparatorScoringNoMaxScoreCollector( - FieldValueHitQueue queue, int numHits, boolean fillFields) { - super(queue, numHits, fillFields); + public OutOfOrderMultiComparatorScoringNoMaxScoreSubCollector(AtomicReaderContext context) throws IOException { + super(context); } @Override @@ -844,37 +806,40 @@ public abstract class TopFieldCollector extends TopDocsCollector { /* * Implements a TopFieldCollector when after != null. */ - private final static class PagingFieldCollector extends TopFieldCollector { + private final class PagingFieldSubCollector extends TopFieldSubCollector { - Scorer scorer; - int collectedHits; - final FieldComparator[] comparators; - final int[] reverseMul; - final FieldValueHitQueue queue; + final FieldDoc after; final boolean trackDocScores; final boolean trackMaxScore; - final FieldDoc after; - int afterDoc; - - public PagingFieldCollector( - FieldValueHitQueue queue, FieldDoc after, int numHits, boolean fillFields, - boolean trackDocScores, boolean trackMaxScore) { - super(queue, numHits, fillFields); - this.queue = queue; + final int afterDoc; + + final FieldComparator[] comparators; + final int[] reverseMul; + + Scorer scorer; + + public PagingFieldSubCollector(FieldDoc after, boolean trackDocScores, boolean trackMaxScore, + AtomicReaderContext context) throws IOException { + super(context); + + this.after = after; this.trackDocScores = trackDocScores; this.trackMaxScore = trackMaxScore; - this.after = after; - comparators = queue.getComparators(); - reverseMul = queue.getReverseMul(); - // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN. - maxScore = Float.NEGATIVE_INFINITY; + afterDoc = after.doc - docBase; + + comparators = queue.comparators; + reverseMul = queue.reverseMul; + + if (trackMaxScore) { + initMaxScore(); + } } - + void updateBottom(int doc, float score) { bottom.doc = docBase + doc; bottom.score = score; - bottom = pq.updateTop(); + bottom = queue.updateTop(); } @SuppressWarnings({"unchecked", "rawtypes"}) @@ -967,7 +932,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { if (trackDocScores && !trackMaxScore) { score = scorer.score(); } - bottom = pq.add(new Entry(slot, docBase + doc, score)); + bottom = queue.add(new Entry(slot, docBase + doc, score)); queueFull = collectedHits == numHits; if (queueFull) { for (int i = 0; i < comparators.length; i++) { @@ -984,44 +949,166 @@ public abstract class TopFieldCollector extends TopDocsCollector { comparators[i].setScorer(scorer); } } - + @Override public boolean acceptsDocsOutOfOrder() { return true; } + } + + private static interface Creator { + TopFieldCollector create(FieldValueHitQueue q, boolean docsScoredInOrder) throws IOException; + } + + private static final class FakeScorer extends Scorer { + float score; + int doc; + + public FakeScorer() { + super(null); + } + @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - docBase = context.docBase; - afterDoc = after.doc - docBase; - for (int i = 0; i < comparators.length; i++) { - queue.setComparator(i, comparators[i].setNextReader(context)); + public int advance(int target) { + throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)"); + } + + @Override + public int docID() { + return doc; + } + + @Override + public int freq() { + throw new UnsupportedOperationException("FakeScorer doesn't support freq()"); + } + + @Override + public int nextDoc() { + throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()"); + } + + @Override + public float score() { + return score; + } + + @Override + public long cost() { + return 1; + } + } + + private abstract class TopFieldSubCollector implements SubCollector { + + final AtomicReaderContext context; + final int docBase; + + final FieldValueHitQueue queue; + int totalHits; + int collectedHits; + float maxScore = Float.NaN; + + FieldValueHitQueue.Entry bottom; + boolean queueFull; + + TopFieldSubCollector(AtomicReaderContext context) throws IOException { + this.context = context; + this.docBase = context.docBase; + + if (parallelized) { + queue = FieldValueHitQueue.create(TopFieldCollector.this.queue.getFields(), numHits); + } else { // pick up where we left of + queue = TopFieldCollector.this.queue; + totalHits = TopFieldCollector.this.totalHits; + collectedHits = TopFieldCollector.this.collectedHits; + bottom = queue.top(); + queueFull = totalHits >= numHits; + } + + for (int i = 0; i < queue.comparators.length; i++) { + queue.setComparator(i, queue.comparators[i].setNextReader(context)); + } + } + + // must be invoked by subclasses that care about maxScore to correctly initialize it to a non-NaN value + final void initMaxScore() { + if (parallelized) { + maxScore = Float.NEGATIVE_INFINITY; + } else { + maxScore = Float.isNaN(TopFieldCollector.this.maxScore) + ? Float.NEGATIVE_INFINITY + : Math.max(Float.NEGATIVE_INFINITY, TopFieldCollector.this.maxScore); + } + } + + final void add(int slot, int doc, float score) { + bottom = queue.add(new Entry(slot, docBase + doc, score)); + queueFull = totalHits == numHits; + } + + @Override + public void done() throws IOException { + if (parallelized) { // merge our state + if (mergeAccumulator == null) { + mergeAccumulator = recreator.create(TopFieldCollector.this.queue, false); + } + + final FakeScorer fakeScorer = new FakeScorer(); + final SubCollector sub = mergeAccumulator.subCollector(context); + sub.setScorer(fakeScorer); + for (final Entry e: queue) { + fakeScorer.doc = e.doc - docBase; + fakeScorer.score = e.score; + sub.collect(fakeScorer.doc); + } + sub.done(); + + TopFieldCollector.this.totalHits += totalHits; + TopFieldCollector.this.collectedHits += collectedHits; + TopFieldCollector.this.maxScore = Float.isNaN(TopFieldCollector.this.maxScore) + ? maxScore : Math.max(maxScore, TopFieldCollector.this.maxScore); + } else { // stash our state + TopFieldCollector.this.totalHits = totalHits; + TopFieldCollector.this.collectedHits = collectedHits; + TopFieldCollector.this.maxScore = maxScore; } } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } } private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0]; - - private final boolean fillFields; - /* - * Stores the maximum score value encountered, needed for normalizing. If - * document scores are not tracked, this value is initialized to NaN. - */ - float maxScore = Float.NaN; + final Creator recreator; + final FieldValueHitQueue queue; final int numHits; - FieldValueHitQueue.Entry bottom = null; - boolean queueFull; - int docBase; - + final boolean fillFields; + + boolean parallelized; + + int collectedHits; + + TopFieldCollector mergeAccumulator; // used during SubCollector.done() merge legwork in parallel collection + + // Stores the maximum score value encountered, needed for normalizing. + // If document scores are not tracked, this value will stay NaN. + float maxScore = Float.NaN; + // Declaring the constructor private prevents extending this class by anyone // else. Note that the class cannot be final since it's extended by the // internal versions. If someone will define a constructor with any other // visibility, then anyone will be able to extend the class, which is not what // we want. - private TopFieldCollector(PriorityQueue pq, int numHits, boolean fillFields) { - super(pq); + private TopFieldCollector(Creator recreator, FieldValueHitQueue queue, int numHits, boolean fillFields) { + super(queue); + this.recreator = recreator; + this.queue = queue; this.numHits = numHits; this.fillFields = fillFields; } @@ -1057,7 +1144,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { * trackDocScores to true as well. * @param docsScoredInOrder * specifies whether documents are scored in doc Id order or not by - * the given {@link Scorer} in {@link #setScorer(Scorer)}. + * the given {@link Scorer} in {@link SubCollector#setScorer(Scorer)}. * @return a {@link TopFieldCollector} instance which will sort the results by * the sort criteria. * @throws IOException if there is a low-level I/O error @@ -1102,81 +1189,117 @@ public abstract class TopFieldCollector extends TopDocsCollector { * trackDocScores to true as well. * @param docsScoredInOrder * specifies whether documents are scored in doc Id order or not by - * the given {@link Scorer} in {@link #setScorer(Scorer)}. + * the given {@link Scorer} in {@link SubCollector#setScorer(Scorer)}. * @return a {@link TopFieldCollector} instance which will sort the results by * the sort criteria. * @throws IOException if there is a low-level I/O error */ - public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after, - boolean fillFields, boolean trackDocScores, boolean trackMaxScore, - boolean docsScoredInOrder) + public static TopFieldCollector create(final Sort sort, final int numHits, final FieldDoc after, + final boolean fillFields, final boolean trackDocScores, final boolean trackMaxScore, + final boolean docsScoredInOrder) throws IOException { if (sort.fields.length == 0) { throw new IllegalArgumentException("Sort must contain at least one field"); } - + if (numHits <= 0) { throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count"); } - FieldValueHitQueue queue = FieldValueHitQueue.create(sort.fields, numHits); + final boolean needScores = trackDocScores || sort.needsScores(); + final Creator creator; if (after == null) { - if (queue.getComparators().length == 1) { - if (docsScoredInOrder) { - if (trackMaxScore) { - return new OneComparatorScoringMaxScoreCollector(queue, numHits, fillFields); - } else if (trackDocScores) { - return new OneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields); - } else { - return new OneComparatorNonScoringCollector(queue, numHits, fillFields); - } - } else { - if (trackMaxScore) { - return new OutOfOrderOneComparatorScoringMaxScoreCollector(queue, numHits, fillFields); - } else if (trackDocScores) { - return new OutOfOrderOneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields); - } else { - return new OutOfOrderOneComparatorNonScoringCollector(queue, numHits, fillFields); - } - } - } - // multiple comparators. - if (docsScoredInOrder) { - if (trackMaxScore) { - return new MultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields); - } else if (trackDocScores) { - return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields); - } else { - return new MultiComparatorNonScoringCollector(queue, numHits, fillFields); - } - } else { - if (trackMaxScore) { - return new OutOfOrderMultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields); - } else if (trackDocScores) { - return new OutOfOrderMultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields); - } else { - return new OutOfOrderMultiComparatorNonScoringCollector(queue, numHits, fillFields); + creator = new Creator() { + @Override + public TopFieldCollector create( + final FieldValueHitQueue queue, + final boolean docsScoredInOrder + ) throws IOException { + return new TopFieldCollector(this, queue, numHits, fillFields) { + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + + if (queue.getComparators().length == 1) { + + if (docsScoredInOrder) { + if (trackMaxScore) { + return new OneComparatorScoringMaxScoreSubCollector(context); + } else if (needScores) { + return new OneComparatorScoringNoMaxScoreSubCollector(context); + } else { + return new OneComparatorNonScoringSubCollector(context); + } + } else { + if (trackMaxScore) { + return new OutOfOrderOneComparatorScoringMaxScoreSubCollector(context); + } else if (needScores) { + return new OutOfOrderOneComparatorScoringNoMaxScoreSubCollector(context); + } else { + return new OutOfOrderOneComparatorNonScoringSubCollector(context); + } + } + + } else { + + // multiple comparators. + if (docsScoredInOrder) { + if (trackMaxScore) { + return new MultiComparatorScoringMaxScoreSubCollector(context); + } else if (needScores) { + return new MultiComparatorScoringNoMaxScoreSubCollector(context); + } else { + return new MultiComparatorNonScoringSubCollector(context); + } + } else { + if (trackMaxScore) { + return new OutOfOrderMultiComparatorScoringMaxScoreSubCollector(context); + } else if (needScores) { + return new OutOfOrderMultiComparatorScoringNoMaxScoreSubCollector(context); + } else { + return new OutOfOrderMultiComparatorNonScoringSubCollector(context); + } + } + + } + } + }; } - } + }; + } else { + if (after.fields == null) { throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search"); } if (after.fields.length != sort.getSort().length) { - throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length); + throw new IllegalArgumentException("after.fields has " + + after.fields.length + " values but sort has " + sort.getSort().length); } - return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore); + creator = new Creator() { + @Override + public TopFieldCollector create( + final FieldValueHitQueue queue, + final boolean docsScoredInOrder + ) throws IOException { + + return new TopFieldCollector(this, queue, numHits, fillFields) { + @Override + public TopFieldSubCollector subCollector(AtomicReaderContext context) throws IOException { + return new PagingFieldSubCollector(after, needScores, trackMaxScore, context); + } + }; + + } + + }; } - } - - final void add(int slot, int doc, float score) { - bottom = pq.add(new Entry(slot, docBase + doc, score)); - queueFull = totalHits == numHits; + + return creator.create(FieldValueHitQueue.create(sort.fields, numHits), docsScoredInOrder); } /* @@ -1188,7 +1311,6 @@ public abstract class TopFieldCollector extends TopDocsCollector { protected void populateResults(ScoreDoc[] results, int howMany) { if (fillFields) { // avoid casting if unnecessary. - FieldValueHitQueue queue = (FieldValueHitQueue) pq; for (int i = howMany - 1; i >= 0; i--) { results[i] = queue.fillFields(queue.pop()); } @@ -1209,11 +1331,17 @@ public abstract class TopFieldCollector extends TopDocsCollector { } // If this is a maxScoring tracking collector and there were no results, - return new TopFieldDocs(totalHits, results, ((FieldValueHitQueue) pq).getFields(), maxScore); + return new TopFieldDocs(totalHits, results, queue.getFields(), maxScore); } - + @Override - public boolean acceptsDocsOutOfOrder() { - return false; + public void setParallelized() { + parallelized = true; } + + @Override + public boolean isParallelizable() { + return true; + } + } diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java index 0674779..c140e9d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java @@ -17,9 +17,11 @@ package org.apache.lucene.search; * limitations under the License. */ -import java.io.IOException; - import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.util.PriorityQueue; + +import java.io.IOException; +import java.util.List; /** * A {@link Collector} implementation that collects the top-scoring hits, @@ -41,88 +43,89 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { private InOrderTopScoreDocCollector(int numHits) { super(numHits); } - - @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(); - } - + @Override - public boolean acceptsDocsOutOfOrder() { - return false; + public TopScoreDocSubCollector subCollector(AtomicReaderContext context) { + final int docBase = context.docBase; + return new TopScoreDocSubCollector() { + @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(); + collectedHits++; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + }; } } - + // Assumes docs are scored in order. private static class InOrderPagingScoreDocCollector 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 InOrderPagingScoreDocCollector(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 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; - } + public TopScoreDocSubCollector subCollector(AtomicReaderContext context) { + final int docBase = context.docBase; + // this is always after.doc - docBase, to save an add when score == after.score + final int afterDoc = after.doc - docBase; + return new TopScoreDocSubCollector() { + @Override + public void collect(int doc) throws IOException { + float score = scorer.score(); - @Override - public void setNextReader(AtomicReaderContext context) { - super.setNextReader(context); - afterDoc = after.doc - docBase; - } + // This collector cannot handle these scores: + assert score != Float.NEGATIVE_INFINITY; + assert !Float.isNaN(score); - @Override - protected int topDocsSize() { - return collectedHits < pq.size() ? collectedHits : pq.size(); + 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; + } + pqTop.doc = doc + docBase; + pqTop.score = score; + pqTop = pq.updateTop(); + collectedHits++; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + }; } - + @Override protected TopDocs newTopDocs(ScoreDoc[] results, int start) { return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results); @@ -134,90 +137,91 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { private OutOfOrderTopScoreDocCollector(int numHits) { super(numHits); } - + @Override - public void collect(int doc) throws IOException { - float score = scorer.score(); + public TopScoreDocSubCollector subCollector(AtomicReaderContext context) { + final int docBase = context.docBase; + return new TopScoreDocSubCollector() { + @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 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(); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return true; + 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(); + collectedHits++; + } + + @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(); + public TopScoreDocSubCollector subCollector(AtomicReaderContext context) { + final int docBase = context.docBase; + // this is always after.doc - docBase, to save an add when score == after.score + final int afterDoc = after.doc - docBase; + return new TopScoreDocSubCollector() { + @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 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 - public void setNextReader(AtomicReaderContext context) { - super.setNextReader(context); - afterDoc = after.doc - docBase; - } - - @Override - protected int topDocsSize() { - return collectedHits < pq.size() ? collectedHits : pq.size(); + 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; + } + pqTop.doc = doc; + pqTop.score = score; + pqTop = pq.updateTop(); + collectedHits++; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + }; } - + @Override protected TopDocs newTopDocs(ScoreDoc[] results, int start) { return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results); @@ -227,7 +231,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { /** * 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 SubCollector#setScorer(Scorer)}. * *

NOTE: The instances returned by this method * pre-allocate a full array of length @@ -237,11 +241,11 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { 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 SubCollector#setScorer(Scorer)}. * *

NOTE: The instances returned by this method * pre-allocate a full array of length @@ -249,33 +253,85 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { * 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); + return after == null + ? new InOrderTopScoreDocCollector(numHits) + : new InOrderPagingScoreDocCollector(after, numHits); } else { return after == null - ? new OutOfOrderTopScoreDocCollector(numHits) - : new OutOfOrderPagingScoreDocCollector(after, numHits); + ? new OutOfOrderTopScoreDocCollector(numHits) + : new OutOfOrderPagingScoreDocCollector(after, numHits); } - + + } + + private abstract class TopScoreDocSubCollector implements SubCollector { + + final PriorityQueue pq; + ScoreDoc pqTop; + + Scorer scorer; + + int totalHits; + int collectedHits; + + private TopScoreDocSubCollector() { + pq = parallelized ? new HitQueue(numHits, true) : TopScoreDocCollector.this.pq; + // HitQueue implements getSentinelObject to return a ScoreDoc, so we know + // that at this point top() is already initialized. + pqTop = pq.top(); + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + } + + @Override + public void done() throws IOException { + TopScoreDocCollector.this.totalHits += totalHits; + TopScoreDocCollector.this.collectedHits += collectedHits; + if (parallelized) { + for (ScoreDoc scoreDoc: pq) { + TopScoreDocCollector.this.pq.insertWithOverflow(scoreDoc); + } + } + } + } - - ScoreDoc pqTop; - int docBase = 0; - Scorer scorer; - - // prevents instantiation - private TopScoreDocCollector(int numHits) { + + protected final int numHits; + + protected boolean parallelized; + + protected int totalHits; + protected int collectedHits; + + protected TopScoreDocCollector(int numHits) { super(new HitQueue(numHits, true)); - // HitQueue implements getSentinelObject to return a ScoreDoc, so we know - // that at this point top() is already initialized. - pqTop = pq.top(); + this.numHits = numHits; + } + + public abstract TopScoreDocSubCollector subCollector(AtomicReaderContext context); + + @Override + public void setParallelized() { + parallelized = true; + } + + @Override + public boolean isParallelizable() { + return true; + } + + @Override + public int getTotalHits() { + return totalHits; } @Override @@ -283,7 +339,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { 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 +351,13 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { for (int i = pq.size(); i > 1; i--) { pq.pop(); } maxScore = pq.pop().score; } - + return new TopDocs(totalHits, results, maxScore); } - - @Override - public void setNextReader(AtomicReaderContext context) { - docBase = context.docBase; - } - + @Override - public void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; + protected int topDocsSize() { + return Math.min(collectedHits, pq.size()); } + } diff --git a/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java index 1704d8b..d07d2cd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java @@ -19,11 +19,13 @@ package org.apache.lucene.search; import org.apache.lucene.index.AtomicReaderContext; +import java.io.IOException; + /** * Just counts the total number of hits. */ -public class TotalHitCountCollector extends Collector { +public class TotalHitCountCollector implements Collector { private int totalHits; /** Returns how many hits matched the search. */ @@ -32,20 +34,37 @@ public class TotalHitCountCollector extends Collector { } @Override - public void setScorer(Scorer scorer) { - } + public SubCollector subCollector(AtomicReaderContext ctx) { + return new SubCollector() { + int totalHits; - @Override - public void collect(int doc) { - totalHits++; + @Override + public void setScorer(Scorer scorer) { + } + + @Override + public void collect(int doc) { + totalHits++; + } + + @Override + public void done() throws IOException { + TotalHitCountCollector.this.totalHits += totalHits; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + }; } @Override - public void setNextReader(AtomicReaderContext context) { + public void setParallelized() { } @Override - public boolean acceptsDocsOutOfOrder() { + public boolean isParallelizable() { return true; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/TotalHitCountWithTopScoreCollector.java b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountWithTopScoreCollector.java new file mode 100644 index 0000000..9757339 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountWithTopScoreCollector.java @@ -0,0 +1,75 @@ +package org.apache.lucene.search; + +import org.apache.lucene.index.AtomicReaderContext; + +import java.io.IOException; + +public class TotalHitCountWithTopScoreCollector implements Collector { + + private final boolean scoringEnabled; + private int totalHits = 0; + private float topScore = Float.NEGATIVE_INFINITY; + + public TotalHitCountWithTopScoreCollector(boolean scoringEnabled) { + this.scoringEnabled = scoringEnabled; + } + + public int getTotalHits() { + return totalHits; + } + + public float getTopScore() { + return topScore; + } + + @Override + public SubCollector subCollector(AtomicReaderContext ctx) throws IOException { + return new SubCollector() { + + private int totalHits = 0; + private float topScore = Float.NEGATIVE_INFINITY; + private Scorer scorer; + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + } + + @Override + public void collect(int doc) throws IOException { + totalHits++; + if (scoringEnabled) { + float score = scorer.score(); + if (score > topScore) { + topScore = score; + } + } + } + + @Override + public void done() throws IOException { + TotalHitCountWithTopScoreCollector.this.totalHits += totalHits; + if (scoringEnabled) { + if (TotalHitCountWithTopScoreCollector.this.topScore > topScore) { + TotalHitCountWithTopScoreCollector.this.topScore = topScore; + } + } + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + }; + } + + @Override + public void setParallelized() { + } + + @Override + public boolean isParallelizable() { + return true; + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index 48dd209..8b9c890 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -100,7 +100,7 @@ public abstract class Weight { * and/or {@link Scorer#advance(int)}, therefore it is recommended to * request an in-order scorer if use of these methods is required. * @param topScorer - * if true, {@link Scorer#score(Collector)} will be called; if false, + * if true, {@link Scorer#score(SubCollector)} will be called; if false, * {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will * be called. * @param acceptDocs @@ -116,7 +116,7 @@ public abstract class Weight { /** * Returns true iff this implementation scores docs only out of order. This * method is used in conjunction with {@link Collector}'s - * {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and + * {@link SubCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and * {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)} to * create a matching {@link Scorer} instance for a given {@link Collector}, or * vice versa. diff --git a/lucene/core/src/java/org/apache/lucene/search/WrappingCollector.java b/lucene/core/src/java/org/apache/lucene/search/WrappingCollector.java new file mode 100644 index 0000000..31c7459 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/WrappingCollector.java @@ -0,0 +1,52 @@ +package org.apache.lucene.search; + +import java.io.IOException; + +public abstract class WrappingCollector implements Collector { + + public static class WrappingSubCollector implements SubCollector { + + protected final SubCollector delegate; + + public WrappingSubCollector(SubCollector delegate) { + this.delegate = delegate; + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + delegate.setScorer(scorer); + } + + @Override + public void collect(int doc) throws IOException { + delegate.collect(doc); + } + + @Override + public void done() throws IOException { + delegate.done(); + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return delegate.acceptsDocsOutOfOrder(); + } + } + + protected final Collector delegate; + + protected WrappingCollector(Collector delegate) { + this.delegate = delegate; + } + + @Override + public void setParallelized() { + delegate.setParallelized(); + } + + @Override + public boolean isParallelizable() { + return delegate.isParallelizable(); + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/package.html b/lucene/core/src/java/org/apache/lucene/search/package.html index 1a9e577..a6a3dc0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/package.html +++ b/lucene/core/src/java/org/apache/lucene/search/package.html @@ -539,7 +539,7 @@ on the built-in available scoring models and extending or changing Similarity.

If a Filter is being used, some initial setup is done to determine which docs to include. Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each {@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling - {@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.Collector) Scorer.score()}. + {@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.SubCollector) Scorer.score()}.

At last, we are actually going to score some documents. The score method takes in the Collector (most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here diff --git a/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java b/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java index ef658bc..c2f2420 100644 --- a/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java +++ b/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java @@ -17,6 +17,8 @@ package org.apache.lucene.util; * limitations under the License. */ +import java.util.Iterator; + /** A PriorityQueue maintains a partial ordering of its elements such that the * least element can always be found in constant time. Put()'s and pop()'s * require log(size) time. @@ -28,7 +30,7 @@ package org.apache.lucene.util; * * @lucene.internal */ -public abstract class PriorityQueue { +public abstract class PriorityQueue implements Iterable { private int size; private final int maxSize; private final T[] heap; @@ -262,4 +264,33 @@ public abstract class PriorityQueue { protected final Object[] getHeapArray() { return (Object[]) heap; } + + /** + * Provides an iterator over the contents of this queue. + * + * If you need entries in sorted order, you should use {@link #pop()} instead. + * + * @return an iterator over this queue's entries, not necessarily in sorted order + */ + @Override + public Iterator iterator() { + return new Iterator() { + int offset = 1; + + @Override + public boolean hasNext() { + return offset <= size; + } + + @Override + public T next() { + return heap[offset++]; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java index 47abea2..b5371f8 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java @@ -34,6 +34,7 @@ import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermStatistics; import org.apache.lucene.search.similarities.TFIDFSimilarity; @@ -414,7 +415,7 @@ public class TestOmitTf extends LuceneTestCase { dir.close(); } - public static class CountingHitCollector extends Collector { + public static class CountingHitCollector extends SerialCollector { static int count=0; static int sum=0; private int docBase = -1; diff --git a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java index b4bf0a4..10e8636 100644 --- a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java +++ b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java @@ -39,30 +39,49 @@ final class JustCompileSearch { private static final String UNSUPPORTED_MSG = "unsupported: used for back-compat testing only !"; - static final class JustCompileCollector extends Collector { + static final class JustCompileSubCollector implements SubCollector { @Override - public void collect(int doc) { + public void setScorer(Scorer scorer) throws IOException { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @Override - public void setNextReader(AtomicReaderContext context) { + public void collect(int doc) throws IOException { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @Override - public void setScorer(Scorer scorer) { + public void done() throws IOException { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } - + @Override public boolean acceptsDocsOutOfOrder() { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } } - + + static final class JustCompileCollector implements Collector { + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED_MSG); + } + + @Override + public void setParallelized() { + throw new UnsupportedOperationException(UNSUPPORTED_MSG); + } + + @Override + public boolean isParallelizable() { + throw new UnsupportedOperationException(UNSUPPORTED_MSG); + } + + } + static final class JustCompileDocIdSet extends DocIdSet { @Override @@ -227,7 +246,7 @@ final class JustCompileSearch { } @Override - public boolean score(Collector collector, int max, int firstDocID) { + public boolean score(SubCollector collector, int max, int firstDocID) { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @@ -287,40 +306,19 @@ final class JustCompileSearch { } @Override - public void collect(int doc) { + public SubCollector subCollector(AtomicReaderContext context) throws IOException { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @Override - public void setNextReader(AtomicReaderContext context) { + public void setParallelized() { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @Override - public void setScorer(Scorer scorer) { + public boolean isParallelizable() { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } - - @Override - public boolean acceptsDocsOutOfOrder() { - throw new UnsupportedOperationException(UNSUPPORTED_MSG); - } - - @Override - public TopDocs topDocs() { - throw new UnsupportedOperationException( UNSUPPORTED_MSG ); - } - - @Override - public TopDocs topDocs( int start ) { - throw new UnsupportedOperationException( UNSUPPORTED_MSG ); - } - - @Override - public TopDocs topDocs( int start, int end ) { - throw new UnsupportedOperationException( UNSUPPORTED_MSG ); - } - } static final class JustCompileWeight extends Weight { diff --git a/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java b/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java index 2a63f49..e0d9f51 100644 --- a/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java +++ b/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java @@ -27,7 +27,7 @@ import org.junit.Test; public class MultiCollectorTest extends LuceneTestCase { - private static class DummyCollector extends Collector { + private static class DummyCollector extends SerialCollector { boolean acceptsDocsOutOfOrderCalled = false; boolean collectCalled = false; @@ -71,10 +71,10 @@ public class MultiCollectorTest extends LuceneTestCase { // doesn't, an NPE would be thrown. Collector c = MultiCollector.wrap(new DummyCollector(), null, new DummyCollector()); assertTrue(c instanceof MultiCollector); - assertTrue(c.acceptsDocsOutOfOrder()); - c.collect(1); - c.setNextReader(null); - c.setScorer(null); + final SubCollector subCollector = c.subCollector(null); + assertTrue(subCollector.acceptsDocsOutOfOrder()); + subCollector.collect(1); + subCollector.setScorer(null); } @Test @@ -93,10 +93,10 @@ public class MultiCollectorTest extends LuceneTestCase { // doesn't, an NPE would be thrown. DummyCollector[] dcs = new DummyCollector[] { new DummyCollector(), new DummyCollector() }; Collector c = MultiCollector.wrap(dcs); - assertTrue(c.acceptsDocsOutOfOrder()); - c.collect(1); - c.setNextReader(null); - c.setScorer(null); + final SubCollector subCollector = c.subCollector(null); + assertTrue(subCollector.acceptsDocsOutOfOrder()); + subCollector.collect(1); + subCollector.setScorer(null); for (DummyCollector dc : dcs) { assertTrue(dc.acceptsDocsOutOfOrderCalled); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java index a25121b..7a6ef27 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java @@ -188,26 +188,27 @@ public class TestBooleanOr extends LuceneTestCase { final FixedBitSet hits = new FixedBitSet(docCount); final AtomicInteger end = new AtomicInteger(); - Collector c = new Collector() { - @Override - public void setNextReader(AtomicReaderContext sub) { - } - - @Override - public void collect(int doc) { - assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue()); - hits.set(doc); - } - - @Override - public void setScorer(Scorer scorer) { - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return true; - } - }; + SubCollector c = new SubCollector() { + + @Override + public void collect(int doc) { + assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue()); + hits.set(doc); + } + + @Override + public void done() throws IOException { + } + + @Override + public void setScorer(Scorer scorer) { + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + }; while (end.intValue() < docCount) { final int inc = _TestUtil.nextInt(random(), 1, 1000); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java index 744ab68..837b80b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java @@ -121,48 +121,15 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase { return collector.docCounts; } - static class MyCollector extends Collector { + static class MyCollector extends WrappingCollector { - private TopDocsCollector collector; - private int docBase; - public final Map docCounts = new HashMap(); private final Set tqsSet = new HashSet(); MyCollector() { - collector = TopScoreDocCollector.create(10, true); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return false; + super(TopScoreDocCollector.create(10, true)); } - @Override - public void collect(int doc) throws IOException { - int freq = 0; - for(Scorer scorer : tqsSet) { - if (doc == scorer.docID()) { - freq += scorer.freq(); - } - } - docCounts.put(doc + docBase, freq); - collector.collect(doc); - } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.docBase = context.docBase; - collector.setNextReader(context); - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - collector.setScorer(scorer); - tqsSet.clear(); - fillLeaves(scorer, tqsSet); - } - private void fillLeaves(Scorer scorer, Set set) { if (scorer.getWeight().getQuery() instanceof TermQuery) { set.add(scorer); @@ -174,11 +141,48 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase { } public TopDocs topDocs(){ - return collector.topDocs(); + return ((TopScoreDocCollector) super.delegate).topDocs(); } public int freq(int doc) throws IOException { return docCounts.get(doc); } + + @Override + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final int docBase = context.docBase; + return new WrappingSubCollector(delegate.subCollector(context)) { + + @Override + public void setScorer(Scorer scorer) throws IOException { + delegate.setScorer(scorer); + tqsSet.clear(); + fillLeaves(scorer, tqsSet); + } + + @Override + public void collect(int doc) throws IOException { + int freq = 0; + for(Scorer scorer : tqsSet) { + if (doc == scorer.docID()) { + freq += scorer.freq(); + } + } + docCounts.put(doc + docBase, freq); + delegate.collect(doc); + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + + }; + } + + @Override + public boolean isParallelizable() { + return false; + } } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java index ffeac52..56a2264 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java @@ -101,7 +101,7 @@ public class TestBooleanScorer extends LuceneTestCase BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), null, scorers.length); final List hits = new ArrayList(); - bs.score(new Collector() { + bs.score(new SerialCollector() { int docBase; @Override public void setScorer(Scorer scorer) { @@ -151,7 +151,7 @@ public class TestBooleanScorer extends LuceneTestCase BooleanClause.Occur.SHOULD)); final int[] count = new int[1]; - s.search(q, new Collector() { + s.search(q, new SerialCollector() { @Override public void setScorer(Scorer scorer) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java index f493d42..4761eec 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java @@ -53,7 +53,7 @@ public class TestCachingCollector extends LuceneTestCase { } } - private static class NoOpCollector extends Collector { + private static class NoOpCollector extends SerialCollector { private final boolean acceptDocsOutOfOrder; @@ -80,22 +80,27 @@ public class TestCachingCollector extends LuceneTestCase { public void testBasic() throws Exception { for (boolean cacheScores : new boolean[] { false, true }) { CachingCollector cc = CachingCollector.create(new NoOpCollector(false), cacheScores, 1.0); - cc.setScorer(new MockScorer()); + final SubCollector sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); // collect 1000 docs for (int i = 0; i < 1000; i++) { - cc.collect(i); + sc.collect(i); } + sc.done(); + // now replay them - cc.replay(new Collector() { + cc.replay(new SerialCollector() { int prevDocID = -1; @Override - public void setScorer(Scorer scorer) {} + public void setScorer(Scorer scorer) { + } @Override - public void setNextReader(AtomicReaderContext context) {} + public void setNextReader(AtomicReaderContext context) { + } @Override public void collect(int doc) { @@ -113,13 +118,16 @@ public class TestCachingCollector extends LuceneTestCase { public void testIllegalStateOnReplay() throws Exception { CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 50 * ONE_BYTE); - cc.setScorer(new MockScorer()); + final SubCollector sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); // collect 130 docs, this should be enough for triggering cache abort. for (int i = 0; i < 130; i++) { - cc.collect(i); + sc.collect(i); } - + + sc.done(); + assertFalse("CachingCollector should not be cached due to low memory limit", cc.isCached()); try { @@ -136,15 +144,21 @@ public class TestCachingCollector extends LuceneTestCase { // 'src' Collector does not support out-of-order CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 50 * ONE_BYTE); - cc.setScorer(new MockScorer()); + SubCollector sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); for (int i = 0; i < 10; i++) cc.collect(i); + sc.done(); + cc.replay(new NoOpCollector(true)); // this call should not fail cc.replay(new NoOpCollector(false)); // this call should not fail // 'src' Collector supports out-of-order cc = CachingCollector.create(new NoOpCollector(true), true, 50 * ONE_BYTE); - cc.setScorer(new MockScorer()); + sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); for (int i = 0; i < 10; i++) cc.collect(i); + sc.done(); + cc.replay(new NoOpCollector(true)); // this call should not fail try { cc.replay(new NoOpCollector(false)); // this call should fail @@ -165,13 +179,17 @@ public class TestCachingCollector extends LuceneTestCase { int bytesPerDoc = cacheScores ? 8 : 4; CachingCollector cc = CachingCollector.create(new NoOpCollector(false), cacheScores, bytesPerDoc * ONE_BYTE * numDocs); - cc.setScorer(new MockScorer()); - for (int i = 0; i < numDocs; i++) cc.collect(i); + final SubCollector sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); + for (int i = 0; i < numDocs; i++) sc.collect(i); + assertTrue(cc.isCached()); // The 151's document should terminate caching - cc.collect(numDocs); + sc.collect(numDocs); assertFalse(cc.isCached()); + + sc.done(); } } @@ -179,10 +197,11 @@ public class TestCachingCollector extends LuceneTestCase { for (boolean cacheScores : new boolean[] { false, true }) { // create w/ null wrapped collector, and test that the methods work CachingCollector cc = CachingCollector.create(true, cacheScores, 50 * ONE_BYTE); - cc.setNextReader(null); - cc.setScorer(new MockScorer()); - cc.collect(0); - + final SubCollector sc = cc.subCollector(null); + sc.setScorer(new MockScorer()); + sc.collect(0); + sc.done(); + assertTrue(cc.isCached()); cc.replay(new NoOpCollector(true)); } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java index 1fa8526..293ed17 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java @@ -50,7 +50,7 @@ public class TestConstantScoreQuery extends LuceneTestCase { private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final String scorerClassName, final String innerScorerClassName) throws IOException { final int[] count = new int[1]; - searcher.search(q, new Collector() { + searcher.search(q, new SerialCollector() { private Scorer scorer; @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java b/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java index 3cf2e85..611a7b2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java @@ -59,7 +59,7 @@ public class TestDocBoost extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader); searcher.search (new TermQuery(new Term("field", "word")), - new Collector() { + new SerialCollector() { private int base = 0; private Scorer scorer; @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java index acbdaf7..b96ae41 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java @@ -58,7 +58,7 @@ public class TestEarlyTermination extends LuceneTestCase { for (int i = 0; i < iters; ++i) { final IndexSearcher searcher = newSearcher(reader); - final Collector collector = new Collector() { + final Collector collector = new SerialCollector() { final boolean outOfOrder = random().nextBoolean(); boolean collectionTerminated = true; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java b/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java index d286d54..dca8288 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java @@ -226,7 +226,7 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter { search.setSimilarity(new DefaultSimilarity()); Query q = csrq("data", "1", "6", T, T); q.setBoost(100); - search.search(q, null, new Collector() { + search.search(q, null, new SerialCollector() { private int base = 0; private Scorer scorer; @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java index 1f52b21..4511345 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java @@ -17,6 +17,7 @@ package org.apache.lucene.search; * limitations under the License. */ +import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; @@ -78,6 +79,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase { Directory directory = newDirectory(); RandomIndexWriter writer = new RandomIndexWriter(random(), directory); + writer.addDocument(new Document()); writer.commit(); IndexReader ir = writer.getReader(); writer.close(); @@ -86,10 +88,12 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase { Scorer s = new SimpleScorer(fake); TopDocsCollector tdc = TopScoreDocCollector.create(scores.length, true); Collector c = new PositiveScoresOnlyCollector(tdc); - c.setScorer(s); + final SubCollector sub = c.subCollector(searcher.leafContexts.get(0)); + sub.setScorer(s); while (s.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { - c.collect(0); + sub.collect(0); } + sub.done(); TopDocs td = tdc.topDocs(); ScoreDoc[] sd = td.scoreDocs; assertEquals(numPositiveScores, td.totalHits); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java index 9c6f486..abe6eef 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java @@ -65,7 +65,7 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase { } } - private static final class ScoreCachingCollector extends Collector { + private static final class ScoreCachingCollector extends SerialCollector { private int idx = 0; private Scorer scorer; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java index 48cbaee..8da5454 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java @@ -97,7 +97,7 @@ public class TestScorerPerf extends LuceneTestCase { return sets; } - public static class CountingHitCollector extends Collector { + public static class CountingHitCollector extends SerialCollector { int count=0; int sum=0; protected int docBase = 0; @@ -133,7 +133,7 @@ public class TestScorerPerf extends LuceneTestCase { } public void collect(int doc, float score) { - + pos = answer.nextSetBit(pos+1); if (pos != doc + docBase) { throw new RuntimeException("Expected doc " + pos + " but got " + doc + docBase); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java index 31ebc14..7086229 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java @@ -76,7 +76,7 @@ public class TestSimilarity extends LuceneTestCase { Term b = new Term("field", "b"); Term c = new Term("field", "c"); - searcher.search(new TermQuery(b), new Collector() { + searcher.search(new TermQuery(b), new SerialCollector() { private Scorer scorer; @Override public void setScorer(Scorer scorer) { @@ -98,7 +98,7 @@ public class TestSimilarity extends LuceneTestCase { bq.add(new TermQuery(a), BooleanClause.Occur.SHOULD); bq.add(new TermQuery(b), BooleanClause.Occur.SHOULD); //System.out.println(bq.toString("field")); - searcher.search(bq, new Collector() { + searcher.search(bq, new SerialCollector() { private int base = 0; private Scorer scorer; @Override @@ -125,7 +125,7 @@ public class TestSimilarity extends LuceneTestCase { pq.add(c); //System.out.println(pq.toString("field")); searcher.search(pq, - new Collector() { + new SerialCollector() { private Scorer scorer; @Override public void setScorer(Scorer scorer) { @@ -146,7 +146,7 @@ public class TestSimilarity extends LuceneTestCase { pq.setSlop(2); //System.out.println(pq.toString("field")); - searcher.search(pq, new Collector() { + searcher.search(pq, new SerialCollector() { private Scorer scorer; @Override public void setScorer(Scorer scorer) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java index fdde532..3f6dfae 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java @@ -175,7 +175,7 @@ public class TestSloppyPhraseQuery extends LuceneTestCase { return query; } - static class MaxFreqCollector extends Collector { + static class MaxFreqCollector extends SerialCollector { float max; int totalHits; Scorer scorer; @@ -203,7 +203,7 @@ public class TestSloppyPhraseQuery extends LuceneTestCase { /** checks that no scores or freqs are infinite */ private void assertSaneScoring(PhraseQuery pq, IndexSearcher searcher) throws Exception { - searcher.search(pq, new Collector() { + searcher.search(pq, new SerialCollector() { Scorer scorer; @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java index a51f118..576f44d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java @@ -65,13 +65,10 @@ public class TestSubScorerFreqs extends LuceneTestCase { dir = null; } - private static class CountingCollector extends Collector { - private final Collector other; - private int docBase; + private static class CountingCollector extends WrappingCollector { public final Map> docCounts = new HashMap>(); - private final Map subScorers = new HashMap(); private final Set relationships; public CountingCollector(Collector other) { @@ -79,48 +76,50 @@ public class TestSubScorerFreqs extends LuceneTestCase { } public CountingCollector(Collector other, Set relationships) { - this.other = other; + super(other); this.relationships = relationships; } @Override - public void setScorer(Scorer scorer) throws IOException { - other.setScorer(scorer); - subScorers.clear(); - setSubScorers(scorer, "TOP"); - } - - public void setSubScorers(Scorer scorer, String relationship) { - for (ChildScorer child : scorer.getChildren()) { - if (relationships.contains(child.relationship)) { - setSubScorers(child.child, child.relationship); + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final int docBase = context.docBase; + return new WrappingSubCollector(delegate.subCollector(context)) { + + private final Map subScorers = new HashMap(); + + private void setSubScorers(Scorer scorer) { + for (ChildScorer child : scorer.getChildren()) { + if (relationships.contains(child.relationship)) { + setSubScorers(child.child); + } + } + subScorers.put(scorer.getWeight().getQuery(), scorer); } - } - subScorers.put(scorer.getWeight().getQuery(), scorer); - } - @Override - public void collect(int doc) throws IOException { - final Map freqs = new HashMap(); - for (Map.Entry ent : subScorers.entrySet()) { - Scorer value = ent.getValue(); - int matchId = value.docID(); - freqs.put(ent.getKey(), matchId == doc ? value.freq() : 0.0f); - } - docCounts.put(doc + docBase, freqs); - other.collect(doc); - } + @Override + public void setScorer(Scorer scorer) throws IOException { + delegate.setScorer(scorer); + setSubScorers(scorer); + } - @Override - public void setNextReader(AtomicReaderContext context) - throws IOException { - docBase = context.docBase; - other.setNextReader(context); + @Override + public void collect(int doc) throws IOException { + final Map freqs = new HashMap(); + for (Map.Entry ent : subScorers.entrySet()) { + Scorer value = ent.getValue(); + int matchId = value.docID(); + freqs.put(ent.getKey(), matchId == doc ? value.freq() : 0.0f); + } + docCounts.put(doc + docBase, freqs); + delegate.collect(doc); + } + + }; } @Override - public boolean acceptsDocsOutOfOrder() { - return other.acceptsDocsOutOfOrder(); + public boolean isParallelizable() { + return false; } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java index 0c4e229..5885286 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java @@ -84,7 +84,7 @@ public class TestTermScorer extends LuceneTestCase { final List docs = new ArrayList(); // must call next first - ts.score(new Collector() { + ts.score(new SerialCollector() { private int base = 0; private Scorer scorer; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java index 537e30f..27f1492 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java @@ -307,7 +307,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase { } // counting collector that can slow down at collect(). - private class MyHitCollector extends Collector { + private class MyHitCollector extends SerialCollector { private final BitSet bits = new BitSet(); private int slowdown = 0; private int lastDocCollected = -1; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java index f8f728f..934fdc1 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java @@ -28,7 +28,7 @@ import org.apache.lucene.util.LuceneTestCase; public class TestTopDocsCollector extends LuceneTestCase { - private static final class MyTopsDocCollector extends TopDocsCollector { + private static final class MyTopsDocCollector extends TopDocsSerialCollector { private int idx = 0; private int base = 0; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java index 0492c2d..32efa15 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java @@ -53,13 +53,16 @@ public class TestTopFieldCollector extends LuceneTestCase { } public void testSortWithoutFillFields() throws Exception { - + // There was previously a bug in TopFieldCollector when fillFields was set // to false - the same doc and score was set in ScoreDoc[] array. This test // asserts that if fillFields is false, the documents are set properly. It // does not use Searcher's default search methods (with Sort) since all set // fillFields to true. - Sort[] sort = new Sort[] { new Sort(SortField.FIELD_DOC), new Sort() }; + Sort[] sort = new Sort[] { + new Sort(SortField.FIELD_DOC), + new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) + }; for(int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, false, @@ -76,70 +79,55 @@ public class TestTopFieldCollector extends LuceneTestCase { } public void testSortWithoutScoreTracking() throws Exception { - - // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; - for(int i = 0; i < sort.length; i++) { - Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, false, - false, true); - - is.search(q, tdc); - - TopDocs td = tdc.topDocs(); - ScoreDoc[] sd = td.scoreDocs; - for(int j = 0; j < sd.length; j++) { - assertTrue(Float.isNaN(sd[j].score)); - } - assertTrue(Float.isNaN(td.getMaxScore())); + Query q = new MatchAllDocsQuery(); + TopDocsCollector tdc = TopFieldCollector.create(new Sort(SortField.FIELD_DOC), 10, true, false, false, true); + is.search(q, tdc); + TopDocs td = tdc.topDocs(); + ScoreDoc[] sd = td.scoreDocs; + for(int j = 0; j < sd.length; j++) { + assertTrue(Float.isNaN(sd[j].score)); } + assertTrue(Float.isNaN(td.getMaxScore())); } public void testSortWithScoreNoMaxScoreTracking() throws Exception { - - // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; - for(int i = 0; i < sort.length; i++) { - Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, - false, true); - - is.search(q, tdc); - - TopDocs td = tdc.topDocs(); - ScoreDoc[] sd = td.scoreDocs; - for(int j = 0; j < sd.length; j++) { - assertTrue(!Float.isNaN(sd[j].score)); - } - assertTrue(Float.isNaN(td.getMaxScore())); + Sort singleFieldSort = new Sort(SortField.FIELD_DOC); + Query q = new MatchAllDocsQuery(); + TopDocsCollector tdc = TopFieldCollector.create(singleFieldSort, 10, true, true, false, true); + + is.search(q, tdc); + + TopDocs td = tdc.topDocs(); + ScoreDoc[] sd = td.scoreDocs; + for(int j = 0; j < sd.length; j++) { + assertTrue(!Float.isNaN(sd[j].score)); } + assertTrue(Float.isNaN(td.getMaxScore())); } - // MultiComparatorScoringNoMaxScoreCollector + // MultiComparatorScoringNoMaxScoreSubCollector public void testSortWithScoreNoMaxScoreTrackingMulti() throws Exception { - - // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) }; - for(int i = 0; i < sort.length; i++) { - Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, - false, true); + Sort multiFieldSort = new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE); + Query q = new MatchAllDocsQuery(); + TopDocsCollector tdc = TopFieldCollector.create(multiFieldSort, 10, true, true, false, true); - is.search(q, tdc); - - TopDocs td = tdc.topDocs(); - ScoreDoc[] sd = td.scoreDocs; - for(int j = 0; j < sd.length; j++) { - assertTrue(!Float.isNaN(sd[j].score)); - } - assertTrue(Float.isNaN(td.getMaxScore())); + is.search(q, tdc); + + TopDocs td = tdc.topDocs(); + ScoreDoc[] sd = td.scoreDocs; + for(int j = 0; j < sd.length; j++) { + assertTrue(!Float.isNaN(sd[j].score)); } + assertTrue(Float.isNaN(td.getMaxScore())); } - + public void testSortWithScoreAndMaxScoreTracking() throws Exception { - + // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; + Sort[] sort = new Sort[] { + new Sort(SortField.FIELD_DOC), + new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) + }; for(int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, @@ -158,8 +146,6 @@ public class TestTopFieldCollector extends LuceneTestCase { public void testOutOfOrderDocsScoringSort() throws Exception { - // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; boolean[][] tfcOptions = new boolean[][] { new boolean[] { false, false, false }, new boolean[] { false, false, true }, @@ -171,14 +157,14 @@ public class TestTopFieldCollector extends LuceneTestCase { new boolean[] { true, true, true }, }; String[] actualTFCClasses = new String[] { - "OutOfOrderOneComparatorNonScoringCollector", - "OutOfOrderOneComparatorScoringMaxScoreCollector", - "OutOfOrderOneComparatorScoringNoMaxScoreCollector", - "OutOfOrderOneComparatorScoringMaxScoreCollector", - "OutOfOrderOneComparatorNonScoringCollector", - "OutOfOrderOneComparatorScoringMaxScoreCollector", - "OutOfOrderOneComparatorScoringNoMaxScoreCollector", - "OutOfOrderOneComparatorScoringMaxScoreCollector" + "OutOfOrderOneComparatorNonScoringSubCollector", + "OutOfOrderOneComparatorScoringMaxScoreSubCollector", + "OutOfOrderOneComparatorScoringNoMaxScoreSubCollector", + "OutOfOrderOneComparatorScoringMaxScoreSubCollector", + "OutOfOrderOneComparatorNonScoringSubCollector", + "OutOfOrderOneComparatorScoringMaxScoreSubCollector", + "OutOfOrderOneComparatorScoringNoMaxScoreSubCollector", + "OutOfOrderOneComparatorScoringMaxScoreSubCollector" }; BooleanQuery bq = new BooleanQuery(); @@ -188,27 +174,25 @@ public class TestTopFieldCollector extends LuceneTestCase { // Set minNrShouldMatch to 1 so that BQ will not optimize rewrite to return // the clause instead of BQ. bq.setMinimumNumberShouldMatch(1); - for(int i = 0; i < sort.length; i++) { - for(int j = 0; j < tfcOptions.length; j++) { - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, - tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false); + for(int j = 0; j < tfcOptions.length; j++) { + TopDocsCollector tdc = TopFieldCollector.create(new Sort(SortField.FIELD_DOC), 10, + tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false); - assertTrue(tdc.getClass().getName().endsWith("$"+actualTFCClasses[j])); - - is.search(bq, tdc); - - TopDocs td = tdc.topDocs(); - ScoreDoc[] sd = td.scoreDocs; - assertEquals(10, sd.length); - } + SubCollector sub = tdc.subCollector(ir.leaves().get(0)); + + assertTrue(sub.getClass().getName().endsWith("$" + actualTFCClasses[j])); + + is.search(bq, tdc); + + TopDocs td = tdc.topDocs(); + ScoreDoc[] sd = td.scoreDocs; + assertEquals(10, sd.length); } } - + // OutOfOrderMulti*Collector public void testOutOfOrderDocsScoringSortMulti() throws Exception { - - // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) }; + final Sort multiFieldSort = new Sort(SortField.FIELD_DOC, SortField.FIELD_DOC); boolean[][] tfcOptions = new boolean[][] { new boolean[] { false, false, false }, new boolean[] { false, false, true }, @@ -220,14 +204,14 @@ public class TestTopFieldCollector extends LuceneTestCase { new boolean[] { true, true, true }, }; String[] actualTFCClasses = new String[] { - "OutOfOrderMultiComparatorNonScoringCollector", - "OutOfOrderMultiComparatorScoringMaxScoreCollector", - "OutOfOrderMultiComparatorScoringNoMaxScoreCollector", - "OutOfOrderMultiComparatorScoringMaxScoreCollector", - "OutOfOrderMultiComparatorNonScoringCollector", - "OutOfOrderMultiComparatorScoringMaxScoreCollector", - "OutOfOrderMultiComparatorScoringNoMaxScoreCollector", - "OutOfOrderMultiComparatorScoringMaxScoreCollector" + "OutOfOrderMultiComparatorNonScoringSubCollector", + "OutOfOrderMultiComparatorScoringMaxScoreSubCollector", + "OutOfOrderMultiComparatorScoringNoMaxScoreSubCollector", + "OutOfOrderMultiComparatorScoringMaxScoreSubCollector", + "OutOfOrderMultiComparatorNonScoringSubCollector", + "OutOfOrderMultiComparatorScoringMaxScoreSubCollector", + "OutOfOrderMultiComparatorScoringNoMaxScoreSubCollector", + "OutOfOrderMultiComparatorScoringMaxScoreSubCollector" }; BooleanQuery bq = new BooleanQuery(); @@ -237,26 +221,29 @@ public class TestTopFieldCollector extends LuceneTestCase { // Set minNrShouldMatch to 1 so that BQ will not optimize rewrite to return // the clause instead of BQ. bq.setMinimumNumberShouldMatch(1); - for(int i = 0; i < sort.length; i++) { - for(int j = 0; j < tfcOptions.length; j++) { - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, - tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false); + for(int j = 0; j < tfcOptions.length; j++) { + TopDocsCollector tdc = TopFieldCollector.create(multiFieldSort, 10, + tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false); - assertTrue(tdc.getClass().getName().endsWith("$"+actualTFCClasses[j])); - - is.search(bq, tdc); - - TopDocs td = tdc.topDocs(); - ScoreDoc[] sd = td.scoreDocs; - assertEquals(10, sd.length); - } + SubCollector sub = tdc.subCollector(ir.leaves().get(0)); + + assertTrue(sub.getClass().getName().endsWith("$" + actualTFCClasses[j])); + + is.search(bq, tdc); + + TopDocs td = tdc.topDocs(); + ScoreDoc[] sd = td.scoreDocs; + assertEquals(10, sd.length); } } public void testSortWithScoreAndMaxScoreTrackingNoResults() throws Exception { - + // Two Sort criteria to instantiate the multi/single comparators. - Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; + Sort[] sort = new Sort[] { + new Sort(SortField.FIELD_DOC), + new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) + }; for(int i = 0; i < sort.length; i++) { TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true); TopDocs td = tdc.topDocs(); diff --git a/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java index b2654ce..19f83bd 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java @@ -30,6 +30,7 @@ import org.apache.lucene.facet.params.FacetSearchParams; import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields; import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState; import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; @@ -44,6 +45,7 @@ import org.apache.lucene.search.MultiCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldCollector; @@ -166,7 +168,8 @@ public class DrillSideways { */ @SuppressWarnings({"rawtypes","unchecked"}) public DrillSidewaysResult search(DrillDownQuery query, - Collector hitCollector, FacetSearchParams fsp) throws IOException { + final Collector hitCollector, + FacetSearchParams fsp) throws IOException { if (query.fip != fsp.indexingParams) { throw new IllegalArgumentException("DrillDownQuery's FacetIndexingParams should match FacetSearchParams'"); @@ -213,9 +216,9 @@ public class DrillSideways { startClause = 1; } - FacetsCollector drillDownCollector = fsp2 == null ? null : FacetsCollector.create(getDrillDownAccumulator(fsp2)); + final FacetsCollector drillDownCollector = fsp2 == null ? null : FacetsCollector.create(getDrillDownAccumulator(fsp2)); - FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[drillDownDims.size()]; + final FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[drillDownDims.size()]; int idx = 0; for(String dim : drillDownDims.keySet()) { @@ -279,7 +282,44 @@ public class DrillSideways { collectorMethod(query, baseQuery, startClause, hitCollector, drillDownCollector, drillSidewaysCollectors); } else { DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms); - searcher.search(dsq, hitCollector); + searcher.search(dsq, new Collector() { + + // The collector provided to IndexSearcher must account for the down & sideways collectors, since + // DrillSidewaysScorer internally manages their sub-collectors. + // in case of collectorMethod(), DrillSidewaysCollector is used which takes care this in one place (much nicer) + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + return hitCollector.subCollector(context); + }; + + @Override + public void setParallelized() { + hitCollector.setParallelized(); + if (drillDownCollector != null) { + drillDownCollector.setParallelized(); + } + for (Collector c: drillSidewaysCollectors) { + c.setParallelized(); + } + } + + @Override + public boolean isParallelizable() { + if (!hitCollector.isParallelizable()) { + return false; + } + if (drillDownCollector != null && !drillDownCollector.isParallelizable()) { + return false; + } + for (Collector c: drillSidewaysCollectors) { + if (!c.isParallelizable()) { + return false; + } + } + return true; + } + }); } int numDims = drillDownDims.size(); @@ -482,7 +522,7 @@ public class DrillSideways { * default is false. Note that if you return true from * this method (in a subclass) be sure your collector * also returns false from {@link - * Collector#acceptsDocsOutOfOrder}: this will trick + * SubCollector#acceptsDocsOutOfOrder}: this will trick * BooleanQuery into also scoring all subDocs at once. */ protected boolean scoreSubDocsAtOnce() { return false; diff --git a/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java index 01dded7..9377916 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java @@ -18,7 +18,6 @@ package org.apache.lucene.facet.search; */ import java.io.IOException; -import java.util.Arrays; import java.util.IdentityHashMap; import java.util.Map; @@ -26,18 +25,19 @@ import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer.ChildScorer; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.search.Weight; /** Collector that scrutinizes each hit to determine if it * passed all constraints (a true hit) or if it missed * exactly one dimension (a near-miss, to count for * drill-sideways counts on that dimension). */ -class DrillSidewaysCollector extends Collector { +class DrillSidewaysCollector implements Collector { private final Collector hitCollector; private final Collector drillDownCollector; private final Collector[] drillSidewaysCollectors; - private final Scorer[] subScorers; + private final int dimsSize; private final int exactCount; // Maps Weight to either -1 (mainQuery) or to integer @@ -46,143 +46,192 @@ class DrillSidewaysCollector extends Collector { // right scorers: private final Map weightToIndex = new IdentityHashMap(); - private Scorer mainScorer; - - public DrillSidewaysCollector(Collector hitCollector, Collector drillDownCollector, Collector[] drillSidewaysCollectors, - Map dims) { + public DrillSidewaysCollector(Collector hitCollector, + Collector drillDownCollector, + Collector[] drillSidewaysCollectors, + Map dims) { this.hitCollector = hitCollector; this.drillDownCollector = drillDownCollector; this.drillSidewaysCollectors = drillSidewaysCollectors; - subScorers = new Scorer[dims.size()]; - - if (dims.size() == 1) { + this.dimsSize = dims.size(); + if (dimsSize == 1) { // When we have only one dim, we insert the // MatchAllDocsQuery, bringing the clause count to // 2: exactCount = 2; } else { - exactCount = dims.size(); + exactCount = dimsSize; } } - @Override - public void collect(int doc) throws IOException { - //System.out.println("collect doc=" + doc + " main.freq=" + mainScorer.freq() + " main.doc=" + mainScorer.docID() + " exactCount=" + exactCount); - - if (mainScorer == null) { - // This segment did not have any docs with any - // drill-down field & value: - return; + void setWeight(Weight weight, int index) { + assert !weightToIndex.containsKey(weight); + weightToIndex.put(weight, index); + } + + private final class DrillSidewaysSubCollector implements SubCollector { + + private final SubCollector hitSubCollector; + private final SubCollector drillDownSubCollector; + private final SubCollector[] drillSideWaysSubCollectors; + + private Scorer mainScorer; + private Scorer[] subScorers; + + private DrillSidewaysSubCollector(AtomicReaderContext leaf) throws IOException { + hitSubCollector = hitCollector.subCollector(leaf); + drillDownSubCollector = (drillDownCollector != null) ? drillDownCollector.subCollector(leaf) : null; + drillSideWaysSubCollectors = new SubCollector[drillSidewaysCollectors.length]; + for (int i = 0; i < drillSidewaysCollectors.length; i++) { + drillSideWaysSubCollectors[i] = drillSidewaysCollectors[i].subCollector(leaf); + } } - if (mainScorer.freq() == exactCount) { - // All sub-clauses from the drill-down filters - // matched, so this is a "real" hit, so we first - // collect in both the hitCollector and the - // drillDown collector: - //System.out.println(" hit " + drillDownCollector); - hitCollector.collect(doc); - if (drillDownCollector != null) { - drillDownCollector.collect(doc); + private void findScorers(Scorer scorer) { + Integer index = weightToIndex.get(scorer.getWeight()); + if (index != null) { + if (index.intValue() == -1) { + mainScorer = scorer; + } else { + subScorers[index] = scorer; + } } + for (ChildScorer child : scorer.getChildren()) { + findScorers(child.child); + } + } - // Also collect across all drill-sideways counts so - // we "merge in" drill-down counts for this - // dimension. - for(int i=0;i doc: "subDoc=" + subDoc + " doc=" + doc; - drillSidewaysCollectors[i].collect(doc); - assert allMatchesFrom(i+1, doc); - found = true; - break; + + } else { + boolean found = false; + for(int i=0;i doc: "subDoc=" + subDoc + " doc=" + doc; + drillSideWaysSubCollectors[i].collect(doc); + assert allMatchesFrom(i+1, doc); + found = true; + break; + } } + assert found; } - assert found; } - } - // Only used by assert: - private boolean allMatchesFrom(int startFrom, int doc) { - for(int i=startFrom;i 1 && (dims[1].maxCost < baseQueryCost/10)) { //System.out.println("drillDownAdvance"); - doDrillDownAdvanceScoring(collector, docsEnums, sidewaysCollectors); + doDrillDownAdvanceScoring(collector, docsEnums, drillDownSubCollector, drillSidewaysSubCollectors); } else { //System.out.println("union"); - doUnionScoring(collector, docsEnums, sidewaysCollectors); + doUnionScoring(collector, docsEnums, drillDownSubCollector, drillSidewaysSubCollectors); + } + + if (drillDownSubCollector != null) { + drillDownSubCollector.done(); + } + for (SubCollector s: drillSidewaysSubCollectors) { + s.done(); } } /** Used when drill downs are highly constraining vs * baseQuery. */ - private void doDrillDownAdvanceScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException { + private void doDrillDownAdvanceScoring(SubCollector collector, DocsEnum[][] docsEnums, + SubCollector downCollector, + SubCollector[] sidewaysCollectors) throws IOException { final int maxDoc = context.reader().maxDoc(); final int numDims = dims.length; @@ -317,7 +331,7 @@ class DrillSidewaysScorer extends Scorer { // System.out.println(" docID=" + docIDs[slot] + " count=" + counts[slot]); //} if (counts[slot] == 1+numDims) { - collectHit(collector, sidewaysCollectors); + collectHit(collector, downCollector, sidewaysCollectors); } else if (counts[slot] == numDims) { collectNearMiss(sidewaysCollectors, missingDims[slot]); } @@ -334,7 +348,8 @@ class DrillSidewaysScorer extends Scorer { /** Used when base query is highly constraining vs the * drilldowns; in this case we just .next() on base and * .advance() on the dims. */ - private void doBaseAdvanceScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException { + private void doBaseAdvanceScoring(SubCollector collector, DocsEnum[][] docsEnums, + SubCollector downCollector, SubCollector[] sidewaysCollectors) throws IOException { //if (DEBUG) { // System.out.println(" doBaseAdvanceScoring"); //} @@ -380,7 +395,7 @@ class DrillSidewaysScorer extends Scorer { collectScore = baseScorer.score(); if (failedDim == -1) { - collectHit(collector, sidewaysCollectors); + collectHit(collector, downCollector, sidewaysCollectors); } else { collectNearMiss(sidewaysCollectors, failedDim); } @@ -389,14 +404,14 @@ class DrillSidewaysScorer extends Scorer { } } - private void collectHit(Collector collector, Collector[] sidewaysCollectors) throws IOException { + private void collectHit(SubCollector collector, SubCollector downCollector, SubCollector[] sidewaysCollectors) throws IOException { //if (DEBUG) { // System.out.println(" hit"); //} collector.collect(collectDocID); - if (drillDownCollector != null) { - drillDownCollector.collect(collectDocID); + if (downCollector != null) { + downCollector.collect(collectDocID); } // TODO: we could "fix" faceting of the sideways counts @@ -409,14 +424,15 @@ class DrillSidewaysScorer extends Scorer { } } - private void collectNearMiss(Collector[] sidewaysCollectors, int dim) throws IOException { + private void collectNearMiss(SubCollector[] sidewaysCollectors, int dim) throws IOException { //if (DEBUG) { // System.out.println(" missingDim=" + dim); //} sidewaysCollectors[dim].collect(collectDocID); } - private void doUnionScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException { + private void doUnionScoring(SubCollector collector, DocsEnum[][] docsEnums, + SubCollector downCollector, SubCollector[] sidewaysCollectors) throws IOException { //if (DEBUG) { // System.out.println(" doUnionScoring"); //} @@ -583,7 +599,7 @@ class DrillSidewaysScorer extends Scorer { //System.out.println(" collect doc=" + collectDocID + " main.freq=" + (counts[slot]-1) + " main.doc=" + collectDocID + " exactCount=" + numDims); if (counts[slot] == 1+numDims) { //System.out.println(" hit"); - collectHit(collector, sidewaysCollectors); + collectHit(collector, downCollector, sidewaysCollectors); } else if (counts[slot] == numDims) { //System.out.println(" sw"); collectNearMiss(sidewaysCollectors, missingDims[slot]); diff --git a/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java index f27d3e2..81078ce 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java @@ -11,6 +11,8 @@ import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.FixedBitSet; @@ -40,107 +42,90 @@ import org.apache.lucene.util.FixedBitSet; * * @lucene.experimental */ -public abstract class FacetsCollector extends Collector { +public abstract class FacetsCollector implements Collector { private static final class DocsAndScoresCollector extends FacetsCollector { - private AtomicReaderContext context; - private Scorer scorer; - private FixedBitSet bits; - private int totalHits; - private float[] scores; - public DocsAndScoresCollector(FacetsAccumulator accumulator) { super(accumulator); } - - @Override - protected final void finish() { - if (bits != null) { - matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, scores)); - bits = null; - scores = null; - context = null; - } - } - - @Override - public final boolean acceptsDocsOutOfOrder() { - return false; - } @Override - public final void collect(int doc) throws IOException { - bits.set(doc); - if (totalHits >= scores.length) { - float[] newScores = new float[ArrayUtil.oversize(totalHits + 1, 4)]; - System.arraycopy(scores, 0, newScores, 0, totalHits); - scores = newScores; - } - scores[totalHits] = scorer.score(); - totalHits++; + protected SubCollector createSubCollector(final AtomicReaderContext context) throws IOException { + return new SubCollector() { + + final FixedBitSet bits = new FixedBitSet(context.reader().maxDoc()); + int totalHits; + float[] scores = new float[64]; + + Scorer scorer; + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + } + + @Override + public final void collect(int doc) throws IOException { + bits.set(doc); + if (totalHits >= scores.length) { + float[] newScores = new float[ArrayUtil.oversize(totalHits + 1, 4)]; + System.arraycopy(scores, 0, newScores, 0, totalHits); + scores = newScores; + } + scores[totalHits] = scorer.score(); + totalHits++; + } + + @Override + public final boolean acceptsDocsOutOfOrder() { + return false; + } + + @Override + public void done() throws IOException { + matchingDocs.add(new MatchingDocs(context, bits, totalHits, scores)); + } + }; } - - @Override - public final void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; - } - - @Override - protected final void doSetNextReader(AtomicReaderContext context) throws IOException { - if (bits != null) { - matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, scores)); - } - bits = new FixedBitSet(context.reader().maxDoc()); - totalHits = 0; - scores = new float[64]; // some initial size - this.context = context; - } - } private final static class DocsOnlyCollector extends FacetsCollector { - private AtomicReaderContext context; - private FixedBitSet bits; - private int totalHits; - public DocsOnlyCollector(FacetsAccumulator accumulator) { super(accumulator); } - - @Override - protected final void finish() { - if (bits != null) { - matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, null)); - bits = null; - context = null; - } - } - - @Override - public final boolean acceptsDocsOutOfOrder() { - return true; - } @Override - public final void collect(int doc) throws IOException { - totalHits++; - bits.set(doc); - } + protected SubCollector createSubCollector(final AtomicReaderContext context) throws IOException { + return new SubCollector() { - @Override - public final void setScorer(Scorer scorer) throws IOException {} - - @Override - protected final void doSetNextReader(AtomicReaderContext context) throws IOException { - if (bits != null) { - matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, null)); - } - bits = new FixedBitSet(context.reader().maxDoc()); - totalHits = 0; - this.context = context; + final FixedBitSet bits = new FixedBitSet(context.reader().maxDoc()); + int totalHits; + + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public final void collect(int doc) throws IOException { + totalHits++; + bits.set(doc); + } + + @Override + public final boolean acceptsDocsOutOfOrder() { + return true; + } + + @Override + public void done() throws IOException { + matchingDocs.add(new MatchingDocs(context, bits, totalHits, null)); + } + + }; } + } /** @@ -184,21 +169,15 @@ public abstract class FacetsCollector extends Collector { private final FacetsAccumulator accumulator; private List cachedResults; - + protected final List matchingDocs = new ArrayList(); protected FacetsCollector(FacetsAccumulator accumulator) { this.accumulator = accumulator; } - - /** - * Called when the Collector has finished, so that the last - * {@link MatchingDocs} can be added. - */ - protected abstract void finish(); - - /** Performs the actual work of {@link #setNextReader(AtomicReaderContext)}. */ - protected abstract void doSetNextReader(AtomicReaderContext context) throws IOException; + + /** Performs the actual work of {@link #subCollector(AtomicReaderContext)}. */ + protected abstract SubCollector createSubCollector(AtomicReaderContext context) throws IOException; /** * Returns a {@link FacetResult} per {@link FacetRequest} set in @@ -210,10 +189,9 @@ public abstract class FacetsCollector extends Collector { // LUCENE-4893: if results are not cached, counts are multiplied as many // times as this method is called. if (cachedResults == null) { - finish(); cachedResults = accumulator.accumulate(matchingDocs); } - + return cachedResults; } @@ -222,7 +200,6 @@ public abstract class FacetsCollector extends Collector { * visited segment. */ public final List getMatchingDocs() { - finish(); return matchingDocs; } @@ -232,18 +209,27 @@ public abstract class FacetsCollector extends Collector { * results), and does not attempt to reuse allocated memory spaces. */ public final void reset() { - finish(); matchingDocs.clear(); cachedResults = null; } @Override - public final void setNextReader(AtomicReaderContext context) throws IOException { + public final SubCollector subCollector(AtomicReaderContext context) throws IOException { // clear cachedResults - needed in case someone called getFacetResults() // before doing a search and didn't call reset(). Defensive code to prevent // traps. cachedResults = null; - doSetNextReader(context); + return createSubCollector(context); } - + + @Override + public void setParallelized() { + // this collector has the same approach for serial & parallel collection + } + + @Override + public boolean isParallelizable() { + return true; + } + } diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/AssertingSubDocsAtOnceCollector.java b/lucene/facet/src/test/org/apache/lucene/facet/search/AssertingSubDocsAtOnceCollector.java index ba2786d..5452e2b 100644 --- a/lucene/facet/src/test/org/apache/lucene/facet/search/AssertingSubDocsAtOnceCollector.java +++ b/lucene/facet/src/test/org/apache/lucene/facet/search/AssertingSubDocsAtOnceCollector.java @@ -24,10 +24,12 @@ import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer.ChildScorer; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; +import org.apache.lucene.search.SubCollector; /** Verifies in collect() that all child subScorers are on * the collected doc. */ -class AssertingSubDocsAtOnceCollector extends Collector { +class AssertingSubDocsAtOnceCollector extends SerialCollector { // TODO: allow wrapping another Collector diff --git a/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java b/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java index 25b8e4c..dcde7cf 100644 --- a/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java +++ b/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java @@ -60,6 +60,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortField.Type; @@ -746,7 +747,7 @@ public class TestDrillSideways extends FacetTestCase { // had an AssertingScorer it could catch it when // Weight.scoresDocsOutOfOrder lies!: new DrillSideways(s, tr).search(ddq, - new Collector() { + new SerialCollector() { int lastDocID; @Override diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java index 1661461..7b0e6cb 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java @@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Collector; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.util.FixedBitSet; import java.io.IOException; @@ -30,7 +31,7 @@ import java.util.Collection; * @lucene.experimental */ @SuppressWarnings({"unchecked","rawtypes"}) -public abstract class AbstractAllGroupHeadsCollector extends Collector { +public abstract class AbstractAllGroupHeadsCollector extends SerialCollector { protected final int[] reversed; protected final int compIDXEnd; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java index 3cd9164..87eefc4 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java @@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.util.BytesRef; import java.io.IOException; @@ -36,7 +37,7 @@ import java.util.Collection; * * @lucene.experimental */ -public abstract class AbstractAllGroupsCollector extends Collector { +public abstract class AbstractAllGroupsCollector extends SerialCollector { /** * Returns the total number of groups for the executed search. diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java index 07fc35e..0af45d7 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java @@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import java.io.IOException; import java.util.*; @@ -28,7 +29,7 @@ import java.util.*; * * @lucene.experimental */ -public abstract class AbstractDistinctValuesCollector> extends Collector { +public abstract class AbstractDistinctValuesCollector> extends SerialCollector { /** * Returns all unique values for each top N group. diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java index 19b1d36..4f7e7b0 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java @@ -33,7 +33,7 @@ import java.util.*; * * @lucene.experimental */ -abstract public class AbstractFirstPassGroupingCollector extends Collector { +abstract public class AbstractFirstPassGroupingCollector extends SerialCollector { private final Sort groupSort; private final FieldComparator[] comparators; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java index 8db044c..d611ea0 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java @@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.PriorityQueue; @@ -30,7 +31,7 @@ import java.util.*; * * @lucene.experimental */ -public abstract class AbstractGroupFacetCollector extends Collector { +public abstract class AbstractGroupFacetCollector extends SerialCollector { protected final String groupField; protected final String facetField; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java index 7b00012..d9a51db 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java @@ -37,7 +37,7 @@ import java.util.Map; * * @lucene.experimental */ -public abstract class AbstractSecondPassGroupingCollector extends Collector { +public abstract class AbstractSecondPassGroupingCollector implements Collector { protected final Map> groupMap; private final int maxDocsPerGroup; @@ -80,23 +80,6 @@ public abstract class AbstractSecondPassGroupingCollector exte } } - @Override - public void setScorer(Scorer scorer) throws IOException { - for (SearchGroupDocs group : groupMap.values()) { - group.collector.setScorer(scorer); - } - } - - @Override - public void collect(int doc) throws IOException { - totalHitCount++; - SearchGroupDocs group = retrieveGroup(doc); - if (group != null) { - totalGroupedHitCount++; - group.collector.collect(doc); - } - } - /** * Returns the group the specified doc belongs to or null if no group could be retrieved. * @@ -107,15 +90,55 @@ public abstract class AbstractSecondPassGroupingCollector exte protected abstract SearchGroupDocs retrieveGroup(int doc) throws IOException; @Override - public void setNextReader(AtomicReaderContext readerContext) throws IOException { + public SubCollector subCollector(AtomicReaderContext readerContext) throws IOException { //System.out.println("SP.setNextReader"); for (SearchGroupDocs group : groupMap.values()) { - group.collector.setNextReader(readerContext); + group.subCollector = group.collector.subCollector(readerContext); } + + return new SubCollector() { + + @Override + public void setScorer(Scorer scorer) throws IOException { + for (SearchGroupDocs group : groupMap.values()) { + group.subCollector.setScorer(scorer); + } + } + + @Override + public void collect(int doc) throws IOException { + totalHitCount++; + SearchGroupDocs group = retrieveGroup(doc); + if (group != null) { + totalGroupedHitCount++; + group.subCollector.collect(doc); + } + } + + @Override + public void done() throws IOException { + for (SearchGroupDocs group : groupMap.values()) { + group.subCollector.done(); + group.subCollector = null; + } + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + + }; + + } + + @Override + public void setParallelized() { + throw new UnsupportedOperationException(); } @Override - public boolean acceptsDocsOutOfOrder() { + public boolean isParallelizable() { return false; } @@ -152,6 +175,8 @@ public abstract class AbstractSecondPassGroupingCollector exte public final GROUP_VALUE_TYPE groupValue; public final TopDocsCollector collector; + public SubCollector subCollector; + public SearchGroupDocs(GROUP_VALUE_TYPE groupValue, TopDocsCollector collector) { this.groupValue = groupValue; this.collector = collector; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index a39678d..f4aab15 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -54,7 +54,7 @@ import java.io.IOException; * @lucene.experimental */ -public class BlockGroupingCollector extends Collector { +public class BlockGroupingCollector extends SerialCollector { private int[] pendingSubDocs; private float[] pendingSubScores; @@ -338,16 +338,18 @@ public class BlockGroupingCollector extends Collector { collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores, true); } - collector.setScorer(fakeScorer); - collector.setNextReader(og.readerContext); + final SubCollector subCollector = collector.subCollector(og.readerContext); + subCollector.setScorer(fakeScorer); for(int docIDX=0;docIDX joinValueToJoinScores = new HashMap<>(); if (multipleValuesPerDocument) { - fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() { + fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SerialCollector() { private Scorer scorer; private SortedSetDocValues docTermOrds; @@ -508,7 +508,7 @@ public class TestJoinUtil extends LuceneTestCase { } }); } else { - fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() { + fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SerialCollector() { private Scorer scorer; private BinaryDocValues terms; @@ -575,7 +575,7 @@ public class TestJoinUtil extends LuceneTestCase { } } } else { - toSearcher.search(new MatchAllDocsQuery(), new Collector() { + toSearcher.search(new MatchAllDocsQuery(), new SerialCollector() { private SortedSetDocValues docTermOrds; private final BytesRef scratch = new BytesRef(); @@ -613,7 +613,7 @@ public class TestJoinUtil extends LuceneTestCase { }); } } else { - toSearcher.search(new MatchAllDocsQuery(), new Collector() { + toSearcher.search(new MatchAllDocsQuery(), new SerialCollector() { private BinaryDocValues terms; private int docBase; diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index ff2b1c5..63d4e7e 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -54,6 +54,7 @@ import org.apache.lucene.search.Collector; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.RAMDirectory; // for javadocs import org.apache.lucene.util.ArrayUtil; @@ -503,7 +504,7 @@ public class MemoryIndex { IndexSearcher searcher = createSearcher(); try { final float[] scores = new float[1]; // inits to 0.0f (no match) - searcher.search(query, new Collector() { + searcher.search(query, new SerialCollector() { private Scorer scorer; @Override diff --git a/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java b/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java index 92f6f10..611347f 100644 --- a/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java +++ b/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java @@ -23,9 +23,10 @@ import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.search.SubCollector; +import org.apache.lucene.search.TopDocsCollector; // javadoc import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.search.WrappingCollector; /** * A {@link Collector} that early terminates collection of documents on a @@ -61,16 +62,10 @@ import org.apache.lucene.search.TotalHitCountCollector; * * @lucene.experimental */ -public class EarlyTerminatingSortingCollector extends Collector { +public class EarlyTerminatingSortingCollector extends WrappingCollector { - protected final Collector in; protected final Sorter sorter; protected final int numDocsToCollect; - - protected int segmentTotalCollect; - protected boolean segmentSorted; - - private int numCollected; /** * Create a new {@link EarlyTerminatingSortingCollector} instance. @@ -86,38 +81,46 @@ public class EarlyTerminatingSortingCollector extends Collector { * hits. */ public EarlyTerminatingSortingCollector(Collector in, Sorter sorter, int numDocsToCollect) { + super(in); if (numDocsToCollect <= 0) { - throw new IllegalStateException("numDocsToCollect must always be > 0, got " + segmentTotalCollect); + throw new IllegalStateException("numDocsToCollect must always be > 0, got " + numDocsToCollect); } - this.in = in; this.sorter = sorter; this.numDocsToCollect = numDocsToCollect; } @Override - public void setScorer(Scorer scorer) throws IOException { - in.setScorer(scorer); + public EarlyTerminatingSortingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final boolean segmentSorted = SortingMergePolicy.isSorted(context.reader(), sorter); + final int segmentTotalCollect = segmentSorted ? numDocsToCollect : Integer.MAX_VALUE; + return new EarlyTerminatingSortingSubCollector(delegate.subCollector(context), segmentTotalCollect, segmentSorted); } - @Override - public void collect(int doc) throws IOException { - in.collect(doc); - if (++numCollected >= segmentTotalCollect) { - throw new CollectionTerminatedException(); + protected static class EarlyTerminatingSortingSubCollector extends WrappingSubCollector { + + final int segmentTotalCollect; + final boolean segmentSorted; + + int numCollected; + + public EarlyTerminatingSortingSubCollector(SubCollector delegate, int segmentTotalCollect, boolean segmentSorted) { + super(delegate); + this.segmentTotalCollect = segmentTotalCollect; + this.segmentSorted = segmentSorted; } - } - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - in.setNextReader(context); - segmentSorted = SortingMergePolicy.isSorted(context.reader(), sorter); - segmentTotalCollect = segmentSorted ? numDocsToCollect : Integer.MAX_VALUE; - numCollected = 0; - } + @Override + public void collect(int doc) throws IOException { + delegate.collect(doc); + if (++numCollected >= segmentTotalCollect) { + throw new CollectionTerminatedException(); + } + } - @Override - public boolean acceptsDocsOutOfOrder() { - return !segmentSorted && in.acceptsDocsOutOfOrder(); - } + @Override + public boolean acceptsDocsOutOfOrder() { + return !segmentSorted && delegate.acceptsDocsOutOfOrder(); + } + } } diff --git a/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java b/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java index ddbb922..85bf24d 100644 --- a/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java +++ b/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java @@ -38,6 +38,7 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopFieldCollector; import org.apache.lucene.store.Directory; @@ -146,9 +147,10 @@ public class TestEarlyTermination extends LuceneTestCase { searcher.search(query, collector1); searcher.search(query, new EarlyTerminatingSortingCollector(collector2, new NumericDocValuesSorter("ndv2"), numHits) { @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - super.setNextReader(context); - assertFalse("segment should not be recognized as sorted as different sorter was used", segmentSorted); + public EarlyTerminatingSortingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final EarlyTerminatingSortingSubCollector actual = super.subCollector(context); + assertFalse("segment should not be recognized as sorted as different sorter was used", actual.segmentSorted); + return actual; } }); } diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java index 35a402d..79514b6 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java @@ -29,6 +29,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.queryparser.surround.parser.QueryParser; +import org.apache.lucene.search.SerialCollector; import org.junit.Assert; public class BooleanQueryTst { @@ -57,7 +58,7 @@ public class BooleanQueryTst { public void setVerbose(boolean verbose) {this.verbose = verbose;} - class TestCollector extends Collector { // FIXME: use check hits from Lucene tests + class TestCollector extends SerialCollector { // FIXME: use check hits from Lucene tests int totalMatched; boolean[] encountered; private Scorer scorer = null; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java index 7d388aa..92e8244 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java @@ -63,6 +63,7 @@ import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SerialCollector; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs @@ -534,7 +535,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable { } } - private static class FirstNDocsCollector extends Collector { + private static class FirstNDocsCollector extends SerialCollector { private int docBase; private final int[] hits; private int hitCount; diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java index 8ab2926..2998dc6 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java @@ -25,46 +25,41 @@ import org.apache.lucene.index.AtomicReaderContext; /** Wraps another Collector and checks that * acceptsDocsOutOfOrder is respected. */ -public class AssertingCollector extends Collector { +public class AssertingCollector extends WrappingCollector { public static Collector wrap(Random random, Collector other, boolean inOrder) { return other instanceof AssertingCollector ? other : new AssertingCollector(random, other, inOrder); } final Random random; - final Collector in; final boolean inOrder; - int lastCollected; AssertingCollector(Random random, Collector in, boolean inOrder) { + super(in); this.random = random; - this.in = in; this.inOrder = inOrder; - lastCollected = -1; } @Override - public void setScorer(Scorer scorer) throws IOException { - in.setScorer(AssertingScorer.getAssertingScorer(random, scorer)); - } + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + return new WrappingSubCollector(delegate.subCollector(context)) { - @Override - public void collect(int doc) throws IOException { - if (inOrder || !acceptsDocsOutOfOrder()) { - assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc; - } - in.collect(doc); - lastCollected = doc; - } + int lastCollected = -1; - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - lastCollected = -1; - } + @Override + public void setScorer(Scorer scorer) throws IOException { + delegate.setScorer(AssertingScorer.getAssertingScorer(random, scorer)); + } - @Override - public boolean acceptsDocsOutOfOrder() { - return in.acceptsDocsOutOfOrder(); + @Override + public void collect(int doc) throws IOException { + if (inOrder || !acceptsDocsOutOfOrder()) { + assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc; + } + delegate.collect(doc); + lastCollected = doc; + } + }; } } diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java index 635703c..d8492bd 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java @@ -36,8 +36,8 @@ public class AssertingScorer extends Scorer { YES, NO, UNKNOWN; } - private static final VirtualMethod SCORE_COLLECTOR = new VirtualMethod(Scorer.class, "score", Collector.class); - private static final VirtualMethod SCORE_COLLECTOR_RANGE = new VirtualMethod(Scorer.class, "score", Collector.class, int.class, int.class); + private static final VirtualMethod SCORE_COLLECTOR = new VirtualMethod(Scorer.class, "score", SubCollector.class); + private static final VirtualMethod SCORE_COLLECTOR_RANGE = new VirtualMethod(Scorer.class, "score", SubCollector.class, int.class, int.class); // we need to track scorers using a weak hash map because otherwise we // could loose references because of eg. @@ -117,7 +117,7 @@ public class AssertingScorer extends Scorer { } @Override - public void score(Collector collector) throws IOException { + public void score(SubCollector collector) throws IOException { assert topScorer != TopScorer.NO; if (SCORE_COLLECTOR.isOverriddenAsOf(this.in.getClass())) { if (random.nextBoolean()) { @@ -138,7 +138,7 @@ public class AssertingScorer extends Scorer { } @Override - public boolean score(Collector collector, int max, int firstDocID) throws IOException { + public boolean score(SubCollector collector, int max, int firstDocID) throws IOException { assert topScorer != TopScorer.NO; if (SCORE_COLLECTOR_RANGE.isOverriddenAsOf(this.in.getClass())) { return in.score(collector, max, firstDocID); diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java index 730f253..70b3112 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java @@ -123,7 +123,7 @@ public class CheckHits { /** * Just collects document ids into a set. */ - public static class SetCollector extends Collector { + public static class SetCollector extends SerialCollector { final Set bag; public SetCollector(Set bag) { this.bag = bag; @@ -464,7 +464,7 @@ public class CheckHits { * * @see CheckHits#verifyExplanation */ - public static class ExplanationAsserter extends Collector { + public static class ExplanationAsserter extends SerialCollector { Query q; IndexSearcher s; diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java index 621baca..4aef86f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java @@ -249,7 +249,7 @@ public class QueryUtils { final float maxDiff = 1e-5f; final AtomicReader lastReader[] = {null}; - s.search(q, new Collector() { + s.search(q, new SerialCollector() { private Scorer sc; private Scorer scorer; private int leafPtr; @@ -357,7 +357,7 @@ public class QueryUtils { final int lastDoc[] = {-1}; final AtomicReader lastReader[] = {null}; final List context = s.getTopReaderContext().leaves(); - s.search(q,new Collector() { + s.search(q,new SerialCollector() { private Scorer scorer; private int leafPtr; private Bits liveDocs; diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonType.java b/solr/core/src/java/org/apache/solr/schema/LatLonType.java index 70235ac..be726b9 100644 --- a/solr/core/src/java/org/apache/solr/schema/LatLonType.java +++ b/solr/core/src/java/org/apache/solr/schema/LatLonType.java @@ -37,6 +37,7 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.search.Weight; import org.apache.lucene.util.Bits; import org.apache.solr.common.SolrException; @@ -537,24 +538,40 @@ class SpatialDistanceQuery extends ExtendedQueryBase implements PostFilter { class SpatialCollector extends DelegatingCollector { final SpatialWeight weight; - SpatialScorer spatialScorer; - int maxdoc; - + public SpatialCollector(SpatialWeight weight) { this.weight = weight; } @Override - public void collect(int doc) throws IOException { - spatialScorer.doc = doc; - if (spatialScorer.match()) delegate.collect(doc); - } + public SubCollector subCollector(final AtomicReaderContext context) + throws IOException { + final SpatialScorer spatialScorer = new SpatialScorer(context, null, weight, 1.0f); + final SubCollector delegateSub = delegate.subCollector(context); + return new SubCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + delegateSub.setScorer(scorer); + } - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - maxdoc = context.reader().maxDoc(); - spatialScorer = new SpatialScorer(context, null, weight, 1.0f); - super.setNextReader(context); + @Override + public void collect(int doc) throws IOException { + spatialScorer.doc = doc; + if (spatialScorer.match()) { + delegateSub.collect(doc); + } + } + + @Override + public void done() throws IOException { + delegateSub.done(); + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return delegateSub.acceptsDocsOutOfOrder(); + } + }; } } diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java index 22d2f85..17b0f9a 100644 --- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java @@ -392,40 +392,6 @@ public class CollapsingQParserPlugin extends QParserPlugin { } } - public boolean acceptsDocsOutOfOrder() { - //Documents must be sent in order to this collector. - return false; - } - - public void setNextReader(AtomicReaderContext context) throws IOException { - this.contexts[context.ord] = context; - this.docBase = context.docBase; - } - - public void collect(int docId) throws IOException { - int globalDoc = docId+this.docBase; - int ord = values.getOrd(globalDoc); - if(ord > -1) { - float score = scorer.score(); - if(score > scores[ord]) { - ords[ord] = globalDoc; - scores[ord] = score; - } - } else if (this.collapsedSet.fastGet(globalDoc)) { - //The doc is elevated so score does not matter - //We just want to be sure it doesn't fall into the null policy - } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { - float score = scorer.score(); - if(score > nullScore) { - nullScore = score; - nullDoc = globalDoc; - } - } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { - collapsedSet.fastSet(globalDoc); - nullScores.add(scorer.score()); - } - } - public void finish() throws IOException { if(contexts.length == 0) { return; @@ -442,44 +408,95 @@ public class CollapsingQParserPlugin extends QParserPlugin { } } - int currentContext = 0; - int currentDocBase = 0; - int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; - delegate.setNextReader(contexts[currentContext]); - DummyScorer dummy = new DummyScorer(); - delegate.setScorer(dummy); - DocIdSetIterator it = collapsedSet.iterator(); - int docId = -1; + final DocIdSetIterator it = collapsedSet.iterator(); int nullScoreIndex = 0; - while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { - - int ord = values.getOrd(docId); - if(ord > -1) { - dummy.score = scores[ord]; - } else if(this.boostDocs != null && boostDocs.contains(docId)) { - //Elevated docs don't need a score. - dummy.score = 0F; - } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { - dummy.score = nullScore; - } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { - dummy.score = nullScores.get(nullScoreIndex++); - } - - while(docId >= nextDocBase) { - currentContext++; - currentDocBase = contexts[currentContext].docBase; - nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; - delegate.setNextReader(contexts[currentContext]); + for (int currentContext = 0; currentContext < contexts.length; currentContext++) { + final int currentDocBase = contexts[currentContext].docBase; + final int nextDocBase = (currentContext + 1 < contexts.length) + ? contexts[currentContext + 1].docBase + : DocIdSetIterator.NO_MORE_DOCS; + + final DummyScorer dummy = new DummyScorer(); + final SubCollector delegateSub = delegate.subCollector(contexts[currentContext]); + delegateSub.setScorer(dummy); + + int docId; + while ((docId = it.nextDoc()) < nextDocBase) { + int ord = values.getOrd(docId); + if (ord > -1) { + dummy.score = scores[ord]; + } else if(this.boostDocs != null && boostDocs.contains(docId)) { + //Elevated docs don't need a score. + dummy.score = 0F; + } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { + dummy.score = nullScore; + } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { + dummy.score = nullScores.get(nullScoreIndex++); + } + delegateSub.collect(docId - currentDocBase); } - int contextDoc = docId-currentDocBase; - delegate.collect(contextDoc); + delegateSub.done(); } if(delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); } } + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + this.contexts[context.ord] = context; + this.docBase = context.docBase; + + return new SubCollector() { + + Scorer scorer; + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + } + + public void collect(int docId) throws IOException { + int globalDoc = docId+docBase; + int ord = values.getOrd(globalDoc); + if(ord > -1) { + float score = scorer.score(); + if(score > scores[ord]) { + ords[ord] = globalDoc; + scores[ord] = score; + } + } else if (collapsedSet.fastGet(globalDoc)) { + //The doc is elevated so score does not matter + //We just want to be sure it doesn't fall into the null policy + } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { + float score = scorer.score(); + if(score > nullScore) { + nullScore = score; + nullDoc = globalDoc; + } + } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { + collapsedSet.fastSet(globalDoc); + nullScores.add(scorer.score()); + } + } + + @Override + public void done() throws IOException { + } + + public boolean acceptsDocsOutOfOrder() { + //Documents must be sent in order to this collector. + return false; + } + }; + } + + @Override + public boolean isParallelizable() { + return false; + } } private class CollapsingFieldValueCollector extends DelegatingCollector { @@ -523,75 +540,88 @@ public class CollapsingQParserPlugin extends QParserPlugin { } } - public boolean acceptsDocsOutOfOrder() { - //Documents must be sent in order to this collector. - return false; - } - - public void setScorer(Scorer scorer) { - this.fieldValueCollapse.setScorer(scorer); - } - - public void setNextReader(AtomicReaderContext context) throws IOException { - this.contexts[context.ord] = context; - this.docBase = context.docBase; - this.fieldValueCollapse.setNextReader(context); - } - - public void collect(int docId) throws IOException { - int globalDoc = docId+this.docBase; - int ord = values.getOrd(globalDoc); - fieldValueCollapse.collapse(ord, docId, globalDoc); - } - public void finish() throws IOException { if(contexts.length == 0) { return; } - int currentContext = 0; - int currentDocBase = 0; - int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; - delegate.setNextReader(contexts[currentContext]); - DummyScorer dummy = new DummyScorer(); - delegate.setScorer(dummy); - DocIdSetIterator it = fieldValueCollapse.getCollapsedSet().iterator(); - int docId = -1; + final float[] scores = fieldValueCollapse.getScores(); + final FloatArrayList nullScores = fieldValueCollapse.getNullScores(); + final float nullScore = fieldValueCollapse.getNullScore(); + final DocIdSetIterator it = fieldValueCollapse.getCollapsedSet().iterator(); int nullScoreIndex = 0; - float[] scores = fieldValueCollapse.getScores(); - FloatArrayList nullScores = fieldValueCollapse.getNullScores(); - float nullScore = fieldValueCollapse.getNullScore(); - while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { - - if(this.needsScores){ - int ord = values.getOrd(docId); - if(ord > -1) { - dummy.score = scores[ord]; - } else if (boostDocs != null && boostDocs.contains(docId)) { - //Its an elevated doc so no score is needed - dummy.score = 0F; - } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { - dummy.score = nullScore; - } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { - dummy.score = nullScores.get(nullScoreIndex++); + for (int currentContext = 0; currentContext < contexts.length; currentContext++) { + + final int currentDocBase = contexts[currentContext].docBase; + final int nextDocBase = (currentContext + 1 < contexts.length) + ? contexts[currentContext + 1].docBase + : DocIdSetIterator.NO_MORE_DOCS; + + final SubCollector delegateSub = delegate.subCollector(contexts[currentContext]); + final DummyScorer dummy = new DummyScorer(); + delegateSub.setScorer(dummy); + + int docId; + while ((docId = it.nextDoc()) < nextDocBase) { + if(this.needsScores){ + int ord = values.getOrd(docId); + if(ord > -1) { + dummy.score = scores[ord]; + } else if (boostDocs != null && boostDocs.contains(docId)) { + //Its an elevated doc so no score is needed + dummy.score = 0F; + } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) { + dummy.score = nullScore; + } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) { + dummy.score = nullScores.get(nullScoreIndex++); + } } + delegateSub.collect(docId-currentDocBase); } - while(docId >= nextDocBase) { - currentContext++; - currentDocBase = contexts[currentContext].docBase; - nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; - delegate.setNextReader(contexts[currentContext]); - } - - int contextDoc = docId-currentDocBase; - delegate.collect(contextDoc); + delegateSub.done(); } if(delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); } } + + @Override + public SubCollector subCollector(AtomicReaderContext context) throws IOException { + this.contexts[context.ord] = context; + this.docBase = context.docBase; + this.fieldValueCollapse.setNextReader(context); + + return new SubCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + fieldValueCollapse.setScorer(scorer); + } + + @Override + public void collect(int docId) throws IOException { + int globalDoc = docId+docBase; + int ord = values.getOrd(globalDoc); + fieldValueCollapse.collapse(ord, docId, globalDoc); + } + + @Override + public void done() throws IOException { + } + + @Override + public boolean acceptsDocsOutOfOrder() { + //Documents must be sent in order to this collector. + return false; + } + }; + } + + @Override + public boolean isParallelizable() { + return false; + } } private abstract class FieldValueCollapse { diff --git a/solr/core/src/java/org/apache/solr/search/DelegatingCollector.java b/solr/core/src/java/org/apache/solr/search/DelegatingCollector.java index 97045e8..792d84b 100644 --- a/solr/core/src/java/org/apache/solr/search/DelegatingCollector.java +++ b/solr/core/src/java/org/apache/solr/search/DelegatingCollector.java @@ -18,24 +18,19 @@ package org.apache.solr.search; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SubCollector; import java.io.IOException; - /** A simple delegating collector where one can set the delegate after creation */ -public class DelegatingCollector extends Collector { +public abstract class DelegatingCollector implements Collector { /* for internal testing purposes only to determine the number of times a delegating collector chain was used */ public static int setLastDelegateCount; protected Collector delegate; - protected Scorer scorer; - protected AtomicReaderContext context; - protected int docBase; public Collector getDelegate() { return delegate; @@ -54,32 +49,20 @@ public class DelegatingCollector extends Collector { } @Override - public void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; - delegate.setScorer(scorer); - } - - @Override - public void collect(int doc) throws IOException { - delegate.collect(doc); + public void setParallelized() { + delegate.setParallelized(); } @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.context = context; - this.docBase = context.docBase; - delegate.setNextReader(context); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return delegate.acceptsDocsOutOfOrder(); + public boolean isParallelizable() { + return delegate.isParallelizable(); } public void finish() throws IOException { - if(delegate instanceof DelegatingCollector) { + if (delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); } } + } diff --git a/solr/core/src/java/org/apache/solr/search/DocSetCollector.java b/solr/core/src/java/org/apache/solr/search/DocSetCollector.java index 36f7162..78a806c 100644 --- a/solr/core/src/java/org/apache/solr/search/DocSetCollector.java +++ b/solr/core/src/java/org/apache/solr/search/DocSetCollector.java @@ -17,80 +17,173 @@ package org.apache.solr.search; * limitations under the License. */ -import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.Collector; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.util.OpenBitSet; import java.io.IOException; -/** - * - */ +public class DocSetCollector implements Collector { + + private static final SortedIntDocSet NULL_SET = new SortedIntDocSet(new int[0], 0); + + private final int smallSetSize; + private final int maxDoc; + private final Collector delegate; + + private boolean parallelized; -public class DocSetCollector extends Collector { - int pos=0; - OpenBitSet bits; - final int maxDoc; - final int smallSetSize; - int base; + // In case there aren't that many hits, we may not want a very sparse bit array. + // Optimistically collect the first few docs in an array in case there are only a few. + private int[] smallSet; - // in case there aren't that many hits, we may not want a very sparse - // bit array. Optimistically collect the first few docs in an array - // in case there are only a few. - final int[] scratch; + private OpenBitSet bits; + private int size; public DocSetCollector(int smallSetSize, int maxDoc) { + this(smallSetSize, maxDoc, null); + } + + public DocSetCollector(int smallSetSize, int maxDoc, Collector delegate) { this.smallSetSize = smallSetSize; this.maxDoc = maxDoc; - this.scratch = new int[smallSetSize]; + this.delegate = delegate; } - @Override - public void collect(int doc) throws IOException { - doc += base; - // optimistically collect the first docs in an array - // in case the total number will be small enough to represent - // as a small set like SortedIntDocSet instead... - // Storing in this array will be quicker to convert - // than scanning through a potentially huge bit vector. - // FUTURE: when search methods all start returning docs in order, maybe - // we could have a ListDocSet() and use the collected array directly. - if (pos < scratch.length) { - scratch[pos]=doc; - } else { - // this conditional could be removed if BitSet was preallocated, but that - // would take up more memory, and add more GC time... - if (bits==null) bits = new OpenBitSet(maxDoc); - bits.fastSet(doc); + public DocSet getDocSet() { + if (smallSet != null) { + if (size <= smallSet.length) { + return new SortedIntDocSet(smallSet, size); + } + // set bits for docs that are only in the smallSet + for (int i = 0; i < smallSet.length; i++) { + bits.fastSet(smallSet[i]); + } } - - pos++; + if (bits == null) { + return NULL_SET; + } + return new BitDocSet(bits, size); } - public DocSet getDocSet() { - if (pos<=scratch.length) { - // assumes docs were collected in sorted order! - return new SortedIntDocSet(scratch, pos); - } else { - // set the bits for ids that were collected in the array - for (int i=0; i 0 && doc < 64) { + first64 |= 1L << doc; + } else { + bits.fastSet(doc + context.docBase); + } + size++; + } + + @Override + public void done() throws IOException { + super.done(); + for (int doc = 0; first64 != 0; first64 >>>= 1, doc++) { + if ((first64 & 1) == 1) { + bits.fastSet(context.docBase + doc); + } + } + DocSetCollector.this.size += size; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return delegateSub == null || delegateSub.acceptsDocsOutOfOrder(); + } + }; + } + + private SubCollector serialSubCollector(final AtomicReaderContext context, final SubCollector delegateSub) { + if (smallSet == null) { + smallSet = new int[smallSetSize]; + } + + return new DocSetSubCollector(delegateSub) { + + @Override + public void collect(int doc) throws IOException { + super.collect(doc); + if (size < smallSet.length) { + smallSet[size] = doc + context.docBase; + } else { + if (bits == null) { + bits = new OpenBitSet(maxDoc); + } + bits.fastSet(doc + context.docBase); + } + size++; + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; // smallSet needs to be collected in sorted order + } + + }; } @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.base = context.docBase; + public void setParallelized() { + if (delegate != null) { + delegate.setParallelized(); + } + parallelized = true; } @Override - public boolean acceptsDocsOutOfOrder() { - return false; + public boolean isParallelizable() { + return delegate == null || delegate.isParallelizable(); } + } diff --git a/solr/core/src/java/org/apache/solr/search/DocSetDelegateCollector.java b/solr/core/src/java/org/apache/solr/search/DocSetDelegateCollector.java deleted file mode 100644 index aa966ec..0000000 --- a/solr/core/src/java/org/apache/solr/search/DocSetDelegateCollector.java +++ /dev/null @@ -1,85 +0,0 @@ -package org.apache.solr.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 org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.util.OpenBitSet; - -import java.io.IOException; - -/** - * - */ -public class DocSetDelegateCollector extends DocSetCollector { - final Collector collector; - - public DocSetDelegateCollector(int smallSetSize, int maxDoc, Collector collector) { - super(smallSetSize, maxDoc); - this.collector = collector; - } - - @Override - public void collect(int doc) throws IOException { - collector.collect(doc); - - doc += base; - // optimistically collect the first docs in an array - // in case the total number will be small enough to represent - // as a small set like SortedIntDocSet instead... - // Storing in this array will be quicker to convert - // than scanning through a potentially huge bit vector. - // FUTURE: when search methods all start returning docs in order, maybe - // we could have a ListDocSet() and use the collected array directly. - if (pos < scratch.length) { - scratch[pos]=doc; - } else { - // this conditional could be removed if BitSet was preallocated, but that - // would take up more memory, and add more GC time... - if (bits==null) bits = new OpenBitSet(maxDoc); - bits.fastSet(doc); - } - - pos++; - } - - @Override - public DocSet getDocSet() { - if (pos<=scratch.length) { - // assumes docs were collected in sorted order! - return new SortedIntDocSet(scratch, pos); - } else { - // set the bits for ids that were collected in the array - for (int i=0; i * A wrapper {@link Collector} that throws {@link EarlyTerminatingCollectorException}) * once a specified maximum number of documents are collected. *

*/ -public class EarlyTerminatingCollector extends Collector { +public class EarlyTerminatingCollector extends WrappingCollector { private final int maxDocsToCollect; - private final Collector delegate; - private int numCollected = 0; - private int prevReaderCumulativeSize = 0; - private int currentReaderSize = 0; + private final AtomicInteger numCollected = new AtomicInteger(); + private final AtomicInteger prevReaderCumulativeSize = new AtomicInteger(); /** *

@@ -47,40 +47,42 @@ public class EarlyTerminatingCollector extends Collector { * */ public EarlyTerminatingCollector(Collector delegate, int maxDocsToCollect) { + super(delegate); assert 0 < maxDocsToCollect; assert null != delegate; - - this.delegate = delegate; this.maxDocsToCollect = maxDocsToCollect; } - /** - * This collector requires that docs be collected in order, otherwise - * the computed number of scanned docs in the resulting - * {@link EarlyTerminatingCollectorException} will be meaningless. - */ @Override - public boolean acceptsDocsOutOfOrder() { - return false; - } + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final int maxDoc = context.reader().maxDoc(); + return new WrappingSubCollector(delegate.subCollector(context)) { - @Override - public void collect(int doc) throws IOException { - delegate.collect(doc); - numCollected++; - if(maxDocsToCollect <= numCollected) { - throw new EarlyTerminatingCollectorException - (numCollected, prevReaderCumulativeSize + (doc + 1)); - } - } - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - prevReaderCumulativeSize += currentReaderSize; // not current any more - currentReaderSize = context.reader().maxDoc() - 1; - delegate.setNextReader(context); - } - @Override - public void setScorer(Scorer scorer) throws IOException { - delegate.setScorer(scorer); + @Override + public void collect(int doc) throws IOException { + delegate.collect(doc); + if (maxDocsToCollect <= numCollected.incrementAndGet()) { + throw new EarlyTerminatingCollectorException(numCollected.get(), prevReaderCumulativeSize.get() + (doc + 1)); + } + } + + @Override + public void done() throws IOException { + delegate.done(); + prevReaderCumulativeSize.addAndGet(maxDoc - 1); + } + + /** + * This collector requires that docs be collected in order, otherwise + * the computed number of scanned docs in the resulting + * {@link EarlyTerminatingCollectorException} will be meaningless. + */ + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + + }; } + } diff --git a/solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java b/solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java index 4e913dd..12bde81 100644 --- a/solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java +++ b/solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java @@ -23,6 +23,8 @@ import org.apache.lucene.queries.function.FunctionValues; import org.apache.lucene.queries.function.ValueSource; import org.apache.lucene.queries.function.ValueSourceScorer; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.SubCollector; import org.apache.solr.search.function.ValueSourceRangeFilter; import java.io.IOException; @@ -45,26 +47,49 @@ public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFi class FunctionRangeCollector extends DelegatingCollector { final Map fcontext; - ValueSourceScorer scorer; - int maxdoc; public FunctionRangeCollector(Map fcontext) { this.fcontext = fcontext; } @Override - public void collect(int doc) throws IOException { - if (doc> 6, maxDoc, allCollectors); + setCollector = new DocSetCollector(maxDoc >> 6, maxDoc, allCollectors); allCollectors = setCollector; } diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index bad1d95..f1c1650 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -74,6 +74,7 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SubCollector; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TimeLimitingCollector; import org.apache.lucene.search.TopDocs; @@ -81,6 +82,7 @@ import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollector; import org.apache.lucene.search.TopScoreDocCollector; import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.search.TotalHitCountWithTopScoreCollector; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; @@ -103,7 +105,6 @@ import org.apache.solr.request.UnInvertedField; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.SchemaField; -import org.apache.solr.spelling.QueryConverter; import org.apache.solr.update.SolrIndexConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -898,19 +899,20 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn if (idIter == null) continue; } - collector.setNextReader(leaf); + final SubCollector subCollector = collector.subCollector(leaf); int max = reader.maxDoc(); if (idIter == null) { for (int docid = 0; docid topscore[0]) topscore[0]=score; - } - @Override - public void setNextReader(AtomicReaderContext context) { - } - @Override - public boolean acceptsDocsOutOfOrder() { - return true; - } - }; - } + final TotalHitCountWithTopScoreCollector totalHitCountWithTopScoreCollector = new TotalHitCountWithTopScoreCollector(needScores); + Collector collector = totalHitCountWithTopScoreCollector; if (terminateEarly) { collector = new EarlyTerminatingCollector(collector, cmd.len); } @@ -1473,8 +1433,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn nDocsReturned=0; ids = new int[nDocsReturned]; scores = new float[nDocsReturned]; - totalHits = numHits[0]; - maxScore = totalHits>0 ? topscore[0] : 0.0f; + totalHits = totalHitCountWithTopScoreCollector.getTotalHits(); + maxScore = totalHits > 0 ? totalHitCountWithTopScoreCollector.getTopScore() : 0.0f; } else { TopDocsCollector topCollector; if (cmd.getSort() == null) { @@ -1554,34 +1514,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn // handle zero case... if (lastDocRequested<=0) { - final float[] topscore = new float[] { Float.NEGATIVE_INFINITY }; - - Collector collector; - DocSetCollector setCollector; - - if (!needScores) { - collector = setCollector = new DocSetCollector(smallSetSize, maxDoc); - } else { - collector = setCollector = new DocSetDelegateCollector(smallSetSize, maxDoc, new Collector() { - Scorer scorer; - @Override - public void setScorer(Scorer scorer) { - this.scorer = scorer; - } - @Override - public void collect(int doc) throws IOException { - float score = scorer.score(); - if (score > topscore[0]) topscore[0]=score; - } - @Override - public void setNextReader(AtomicReaderContext context) { - } - @Override - public boolean acceptsDocsOutOfOrder() { - return false; - } - }); - } + + TotalHitCountWithTopScoreCollector totalHitCountWithTopScoreCollector = new TotalHitCountWithTopScoreCollector(needScores); + DocSetCollector setCollector = new DocSetCollector(smallSetSize, maxDoc, totalHitCountWithTopScoreCollector); + Collector collector = setCollector; if (terminateEarly) { collector = new EarlyTerminatingCollector(collector, cmd.len); } @@ -1609,8 +1545,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn nDocsReturned = 0; ids = new int[nDocsReturned]; scores = new float[nDocsReturned]; - totalHits = set.size(); - maxScore = totalHits>0 ? topscore[0] : 0.0f; + totalHits = totalHitCountWithTopScoreCollector.getTotalHits(); + maxScore = totalHits > 0 ? totalHitCountWithTopScoreCollector.getTopScore() : 0.0f; } else { TopDocsCollector topCollector; @@ -1621,7 +1557,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn topCollector = TopFieldCollector.create(weightSort(cmd.getSort()), len, false, needScores, needScores, true); } - DocSetCollector setCollector = new DocSetDelegateCollector(maxDoc>>6, maxDoc, topCollector); + DocSetCollector setCollector = new DocSetCollector(maxDoc>>6, maxDoc, topCollector); Collector collector = setCollector; if (terminateEarly) { collector = new EarlyTerminatingCollector(collector, cmd.len); @@ -1910,6 +1846,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn TopDocsCollector topCollector = TopFieldCollector.create(weightSort(sort), nDocs, false, false, false, inOrder); DocIterator iter = set.iterator(); + SubCollector subCollector = null; int base=0; int end=0; int readerIndex = 0; @@ -1920,12 +1857,18 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn AtomicReaderContext leaf = leafContexts.get(readerIndex++); base = leaf.docBase; end = base + leaf.reader().maxDoc(); - topCollector.setNextReader(leaf); + if (subCollector != null) { + subCollector.done(); + } + subCollector = topCollector.subCollector(leaf); // we should never need to set the scorer given the settings for the collector } - topCollector.collect(doc-base); + subCollector.collect(doc-base); } - + if (subCollector != null) { + subCollector.done(); + } + TopDocs topDocs = topCollector.topDocs(0, nDocs); int nDocsReturned = topDocs.scoreDocs.length; diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java index 74e7e46..c77429f 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java @@ -171,7 +171,7 @@ public class CommandHandler { docSetCollector = new DocSetCollector(maxDoc >> 6, maxDoc); } else { Collector wrappedCollectors = MultiCollector.wrap(collectors.toArray(new Collector[collectors.size()])); - docSetCollector = new DocSetDelegateCollector(maxDoc >> 6, maxDoc, wrappedCollectors); + docSetCollector = new DocSetCollector(maxDoc >> 6, maxDoc, wrappedCollectors); } searchWithTimeLimiter(query, filter, docSetCollector); return docSetCollector.getDocSet(); diff --git a/solr/core/src/java/org/apache/solr/search/grouping/collector/FilterCollector.java b/solr/core/src/java/org/apache/solr/search/grouping/collector/FilterCollector.java index 5ab7f18..17d09b6 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/collector/FilterCollector.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/collector/FilterCollector.java @@ -19,7 +19,7 @@ package org.apache.solr.search.grouping.collector; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.WrappingCollector; import org.apache.solr.search.DocSet; import java.io.IOException; @@ -29,40 +29,15 @@ import java.io.IOException; * * @lucene.experimental */ -public class FilterCollector extends Collector { +public class FilterCollector extends WrappingCollector { private final DocSet filter; - private final Collector delegate; - private int docBase; + private int matches; public FilterCollector(DocSet filter, Collector delegate) { + super(delegate); this.filter = filter; - this.delegate = delegate; - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - delegate.setScorer(scorer); - } - - @Override - public void collect(int doc) throws IOException { - matches++; - if (filter.exists(doc + docBase)) { - delegate.collect(doc); - } - } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.docBase = context.docBase; - delegate.setNextReader(context); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return delegate.acceptsDocsOutOfOrder(); } public int getMatches() { @@ -77,4 +52,28 @@ public class FilterCollector extends Collector { public Collector getDelegate() { return delegate; } + + @Override + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final int docBase = context.docBase; + return new WrappingSubCollector(delegate.subCollector(context)) { + + int subMatches = 0; + + @Override + public void collect(int doc) throws IOException { + subMatches++; + if (filter.exists(doc + docBase)) { + delegate.collect(doc); + } + } + + @Override + public void done() throws IOException { + delegate.done(); + FilterCollector.this.matches += subMatches; + } + }; + } + } diff --git a/solr/core/src/test/org/apache/solr/search/TestSort.java b/solr/core/src/test/org/apache/solr/search/TestSort.java index 1f391c5..46b45d9 100644 --- a/solr/core/src/test/org/apache/solr/search/TestSort.java +++ b/solr/core/src/test/org/apache/solr/search/TestSort.java @@ -241,29 +241,22 @@ public class TestSort extends SolrTestCaseJ4 { final List collectedDocs = new ArrayList(); // delegate and collect docs ourselves - Collector myCollector = new Collector() { - int docBase; - - @Override - public void setScorer(Scorer scorer) throws IOException { - topCollector.setScorer(scorer); - } - - @Override - public void collect(int doc) throws IOException { - topCollector.collect(doc); - collectedDocs.add(mydocs[doc + docBase]); - } - + Collector myCollector = new WrappingCollector(topCollector) { @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - topCollector.setNextReader(context); - docBase = context.docBase; + public WrappingSubCollector subCollector(AtomicReaderContext context) throws IOException { + final int docBase = context.docBase; + return new WrappingSubCollector(topCollector.subCollector(context)) { + @Override + public void collect(int doc) throws IOException { + delegate.collect(doc); + collectedDocs.add(mydocs[doc + docBase]); + } + }; } @Override - public boolean acceptsDocsOutOfOrder() { - return topCollector.acceptsDocsOutOfOrder(); + public boolean isParallelizable() { + return false; } };