blob: c5118889110a361305954822a50c89db96132cae [file] [log] [blame]
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
index 9b5c987..a8a7db1 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
@@ -65,10 +65,10 @@ public class CompletionScorer extends BulkScorer {
@Override
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
- if (!(collector instanceof TopSuggestDocsCollector)) {
- throw new IllegalArgumentException("collector is not of type TopSuggestDocsCollector");
+ if (!(collector instanceof TopSuggestionsLeafCollector)) {
+ throw new IllegalArgumentException("collector is not of type TopSuggestionsCollector");
}
- suggester.lookup(this, acceptDocs, ((TopSuggestDocsCollector) collector));
+ suggester.lookup(this, acceptDocs, ((TopSuggestionsLeafCollector) collector));
return max;
}
@@ -103,4 +103,5 @@ public class CompletionScorer extends BulkScorer {
}
return weight * boost;
}
+
}
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
index db09482..c431c1c 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
@@ -45,7 +45,7 @@ import static org.apache.lucene.search.suggest.document.NRTSuggester.PayLoadProc
* <p>
* NRTSuggester executes Top N search on a weighted FST specified by a {@link CompletionScorer}
* <p>
- * See {@link #lookup(CompletionScorer, Bits, TopSuggestDocsCollector)} for more implementation
+ * See {@link #lookup(CompletionScorer, Bits, TopSuggestionsLeafCollector)} for more implementation
* details.
* <p>
* FST Format:
@@ -115,7 +115,7 @@ public final class NRTSuggester implements Accountable {
}
/**
- * Collects at most {@link TopSuggestDocsCollector#getCountToCollect()} completions that
+ * Collects at most {@link TopSuggestionsLeafCollector#getCountToCollect()} completions that
* match the provided {@link CompletionScorer}.
* <p>
* The {@link CompletionScorer#automaton} is intersected with the {@link #fst}.
@@ -124,9 +124,9 @@ public final class NRTSuggester implements Accountable {
* the matched partial paths. Upon reaching a completed path, {@link CompletionScorer#accept(int, Bits)}
* and {@link CompletionScorer#score(float, float)} is used on the document id, index weight
* and query boost to filter and score the entry, before being collected via
- * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
+ * {@link TopSuggestionsLeafCollector#collect(int, CharSequence, CharSequence, float)}
*/
- public void lookup(final CompletionScorer scorer, Bits acceptDocs, final TopSuggestDocsCollector collector) throws IOException {
+ public void lookup(final CompletionScorer scorer, Bits acceptDocs, final TopSuggestionsLeafCollector collector) throws IOException {
final double liveDocsRatio = calculateLiveDocRatio(scorer.reader.numDocs(), scorer.reader.maxDoc());
if (liveDocsRatio == -1) {
return;
@@ -144,6 +144,7 @@ public final class NRTSuggester implements Accountable {
final int topN = collector.getCountToCollect() * prefixPaths.size();
final int queueSize = getMaxTopNSearcherQueueSize(topN, scorer.reader.numDocs(), liveDocsRatio, scorer.filtered);
Comparator<Pair<Long, BytesRef>> comparator = getComparator();
+ //TODO: make TopNSearcher iterative such that topN value is not required upfront
Util.TopNSearcher<Pair<Long, BytesRef>> searcher = new Util.TopNSearcher<Pair<Long, BytesRef>>(fst, topN, queueSize, comparator,
new ScoringPathComparator(scorer)) {
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
index 25566dc..08287d4 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
@@ -52,19 +52,16 @@ public class SuggestIndexSearcher extends IndexSearcher {
* <code>query</code>
*/
public TopSuggestDocs suggest(CompletionQuery query, int n) throws IOException {
- TopSuggestDocsCollector collector = new TopSuggestDocsCollector(n);
+ TopSuggestsCollector collector = new TopSuggestsCollector(n);
suggest(query, collector);
- return collector.get();
+ return ((TopSuggestDocs) collector.topDocs());
}
/**
* Lower-level suggest API.
* Collects completion hits through <code>collector</code> for <code>query</code>.
- *
- * <p>{@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
- * is called for every matching completion hit.
*/
- public void suggest(CompletionQuery query, TopSuggestDocsCollector collector) throws IOException {
+ public void suggest(CompletionQuery query, TopSuggestsCollector collector) throws IOException {
// TODO use IndexSearcher.rewrite instead
// have to implement equals() and hashCode() in CompletionQuerys and co
query = (CompletionQuery) query.rewrite(getIndexReader());
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestScoreDocPriorityQueue.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestScoreDocPriorityQueue.java
index 634383c..cd17732 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestScoreDocPriorityQueue.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestScoreDocPriorityQueue.java
@@ -41,16 +41,4 @@ final class SuggestScoreDocPriorityQueue extends PriorityQueue<SuggestScoreDoc>
}
return a.score < b.score;
}
-
- /**
- * Returns the top N results in descending order.
- */
- public SuggestScoreDoc[] getResults() {
- int size = size();
- SuggestScoreDoc[] res = new SuggestScoreDoc[size];
- for (int i = size - 1; i >= 0; i--) {
- res[i] = pop();
- }
- return res;
- }
}
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
index 049f73a..7aefb45 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
@@ -17,6 +17,9 @@ package org.apache.lucene.search.suggest.document;
* limitations under the License.
*/
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.suggest.Lookup;
@@ -38,34 +41,36 @@ public class TopSuggestDocs extends TopDocs {
* {@link org.apache.lucene.search.ScoreDoc} with an
* additional CharSequence key
*/
- public static class SuggestScoreDoc extends ScoreDoc implements Comparable<SuggestScoreDoc> {
+ public static class SuggestScoreDoc extends ScoreDoc {
+
+ /**
+ * Matched completion keys
+ */
+ public final List<CharSequence> keys = new ArrayList<>();
/**
- * Matched completion key
+ * Contexts for the completions
*/
- public final CharSequence key;
+ public final List<CharSequence> contexts = new ArrayList<>();
/**
- * Context for the completion
+ * Scores for the completions
*/
- public final CharSequence context;
+ public final List<Float> scores = new ArrayList<>();
/**
* Creates a SuggestScoreDoc instance
*
* @param doc document id (hit)
* @param key matched completion
+ * @param context context of the matched completion
* @param score weight of the matched completion
*/
public SuggestScoreDoc(int doc, CharSequence key, CharSequence context, float score) {
super(doc, score);
- this.key = key;
- this.context = context;
- }
-
- @Override
- public int compareTo(SuggestScoreDoc o) {
- return Lookup.CHARSEQUENCE_COMPARATOR.compare(key, o.key);
+ this.keys.add(key);
+ this.contexts.add(context);
+ this.scores.add(score);
}
}
@@ -96,20 +101,23 @@ public class TopSuggestDocs extends TopDocs {
* NOTE: assumes every <code>shardHit</code> is already sorted by score
*/
public static TopSuggestDocs merge(int topN, TopSuggestDocs[] shardHits) {
- SuggestScoreDocPriorityQueue priorityQueue = new SuggestScoreDocPriorityQueue(topN);
+ SuggestScoreDocPriorityQueue queue = new SuggestScoreDocPriorityQueue(topN);
for (TopSuggestDocs shardHit : shardHits) {
for (SuggestScoreDoc scoreDoc : shardHit.scoreLookupDocs()) {
- if (scoreDoc == priorityQueue.insertWithOverflow(scoreDoc)) {
+ if (scoreDoc == queue.insertWithOverflow(scoreDoc)) {
break;
}
}
}
- SuggestScoreDoc[] topNResults = priorityQueue.getResults();
+ int size = queue.size() < topN ? queue.size() : topN;
+ SuggestScoreDoc[] topNResults = new SuggestScoreDoc[size];
+ for (int i = size - 1; i >= 0; i--) {
+ topNResults[i] = queue.pop();
+ }
if (topNResults.length > 0) {
return new TopSuggestDocs(topNResults.length, topNResults, topNResults[0].score);
} else {
return TopSuggestDocs.EMPTY;
}
}
-
}
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
deleted file mode 100644
index 1cb3277..0000000
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
+++ /dev/null
@@ -1,129 +0,0 @@
-package org.apache.lucene.search.suggest.document;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.CollectionTerminatedException;
-import org.apache.lucene.search.SimpleCollector;
-
-import static org.apache.lucene.search.suggest.document.TopSuggestDocs.SuggestScoreDoc;
-
-/**
- * {@link org.apache.lucene.search.Collector} that collects completion and
- * score, along with document id
- * <p>
- * Non scoring collector that collect completions in order of their
- * pre-computed scores.
- * <p>
- * NOTE: One document can be collected multiple times if a document
- * is matched for multiple unique completions for a given query
- * <p>
- * Subclasses should only override
- * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}.
- * <p>
- * NOTE: {@link #setScorer(org.apache.lucene.search.Scorer)} and
- * {@link #collect(int)} is not used
- *
- * @lucene.experimental
- */
-public class TopSuggestDocsCollector extends SimpleCollector {
-
- private final SuggestScoreDocPriorityQueue priorityQueue;
- private final int num;
-
- /**
- * Document base offset for the current Leaf
- */
- protected int docBase;
-
- /**
- * Sole constructor
- *
- * Collects at most <code>num</code> completions
- * with corresponding document and weight
- */
- public TopSuggestDocsCollector(int num) {
- if (num <= 0) {
- throw new IllegalArgumentException("'num' must be > 0");
- }
- this.num = num;
- this.priorityQueue = new SuggestScoreDocPriorityQueue(num);
- }
-
- /**
- * Returns the number of results to be collected
- */
- public int getCountToCollect() {
- return num;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- docBase = context.docBase;
- }
-
- /**
- * Called for every matched completion,
- * similar to {@link org.apache.lucene.search.LeafCollector#collect(int)}
- * but for completions.
- *
- * NOTE: collection at the leaf level is guaranteed to be in
- * descending order of score
- */
- public void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException {
- SuggestScoreDoc current = new SuggestScoreDoc(docBase + docID, key, context, score);
- if (current == priorityQueue.insertWithOverflow(current)) {
- // if the current SuggestScoreDoc has overflown from pq,
- // we can assume all of the successive collections from
- // this leaf will be overflown as well
- // TODO: reuse the overflow instance?
- throw new CollectionTerminatedException();
- }
- }
-
- /**
- * Returns at most <code>num</code> Top scoring {@link org.apache.lucene.search.suggest.document.TopSuggestDocs}s
- */
- public TopSuggestDocs get() throws IOException {
- SuggestScoreDoc[] suggestScoreDocs = priorityQueue.getResults();
- if (suggestScoreDocs.length > 0) {
- return new TopSuggestDocs(suggestScoreDocs.length, suggestScoreDocs, suggestScoreDocs[0].score);
- } else {
- return TopSuggestDocs.EMPTY;
- }
- }
-
- /**
- * Ignored
- */
- @Override
- public void collect(int doc) throws IOException {
- // {@link #collect(int, CharSequence, CharSequence, long)} is used
- // instead
- }
-
- /**
- * Ignored
- */
- @Override
- public boolean needsScores() {
- return true;
- }
-}
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionDocumentsCollector.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionDocumentsCollector.java
new file mode 100644
index 0000000..cd3f36b
--- /dev/null
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionDocumentsCollector.java
@@ -0,0 +1,117 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectionTerminatedException;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * A {@link org.apache.lucene.search.TopDocsCollector} that collects
+ * <code>num</code> documents along with completion, pre-computed score
+ * and contexts using {@link TopSuggestionsLeafCollector}
+ * <p>
+ * This collects <code>num</code> unique documents for each leaf and
+ * uses a priority queue for the global top documents.
+ * <p>
+ * NOTE: this collector uses a mutable map across its leaf collectors,
+ * hence it is not thread-safe
+ * @lucene.experimental
+ */
+public class TopSuggestionDocumentsCollector extends TopSuggestsCollector {
+ private final Map<Integer, TopSuggestDocs.SuggestScoreDoc> scoreDocMap;
+
+ /**
+ * Collects at most <code>num</code> documents
+ * along with completions, pre-computed scores and
+ * contexts
+ */
+ public TopSuggestionDocumentsCollector(int num) {
+ this(num, new SuggestScoreDocPriorityQueue(num));
+ }
+
+ /**
+ * Sub-classes can pass in their pq implementation
+ */
+ protected TopSuggestionDocumentsCollector(int num, PriorityQueue<TopSuggestDocs.SuggestScoreDoc> pq) {
+ super(num, pq);
+ this.scoreDocMap = new LinkedHashMap<>(num);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ final LeafReaderContext ctx = context;
+ updateResults();
+ // NOTE: we set countToCollect to max doc, to increase the topN value set
+ // in the underlying TopNSearcher, allowing it to collect more paths
+ // that might belong to a document that has already been collected
+ // (same document can have more than one path in the FST).
+ // This can still lead to collecting less paths then needed, in case of having
+ // more paths for a document than the number of documents in the segment.
+ // This case has been worked around for the case of completions with contexts
+ // as we can increase this value by the number of contexts that matched a query
+ return new TopSuggestionsLeafCollector(context.reader().maxDoc()) {
+ @Override
+ public void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException {
+ if (scoreDocMap.containsKey(docID)) {
+ TopSuggestDocs.SuggestScoreDoc suggestDoc = scoreDocMap.get(docID);
+ suggestDoc.keys.add(key);
+ suggestDoc.contexts.add(context);
+ suggestDoc.scores.add(score);
+ } else if (scoreDocMap.size() <= num) {
+ scoreDocMap.put(docID, new TopSuggestDocs.SuggestScoreDoc(ctx.docBase + docID, key, context, score));
+ } else {
+ throw new CollectionTerminatedException();
+ }
+ }
+ };
+ }
+
+ private void updateResults() {
+ // populate the pq with documents collected by the last leaf collector
+ for (TopSuggestDocs.SuggestScoreDoc currentSuggestDoc : scoreDocMap.values()) {
+ TopSuggestDocs.SuggestScoreDoc overflow = pq.insertWithOverflow(currentSuggestDoc);
+ if (overflow == currentSuggestDoc) {
+ break;
+ } else if (overflow == null) {
+ totalHits++;
+ }
+ }
+ // clear the map for the next leaf collector to start populating
+ scoreDocMap.clear();
+ }
+
+ @Override
+ public TopDocs topDocs(int start, int howMany) {
+ updateResults();
+ // we pass in topDocsSize() instead of howMany
+ // as updateResults add suggest docs from the
+ // last leaf collector to the pq
+ return super.topDocs(start, topDocsSize());
+ }
+}
+
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionsLeafCollector.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionsLeafCollector.java
new file mode 100644
index 0000000..49194db
--- /dev/null
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestionsLeafCollector.java
@@ -0,0 +1,83 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorer;
+
+/**
+ * {@link org.apache.lucene.search.LeafCollector} that collects completion
+ * and score, along with document id in order of their pre-computed scores
+ *
+ * <p>
+ * NOTE:
+ * <ul>
+ * <li>
+ * A document can be collected multiple times if a document is matched
+ * for multiple unique completions
+ * </li>
+ * <li>
+ * Documents are not collected in order of their pre-computed score
+ * rather than their docIDs
+ * </li>
+ * </ul>
+ * @lucene.experimental
+ */
+public abstract class TopSuggestionsLeafCollector implements LeafCollector {
+
+ /**
+ * Number of results to collect
+ */
+ protected final int countToCollect;
+
+ /**
+ * Collects <code>countToCollect</code> completions
+ */
+ protected TopSuggestionsLeafCollector(int countToCollect) {
+ this.countToCollect = countToCollect;
+ }
+
+ /**
+ * Returns the number of results to be collected
+ */
+ public int getCountToCollect() {
+ return countToCollect;
+ }
+
+ /**
+ * Called for every matched completion,
+ * similar to {@link org.apache.lucene.search.LeafCollector#collect(int)}
+ * but for completions.
+ *
+ * NOTE: collection at the leaf level is guaranteed to be in
+ * descending order of score
+ */
+ public abstract void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException;
+
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ // no-op
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ // no-op
+ }
+}
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestsCollector.java lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestsCollector.java
new file mode 100644
index 0000000..21b58fe
--- /dev/null
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestsCollector.java
@@ -0,0 +1,110 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectionTerminatedException;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.util.PriorityQueue;
+
+import static org.apache.lucene.search.suggest.document.TopSuggestDocs.SuggestScoreDoc;
+
+/**
+ * A {@link TopDocsCollector} that collects <code>num</code> completions,
+ * pre-computed score and contexts using {@link TopSuggestionsLeafCollector}
+ *
+ * NOTE: In case of matching a document multiple times (e.g. docs with multiple contexts),
+ * a single document will be collected more than once. To ensure collecting
+ * <code>num</code> unique documents, use {@link TopSuggestionDocumentsCollector}
+ * instead
+ *
+ * @lucene.experimental
+ */
+public class TopSuggestsCollector extends TopDocsCollector<SuggestScoreDoc> {
+
+ /**
+ * Number of completions to collect
+ */
+ protected final int num;
+
+ /**
+ * Collects at most <code>num</code> completions
+ * with corresponding document and weight
+ */
+ public TopSuggestsCollector(int num) {
+ this(num, new SuggestScoreDocPriorityQueue(num));
+ }
+
+ /**
+ * Sub-classes can pass in their pq implementation
+ */
+ protected TopSuggestsCollector(int num, PriorityQueue<SuggestScoreDoc> pq) {
+ super(pq);
+ this.num = num;
+ }
+
+ /**
+ * Creates a {@link TopSuggestionsLeafCollector} for the given <code>context</code>
+ */
+ @Override
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ final LeafReaderContext ctx = context;
+ return new TopSuggestionsLeafCollector(num) {
+ @Override
+ public void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException {
+ SuggestScoreDoc current = new SuggestScoreDoc(ctx.docBase + docID, key, context, score);
+ SuggestScoreDoc overflow = pq.insertWithOverflow(current);
+ if (overflow == current) {
+ // if the current SuggestScoreDoc has overflown from pq,
+ // we can assume all of the successive collections from
+ // this leaf will be overflown as well
+ // TODO: reuse the overflow instance?
+ throw new CollectionTerminatedException();
+ } else if (overflow == null) {
+ totalHits++;
+ }
+ }
+ };
+ }
+
+ @Override
+ public TopDocs topDocs(int start, int howMany) {
+ if (start != 0) {
+ throw new IllegalArgumentException("start > 0 is not supported");
+ }
+ int size = topDocsSize();
+ if (size == 0) {
+ return TopSuggestDocs.EMPTY;
+ }
+ TopSuggestDocs.SuggestScoreDoc[] suggestScoreDocs = new TopSuggestDocs.SuggestScoreDoc[size];
+ populateResults(suggestScoreDocs, howMany);
+ return new TopSuggestDocs(totalHits, suggestScoreDocs, suggestScoreDocs[0].score);
+ }
+
+ /**
+ * Ignored
+ */
+ @Override
+ public boolean needsScores() {
+ return true;
+ }
+}
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
index 20a76e6..9d0f10c 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
@@ -475,6 +475,41 @@ public class TestContextQuery extends LuceneTestCase {
}
@Test
+ public void testWithTopSuggestionDocCollector() throws Exception {
+ Analyzer analyzer = new MockAnalyzer(random());
+ RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+ Document document = new Document();
+
+ document.add(new ContextSuggestField("suggest_field", "suggestion1", 4, "type1"));
+ document.add(new ContextSuggestField("suggest_field", "suggestion2", 3, "type2"));
+ document.add(new ContextSuggestField("suggest_field", "suggestion3", 2, "type3"));
+ iw.addDocument(document);
+
+ document = new Document();
+ document.add(new ContextSuggestField("suggest_field", "suggestion4", 1, "type4"));
+ iw.addDocument(document);
+
+ if (rarely()) {
+ iw.commit();
+ }
+
+ DirectoryReader reader = iw.getReader();
+ SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+ ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+ TopSuggestionDocumentsCollector collector = new TopSuggestionDocumentsCollector(2);
+ suggestIndexSearcher.suggest(query, collector);
+ TopSuggestDocs suggestDocs = (TopSuggestDocs) collector.topDocs();
+ assertSuggestions(suggestDocs,
+ new Entry(Arrays.asList("suggestion1", "suggestion2", "suggestion3"),
+ Arrays.asList("type1", "type2", "type3"),
+ Arrays.asList(4f, 3f, 2f)),
+ new Entry("suggestion4", "type4", 1));
+
+ reader.close();
+ iw.close();
+ }
+
+ @Test
public void testRandomContextQueryScoring() throws Exception {
Analyzer analyzer = new MockAnalyzer(random());
try(RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"))) {
@@ -508,11 +543,11 @@ public class TestContextQuery extends LuceneTestCase {
ArrayUtil.introSort(expectedResults, new Comparator<Entry>() {
@Override
public int compare(Entry o1, Entry o2) {
- int cmp = Float.compare(o2.value, o1.value);
+ int cmp = Float.compare(o2.scores.get(0), o1.scores.get(0));
if (cmp != 0) {
return cmp;
} else {
- return o1.output.compareTo(o2.output);
+ return o1.keys.get(0).compareTo(o2.keys.get(0));
}
}
});
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
index 20a2df1..e3ca124 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search.suggest.document;
*/
import java.io.IOException;
+import java.util.Arrays;
import java.util.Objects;
import org.apache.lucene.analysis.Analyzer;
@@ -175,7 +176,7 @@ public class TestPrefixCompletionQuery extends LuceneTestCase {
// the search should be admissible for a single segment
TopSuggestDocs suggest = indexSearcher.suggest(query, num);
assertTrue(suggest.totalHits >= 1);
- assertThat(suggest.scoreLookupDocs()[0].key.toString(), equalTo("abc_" + topScore));
+ assertThat(suggest.scoreLookupDocs()[0].keys.get(0).toString(), equalTo("abc_" + topScore));
assertThat(suggest.scoreLookupDocs()[0].score, equalTo((float) topScore));
filter = new NumericRangeBitsProducer("filter_int_fld", 0, 0);
@@ -309,5 +310,4 @@ public class TestPrefixCompletionQuery extends LuceneTestCase {
reader.close();
iw.close();
}
-
}
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index 9699e5d..f1ff201 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -21,6 +21,7 @@ import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -444,7 +445,7 @@ public class TestSuggestField extends LuceneTestCase {
TopSuggestDocs suggest = indexSearcher.suggest(query, num);
assertEquals(num, suggest.totalHits);
for (SuggestScoreDoc suggestScoreDoc : suggest.scoreLookupDocs()) {
- String key = suggestScoreDoc.key.toString();
+ String key = suggestScoreDoc.keys.get(0).toString();
assertTrue(key.startsWith("abc_"));
String substring = key.substring(4);
int fieldValue = Integer.parseInt(substring);
@@ -489,8 +490,8 @@ public class TestSuggestField extends LuceneTestCase {
assertTrue(topScore >= scoreDoc.score);
}
topScore = scoreDoc.score;
- assertThat((float) mappings.get(scoreDoc.key.toString()), equalTo(scoreDoc.score));
- assertNotNull(mappings.remove(scoreDoc.key.toString()));
+ assertThat((float) mappings.get(scoreDoc.keys.get(0).toString()), equalTo(scoreDoc.score));
+ assertNotNull(mappings.remove(scoreDoc.keys.get(0).toString()));
}
}
@@ -615,18 +616,24 @@ public class TestSuggestField extends LuceneTestCase {
}
static class Entry {
- final String output;
- final float value;
- final String context;
+ final List<String> keys;
+ final List<Float> scores;
+ final List<String> contexts;
- Entry(String output, float value) {
- this(output, null, value);
+ Entry(String keys, float scores) {
+ this(keys, null, scores);
}
- Entry(String output, String context, float value) {
- this.output = output;
- this.value = value;
- this.context = context;
+ Entry(String keys, String contexts, float scores) {
+ this(Collections.singletonList(keys),
+ Collections.singletonList(contexts),
+ Collections.singletonList(scores));
+ }
+
+ Entry(List<String> keys, List<String> contexts, List<Float> scores) {
+ this.keys = keys;
+ this.scores = scores;
+ this.contexts = contexts;
}
}
@@ -636,18 +643,28 @@ public class TestSuggestField extends LuceneTestCase {
for (int i = 0; i < suggestScoreDocs.length; i++) {
SuggestScoreDoc lookupDoc = suggestScoreDocs[i];
String msg = "Expected: " + toString(expected[i]) + " Actual: " + toString(lookupDoc);
- assertThat(msg, lookupDoc.key.toString(), equalTo(expected[i].output));
- assertThat(msg, lookupDoc.score, equalTo(expected[i].value));
- assertThat(msg, lookupDoc.context, equalTo(expected[i].context));
+ assertThat(lookupDoc.keys.size(), equalTo(expected[i].keys.size()));
+ for (int keyIndex = 0; keyIndex < lookupDoc.keys.size(); keyIndex++) {
+ assertThat(msg, lookupDoc.keys.get(keyIndex).toString(), equalTo(expected[i].keys.get(keyIndex)));
+ }
+ assertThat(lookupDoc.scores.size(), equalTo(expected[i].scores.size()));
+ for (int scoreIndex = 0; scoreIndex < lookupDoc.keys.size(); scoreIndex++) {
+ assertThat(msg, lookupDoc.scores.get(scoreIndex), equalTo(expected[i].scores.get(scoreIndex)));
+ }
+ assertThat(msg, lookupDoc.score, equalTo(expected[i].scores.get(0)));
+ assertThat(lookupDoc.contexts.size(), equalTo(expected[i].contexts.size()));
+ for (int contextIndex = 0; contextIndex < lookupDoc.contexts.size(); contextIndex++) {
+ assertThat(msg, lookupDoc.contexts.get(contextIndex), equalTo(expected[i].contexts.get(contextIndex)));
+ }
}
}
private static String toString(Entry expected) {
- return "key:"+ expected.output+" score:"+expected.value+" context:"+expected.context;
+ return "key:"+ expected.keys +" score:"+expected.scores +" context:"+expected.contexts;
}
private static String toString(SuggestScoreDoc actual) {
- return "key:"+ actual.key.toString()+" score:"+actual.score+" context:"+actual.context;
+ return "key:"+ actual.keys.toString()+" score:"+actual.score+" context:"+actual.contexts.toString();
}
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, String... suggestFields) {
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestTopSuggestionDocumentsCollector.java lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestTopSuggestionDocumentsCollector.java
new file mode 100644
index 0000000..d3cf35a
--- /dev/null
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestTopSuggestionDocumentsCollector.java
@@ -0,0 +1,69 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.junit.Test;
+
+import static org.apache.lucene.search.suggest.document.TestSuggestField.iwcWithSuggestField;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+public class TestTopSuggestionDocumentsCollector extends LuceneTestCase {
+
+ @Test
+ public void testEarlyTermination() throws Exception {
+ try (Directory dir = newDirectory()) {
+ Analyzer analyzer = new MockAnalyzer(random());
+ int numSuggestions = atLeast(20);
+ int numContexts = atLeast(5);
+ try (RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"))) {
+ for (int i = 0; i < numSuggestions; i++) {
+ String suggestion = "sugg_" + TestUtil.randomSimpleString(random(), 10);
+ Document document = new Document();
+ for (int j = 0; j < numContexts; j++) {
+ CharSequence context = TestUtil.randomSimpleString(random(), 10) + i;
+ document.add(new ContextSuggestField("suggest_field", suggestion, i + 1, context));
+ }
+ iw.addDocument(document);
+ if (rarely()) {
+ iw.commit();
+ }
+ }
+ try (DirectoryReader reader = iw.getReader()) {
+ SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+ for (int suggestions = 1; suggestions <= numSuggestions; suggestions++) {
+ TopSuggestionDocumentsCollector collector = new TopSuggestionDocumentsCollector(suggestions);
+ CompletionQuery query = new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg"));
+ suggestIndexSearcher.suggest(query, collector);
+ TopDocs topDocs = collector.topDocs();
+ assertThat(topDocs.totalHits, equalTo(suggestions));
+ }
+ }
+ }
+ }
+ }
+}