diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e00e4eaf02a..c8fa285c36e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -77,6 +77,10 @@ New Features
similar to the query parser by overriding a newTermQuery method.
(Simon Willnauer)
+* LUCENE-5477: AnalyzingInfixSuggester now supports near-real-time
+ additions and updates (to change weight or payload of an existing
+ suggestion). (Mike McCandless)
+
API Changes
* LUCENE-5454: Add RandomAccessOrds, an optional extension of SortedSetDocValues
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
index fba75c9a036..9968377f514 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
@@ -562,7 +562,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
termCount++;
}
}
- docCount = (int) visitedDocs.cardinality();
+ docCount = visitedDocs.cardinality();
fst = b.finish();
/*
PrintStream ps = new PrintStream("out.dot");
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index d6fb922233f..29f6d029026 100755
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -35,11 +35,12 @@
-
+
+
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
index 8662d54aec3..4a749e13328 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
@@ -195,7 +195,7 @@ public abstract class Lookup {
* Get the number of entries the lookup was built with
* @return total number of suggester entries
*/
- public abstract long getCount();
+ public abstract long getCount() throws IOException;
/**
* Builds up a new internal {@link Lookup} representation based on the given {@link InputIterator}.
@@ -211,7 +211,7 @@ public abstract class Lookup {
* @param num maximum number of results to return
* @return a list of possible completions, with their relative weight (e.g. popularity)
*/
- public abstract List lookup(CharSequence key, boolean onlyMorePopular, int num);
+ public abstract List lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException;
/**
* Persist the constructed lookup data to a directory. Optional operation.
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 0944587a109..354d1cfb1e8 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
@@ -38,6 +38,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
@@ -53,20 +54,25 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
+import org.apache.lucene.index.sorter.EarlyTerminatingSortingCollector;
import org.apache.lucene.index.sorter.Sorter;
import org.apache.lucene.index.sorter.SortingAtomicReader;
+import org.apache.lucene.index.sorter.SortingMergePolicy;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
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.SearcherManager;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.suggest.InputIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@@ -81,8 +87,9 @@ import org.apache.lucene.util.Version;
// - a PostingsFormat that stores super-high-freq terms as
// a bitset should be a win for the prefix terms?
// (LUCENE-5052)
-// - we could allow NRT here, if we sort index as we go
-// (SortingMergePolicy) -- http://svn.apache.org/viewvc?view=revision&revision=1459808
+// - we could offer a better integration with
+// DocumentDictionary and NRT? so that your suggester
+// "automatically" keeps in sync w/ your index
/** Analyzes the input text and then suggests matches based
* on prefix matches to any tokens in the indexed text.
@@ -101,6 +108,10 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
/** Field name used for the indexed text. */
protected final static String TEXT_FIELD_NAME = "text";
+ /** Field name used for the indexed text, as a
+ * StringField, for exact lookup. */
+ protected final static String EXACT_TEXT_FIELD_NAME = "exacttext";
+
/** Analyzer used at search time */
protected final Analyzer queryAnalyzer;
/** Analyzer used at index time */
@@ -109,25 +120,19 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
private final File indexPath;
final int minPrefixChars;
private Directory dir;
- /** Number of entries the lookup was built with */
- private long count = 0;
+
+ /** Used for ongoing NRT additions/updates. */
+ private IndexWriter writer;
/** {@link IndexSearcher} used for lookups. */
- protected IndexSearcher searcher;
-
- /** DocValuesField holding the payloads; null if payloads were not indexed. */
- protected BinaryDocValues payloadsDV;
-
- /** DocValuesField holding each suggestion's text. */
- protected BinaryDocValues textDV;
-
- /** DocValuesField holding each suggestion's weight. */
- protected NumericDocValues weightsDV;
+ protected SearcherManager searcherMgr;
/** Default minimum number of leading characters before
* PrefixQuery is used (4). */
public static final int DEFAULT_MIN_PREFIX_CHARS = 4;
+ private Sorter sorter;
+
/** Create a new instance, loading from a previously built
* directory, if it exists. */
public AnalyzingInfixSuggester(Version matchVersion, File indexPath, Analyzer analyzer) throws IOException {
@@ -158,24 +163,27 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
if (DirectoryReader.indexExists(dir)) {
// Already built; open it:
- IndexReader reader = DirectoryReader.open(dir);
- searcher = new IndexSearcher(reader);
- // This will just be null if app didn't pass payloads to build():
- // TODO: maybe just stored fields? they compress...
- payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
- weightsDV = MultiDocValues.getNumericValues(searcher.getIndexReader(), "weight");
- textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
- count = reader.numDocs();
- assert textDV != null;
+ initSorter();
+ writer = new IndexWriter(dir,
+ getIndexWriterConfig(matchVersion, getGramAnalyzer(), sorter, IndexWriterConfig.OpenMode.APPEND));
+ searcherMgr = new SearcherManager(writer, true, null);
}
}
/** Override this to customize index settings, e.g. which
- * codec to use. */
- protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer) {
+ * codec to use. Sorter is null if this config is for
+ * the first pass writer. */
+ protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer, Sorter sorter, IndexWriterConfig.OpenMode openMode) {
IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
iwc.setCodec(new Lucene46Codec());
- iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
+ iwc.setOpenMode(openMode);
+
+ if (sorter != null) {
+ // This way all merged segments will be sorted at
+ // merge time, allow for per-segment early termination
+ // when those segments are searched:
+ iwc.setMergePolicy(new SortingMergePolicy(iwc.getMergePolicy(), sorter));
+ }
return iwc;
}
@@ -188,41 +196,26 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
@Override
public void build(InputIterator iter) throws IOException {
- if (searcher != null) {
- searcher.getIndexReader().close();
- searcher = null;
+ if (searcherMgr != null) {
+ searcherMgr.close();
+ searcherMgr = null;
}
+ if (writer != null) {
+ writer.close();
+ writer = null;
+ }
Directory dirTmp = getDirectory(new File(indexPath.toString() + ".tmp"));
IndexWriter w = null;
- IndexWriter w2 = null;
AtomicReader r = null;
boolean success = false;
- count = 0;
try {
- Analyzer gramAnalyzer = new AnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) {
- @Override
- protected Analyzer getWrappedAnalyzer(String fieldName) {
- return indexAnalyzer;
- }
-
- @Override
- protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
- if (fieldName.equals("textgrams") && minPrefixChars > 0) {
- return new TokenStreamComponents(components.getTokenizer(),
- new EdgeNGramTokenFilter(matchVersion,
- components.getTokenStream(),
- 1, minPrefixChars));
- } else {
- return components;
- }
- }
- };
-
+ // First pass: build a temporary normal Lucene index,
+ // just indexing the suggestions as they iterate:
w = new IndexWriter(dirTmp,
- getIndexWriterConfig(matchVersion, gramAnalyzer));
+ getIndexWriterConfig(matchVersion, getGramAnalyzer(), null, IndexWriterConfig.OpenMode.CREATE));
BytesRef text;
Document doc = new Document();
FieldType ft = getTextFieldType();
@@ -232,11 +225,14 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
Field textGramField = new Field("textgrams", "", ft);
doc.add(textGramField);
+ Field exactTextField = new StringField(EXACT_TEXT_FIELD_NAME, "", Field.Store.NO);
+ doc.add(exactTextField);
+
Field textDVField = new BinaryDocValuesField(TEXT_FIELD_NAME, new BytesRef());
doc.add(textDVField);
// TODO: use threads...?
- Field weightField = new NumericDocValuesField("weight", 0);
+ Field weightField = new NumericDocValuesField("weight", 0L);
doc.add(weightField);
Field payloadField;
@@ -250,6 +246,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
while ((text = iter.next()) != null) {
String textString = text.utf8ToString();
textField.setStringValue(textString);
+ exactTextField.setStringValue(textString);
textGramField.setStringValue(textString);
textDVField.setBytesValue(text);
weightField.setLongValue(iter.weight());
@@ -257,72 +254,144 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
payloadField.setBytesValue(iter.payload());
}
w.addDocument(doc);
- count++;
}
//System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
+ // Second pass: sort the entire index:
r = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(w, false));
//long t1 = System.nanoTime();
+
+ // We can rollback the first pass, now that have have
+ // the reader open, because we will discard it anyway
+ // (no sense in fsync'ing it):
w.rollback();
- final int maxDoc = r.maxDoc();
+ initSorter();
- final NumericDocValues weights = r.getNumericDocValues("weight");
-
- final Sorter.DocComparator comparator = new Sorter.DocComparator() {
- @Override
- public int compare(int docID1, int docID2) {
- final long v1 = weights.get(docID1);
- final long v2 = weights.get(docID2);
- // Reverse sort (highest weight first);
- // java7 only:
- //return Long.compare(v2, v1);
- if (v1 > v2) {
- return -1;
- } else if (v1 < v2) {
- return 1;
- } else {
- return 0;
- }
- }
- };
-
- r = SortingAtomicReader.wrap(r, new Sorter() {
- @Override
- public Sorter.DocMap sort(AtomicReader reader) throws IOException {
- return Sorter.sort(maxDoc, comparator);
- }
-
- @Override
- public String getID() {
- return "Weight";
- }
- });
+ r = SortingAtomicReader.wrap(r, sorter);
- w2 = new IndexWriter(dir,
- getIndexWriterConfig(matchVersion, indexAnalyzer));
- w2.addIndexes(new IndexReader[] {r});
+ writer = new IndexWriter(dir,
+ getIndexWriterConfig(matchVersion, getGramAnalyzer(), sorter, IndexWriterConfig.OpenMode.CREATE));
+ writer.addIndexes(new IndexReader[] {r});
r.close();
//System.out.println("sort time: " + ((System.nanoTime()-t1)/1000000) + " msec");
- searcher = new IndexSearcher(DirectoryReader.open(w2, false));
- w2.close();
-
- payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
- weightsDV = MultiDocValues.getNumericValues(searcher.getIndexReader(), "weight");
- textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
- assert textDV != null;
+ searcherMgr = new SearcherManager(writer, true, null);
success = true;
} finally {
if (success) {
- IOUtils.close(w, w2, r, dirTmp);
+ IOUtils.close(w, r, dirTmp);
} else {
- IOUtils.closeWhileHandlingException(w, w2, r, dirTmp);
+ IOUtils.closeWhileHandlingException(w, writer, r, dirTmp);
+ writer = null;
}
}
}
+ private Analyzer getGramAnalyzer() {
+ return new AnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) {
+ @Override
+ protected Analyzer getWrappedAnalyzer(String fieldName) {
+ return indexAnalyzer;
+ }
+
+ @Override
+ protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
+ if (fieldName.equals("textgrams") && minPrefixChars > 0) {
+ return new TokenStreamComponents(components.getTokenizer(),
+ new EdgeNGramTokenFilter(matchVersion,
+ components.getTokenStream(),
+ 1, minPrefixChars));
+ } else {
+ return components;
+ }
+ }
+ };
+ }
+
+ /** Adds a new suggestion. Be sure to use {@link #update}
+ * instead if you want to replace a previous suggestion.
+ * After adding or updating a batch of new suggestions,
+ * you must call {@link #refresh} in the end in order to
+ * see the suggestions in {@link #lookup} */
+ public void add(BytesRef text, long weight, BytesRef payload) throws IOException {
+ String textString = text.utf8ToString();
+ Document doc = new Document();
+ FieldType ft = getTextFieldType();
+ doc.add(new Field(TEXT_FIELD_NAME, textString, ft));
+ doc.add(new Field("textgrams", textString, ft));
+ doc.add(new StringField(EXACT_TEXT_FIELD_NAME, textString, Field.Store.NO));
+ doc.add(new BinaryDocValuesField(TEXT_FIELD_NAME, text));
+ doc.add(new NumericDocValuesField("weight", weight));
+ if (payload != null) {
+ doc.add(new BinaryDocValuesField("payloads", payload));
+ }
+ writer.addDocument(doc);
+ }
+
+ /** Updates a previous suggestion, matching the exact same
+ * text as before. Use this to change the weight or
+ * payload of an already added suggstion. If you know
+ * this text is not already present you can use {@link
+ * #add} instead. After adding or updating a batch of
+ * new suggestions, you must call {@link #refresh} in the
+ * end in order to see the suggestions in {@link #lookup} */
+ public void update(BytesRef text, long weight, BytesRef payload) throws IOException {
+ String textString = text.utf8ToString();
+ Document doc = new Document();
+ FieldType ft = getTextFieldType();
+ doc.add(new Field(TEXT_FIELD_NAME, textString, ft));
+ doc.add(new Field("textgrams", textString, ft));
+ doc.add(new StringField(EXACT_TEXT_FIELD_NAME, textString, Field.Store.NO));
+ doc.add(new BinaryDocValuesField(TEXT_FIELD_NAME, text));
+ doc.add(new NumericDocValuesField("weight", weight));
+ if (payload != null) {
+ doc.add(new BinaryDocValuesField("payloads", payload));
+ }
+ writer.updateDocument(new Term(EXACT_TEXT_FIELD_NAME, textString), doc);
+ }
+
+ /** Reopens the underlying searcher; it's best to "batch
+ * up" many additions/updates, and then call refresh
+ * once in the end. */
+ public void refresh() throws IOException {
+ searcherMgr.maybeRefreshBlocking();
+ }
+
+ private void initSorter() {
+ sorter = new Sorter() {
+
+ @Override
+ public Sorter.DocMap sort(AtomicReader reader) throws IOException {
+ final NumericDocValues weights = reader.getNumericDocValues("weight");
+ final Sorter.DocComparator comparator = new Sorter.DocComparator() {
+ @Override
+ public int compare(int docID1, int docID2) {
+ final long v1 = weights.get(docID1);
+ final long v2 = weights.get(docID2);
+ // Reverse sort (highest weight first);
+ // java7 only:
+ //return Long.compare(v2, v1);
+ if (v1 > v2) {
+ return -1;
+ } else if (v1 < v2) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+ };
+ return Sorter.sort(reader.maxDoc(), comparator);
+ }
+
+ @Override
+ public String getID() {
+ return "BySuggestWeight";
+ }
+ };
+ }
+
/**
* Subclass can override this method to change the field type of the text field
* e.g. to change the index options
@@ -336,7 +405,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
@Override
- public List lookup(CharSequence key, boolean onlyMorePopular, int num) {
+ public List lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException {
return lookup(key, num, true, true);
}
@@ -355,9 +424,9 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
/** Retrieve suggestions, specifying whether all terms
* must match ({@code allTermsRequired}) and whether the hits
* should be highlighted ({@code doHighlight}). */
- public List lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) {
+ public List lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
- if (searcher == null) {
+ if (searcherMgr == null) {
throw new IllegalStateException("suggester was not built");
}
@@ -368,15 +437,19 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
occur = BooleanClause.Occur.SHOULD;
}
+ BooleanQuery query;
+ Set matchedTokens = new HashSet();
+ String prefixToken = null;
+
try (TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()))) {
//long t0 = System.currentTimeMillis();
ts.reset();
final CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
String lastToken = null;
- BooleanQuery query = new BooleanQuery();
+ query = new BooleanQuery();
int maxEndOffset = -1;
- final Set matchedTokens = new HashSet();
+ matchedTokens = new HashSet();
while (ts.incrementToken()) {
if (lastToken != null) {
matchedTokens.add(lastToken);
@@ -389,7 +462,6 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
ts.end();
- String prefixToken = null;
if (lastToken != null) {
Query lastQuery;
if (maxEndOffset == offsetAtt.endOffset()) {
@@ -412,38 +484,44 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
query.add(lastQuery, occur);
}
}
- ts.close();
+ }
- // TODO: we could allow blended sort here, combining
- // weight w/ score. Now we ignore score and sort only
- // by weight:
+ // TODO: we could allow blended sort here, combining
+ // weight w/ score. Now we ignore score and sort only
+ // by weight:
- //System.out.println("INFIX query=" + query);
+ //System.out.println("INFIX query=" + query);
- Query finalQuery = finishQuery(query, allTermsRequired);
+ Query finalQuery = finishQuery(query, allTermsRequired);
- // We sorted postings by weight during indexing, so we
- // only retrieve the first num hits now:
- FirstNDocsCollector c = new FirstNDocsCollector(num);
- try {
- searcher.search(finalQuery, c);
- } catch (FirstNDocsCollector.DoneException done) {
- }
- TopDocs hits = c.getHits();
+ //System.out.println("finalQuery=" + query);
+
+ // Sort by weight, descending:
+ TopFieldCollector c = TopFieldCollector.create(new Sort(new SortField("weight", SortField.Type.LONG, true)),
+ num, true, false, false, false);
+
+ // We sorted postings by weight during indexing, so we
+ // only retrieve the first num hits now:
+ Collector c2 = new EarlyTerminatingSortingCollector(c, sorter, num);
+ IndexSearcher searcher = searcherMgr.acquire();
+ List results = null;
+ try {
+ //System.out.println("got searcher=" + searcher);
+ searcher.search(finalQuery, c2);
+
+ TopFieldDocs hits = (TopFieldDocs) c.topDocs();
// Slower way if postings are not pre-sorted by weight:
// hits = searcher.search(query, null, num, new Sort(new SortField("weight", SortField.Type.LONG, true)));
-
- List results = createResults(hits, num, key, doHighlight, matchedTokens, prefixToken);
-
- //System.out.println((System.currentTimeMillis() - t0) + " msec for infix suggest");
- //System.out.println(results);
-
- return results;
-
- } catch (IOException ioe) {
- throw new RuntimeException(ioe);
+ results = createResults(searcher, hits, num, key, doHighlight, matchedTokens, prefixToken);
+ } finally {
+ searcherMgr.release(searcher);
}
+
+ //System.out.println((System.currentTimeMillis() - t0) + " msec for infix suggest");
+ //System.out.println(results);
+
+ return results;
}
/**
@@ -451,22 +529,28 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
* Can be overridden by subclass to add particular behavior (e.g. weight transformation)
* @throws IOException If there are problems reading fields from the underlying Lucene index.
*/
- protected List createResults(TopDocs hits, int num, CharSequence charSequence,
+ protected List createResults(IndexSearcher searcher, TopFieldDocs hits, int num,
+ CharSequence charSequence,
boolean doHighlight, Set matchedTokens, String prefixToken)
throws IOException {
+ BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
+
+ // This will just be null if app didn't pass payloads to build():
+ // TODO: maybe just stored fields? they compress...
+ BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
List results = new ArrayList();
BytesRef scratch = new BytesRef();
for (int i=0;i results = new TreeSet<>(LOOKUP_COMP);
// we reduce the num to the one initially requested
@@ -145,16 +155,16 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
BytesRef scratch = new BytesRef();
for (int i = 0; i < hits.scoreDocs.length; i++) {
+ FieldDoc fd = (FieldDoc) hits.scoreDocs[i];
- ScoreDoc sd = hits.scoreDocs[i];
- textDV.get(sd.doc, scratch);
+ textDV.get(fd.doc, scratch);
String text = scratch.utf8ToString();
- long weight = weightsDV.get(sd.doc);
+ long weight = (Long) fd.fields[0];
BytesRef payload;
if (payloadsDV != null) {
payload = new BytesRef();
- payloadsDV.get(sd.doc, payload);
+ payloadsDV.get(fd.doc, payload);
} else {
payload = null;
}
@@ -164,7 +174,7 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
// if hit starts with the key, we don't change the score
coefficient = 1;
} else {
- coefficient = createCoefficient(sd.doc, matchedTokens, prefixToken);
+ coefficient = createCoefficient(searcher, fd.doc, matchedTokens, prefixToken);
}
long score = (long) (weight * coefficient);
@@ -212,7 +222,7 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
* @return the coefficient
* @throws IOException If there are problems reading term vectors from the underlying Lucene index.
*/
- private double createCoefficient(int doc, Set matchedTokens, String prefixToken) throws IOException {
+ private double createCoefficient(IndexSearcher searcher, int doc, Set matchedTokens, String prefixToken) throws IOException {
Terms tv = searcher.getIndexReader().getTermVector(doc, TEXT_FIELD_NAME);
TermsEnum it = tv.iterator(TermsEnum.EMPTY);
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index d168fc65cfa..0a59e010434 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -83,7 +83,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
writer.close();
IndexReader ir = DirectoryReader.open(dir);
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
- InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
+ InputIterator inputIterator = dictionary.getEntryIterator();
assertNull(inputIterator.next());
assertEquals(inputIterator.weight(), 0);
@@ -109,7 +109,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
IndexReader ir = DirectoryReader.open(dir);
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
- InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
+ InputIterator inputIterator = dictionary.getEntryIterator();
BytesRef f;
while((f = inputIterator.next())!=null) {
Document doc = docs.remove(f.utf8ToString());
@@ -141,7 +141,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
IndexReader ir = DirectoryReader.open(dir);
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd));
- InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
+ InputIterator inputIterator = dictionary.getEntryIterator();
BytesRef f;
while((f = inputIterator.next())!=null) {
Document doc = docs.remove(f.utf8ToString());
@@ -195,7 +195,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2)};
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
- InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
+ InputIterator inputIterator = dictionary.getEntryIterator();
BytesRef f;
while((f = inputIterator.next())!=null) {
Document doc = docs.remove(f.utf8ToString());
@@ -226,7 +226,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
IndexReader ir = DirectoryReader.open(dir);
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
- InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
+ InputIterator inputIterator = dictionary.getEntryIterator();
BytesRef f;
while((f = inputIterator.next())!=null) {
Document doc = docs.remove(f.utf8ToString());
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
index 4981b33800c..e85713864a4 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
@@ -21,6 +21,10 @@ import java.io.File;
import java.io.IOException;
import java.io.StringReader;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -32,9 +36,9 @@ import org.apache.lucene.analysis.core.StopFilter;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.util.CharArraySet;
-import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.search.suggest.Input;
import org.apache.lucene.search.suggest.InputArrayIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
@@ -433,4 +437,458 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
assertEquals("a bob for apples", results.get(0).key);
suggester.close();
}
+
+ public void testEmptyAtStart() throws Exception {
+ File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
+ Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+ AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
+ @Override
+ protected Directory getDirectory(File path) {
+ return newDirectory();
+ }
+ };
+ suggester.build(new InputArrayIterator(new Input[0]));
+ suggester.add(new BytesRef("a penny saved is a penny earned"), 10, new BytesRef("foobaz"));
+ suggester.add(new BytesRef("lend me your ear"), 8, new BytesRef("foobar"));
+ suggester.refresh();
+ List results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
+ assertEquals(2, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ assertEquals("lend me your ear", results.get(1).key);
+ assertEquals(8, results.get(1).value);
+ assertEquals(new BytesRef("foobar"), results.get(1).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("ear ", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("lend me your ear", results.get(0).key);
+ assertEquals(8, results.get(0).value);
+ assertEquals(new BytesRef("foobar"), results.get(0).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("pen", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("p", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ suggester.close();
+ }
+
+ public void testBothExactAndPrefix() throws Exception {
+ File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
+ Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+ AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
+ @Override
+ protected Directory getDirectory(File path) {
+ return newDirectory();
+ }
+ };
+ suggester.build(new InputArrayIterator(new Input[0]));
+ suggester.add(new BytesRef("the pen is pretty"), 10, new BytesRef("foobaz"));
+ suggester.refresh();
+
+ List results = suggester.lookup(TestUtil.stringToCharSequence("pen p", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("the pen is pretty", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+ suggester.close();
+ }
+
+ private static String randomText() {
+ int numWords = TestUtil.nextInt(random(), 1, 4);
+
+ StringBuilder b = new StringBuilder();
+ for(int i=0;i 0) {
+ b.append(' ');
+ }
+ b.append(TestUtil.randomSimpleString(random(), 1, 10));
+ }
+
+ return b.toString();
+ }
+
+ private static class Update {
+ long weight;
+ int index;
+ }
+
+ private static class LookupThread extends Thread {
+ private final AnalyzingInfixSuggester suggester;
+ private volatile boolean stop;
+
+ public LookupThread(AnalyzingInfixSuggester suggester) {
+ this.suggester = suggester;
+ }
+
+ public void finish() throws InterruptedException {
+ stop = true;
+ this.join();
+ }
+
+ @Override
+ public void run() {
+ while (stop == false) {
+ String query = randomText();
+ int topN = TestUtil.nextInt(random(), 1, 100);
+ boolean allTermsRequired = random().nextBoolean();
+ boolean doHilite = random().nextBoolean();
+ // We don't verify the results; just doing
+ // simultaneous lookups while adding/updating to
+ // see if there are any thread hazards:
+ try {
+ suggester.lookup(TestUtil.stringToCharSequence(query, random()),
+ topN, allTermsRequired, doHilite);
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ }
+ }
+ }
+
+ public void testRandomNRT() throws Exception {
+ final File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
+ Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+ int minPrefixChars = random().nextInt(7);
+ if (VERBOSE) {
+ System.out.println(" minPrefixChars=" + minPrefixChars);
+ }
+
+ AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixChars) {
+ @Override
+ protected Directory getDirectory(File path) {
+ return newFSDirectory(path);
+ }
+ };
+
+ // Initial suggester built with nothing:
+ suggester.build(new InputArrayIterator(new Input[0]));
+
+ LookupThread lookupThread = new LookupThread(suggester);
+ lookupThread.start();
+
+ int iters = atLeast(1000);
+ int visibleUpto = 0;
+
+ Set usedWeights = new HashSet();
+ Set usedKeys = new HashSet();
+
+ List inputs = new ArrayList();
+ List pendingUpdates = new ArrayList();
+
+ for(int iter=0;iter 0 && random().nextInt(4) == 1) {
+ // Update an existing suggestion
+ Update update = new Update();
+ update.index = random().nextInt(inputs.size());
+ update.weight = weight;
+ Input input = inputs.get(update.index);
+ pendingUpdates.add(update);
+ if (VERBOSE) {
+ System.out.println("TEST: iter=" + iter + " update input=" + input.term.utf8ToString() + "/" + weight);
+ }
+ suggester.update(input.term, weight, input.term);
+
+ } else {
+ // Add a new suggestion
+ inputs.add(new Input(text, weight, new BytesRef(text)));
+ if (VERBOSE) {
+ System.out.println("TEST: iter=" + iter + " add input=" + text + "/" + weight);
+ }
+ BytesRef br = new BytesRef(text);
+ suggester.add(br, weight, br);
+ }
+
+ if (random().nextInt(15) == 7) {
+ if (VERBOSE) {
+ System.out.println("TEST: now refresh suggester");
+ }
+ suggester.refresh();
+ visibleUpto = inputs.size();
+ for(Update update : pendingUpdates) {
+ Input oldInput = inputs.get(update.index);
+ Input newInput = new Input(oldInput.term, update.weight, oldInput.payload);
+ inputs.set(update.index, newInput);
+ }
+ pendingUpdates.clear();
+ }
+
+ if (random().nextInt(50) == 7) {
+ if (VERBOSE) {
+ System.out.println("TEST: now close/reopen suggester");
+ }
+ lookupThread.finish();
+ suggester.close();
+ suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixChars) {
+ @Override
+ protected Directory getDirectory(File path) {
+ return newFSDirectory(path);
+ }
+ };
+ lookupThread = new LookupThread(suggester);
+ lookupThread.start();
+
+ visibleUpto = inputs.size();
+ for(Update update : pendingUpdates) {
+ Input oldInput = inputs.get(update.index);
+ Input newInput = new Input(oldInput.term, update.weight, oldInput.payload);
+ inputs.set(update.index, newInput);
+ }
+ pendingUpdates.clear();
+ }
+
+ if (visibleUpto > 0) {
+ String query = randomText();
+ boolean lastPrefix = random().nextInt(5) != 1;
+ if (lastPrefix == false) {
+ query += " ";
+ }
+
+ String[] queryTerms = query.split("\\s");
+ boolean allTermsRequired = random().nextInt(10) == 7;
+ boolean doHilite = random().nextBoolean();
+
+ if (VERBOSE) {
+ System.out.println("TEST: lookup \"" + query + "\" allTermsRequired=" + allTermsRequired + " doHilite=" + doHilite);
+ }
+
+ // Stupid slow but hopefully correct matching:
+ List expected = new ArrayList();
+ for(int i=0;i() {
+ @Override
+ public int compare(Input a, Input b) {
+ if (a.v > b.v) {
+ return -1;
+ } else if (a.v < b.v) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+ });
+
+ if (expected.isEmpty() == false) {
+
+ int topN = TestUtil.nextInt(random(), 1, expected.size());
+
+ List actual = suggester.lookup(TestUtil.stringToCharSequence(query, random()), topN, allTermsRequired, doHilite);
+
+ int expectedCount = Math.min(topN, expected.size());
+
+ if (VERBOSE) {
+ System.out.println(" expected:");
+ for(int i=0;i");
+ b.append(inputTerm);
+ b.append("");
+ matched = true;
+ break;
+ }
+ } else if (inputTerm.startsWith(queryTerm)) {
+ b.append("");
+ b.append(queryTerm);
+ b.append("");
+ b.append(inputTerm.substring(queryTerm.length(), inputTerm.length()));
+ matched = true;
+ break;
+ }
+ }
+
+ if (matched == false) {
+ b.append(inputTerm);
+ }
+ }
+
+ return b.toString();
+ }
+
+ public void testBasicNRT() throws Exception {
+ Input keys[] = new Input[] {
+ new Input("lend me your ear", 8, new BytesRef("foobar")),
+ };
+
+ File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
+
+ Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+ AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
+ @Override
+ protected Directory getDirectory(File path) {
+ return newDirectory();
+ }
+ };
+ suggester.build(new InputArrayIterator(keys));
+
+ List results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("lend me your ear", results.get(0).key);
+ assertEquals(8, results.get(0).value);
+ assertEquals(new BytesRef("foobar"), results.get(0).payload);
+
+ // Add a new suggestion:
+ suggester.add(new BytesRef("a penny saved is a penny earned"), 10, new BytesRef("foobaz"));
+
+ // Must refresh to see any newly added suggestions:
+ suggester.refresh();
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
+ assertEquals(2, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ assertEquals("lend me your ear", results.get(1).key);
+ assertEquals(8, results.get(1).value);
+ assertEquals(new BytesRef("foobar"), results.get(1).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("ear ", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("lend me your ear", results.get(0).key);
+ assertEquals(8, results.get(0).value);
+ assertEquals(new BytesRef("foobar"), results.get(0).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("pen", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("p", random()), 10, true, true);
+ assertEquals(1, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+
+ // Change the weight:
+ suggester.update(new BytesRef("lend me your ear"), 12, new BytesRef("foobox"));
+
+ // Must refresh to see any newly added suggestions:
+ suggester.refresh();
+
+ results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
+ assertEquals(2, results.size());
+ assertEquals("lend me your ear", results.get(0).key);
+ assertEquals(12, results.get(0).value);
+ assertEquals(new BytesRef("foobox"), results.get(0).payload);
+ assertEquals("a penny saved is a penny earned", results.get(1).key);
+ assertEquals(10, results.get(1).value);
+ assertEquals(new BytesRef("foobaz"), results.get(1).payload);
+ suggester.close();
+ }
}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
index b88e3e32a2d..71ac3df23d0 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
@@ -225,7 +225,7 @@ public class BlendedInfixSuggesterTest extends LuceneTestCase {
suggester.close();
}
- private static long getInResults(BlendedInfixSuggester suggester, String prefix, BytesRef payload, int num) {
+ private static long getInResults(BlendedInfixSuggester suggester, String prefix, BytesRef payload, int num) throws IOException {
List responses = suggester.lookup(prefix, num, true, false);
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
index 55cbddb8907..31d2925b29c 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
@@ -78,7 +78,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
protected NamedList initParams;
/**
- * Key is the dictionary name used in SolrConfig, value is the corrosponding {@link SolrSuggester}
+ * Key is the dictionary name used in SolrConfig, value is the corresponding {@link SolrSuggester}
*/
protected Map suggesters = new ConcurrentHashMap();
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
index 74a27596e22..fb72eea51d3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
@@ -126,6 +126,7 @@
FileDictionaryFactory
false
blendedInfixSuggest.txt
+ blendedInfixDir1
linear
@@ -138,6 +139,7 @@
FileDictionaryFactory
false
blendedInfixSuggest.txt
+ blendedInfixDir2
reciprocal