mirror of https://github.com/apache/lucene.git
LUCENE-5528: add contexts to AnalyzingInfixSuggester
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1580510 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
cbd8cacc50
commit
40fc92d3c8
|
@ -127,6 +127,11 @@ New Features
|
||||||
first pass search using scores from a more costly second pass
|
first pass search using scores from a more costly second pass
|
||||||
search. (Simon Willnauer, Robert Muir, Mike McCandless)
|
search. (Simon Willnauer, Robert Muir, Mike McCandless)
|
||||||
|
|
||||||
|
* LUCENE-5528: Add context to suggesters (InputIterator and Lookup
|
||||||
|
classes), and fix AnalyzingInfixSuggester to handle contexts.
|
||||||
|
Suggester contexts allow you to filter suggestions. (Areek Zillur,
|
||||||
|
Mike McCandless)
|
||||||
|
|
||||||
* LUCENE-5545: Add SortRescorer and Expression.getRescorer, to
|
* LUCENE-5545: Add SortRescorer and Expression.getRescorer, to
|
||||||
resort the hits from a first pass search using a Sort or an
|
resort the hits from a first pass search using a Sort or an
|
||||||
Expression. (Simon Willnauer, Robert Muir, Mike McCandless)
|
Expression. (Simon Willnauer, Robert Muir, Mike McCandless)
|
||||||
|
|
|
@ -1,7 +1,5 @@
|
||||||
package org.apache.lucene.document;
|
package org.apache.lucene.document;
|
||||||
|
|
||||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
@ -19,6 +17,8 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||||
|
|
||||||
/** A field that is indexed but not tokenized: the entire
|
/** A field that is indexed but not tokenized: the entire
|
||||||
* String value is indexed as a single token. For example
|
* String value is indexed as a single token. For example
|
||||||
* this might be used for a 'country' field or an 'id'
|
* this might be used for a 'country' field or an 'id'
|
||||||
|
|
|
@ -18,13 +18,13 @@
|
||||||
package org.apache.lucene.search.spell;
|
package org.apache.lucene.search.spell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.TermsEnum;
|
import org.apache.lucene.index.TermsEnum;
|
||||||
import org.apache.lucene.index.Terms;
|
import org.apache.lucene.index.Terms;
|
||||||
import org.apache.lucene.index.MultiFields;
|
import org.apache.lucene.index.MultiFields;
|
||||||
import org.apache.lucene.search.suggest.InputIterator;
|
import org.apache.lucene.search.suggest.InputIterator;
|
||||||
import org.apache.lucene.util.BytesRefIterator;
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -109,5 +109,15 @@ public class HighFrequencyDictionary implements Dictionary {
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,6 +18,9 @@ package org.apache.lucene.search.suggest;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -34,6 +37,8 @@ public class BufferedInputIterator implements InputIterator {
|
||||||
protected BytesRefArray entries = new BytesRefArray(Counter.newCounter());
|
protected BytesRefArray entries = new BytesRefArray(Counter.newCounter());
|
||||||
/** buffered payload entries */
|
/** buffered payload entries */
|
||||||
protected BytesRefArray payloads = new BytesRefArray(Counter.newCounter());
|
protected BytesRefArray payloads = new BytesRefArray(Counter.newCounter());
|
||||||
|
/** buffered context set entries */
|
||||||
|
protected List<Set<BytesRef>> contextSets = new ArrayList<>();
|
||||||
/** current buffer position */
|
/** current buffer position */
|
||||||
protected int curPos = -1;
|
protected int curPos = -1;
|
||||||
/** buffered weights, parallel with {@link #entries} */
|
/** buffered weights, parallel with {@link #entries} */
|
||||||
|
@ -41,17 +46,22 @@ public class BufferedInputIterator implements InputIterator {
|
||||||
private final BytesRef spare = new BytesRef();
|
private final BytesRef spare = new BytesRef();
|
||||||
private final BytesRef payloadSpare = new BytesRef();
|
private final BytesRef payloadSpare = new BytesRef();
|
||||||
private final boolean hasPayloads;
|
private final boolean hasPayloads;
|
||||||
|
private final boolean hasContexts;
|
||||||
|
|
||||||
/** Creates a new iterator, buffering entries from the specified iterator */
|
/** Creates a new iterator, buffering entries from the specified iterator */
|
||||||
public BufferedInputIterator(InputIterator source) throws IOException {
|
public BufferedInputIterator(InputIterator source) throws IOException {
|
||||||
BytesRef spare;
|
BytesRef spare;
|
||||||
int freqIndex = 0;
|
int freqIndex = 0;
|
||||||
hasPayloads = source.hasPayloads();
|
hasPayloads = source.hasPayloads();
|
||||||
|
hasContexts = source.hasContexts();
|
||||||
while((spare = source.next()) != null) {
|
while((spare = source.next()) != null) {
|
||||||
entries.append(spare);
|
entries.append(spare);
|
||||||
if (hasPayloads) {
|
if (hasPayloads) {
|
||||||
payloads.append(source.payload());
|
payloads.append(source.payload());
|
||||||
}
|
}
|
||||||
|
if (hasContexts) {
|
||||||
|
contextSets.add(source.contexts());
|
||||||
|
}
|
||||||
if (freqIndex >= freqs.length) {
|
if (freqIndex >= freqs.length) {
|
||||||
freqs = ArrayUtil.grow(freqs, freqs.length+1);
|
freqs = ArrayUtil.grow(freqs, freqs.length+1);
|
||||||
}
|
}
|
||||||
|
@ -86,4 +96,17 @@ public class BufferedInputIterator implements InputIterator {
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
return hasPayloads;
|
return hasPayloads;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
if (hasContexts && curPos < contextSets.size()) {
|
||||||
|
return contextSets.get(curPos);
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return hasContexts;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,12 +29,11 @@ import org.apache.lucene.index.StoredDocument;
|
||||||
import org.apache.lucene.search.spell.Dictionary;
|
import org.apache.lucene.search.spell.Dictionary;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.BytesRefIterator;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* Dictionary with terms, weights and optionally payload information
|
* Dictionary with terms, weights, payload (optional) and contexts (optional)
|
||||||
* taken from stored/indexed fields in a Lucene index.
|
* information taken from stored/indexed fields in a Lucene index.
|
||||||
* </p>
|
* </p>
|
||||||
* <b>NOTE:</b>
|
* <b>NOTE:</b>
|
||||||
* <ul>
|
* <ul>
|
||||||
|
@ -60,6 +59,8 @@ public class DocumentDictionary implements Dictionary {
|
||||||
|
|
||||||
/** Field to read payload from */
|
/** Field to read payload from */
|
||||||
protected final String payloadField;
|
protected final String payloadField;
|
||||||
|
/** Field to read contexts from */
|
||||||
|
protected final String contextsField;
|
||||||
private final String field;
|
private final String field;
|
||||||
private final String weightField;
|
private final String weightField;
|
||||||
|
|
||||||
|
@ -79,15 +80,26 @@ public class DocumentDictionary implements Dictionary {
|
||||||
* for the entry.
|
* for the entry.
|
||||||
*/
|
*/
|
||||||
public DocumentDictionary(IndexReader reader, String field, String weightField, String payloadField) {
|
public DocumentDictionary(IndexReader reader, String field, String weightField, String payloadField) {
|
||||||
|
this(reader, field, weightField, payloadField, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new dictionary with the contents of the fields named <code>field</code>
|
||||||
|
* for the terms, <code>weightField</code> for the weights that will be used for the
|
||||||
|
* the corresponding terms, <code>payloadField</code> for the corresponding payloads
|
||||||
|
* for the entry and <code>contextsFeild</code> for associated contexts.
|
||||||
|
*/
|
||||||
|
public DocumentDictionary(IndexReader reader, String field, String weightField, String payloadField, String contextsField) {
|
||||||
this.reader = reader;
|
this.reader = reader;
|
||||||
this.field = field;
|
this.field = field;
|
||||||
this.weightField = weightField;
|
this.weightField = weightField;
|
||||||
this.payloadField = payloadField;
|
this.payloadField = payloadField;
|
||||||
|
this.contextsField = contextsField;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InputIterator getEntryIterator() throws IOException {
|
public InputIterator getEntryIterator() throws IOException {
|
||||||
return new DocumentInputIterator(payloadField!=null);
|
return new DocumentInputIterator(payloadField!=null, contextsField!=null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Implements {@link InputIterator} from stored fields. */
|
/** Implements {@link InputIterator} from stored fields. */
|
||||||
|
@ -96,10 +108,12 @@ public class DocumentDictionary implements Dictionary {
|
||||||
private final int docCount;
|
private final int docCount;
|
||||||
private final Set<String> relevantFields;
|
private final Set<String> relevantFields;
|
||||||
private final boolean hasPayloads;
|
private final boolean hasPayloads;
|
||||||
|
private final boolean hasContexts;
|
||||||
private final Bits liveDocs;
|
private final Bits liveDocs;
|
||||||
private int currentDocId = -1;
|
private int currentDocId = -1;
|
||||||
private long currentWeight = 0;
|
private long currentWeight = 0;
|
||||||
private BytesRef currentPayload = null;
|
private BytesRef currentPayload = null;
|
||||||
|
private Set<BytesRef> currentContexts;
|
||||||
private final NumericDocValues weightValues;
|
private final NumericDocValues weightValues;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -107,12 +121,13 @@ public class DocumentDictionary implements Dictionary {
|
||||||
* index. setting <code>withPayload</code> to false, implies an iterator
|
* index. setting <code>withPayload</code> to false, implies an iterator
|
||||||
* over only term and weight.
|
* over only term and weight.
|
||||||
*/
|
*/
|
||||||
public DocumentInputIterator(boolean hasPayloads) throws IOException {
|
public DocumentInputIterator(boolean hasPayloads, boolean hasContexts) throws IOException {
|
||||||
this.hasPayloads = hasPayloads;
|
this.hasPayloads = hasPayloads;
|
||||||
|
this.hasContexts = hasContexts;
|
||||||
docCount = reader.maxDoc() - 1;
|
docCount = reader.maxDoc() - 1;
|
||||||
weightValues = (weightField != null) ? MultiDocValues.getNumericValues(reader, weightField) : null;
|
weightValues = (weightField != null) ? MultiDocValues.getNumericValues(reader, weightField) : null;
|
||||||
liveDocs = (reader.leaves().size() > 0) ? MultiFields.getLiveDocs(reader) : null;
|
liveDocs = (reader.leaves().size() > 0) ? MultiFields.getLiveDocs(reader) : null;
|
||||||
relevantFields = getRelevantFields(new String [] {field, weightField, payloadField});
|
relevantFields = getRelevantFields(new String [] {field, weightField, payloadField, contextsField});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -132,6 +147,7 @@ public class DocumentDictionary implements Dictionary {
|
||||||
|
|
||||||
BytesRef tempPayload = null;
|
BytesRef tempPayload = null;
|
||||||
BytesRef tempTerm = null;
|
BytesRef tempTerm = null;
|
||||||
|
Set<BytesRef> tempContexts = new HashSet<>();
|
||||||
|
|
||||||
if (hasPayloads) {
|
if (hasPayloads) {
|
||||||
StorableField payload = doc.getField(payloadField);
|
StorableField payload = doc.getField(payloadField);
|
||||||
|
@ -141,6 +157,17 @@ public class DocumentDictionary implements Dictionary {
|
||||||
tempPayload = (payload.binaryValue() != null) ? payload.binaryValue() : new BytesRef(payload.stringValue());
|
tempPayload = (payload.binaryValue() != null) ? payload.binaryValue() : new BytesRef(payload.stringValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (hasContexts) {
|
||||||
|
final StorableField[] contextFields = doc.getFields(contextsField);
|
||||||
|
for (StorableField contextField : contextFields) {
|
||||||
|
if (contextField.binaryValue() == null && contextField.stringValue() == null) {
|
||||||
|
continue;
|
||||||
|
} else {
|
||||||
|
tempContexts.add((contextField.binaryValue() != null) ? contextField.binaryValue() : new BytesRef(contextField.stringValue()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
StorableField fieldVal = doc.getField(field);
|
StorableField fieldVal = doc.getField(field);
|
||||||
if (fieldVal == null || (fieldVal.binaryValue() == null && fieldVal.stringValue() == null)) {
|
if (fieldVal == null || (fieldVal.binaryValue() == null && fieldVal.stringValue() == null)) {
|
||||||
continue;
|
continue;
|
||||||
|
@ -148,6 +175,7 @@ public class DocumentDictionary implements Dictionary {
|
||||||
tempTerm = (fieldVal.stringValue() != null) ? new BytesRef(fieldVal.stringValue()) : fieldVal.binaryValue();
|
tempTerm = (fieldVal.stringValue() != null) ? new BytesRef(fieldVal.stringValue()) : fieldVal.binaryValue();
|
||||||
|
|
||||||
currentPayload = tempPayload;
|
currentPayload = tempPayload;
|
||||||
|
currentContexts = tempContexts;
|
||||||
currentWeight = getWeight(doc, currentDocId);
|
currentWeight = getWeight(doc, currentDocId);
|
||||||
|
|
||||||
return tempTerm;
|
return tempTerm;
|
||||||
|
@ -191,5 +219,18 @@ public class DocumentDictionary implements Dictionary {
|
||||||
}
|
}
|
||||||
return relevantFields;
|
return relevantFields;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
if (hasContexts) {
|
||||||
|
return currentContexts;
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return hasContexts;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.StoredDocument;
|
import org.apache.lucene.index.StoredDocument;
|
||||||
import org.apache.lucene.queries.function.FunctionValues;
|
import org.apache.lucene.queries.function.FunctionValues;
|
||||||
import org.apache.lucene.queries.function.ValueSource;
|
import org.apache.lucene.queries.function.ValueSource;
|
||||||
import org.apache.lucene.util.BytesRefIterator;
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -68,6 +67,17 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
|
||||||
|
|
||||||
private final ValueSource weightsValueSource;
|
private final ValueSource weightsValueSource;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new dictionary with the contents of the fields named <code>field</code>
|
||||||
|
* for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
|
||||||
|
* for the associated contexts and uses the <code>weightsValueSource</code> supplied
|
||||||
|
* to determine the score.
|
||||||
|
*/
|
||||||
|
public DocumentValueSourceDictionary(IndexReader reader, String field,
|
||||||
|
ValueSource weightsValueSource, String payload, String contexts) {
|
||||||
|
super(reader, field, null, payload, contexts);
|
||||||
|
this.weightsValueSource = weightsValueSource;
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Creates a new dictionary with the contents of the fields named <code>field</code>
|
* Creates a new dictionary with the contents of the fields named <code>field</code>
|
||||||
* for the terms, <code>payloadField</code> for the corresponding payloads
|
* for the terms, <code>payloadField</code> for the corresponding payloads
|
||||||
|
@ -93,7 +103,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InputIterator getEntryIterator() throws IOException {
|
public InputIterator getEntryIterator() throws IOException {
|
||||||
return new DocumentValueSourceInputIterator(payloadField!=null);
|
return new DocumentValueSourceInputIterator(payloadField!=null, contextsField!=null);
|
||||||
}
|
}
|
||||||
|
|
||||||
final class DocumentValueSourceInputIterator extends DocumentDictionary.DocumentInputIterator {
|
final class DocumentValueSourceInputIterator extends DocumentDictionary.DocumentInputIterator {
|
||||||
|
@ -106,9 +116,9 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
|
||||||
/** current leave index */
|
/** current leave index */
|
||||||
private int currentLeafIndex = 0;
|
private int currentLeafIndex = 0;
|
||||||
|
|
||||||
public DocumentValueSourceInputIterator(boolean hasPayloads)
|
public DocumentValueSourceInputIterator(boolean hasPayloads, boolean hasContexts)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(hasPayloads);
|
super(hasPayloads, hasContexts);
|
||||||
leaves = reader.leaves();
|
leaves = reader.leaves();
|
||||||
starts = new int[leaves.size() + 1];
|
starts = new int[leaves.size() + 1];
|
||||||
for (int i = 0; i < leaves.size(); i++) {
|
for (int i = 0; i < leaves.size(); i++) {
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest;
|
||||||
|
|
||||||
|
|
||||||
import java.io.*;
|
import java.io.*;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.spell.Dictionary;
|
import org.apache.lucene.search.spell.Dictionary;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -209,5 +210,15 @@ public class FileDictionary implements Dictionary {
|
||||||
curWeight = (long)Double.parseDouble(weight);
|
curWeight = (long)Double.parseDouble(weight);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.suggest;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
|
import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
|
||||||
import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; // javadocs
|
import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; // javadocs
|
||||||
|
@ -44,6 +45,15 @@ public interface InputIterator extends BytesRefIterator {
|
||||||
/** Returns true if the iterator has payloads */
|
/** Returns true if the iterator has payloads */
|
||||||
public boolean hasPayloads();
|
public boolean hasPayloads();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A term's contexts context can be used to filter suggestions.
|
||||||
|
* May return null, if suggest entries do not have any context
|
||||||
|
* */
|
||||||
|
public Set<BytesRef> contexts();
|
||||||
|
|
||||||
|
/** Returns true if the iterator has contexts */
|
||||||
|
public boolean hasContexts();
|
||||||
|
|
||||||
/** Singleton InputIterator that iterates over 0 BytesRefs. */
|
/** Singleton InputIterator that iterates over 0 BytesRefs. */
|
||||||
public static final InputIterator EMPTY = new InputIteratorWrapper(BytesRefIterator.EMPTY);
|
public static final InputIterator EMPTY = new InputIteratorWrapper(BytesRefIterator.EMPTY);
|
||||||
|
|
||||||
|
@ -82,5 +92,15 @@ public interface InputIterator extends BytesRefIterator {
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.spell.Dictionary;
|
import org.apache.lucene.search.spell.Dictionary;
|
||||||
import org.apache.lucene.store.DataInput;
|
import org.apache.lucene.store.DataInput;
|
||||||
|
@ -40,6 +41,7 @@ public abstract class Lookup {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Result of a lookup.
|
* Result of a lookup.
|
||||||
|
* @lucene.experimental
|
||||||
*/
|
*/
|
||||||
public static final class LookupResult implements Comparable<LookupResult> {
|
public static final class LookupResult implements Comparable<LookupResult> {
|
||||||
/** the key's text */
|
/** the key's text */
|
||||||
|
@ -55,31 +57,53 @@ public abstract class Lookup {
|
||||||
/** the key's payload (null if not present) */
|
/** the key's payload (null if not present) */
|
||||||
public final BytesRef payload;
|
public final BytesRef payload;
|
||||||
|
|
||||||
|
/** the key's contexts (null if not present) */
|
||||||
|
public final Set<BytesRef> contexts;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new result from a key+weight pair.
|
* Create a new result from a key+weight pair.
|
||||||
*/
|
*/
|
||||||
public LookupResult(CharSequence key, long value) {
|
public LookupResult(CharSequence key, long value) {
|
||||||
this(key, value, null);
|
this(key, null, value, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new result from a key+weight+payload triple.
|
* Create a new result from a key+weight+payload triple.
|
||||||
*/
|
*/
|
||||||
public LookupResult(CharSequence key, long value, BytesRef payload) {
|
public LookupResult(CharSequence key, long value, BytesRef payload) {
|
||||||
this.key = key;
|
this(key, null, value, payload, null);
|
||||||
this.highlightKey = null;
|
|
||||||
this.value = value;
|
|
||||||
this.payload = payload;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new result from a key+highlightKey+weight+payload triple.
|
* Create a new result from a key+highlightKey+weight+payload triple.
|
||||||
*/
|
*/
|
||||||
public LookupResult(CharSequence key, Object highlightKey, long value, BytesRef payload) {
|
public LookupResult(CharSequence key, Object highlightKey, long value, BytesRef payload) {
|
||||||
|
this(key, highlightKey, value, payload, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new result from a key+weight+payload+contexts triple.
|
||||||
|
*/
|
||||||
|
public LookupResult(CharSequence key, long value, BytesRef payload, Set<BytesRef> contexts) {
|
||||||
|
this(key, null, value, payload, contexts);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new result from a key+weight+contexts triple.
|
||||||
|
*/
|
||||||
|
public LookupResult(CharSequence key, long value, Set<BytesRef> contexts) {
|
||||||
|
this(key, null, value, null, contexts);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new result from a key+highlightKey+weight+payload+contexts triple.
|
||||||
|
*/
|
||||||
|
public LookupResult(CharSequence key, Object highlightKey, long value, BytesRef payload, Set<BytesRef> contexts) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.highlightKey = highlightKey;
|
this.highlightKey = highlightKey;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
this.payload = payload;
|
this.payload = payload;
|
||||||
|
this.contexts = contexts;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -211,7 +235,20 @@ public abstract class Lookup {
|
||||||
* @param num maximum number of results to return
|
* @param num maximum number of results to return
|
||||||
* @return a list of possible completions, with their relative weight (e.g. popularity)
|
* @return a list of possible completions, with their relative weight (e.g. popularity)
|
||||||
*/
|
*/
|
||||||
public abstract List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException;
|
public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException {
|
||||||
|
return lookup(key, null, onlyMorePopular, num);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Look up a key and return possible completion for this key.
|
||||||
|
* @param key lookup key. Depending on the implementation this may be
|
||||||
|
* a prefix, misspelling, or even infix.
|
||||||
|
* @param contexts contexts to filter the lookup by, or null if all contexts are allowed; if the suggestion contains any of the contexts, it's a match
|
||||||
|
* @param onlyMorePopular return only more popular results
|
||||||
|
* @param num maximum number of results to return
|
||||||
|
* @return a list of possible completions, with their relative weight (e.g. popularity)
|
||||||
|
*/
|
||||||
|
public abstract List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Persist the constructed lookup data to a directory. Optional operation.
|
* Persist the constructed lookup data to a directory. Optional operation.
|
||||||
|
@ -235,4 +272,5 @@ public abstract class Lookup {
|
||||||
* @return ram size of the lookup implementation in bytes
|
* @return ram size of the lookup implementation in bytes
|
||||||
*/
|
*/
|
||||||
public abstract long sizeInBytes();
|
public abstract long sizeInBytes();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.lucene.search.suggest;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.store.ByteArrayDataInput;
|
import org.apache.lucene.store.ByteArrayDataInput;
|
||||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||||
|
@ -42,11 +44,13 @@ public class SortedInputIterator implements InputIterator {
|
||||||
private final ByteSequencesReader reader;
|
private final ByteSequencesReader reader;
|
||||||
private final Comparator<BytesRef> comparator;
|
private final Comparator<BytesRef> comparator;
|
||||||
private final boolean hasPayloads;
|
private final boolean hasPayloads;
|
||||||
|
private final boolean hasContexts;
|
||||||
private boolean done = false;
|
private boolean done = false;
|
||||||
|
|
||||||
private long weight;
|
private long weight;
|
||||||
private final BytesRef scratch = new BytesRef();
|
private final BytesRef scratch = new BytesRef();
|
||||||
private BytesRef payload = new BytesRef();
|
private BytesRef payload = new BytesRef();
|
||||||
|
private Set<BytesRef> contexts = null;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new sorted wrapper, using {@link
|
* Creates a new sorted wrapper, using {@link
|
||||||
|
@ -62,6 +66,7 @@ public class SortedInputIterator implements InputIterator {
|
||||||
*/
|
*/
|
||||||
public SortedInputIterator(InputIterator source, Comparator<BytesRef> comparator) throws IOException {
|
public SortedInputIterator(InputIterator source, Comparator<BytesRef> comparator) throws IOException {
|
||||||
this.hasPayloads = source.hasPayloads();
|
this.hasPayloads = source.hasPayloads();
|
||||||
|
this.hasContexts = source.hasContexts();
|
||||||
this.source = source;
|
this.source = source;
|
||||||
this.comparator = comparator;
|
this.comparator = comparator;
|
||||||
this.reader = sort();
|
this.reader = sort();
|
||||||
|
@ -80,6 +85,9 @@ public class SortedInputIterator implements InputIterator {
|
||||||
if (hasPayloads) {
|
if (hasPayloads) {
|
||||||
payload = decodePayload(scratch, input);
|
payload = decodePayload(scratch, input);
|
||||||
}
|
}
|
||||||
|
if (hasContexts) {
|
||||||
|
contexts = decodeContexts(scratch, input);
|
||||||
|
}
|
||||||
success = true;
|
success = true;
|
||||||
return scratch;
|
return scratch;
|
||||||
}
|
}
|
||||||
|
@ -112,6 +120,16 @@ public class SortedInputIterator implements InputIterator {
|
||||||
return hasPayloads;
|
return hasPayloads;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return contexts;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return hasContexts;
|
||||||
|
}
|
||||||
|
|
||||||
/** Sortes by BytesRef (ascending) then cost (ascending). */
|
/** Sortes by BytesRef (ascending) then cost (ascending). */
|
||||||
private final Comparator<BytesRef> tieBreakByCostComparator = new Comparator<BytesRef>() {
|
private final Comparator<BytesRef> tieBreakByCostComparator = new Comparator<BytesRef>() {
|
||||||
|
|
||||||
|
@ -134,6 +152,10 @@ public class SortedInputIterator implements InputIterator {
|
||||||
decodePayload(leftScratch, input);
|
decodePayload(leftScratch, input);
|
||||||
decodePayload(rightScratch, input);
|
decodePayload(rightScratch, input);
|
||||||
}
|
}
|
||||||
|
if (hasContexts) {
|
||||||
|
decodeContexts(leftScratch, input);
|
||||||
|
decodeContexts(rightScratch, input);
|
||||||
|
}
|
||||||
int cmp = comparator.compare(leftScratch, rightScratch);
|
int cmp = comparator.compare(leftScratch, rightScratch);
|
||||||
if (cmp != 0) {
|
if (cmp != 0) {
|
||||||
return cmp;
|
return cmp;
|
||||||
|
@ -156,7 +178,7 @@ public class SortedInputIterator implements InputIterator {
|
||||||
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
|
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
|
||||||
|
|
||||||
while ((spare = source.next()) != null) {
|
while ((spare = source.next()) != null) {
|
||||||
encode(writer, output, buffer, spare, source.payload(), source.weight());
|
encode(writer, output, buffer, spare, source.payload(), source.contexts(), source.weight());
|
||||||
}
|
}
|
||||||
writer.close();
|
writer.close();
|
||||||
new OfflineSorter(tieBreakByCostComparator).sort(tempInput, tempSorted);
|
new OfflineSorter(tieBreakByCostComparator).sort(tempInput, tempSorted);
|
||||||
|
@ -187,9 +209,15 @@ public class SortedInputIterator implements InputIterator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** encodes an entry (bytes+(payload)+weight) to the provided writer */
|
/** encodes an entry (bytes+(payload)+(contexts)+weight) to the provided writer */
|
||||||
protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, BytesRef payload, long weight) throws IOException {
|
protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, BytesRef payload, Set<BytesRef> contexts, long weight) throws IOException {
|
||||||
int requiredLength = spare.length + 8 + ((hasPayloads) ? 2 + payload.length : 0);
|
int requiredLength = spare.length + 8 + ((hasPayloads) ? 2 + payload.length : 0);
|
||||||
|
if (hasContexts) {
|
||||||
|
for(BytesRef ctx : contexts) {
|
||||||
|
requiredLength += 2 + ctx.length;
|
||||||
|
}
|
||||||
|
requiredLength += 2; // for length of contexts
|
||||||
|
}
|
||||||
if (requiredLength >= buffer.length) {
|
if (requiredLength >= buffer.length) {
|
||||||
buffer = ArrayUtil.grow(buffer, requiredLength);
|
buffer = ArrayUtil.grow(buffer, requiredLength);
|
||||||
}
|
}
|
||||||
|
@ -199,6 +227,13 @@ public class SortedInputIterator implements InputIterator {
|
||||||
output.writeBytes(payload.bytes, payload.offset, payload.length);
|
output.writeBytes(payload.bytes, payload.offset, payload.length);
|
||||||
output.writeShort((short) payload.length);
|
output.writeShort((short) payload.length);
|
||||||
}
|
}
|
||||||
|
if (hasContexts) {
|
||||||
|
for (BytesRef ctx : contexts) {
|
||||||
|
output.writeBytes(ctx.bytes, ctx.offset, ctx.length);
|
||||||
|
output.writeShort((short) ctx.length);
|
||||||
|
}
|
||||||
|
output.writeShort((short) contexts.size());
|
||||||
|
}
|
||||||
output.writeLong(weight);
|
output.writeLong(weight);
|
||||||
writer.write(buffer, 0, output.getPosition());
|
writer.write(buffer, 0, output.getPosition());
|
||||||
}
|
}
|
||||||
|
@ -211,6 +246,27 @@ public class SortedInputIterator implements InputIterator {
|
||||||
return tmpInput.readLong();
|
return tmpInput.readLong();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** decodes the contexts at the current position */
|
||||||
|
protected Set<BytesRef> decodeContexts(BytesRef scratch, ByteArrayDataInput tmpInput) {
|
||||||
|
tmpInput.reset(scratch.bytes);
|
||||||
|
tmpInput.skipBytes(scratch.length - 2); //skip to context set size
|
||||||
|
short ctxSetSize = tmpInput.readShort();
|
||||||
|
scratch.length -= 2;
|
||||||
|
final Set<BytesRef> contextSet = new HashSet<>();
|
||||||
|
for (short i = 0; i < ctxSetSize; i++) {
|
||||||
|
tmpInput.setPosition(scratch.length - 2);
|
||||||
|
short curContextLength = tmpInput.readShort();
|
||||||
|
scratch.length -= 2;
|
||||||
|
tmpInput.setPosition(scratch.length - curContextLength);
|
||||||
|
BytesRef contextSpare = new BytesRef(curContextLength);
|
||||||
|
tmpInput.readBytes(contextSpare.bytes, 0, curContextLength);
|
||||||
|
contextSpare.length = curContextLength;
|
||||||
|
contextSet.add(contextSpare);
|
||||||
|
scratch.length -= curContextLength;
|
||||||
|
}
|
||||||
|
return contextSet;
|
||||||
|
}
|
||||||
|
|
||||||
/** decodes the payload at the current position */
|
/** decodes the payload at the current position */
|
||||||
protected BytesRef decodePayload(BytesRef scratch, ByteArrayDataInput tmpInput) {
|
protected BytesRef decodePayload(BytesRef scratch, ByteArrayDataInput tmpInput) {
|
||||||
tmpInput.reset(scratch.bytes);
|
tmpInput.reset(scratch.bytes);
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
|
@ -75,4 +76,13 @@ public class UnsortedInputIterator extends BufferedInputIterator {
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
if (hasContexts() && curPos < contextSets.size()) {
|
||||||
|
assert currentOrd == ords[curPos];
|
||||||
|
return contextSets.get(currentOrd);
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.document.FieldType;
|
import org.apache.lucene.document.FieldType;
|
||||||
import org.apache.lucene.document.NumericDocValuesField;
|
import org.apache.lucene.document.NumericDocValuesField;
|
||||||
|
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||||
import org.apache.lucene.document.StringField;
|
import org.apache.lucene.document.StringField;
|
||||||
import org.apache.lucene.document.TextField;
|
import org.apache.lucene.document.TextField;
|
||||||
import org.apache.lucene.index.AtomicReader;
|
import org.apache.lucene.index.AtomicReader;
|
||||||
|
@ -49,7 +50,9 @@ import org.apache.lucene.index.FilterAtomicReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.MultiDocValues;
|
import org.apache.lucene.index.MultiDocValues;
|
||||||
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.SegmentReader;
|
import org.apache.lucene.index.SegmentReader;
|
||||||
|
import org.apache.lucene.index.SortedSetDocValues;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.index.sorter.EarlyTerminatingSortingCollector;
|
import org.apache.lucene.index.sorter.EarlyTerminatingSortingCollector;
|
||||||
import org.apache.lucene.index.sorter.SortingMergePolicy;
|
import org.apache.lucene.index.sorter.SortingMergePolicy;
|
||||||
|
@ -96,7 +99,12 @@ import org.apache.lucene.util.Version;
|
||||||
* by the suggest weight; it would be nice to support
|
* by the suggest weight; it would be nice to support
|
||||||
* blended score + weight sort in the future. This means
|
* blended score + weight sort in the future. This means
|
||||||
* this suggester best applies when there is a strong
|
* this suggester best applies when there is a strong
|
||||||
* apriori ranking of all the suggestions. */
|
* apriori ranking of all the suggestions.
|
||||||
|
*
|
||||||
|
* <p>This suggester supports contexts, however the
|
||||||
|
* contexts must be valid utf8 (arbitrary binary terms will
|
||||||
|
* not work).
|
||||||
|
* @lucene.experimental */
|
||||||
|
|
||||||
public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
|
|
||||||
|
@ -104,9 +112,13 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
protected final static String TEXT_FIELD_NAME = "text";
|
protected final static String TEXT_FIELD_NAME = "text";
|
||||||
|
|
||||||
/** Field name used for the indexed text, as a
|
/** Field name used for the indexed text, as a
|
||||||
* StringField, for exact lookup. */
|
* StringField, for exact lookup. */
|
||||||
protected final static String EXACT_TEXT_FIELD_NAME = "exacttext";
|
protected final static String EXACT_TEXT_FIELD_NAME = "exacttext";
|
||||||
|
|
||||||
|
/** Field name used for the indexed context, as a
|
||||||
|
* StringField and a SortedSetDVField, for filtering. */
|
||||||
|
protected final static String CONTEXTS_FIELD_NAME = "contexts";
|
||||||
|
|
||||||
/** Analyzer used at search time */
|
/** Analyzer used at search time */
|
||||||
protected final Analyzer queryAnalyzer;
|
protected final Analyzer queryAnalyzer;
|
||||||
/** Analyzer used at index time */
|
/** Analyzer used at index time */
|
||||||
|
@ -206,45 +218,21 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
// just indexing the suggestions as they iterate:
|
// just indexing the suggestions as they iterate:
|
||||||
writer = new IndexWriter(dir,
|
writer = new IndexWriter(dir,
|
||||||
getIndexWriterConfig(matchVersion, getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
|
getIndexWriterConfig(matchVersion, getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
|
||||||
BytesRef text;
|
|
||||||
Document doc = new Document();
|
|
||||||
FieldType ft = getTextFieldType();
|
|
||||||
Field textField = new Field(TEXT_FIELD_NAME, "", ft);
|
|
||||||
doc.add(textField);
|
|
||||||
|
|
||||||
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", 0L);
|
|
||||||
doc.add(weightField);
|
|
||||||
|
|
||||||
Field payloadField;
|
|
||||||
if (iter.hasPayloads()) {
|
|
||||||
payloadField = new BinaryDocValuesField("payloads", new BytesRef());
|
|
||||||
doc.add(payloadField);
|
|
||||||
} else {
|
|
||||||
payloadField = null;
|
|
||||||
}
|
|
||||||
//long t0 = System.nanoTime();
|
//long t0 = System.nanoTime();
|
||||||
|
|
||||||
|
// TODO: use threads?
|
||||||
|
BytesRef text;
|
||||||
while ((text = iter.next()) != null) {
|
while ((text = iter.next()) != null) {
|
||||||
String textString = text.utf8ToString();
|
BytesRef payload;
|
||||||
textField.setStringValue(textString);
|
|
||||||
exactTextField.setStringValue(textString);
|
|
||||||
textGramField.setStringValue(textString);
|
|
||||||
textDVField.setBytesValue(text);
|
|
||||||
weightField.setLongValue(iter.weight());
|
|
||||||
if (iter.hasPayloads()) {
|
if (iter.hasPayloads()) {
|
||||||
payloadField.setBytesValue(iter.payload());
|
payload = iter.payload();
|
||||||
|
} else {
|
||||||
|
payload = null;
|
||||||
}
|
}
|
||||||
writer.addDocument(doc);
|
|
||||||
|
add(text, iter.contexts(), iter.weight(), payload);
|
||||||
}
|
}
|
||||||
|
|
||||||
//System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
|
//System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
|
||||||
|
|
||||||
searcherMgr = new SearcherManager(writer, true, null);
|
searcherMgr = new SearcherManager(writer, true, null);
|
||||||
|
@ -285,19 +273,8 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
* After adding or updating a batch of new suggestions,
|
* After adding or updating a batch of new suggestions,
|
||||||
* you must call {@link #refresh} in the end in order to
|
* you must call {@link #refresh} in the end in order to
|
||||||
* see the suggestions in {@link #lookup} */
|
* see the suggestions in {@link #lookup} */
|
||||||
public void add(BytesRef text, long weight, BytesRef payload) throws IOException {
|
public void add(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
|
||||||
String textString = text.utf8ToString();
|
writer.addDocument(buildDocument(text, contexts, weight, payload));
|
||||||
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
|
/** Updates a previous suggestion, matching the exact same
|
||||||
|
@ -307,7 +284,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
* #add} instead. After adding or updating a batch of
|
* #add} instead. After adding or updating a batch of
|
||||||
* new suggestions, you must call {@link #refresh} in the
|
* new suggestions, you must call {@link #refresh} in the
|
||||||
* end in order to see the suggestions in {@link #lookup} */
|
* end in order to see the suggestions in {@link #lookup} */
|
||||||
public void update(BytesRef text, long weight, BytesRef payload) throws IOException {
|
public void update(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
|
||||||
|
writer.updateDocument(new Term(EXACT_TEXT_FIELD_NAME, text.utf8ToString()),
|
||||||
|
buildDocument(text, contexts, weight, payload));
|
||||||
|
}
|
||||||
|
|
||||||
|
private Document buildDocument(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
|
||||||
String textString = text.utf8ToString();
|
String textString = text.utf8ToString();
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
FieldType ft = getTextFieldType();
|
FieldType ft = getTextFieldType();
|
||||||
|
@ -319,7 +301,15 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
if (payload != null) {
|
if (payload != null) {
|
||||||
doc.add(new BinaryDocValuesField("payloads", payload));
|
doc.add(new BinaryDocValuesField("payloads", payload));
|
||||||
}
|
}
|
||||||
writer.updateDocument(new Term(EXACT_TEXT_FIELD_NAME, textString), doc);
|
if (contexts != null) {
|
||||||
|
for(BytesRef context : contexts) {
|
||||||
|
// TODO: if we had a BinaryTermField we could fix
|
||||||
|
// this "must be valid ut8f" limitation:
|
||||||
|
doc.add(new StringField(CONTEXTS_FIELD_NAME, context.utf8ToString(), Field.Store.NO));
|
||||||
|
doc.add(new SortedSetDocValuesField(CONTEXTS_FIELD_NAME, context));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return doc;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Reopens the underlying searcher; it's best to "batch
|
/** Reopens the underlying searcher; it's best to "batch
|
||||||
|
@ -342,8 +332,13 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) throws IOException {
|
||||||
return lookup(key, num, true, true);
|
return lookup(key, contexts, num, true, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Lookup, without any context. */
|
||||||
|
public List<LookupResult> lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
|
||||||
|
return lookup(key, null, num, allTermsRequired, doHighlight);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** This is called if the last token isn't ended
|
/** This is called if the last token isn't ended
|
||||||
|
@ -361,7 +356,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
/** Retrieve suggestions, specifying whether all terms
|
/** Retrieve suggestions, specifying whether all terms
|
||||||
* must match ({@code allTermsRequired}) and whether the hits
|
* must match ({@code allTermsRequired}) and whether the hits
|
||||||
* should be highlighted ({@code doHighlight}). */
|
* should be highlighted ({@code doHighlight}). */
|
||||||
public List<LookupResult> lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
|
||||||
|
|
||||||
if (searcherMgr == null) {
|
if (searcherMgr == null) {
|
||||||
throw new IllegalStateException("suggester was not built");
|
throw new IllegalStateException("suggester was not built");
|
||||||
|
@ -421,14 +416,30 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
query.add(lastQuery, occur);
|
query.add(lastQuery, occur);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (contexts != null) {
|
||||||
|
BooleanQuery sub = new BooleanQuery();
|
||||||
|
query.add(sub, BooleanClause.Occur.MUST);
|
||||||
|
for(BytesRef context : contexts) {
|
||||||
|
// NOTE: we "should" wrap this in
|
||||||
|
// ConstantScoreQuery, or maybe send this as a
|
||||||
|
// Filter instead to search, but since all of
|
||||||
|
// these are MUST'd, the change to the score won't
|
||||||
|
// affect the overall ranking. Since we indexed
|
||||||
|
// as DOCS_ONLY, the perf should be the same
|
||||||
|
// either way (no freq int[] blocks to decode):
|
||||||
|
|
||||||
|
// TODO: if we had a BinaryTermField we could fix
|
||||||
|
// this "must be valid ut8f" limitation:
|
||||||
|
sub.add(new TermQuery(new Term(CONTEXTS_FIELD_NAME, context.utf8ToString())), BooleanClause.Occur.SHOULD);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: we could allow blended sort here, combining
|
// TODO: we could allow blended sort here, combining
|
||||||
// weight w/ score. Now we ignore score and sort only
|
// weight w/ score. Now we ignore score and sort only
|
||||||
// by weight:
|
// by weight:
|
||||||
|
|
||||||
//System.out.println("INFIX query=" + query);
|
|
||||||
|
|
||||||
Query finalQuery = finishQuery(query, allTermsRequired);
|
Query finalQuery = finishQuery(query, allTermsRequired);
|
||||||
|
|
||||||
//System.out.println("finalQuery=" + query);
|
//System.out.println("finalQuery=" + query);
|
||||||
|
@ -475,6 +486,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
// This will just be null if app didn't pass payloads to build():
|
// This will just be null if app didn't pass payloads to build():
|
||||||
// TODO: maybe just stored fields? they compress...
|
// TODO: maybe just stored fields? they compress...
|
||||||
BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
|
BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
|
||||||
|
List<AtomicReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||||
List<LookupResult> results = new ArrayList<>();
|
List<LookupResult> results = new ArrayList<>();
|
||||||
BytesRef scratch = new BytesRef();
|
BytesRef scratch = new BytesRef();
|
||||||
for (int i=0;i<hits.scoreDocs.length;i++) {
|
for (int i=0;i<hits.scoreDocs.length;i++) {
|
||||||
|
@ -491,13 +503,30 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
||||||
payload = null;
|
payload = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Must look up sorted-set by segment:
|
||||||
|
int segment = ReaderUtil.subIndex(fd.doc, leaves);
|
||||||
|
SortedSetDocValues contextsDV = leaves.get(segment).reader().getSortedSetDocValues(CONTEXTS_FIELD_NAME);
|
||||||
|
Set<BytesRef> contexts;
|
||||||
|
if (contextsDV != null) {
|
||||||
|
contexts = new HashSet<BytesRef>();
|
||||||
|
contextsDV.setDocument(fd.doc - leaves.get(segment).docBase);
|
||||||
|
long ord;
|
||||||
|
while ((ord = contextsDV.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
|
||||||
|
BytesRef context = new BytesRef();
|
||||||
|
contextsDV.lookupOrd(ord, context);
|
||||||
|
contexts.add(context);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
contexts = null;
|
||||||
|
}
|
||||||
|
|
||||||
LookupResult result;
|
LookupResult result;
|
||||||
|
|
||||||
if (doHighlight) {
|
if (doHighlight) {
|
||||||
Object highlightKey = highlight(text, matchedTokens, prefixToken);
|
Object highlightKey = highlight(text, matchedTokens, prefixToken);
|
||||||
result = new LookupResult(highlightKey.toString(), highlightKey, score, payload);
|
result = new LookupResult(highlightKey.toString(), highlightKey, score, payload, contexts);
|
||||||
} else {
|
} else {
|
||||||
result = new LookupResult(text, score, payload);
|
result = new LookupResult(text, score, payload, contexts);
|
||||||
}
|
}
|
||||||
|
|
||||||
results.add(result);
|
results.add(result);
|
||||||
|
|
|
@ -380,6 +380,9 @@ public class AnalyzingSuggester extends Lookup {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void build(InputIterator iterator) throws IOException {
|
public void build(InputIterator iterator) throws IOException {
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
String prefix = getClass().getSimpleName();
|
String prefix = getClass().getSimpleName();
|
||||||
File directory = OfflineSorter.defaultTempDir();
|
File directory = OfflineSorter.defaultTempDir();
|
||||||
File tempInput = File.createTempFile(prefix, ".input", directory);
|
File tempInput = File.createTempFile(prefix, ".input", directory);
|
||||||
|
@ -639,12 +642,15 @@ public class AnalyzingSuggester extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(final CharSequence key, boolean onlyMorePopular, int num) {
|
public List<LookupResult> lookup(final CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) {
|
||||||
assert num > 0;
|
assert num > 0;
|
||||||
|
|
||||||
if (onlyMorePopular) {
|
if (onlyMorePopular) {
|
||||||
throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
|
throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
|
||||||
}
|
}
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
if (fst == null) {
|
if (fst == null) {
|
||||||
return Collections.emptyList();
|
return Collections.emptyList();
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,6 +50,8 @@ import org.apache.lucene.util.Version;
|
||||||
* the indexed text.
|
* the indexed text.
|
||||||
* Please note that it increases the number of elements searched and applies the
|
* Please note that it increases the number of elements searched and applies the
|
||||||
* ponderation after. It might be costly for long suggestions.
|
* ponderation after. It might be costly for long suggestions.
|
||||||
|
*
|
||||||
|
* @lucene.experimental
|
||||||
*/
|
*/
|
||||||
public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
|
public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
|
||||||
|
|
||||||
|
@ -114,15 +116,15 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Lookup.LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) throws IOException {
|
public List<Lookup.LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) throws IOException {
|
||||||
// here we multiply the number of searched element by the defined factor
|
// here we multiply the number of searched element by the defined factor
|
||||||
return super.lookup(key, onlyMorePopular, num * numFactor);
|
return super.lookup(key, contexts, onlyMorePopular, num * numFactor);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Lookup.LookupResult> lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
|
public List<Lookup.LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {
|
||||||
// here we multiply the number of searched element by the defined factor
|
// here we multiply the number of searched element by the defined factor
|
||||||
return super.lookup(key, num * numFactor, allTermsRequired, doHighlight);
|
return super.lookup(key, contexts, num * numFactor, allTermsRequired, doHighlight);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -286,7 +286,10 @@ public class FreeTextSuggester extends Lookup {
|
||||||
* the weights for the suggestions are ignored. */
|
* the weights for the suggestions are ignored. */
|
||||||
public void build(InputIterator iterator, double ramBufferSizeMB) throws IOException {
|
public void build(InputIterator iterator, double ramBufferSizeMB) throws IOException {
|
||||||
if (iterator.hasPayloads()) {
|
if (iterator.hasPayloads()) {
|
||||||
throw new IllegalArgumentException("payloads are not supported");
|
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||||
|
}
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
}
|
}
|
||||||
|
|
||||||
String prefix = getClass().getSimpleName();
|
String prefix = getClass().getSimpleName();
|
||||||
|
@ -433,8 +436,18 @@ public class FreeTextSuggester extends Lookup {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(final CharSequence key, /* ignored */ boolean onlyMorePopular, int num) {
|
public List<LookupResult> lookup(final CharSequence key, /* ignored */ boolean onlyMorePopular, int num) {
|
||||||
|
return lookup(key, null, onlyMorePopular, num);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Lookup, without any context. */
|
||||||
|
public List<LookupResult> lookup(final CharSequence key, int num) {
|
||||||
|
return lookup(key, null, true, num);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<LookupResult> lookup(final CharSequence key, Set<BytesRef> contexts, /* ignored */ boolean onlyMorePopular, int num) {
|
||||||
try {
|
try {
|
||||||
return lookup(key, num);
|
return lookup(key, contexts, num);
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
// bogus:
|
// bogus:
|
||||||
throw new RuntimeException(ioe);
|
throw new RuntimeException(ioe);
|
||||||
|
@ -458,7 +471,11 @@ public class FreeTextSuggester extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Retrieve suggestions. */
|
/** Retrieve suggestions. */
|
||||||
public List<LookupResult> lookup(final CharSequence key, int num) throws IOException {
|
public List<LookupResult> lookup(final CharSequence key, Set<BytesRef> contexts, int num) throws IOException {
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
|
|
||||||
try (TokenStream ts = queryAnalyzer.tokenStream("", key.toString())) {
|
try (TokenStream ts = queryAnalyzer.tokenStream("", key.toString())) {
|
||||||
TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||||
|
|
|
@ -21,6 +21,7 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.suggest.InputIterator;
|
import org.apache.lucene.search.suggest.InputIterator;
|
||||||
import org.apache.lucene.search.suggest.Lookup;
|
import org.apache.lucene.search.suggest.Lookup;
|
||||||
|
@ -30,16 +31,16 @@ import org.apache.lucene.store.ByteArrayDataInput;
|
||||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||||
import org.apache.lucene.store.DataInput;
|
import org.apache.lucene.store.DataInput;
|
||||||
import org.apache.lucene.store.DataOutput;
|
import org.apache.lucene.store.DataOutput;
|
||||||
import org.apache.lucene.util.fst.FST;
|
|
||||||
import org.apache.lucene.util.fst.NoOutputs;
|
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.CharsRef;
|
import org.apache.lucene.util.CharsRef;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.OfflineSorter;
|
|
||||||
import org.apache.lucene.util.OfflineSorter.SortInfo;
|
import org.apache.lucene.util.OfflineSorter.SortInfo;
|
||||||
|
import org.apache.lucene.util.OfflineSorter;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.apache.lucene.util.UnicodeUtil;
|
import org.apache.lucene.util.UnicodeUtil;
|
||||||
|
import org.apache.lucene.util.fst.FST;
|
||||||
|
import org.apache.lucene.util.fst.NoOutputs;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An adapter from {@link Lookup} API to {@link FSTCompletion}.
|
* An adapter from {@link Lookup} API to {@link FSTCompletion}.
|
||||||
|
@ -150,6 +151,9 @@ public class FSTCompletionLookup extends Lookup {
|
||||||
if (iterator.hasPayloads()) {
|
if (iterator.hasPayloads()) {
|
||||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||||
}
|
}
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
File tempInput = File.createTempFile(
|
File tempInput = File.createTempFile(
|
||||||
FSTCompletionLookup.class.getSimpleName(), ".input", OfflineSorter.defaultTempDir());
|
FSTCompletionLookup.class.getSimpleName(), ".input", OfflineSorter.defaultTempDir());
|
||||||
File tempSorted = File.createTempFile(
|
File tempSorted = File.createTempFile(
|
||||||
|
@ -243,7 +247,10 @@ public class FSTCompletionLookup extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(CharSequence key, boolean higherWeightsFirst, int num) {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean higherWeightsFirst, int num) {
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
final List<Completion> completions;
|
final List<Completion> completions;
|
||||||
if (higherWeightsFirst) {
|
if (higherWeightsFirst) {
|
||||||
completions = higherWeightsCompletion.lookup(key, num);
|
completions = higherWeightsCompletion.lookup(key, num);
|
||||||
|
|
|
@ -17,6 +17,13 @@ package org.apache.lucene.search.suggest.fst;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.suggest.InputIterator;
|
import org.apache.lucene.search.suggest.InputIterator;
|
||||||
import org.apache.lucene.search.suggest.Lookup;
|
import org.apache.lucene.search.suggest.Lookup;
|
||||||
import org.apache.lucene.search.suggest.SortedInputIterator;
|
import org.apache.lucene.search.suggest.SortedInputIterator;
|
||||||
|
@ -31,20 +38,14 @@ import org.apache.lucene.util.IntsRef;
|
||||||
import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
|
import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
|
||||||
import org.apache.lucene.util.UnicodeUtil;
|
import org.apache.lucene.util.UnicodeUtil;
|
||||||
import org.apache.lucene.util.fst.Builder;
|
import org.apache.lucene.util.fst.Builder;
|
||||||
import org.apache.lucene.util.fst.FST;
|
|
||||||
import org.apache.lucene.util.fst.FST.Arc;
|
import org.apache.lucene.util.fst.FST.Arc;
|
||||||
import org.apache.lucene.util.fst.FST.BytesReader;
|
import org.apache.lucene.util.fst.FST.BytesReader;
|
||||||
|
import org.apache.lucene.util.fst.FST;
|
||||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||||
import org.apache.lucene.util.fst.Util.Result;
|
import org.apache.lucene.util.fst.Util.Result;
|
||||||
import org.apache.lucene.util.fst.Util.TopResults;
|
import org.apache.lucene.util.fst.Util.TopResults;
|
||||||
import org.apache.lucene.util.fst.Util;
|
import org.apache.lucene.util.fst.Util;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Suggester based on a weighted FST: it first traverses the prefix,
|
* Suggester based on a weighted FST: it first traverses the prefix,
|
||||||
* then walks the <i>n</i> shortest paths to retrieve top-ranked
|
* then walks the <i>n</i> shortest paths to retrieve top-ranked
|
||||||
|
@ -97,6 +98,9 @@ public class WFSTCompletionLookup extends Lookup {
|
||||||
if (iterator.hasPayloads()) {
|
if (iterator.hasPayloads()) {
|
||||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||||
}
|
}
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
count = 0;
|
count = 0;
|
||||||
BytesRef scratch = new BytesRef();
|
BytesRef scratch = new BytesRef();
|
||||||
InputIterator iter = new WFSTInputIterator(iterator);
|
InputIterator iter = new WFSTInputIterator(iterator);
|
||||||
|
@ -140,7 +144,10 @@ public class WFSTCompletionLookup extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) {
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
assert num > 0;
|
assert num > 0;
|
||||||
|
|
||||||
if (onlyMorePopular) {
|
if (onlyMorePopular) {
|
||||||
|
@ -260,7 +267,7 @@ public class WFSTCompletionLookup extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, BytesRef payload, long weight) throws IOException {
|
protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, BytesRef payload, Set<BytesRef> contexts, long weight) throws IOException {
|
||||||
if (spare.length + 4 >= buffer.length) {
|
if (spare.length + 4 >= buffer.length) {
|
||||||
buffer = ArrayUtil.grow(buffer, spare.length + 4);
|
buffer = ArrayUtil.grow(buffer, spare.length + 4);
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.search.suggest.jaspell;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.suggest.InputIterator;
|
import org.apache.lucene.search.suggest.InputIterator;
|
||||||
import org.apache.lucene.search.suggest.Lookup;
|
import org.apache.lucene.search.suggest.Lookup;
|
||||||
|
@ -28,7 +29,6 @@ import org.apache.lucene.store.DataInput;
|
||||||
import org.apache.lucene.store.DataOutput;
|
import org.apache.lucene.store.DataOutput;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.CharsRef;
|
import org.apache.lucene.util.CharsRef;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
|
||||||
import org.apache.lucene.util.UnicodeUtil;
|
import org.apache.lucene.util.UnicodeUtil;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -56,6 +56,9 @@ public class JaspellLookup extends Lookup {
|
||||||
if (iterator.hasPayloads()) {
|
if (iterator.hasPayloads()) {
|
||||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||||
}
|
}
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
count = 0;
|
count = 0;
|
||||||
trie = new JaspellTernarySearchTrie();
|
trie = new JaspellTernarySearchTrie();
|
||||||
trie.setMatchAlmostDiff(editDistance);
|
trie.setMatchAlmostDiff(editDistance);
|
||||||
|
@ -95,7 +98,10 @@ public class JaspellLookup extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) {
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
List<LookupResult> res = new ArrayList<>();
|
List<LookupResult> res = new ArrayList<>();
|
||||||
List<String> list;
|
List<String> list;
|
||||||
int count = onlyMorePopular ? num * 2 : num;
|
int count = onlyMorePopular ? num * 2 : num;
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.search.suggest.tst;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.search.suggest.InputIterator;
|
import org.apache.lucene.search.suggest.InputIterator;
|
||||||
import org.apache.lucene.search.suggest.Lookup;
|
import org.apache.lucene.search.suggest.Lookup;
|
||||||
|
@ -55,6 +56,9 @@ public class TSTLookup extends Lookup {
|
||||||
if (iterator.hasPayloads()) {
|
if (iterator.hasPayloads()) {
|
||||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||||
}
|
}
|
||||||
|
if (iterator.hasContexts()) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
root = new TernaryTreeNode();
|
root = new TernaryTreeNode();
|
||||||
|
|
||||||
// make sure it's sorted and the comparator uses UTF16 sort order
|
// make sure it's sorted and the comparator uses UTF16 sort order
|
||||||
|
@ -117,7 +121,10 @@ public class TSTLookup extends Lookup {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) {
|
public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) {
|
||||||
|
if (contexts != null) {
|
||||||
|
throw new IllegalArgumentException("this suggester doesn't support contexts");
|
||||||
|
}
|
||||||
List<TernaryTreeNode> list = autocomplete.prefixCompletion(root, key, 0);
|
List<TernaryTreeNode> list = autocomplete.prefixCompletion(root, key, 0);
|
||||||
List<LookupResult> res = new ArrayList<>();
|
List<LookupResult> res = new ArrayList<>();
|
||||||
if (list == null || list.size() == 0) {
|
if (list == null || list.size() == 0) {
|
||||||
|
|
|
@ -4,9 +4,11 @@ import java.io.IOException;
|
||||||
import java.util.AbstractMap.SimpleEntry;
|
import java.util.AbstractMap.SimpleEntry;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
|
@ -49,9 +51,10 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
||||||
static final String FIELD_NAME = "f1";
|
static final String FIELD_NAME = "f1";
|
||||||
static final String WEIGHT_FIELD_NAME = "w1";
|
static final String WEIGHT_FIELD_NAME = "w1";
|
||||||
static final String PAYLOAD_FIELD_NAME = "p1";
|
static final String PAYLOAD_FIELD_NAME = "p1";
|
||||||
|
static final String CONTEXT_FIELD_NAME = "c1";
|
||||||
|
|
||||||
/** Returns Pair(list of invalid document terms, Map of document term -> document) */
|
/** Returns Pair(list of invalid document terms, Map of document term -> document) */
|
||||||
private Map.Entry<List<String>, Map<String, Document>> generateIndexDocuments(int ndocs, boolean requiresPayload) {
|
private Map.Entry<List<String>, Map<String, Document>> generateIndexDocuments(int ndocs, boolean requiresPayload, boolean requiresContexts) {
|
||||||
Map<String, Document> docs = new HashMap<>();
|
Map<String, Document> docs = new HashMap<>();
|
||||||
List<String> invalidDocTerms = new ArrayList<>();
|
List<String> invalidDocTerms = new ArrayList<>();
|
||||||
for(int i = 0; i < ndocs ; i++) {
|
for(int i = 0; i < ndocs ; i++) {
|
||||||
|
@ -77,6 +80,15 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (requiresContexts || usually()) {
|
||||||
|
if (usually()) {
|
||||||
|
for (int j = 0; j < atLeast(2); j++) {
|
||||||
|
doc.add(new StoredField(CONTEXT_FIELD_NAME, new BytesRef("context_" + i + "_"+ j)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// we should allow entries without context
|
||||||
|
}
|
||||||
|
|
||||||
// usually have valid weight field in document
|
// usually have valid weight field in document
|
||||||
if (usually()) {
|
if (usually()) {
|
||||||
Field weight = (rarely()) ?
|
Field weight = (rarely()) ?
|
||||||
|
@ -125,7 +137,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
||||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||||
iwc.setMergePolicy(newLogMergePolicy());
|
iwc.setMergePolicy(newLogMergePolicy());
|
||||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||||
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), true);
|
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), true, false);
|
||||||
Map<String, Document> docs = res.getValue();
|
Map<String, Document> docs = res.getValue();
|
||||||
List<String> invalidDocTerms = res.getKey();
|
List<String> invalidDocTerms = res.getKey();
|
||||||
for(Document doc: docs.values()) {
|
for(Document doc: docs.values()) {
|
||||||
|
@ -160,7 +172,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
||||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||||
iwc.setMergePolicy(newLogMergePolicy());
|
iwc.setMergePolicy(newLogMergePolicy());
|
||||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||||
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), false);
|
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), false, false);
|
||||||
Map<String, Document> docs = res.getValue();
|
Map<String, Document> docs = res.getValue();
|
||||||
List<String> invalidDocTerms = res.getKey();
|
List<String> invalidDocTerms = res.getKey();
|
||||||
for(Document doc: docs.values()) {
|
for(Document doc: docs.values()) {
|
||||||
|
@ -190,13 +202,54 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWithContexts() throws IOException {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||||
|
iwc.setMergePolicy(newLogMergePolicy());
|
||||||
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||||
|
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), true, true);
|
||||||
|
Map<String, Document> docs = res.getValue();
|
||||||
|
List<String> invalidDocTerms = res.getKey();
|
||||||
|
for(Document doc: docs.values()) {
|
||||||
|
writer.addDocument(doc);
|
||||||
|
}
|
||||||
|
writer.commit();
|
||||||
|
writer.close();
|
||||||
|
IndexReader ir = DirectoryReader.open(dir);
|
||||||
|
Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME, PAYLOAD_FIELD_NAME, CONTEXT_FIELD_NAME);
|
||||||
|
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||||
|
BytesRef f;
|
||||||
|
while((f = inputIterator.next())!=null) {
|
||||||
|
Document doc = docs.remove(f.utf8ToString());
|
||||||
|
assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
|
||||||
|
Field weightField = doc.getField(WEIGHT_FIELD_NAME);
|
||||||
|
assertEquals(inputIterator.weight(), (weightField != null) ? weightField.numericValue().longValue() : 0);
|
||||||
|
assertTrue(inputIterator.payload().equals(doc.getField(PAYLOAD_FIELD_NAME).binaryValue()));
|
||||||
|
Set<BytesRef> oriCtxs = new HashSet<>();
|
||||||
|
Set<BytesRef> contextSet = inputIterator.contexts();
|
||||||
|
for (StorableField ctxf : doc.getFields(CONTEXT_FIELD_NAME)) {
|
||||||
|
oriCtxs.add(ctxf.binaryValue());
|
||||||
|
}
|
||||||
|
assertEquals(oriCtxs.size(), contextSet.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String invalidTerm : invalidDocTerms) {
|
||||||
|
assertNotNull(docs.remove(invalidTerm));
|
||||||
|
}
|
||||||
|
assertTrue(docs.isEmpty());
|
||||||
|
|
||||||
|
ir.close();
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithDeletions() throws IOException {
|
public void testWithDeletions() throws IOException {
|
||||||
Directory dir = newDirectory();
|
Directory dir = newDirectory();
|
||||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||||
iwc.setMergePolicy(newLogMergePolicy());
|
iwc.setMergePolicy(newLogMergePolicy());
|
||||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||||
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), false);
|
Map.Entry<List<String>, Map<String, Document>> res = generateIndexDocuments(atLeast(1000), false, false);
|
||||||
Map<String, Document> docs = res.getValue();
|
Map<String, Document> docs = res.getValue();
|
||||||
List<String> invalidDocTerms = res.getKey();
|
List<String> invalidDocTerms = res.getKey();
|
||||||
Random rand = random();
|
Random rand = random();
|
||||||
|
|
|
@ -20,9 +20,11 @@ package org.apache.lucene.search.suggest;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
|
@ -52,6 +54,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
||||||
static final String WEIGHT_FIELD_NAME_2 = "w2";
|
static final String WEIGHT_FIELD_NAME_2 = "w2";
|
||||||
static final String WEIGHT_FIELD_NAME_3 = "w3";
|
static final String WEIGHT_FIELD_NAME_3 = "w3";
|
||||||
static final String PAYLOAD_FIELD_NAME = "p1";
|
static final String PAYLOAD_FIELD_NAME = "p1";
|
||||||
|
static final String CONTEXTS_FIELD_NAME = "c1";
|
||||||
|
|
||||||
private Map<String, Document> generateIndexDocuments(int ndocs) {
|
private Map<String, Document> generateIndexDocuments(int ndocs) {
|
||||||
Map<String, Document> docs = new HashMap<>();
|
Map<String, Document> docs = new HashMap<>();
|
||||||
|
@ -61,12 +64,18 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
||||||
Field weight1 = new NumericDocValuesField(WEIGHT_FIELD_NAME_1, 10 + i);
|
Field weight1 = new NumericDocValuesField(WEIGHT_FIELD_NAME_1, 10 + i);
|
||||||
Field weight2 = new NumericDocValuesField(WEIGHT_FIELD_NAME_2, 20 + i);
|
Field weight2 = new NumericDocValuesField(WEIGHT_FIELD_NAME_2, 20 + i);
|
||||||
Field weight3 = new NumericDocValuesField(WEIGHT_FIELD_NAME_3, 30 + i);
|
Field weight3 = new NumericDocValuesField(WEIGHT_FIELD_NAME_3, 30 + i);
|
||||||
|
Field contexts = new StoredField(CONTEXTS_FIELD_NAME, new BytesRef("ctx_" + i + "_0"));
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
doc.add(field);
|
doc.add(field);
|
||||||
doc.add(payload);
|
doc.add(payload);
|
||||||
doc.add(weight1);
|
doc.add(weight1);
|
||||||
doc.add(weight2);
|
doc.add(weight2);
|
||||||
doc.add(weight3);
|
doc.add(weight3);
|
||||||
|
doc.add(contexts);
|
||||||
|
for(int j = 1; j < atLeast(3); j++) {
|
||||||
|
contexts.setBytesValue(new BytesRef("ctx_" + i + "_" + j));
|
||||||
|
doc.add(contexts);
|
||||||
|
}
|
||||||
docs.put(field.stringValue(), doc);
|
docs.put(field.stringValue(), doc);
|
||||||
}
|
}
|
||||||
return docs;
|
return docs;
|
||||||
|
@ -125,6 +134,43 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWithContext() throws IOException {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||||
|
iwc.setMergePolicy(newLogMergePolicy());
|
||||||
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||||
|
Map<String, Document> docs = generateIndexDocuments(atLeast(100));
|
||||||
|
for(Document doc: docs.values()) {
|
||||||
|
writer.addDocument(doc);
|
||||||
|
}
|
||||||
|
writer.commit();
|
||||||
|
writer.close();
|
||||||
|
|
||||||
|
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, CONTEXTS_FIELD_NAME);
|
||||||
|
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||||
|
BytesRef f;
|
||||||
|
while((f = inputIterator.next())!=null) {
|
||||||
|
Document doc = docs.remove(f.utf8ToString());
|
||||||
|
long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
|
||||||
|
long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
|
||||||
|
long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
|
||||||
|
assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
|
||||||
|
assertEquals(inputIterator.weight(), (w1 + w2 + w3));
|
||||||
|
assertTrue(inputIterator.payload().equals(doc.getField(PAYLOAD_FIELD_NAME).binaryValue()));
|
||||||
|
Set<BytesRef> originalCtxs = new HashSet<>();
|
||||||
|
for (Field ctxf: doc.getFields(CONTEXTS_FIELD_NAME)) {
|
||||||
|
originalCtxs.add(ctxf.binaryValue());
|
||||||
|
}
|
||||||
|
assertEquals(originalCtxs, inputIterator.contexts());
|
||||||
|
}
|
||||||
|
assertTrue(docs.isEmpty());
|
||||||
|
ir.close();
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithoutPayload() throws IOException {
|
public void testWithoutPayload() throws IOException {
|
||||||
Directory dir = newDirectory();
|
Directory dir = newDirectory();
|
||||||
|
|
|
@ -17,6 +17,8 @@ package org.apache.lucene.search.suggest;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
/** corresponds to {@link InputIterator}'s entries */
|
/** corresponds to {@link InputIterator}'s entries */
|
||||||
|
@ -25,28 +27,55 @@ public final class Input {
|
||||||
public final long v;
|
public final long v;
|
||||||
public final BytesRef payload;
|
public final BytesRef payload;
|
||||||
public final boolean hasPayloads;
|
public final boolean hasPayloads;
|
||||||
|
public final Set<BytesRef> contexts;
|
||||||
|
public final boolean hasContexts;
|
||||||
|
|
||||||
public Input(BytesRef term, long v, BytesRef payload) {
|
public Input(BytesRef term, long v, BytesRef payload) {
|
||||||
this(term, v, payload, true);
|
this(term, v, payload, true, null, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Input(String term, long v, BytesRef payload) {
|
public Input(String term, long v, BytesRef payload) {
|
||||||
this(new BytesRef(term), v, payload, true);
|
this(new BytesRef(term), v, payload);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Input(BytesRef term, long v, Set<BytesRef> contexts) {
|
||||||
|
this(term, v, null, false, contexts, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Input(String term, long v, Set<BytesRef> contexts) {
|
||||||
|
this(new BytesRef(term), v, null, false, contexts, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Input(BytesRef term, long v) {
|
public Input(BytesRef term, long v) {
|
||||||
this(term, v, null, false);
|
this(term, v, null, false, null, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Input(String term, long v) {
|
public Input(String term, long v) {
|
||||||
this(new BytesRef(term), v, null, false);
|
this(new BytesRef(term), v, null, false, null, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Input(BytesRef term, long v, BytesRef payload, boolean hasPayloads) {
|
public Input(String term, int v, BytesRef payload, Set<BytesRef> contexts) {
|
||||||
|
this(new BytesRef(term), v, payload, true, contexts, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Input(BytesRef term, long v, BytesRef payload, Set<BytesRef> contexts) {
|
||||||
|
this(term, v, payload, true, contexts, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
public Input(BytesRef term, long v, BytesRef payload, boolean hasPayloads, Set<BytesRef> contexts,
|
||||||
|
boolean hasContexts) {
|
||||||
this.term = term;
|
this.term = term;
|
||||||
this.v = v;
|
this.v = v;
|
||||||
this.payload = payload;
|
this.payload = payload;
|
||||||
this.hasPayloads = hasPayloads;
|
this.hasPayloads = hasPayloads;
|
||||||
|
this.contexts = contexts;
|
||||||
|
this.hasContexts = hasContexts;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return hasContexts;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
|
@ -28,6 +29,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
public final class InputArrayIterator implements InputIterator {
|
public final class InputArrayIterator implements InputIterator {
|
||||||
private final Iterator<Input> i;
|
private final Iterator<Input> i;
|
||||||
private final boolean hasPayloads;
|
private final boolean hasPayloads;
|
||||||
|
private final boolean hasContexts;
|
||||||
private boolean first;
|
private boolean first;
|
||||||
private Input current;
|
private Input current;
|
||||||
private final BytesRef spare = new BytesRef();
|
private final BytesRef spare = new BytesRef();
|
||||||
|
@ -38,8 +40,10 @@ public final class InputArrayIterator implements InputIterator {
|
||||||
current = i.next();
|
current = i.next();
|
||||||
first = true;
|
first = true;
|
||||||
this.hasPayloads = current.hasPayloads;
|
this.hasPayloads = current.hasPayloads;
|
||||||
|
this.hasContexts = current.hasContexts;
|
||||||
} else {
|
} else {
|
||||||
this.hasPayloads = false;
|
this.hasPayloads = false;
|
||||||
|
this.hasContexts = false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -78,4 +82,14 @@ public final class InputArrayIterator implements InputIterator {
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
return hasPayloads;
|
return hasPayloads;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return current.contexts;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return hasContexts;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -19,9 +19,11 @@ package org.apache.lucene.search.suggest;
|
||||||
|
|
||||||
import java.util.AbstractMap.SimpleEntry;
|
import java.util.AbstractMap.SimpleEntry;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -45,21 +47,29 @@ public class TestInputIterator extends LuceneTestCase {
|
||||||
Comparator<BytesRef> comparator = random.nextBoolean() ? BytesRef.getUTF8SortedAsUnicodeComparator() : BytesRef.getUTF8SortedAsUTF16Comparator();
|
Comparator<BytesRef> comparator = random.nextBoolean() ? BytesRef.getUTF8SortedAsUnicodeComparator() : BytesRef.getUTF8SortedAsUTF16Comparator();
|
||||||
TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> sorted = new TreeMap<>(comparator);
|
TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> sorted = new TreeMap<>(comparator);
|
||||||
TreeMap<BytesRef, Long> sortedWithoutPayload = new TreeMap<>(comparator);
|
TreeMap<BytesRef, Long> sortedWithoutPayload = new TreeMap<>(comparator);
|
||||||
|
TreeMap<BytesRef, SimpleEntry<Long, Set<BytesRef>>> sortedWithContext = new TreeMap<>(comparator);
|
||||||
Input[] unsorted = new Input[num];
|
Input[] unsorted = new Input[num];
|
||||||
Input[] unsortedWithoutPayload = new Input[num];
|
Input[] unsortedWithoutPayload = new Input[num];
|
||||||
|
Input[] unsortedWithContexts = new Input[num];
|
||||||
|
Set<BytesRef> ctxs;
|
||||||
for (int i = 0; i < num; i++) {
|
for (int i = 0; i < num; i++) {
|
||||||
BytesRef key;
|
BytesRef key;
|
||||||
BytesRef payload;
|
BytesRef payload;
|
||||||
|
ctxs = new HashSet<>();
|
||||||
do {
|
do {
|
||||||
key = new BytesRef(TestUtil.randomUnicodeString(random));
|
key = new BytesRef(TestUtil.randomUnicodeString(random));
|
||||||
payload = new BytesRef(TestUtil.randomUnicodeString(random));
|
payload = new BytesRef(TestUtil.randomUnicodeString(random));
|
||||||
|
for(int j = 0; j < atLeast(2); j++) {
|
||||||
|
ctxs.add(new BytesRef(TestUtil.randomUnicodeString(random)));
|
||||||
|
}
|
||||||
} while (sorted.containsKey(key));
|
} while (sorted.containsKey(key));
|
||||||
long value = random.nextLong();
|
long value = random.nextLong();
|
||||||
sortedWithoutPayload.put(key, value);
|
sortedWithoutPayload.put(key, value);
|
||||||
sorted.put(key, new SimpleEntry<>(value, payload));
|
sorted.put(key, new SimpleEntry<>(value, payload));
|
||||||
|
sortedWithContext.put(key, new SimpleEntry<>(value, ctxs));
|
||||||
unsorted[i] = new Input(key, value, payload);
|
unsorted[i] = new Input(key, value, payload);
|
||||||
unsortedWithoutPayload[i] = new Input(key, value);
|
unsortedWithoutPayload[i] = new Input(key, value);
|
||||||
|
unsortedWithContexts[i] = new Input(key, value, ctxs);
|
||||||
}
|
}
|
||||||
|
|
||||||
// test the sorted iterator wrapper with payloads
|
// test the sorted iterator wrapper with payloads
|
||||||
|
@ -74,6 +84,18 @@ public class TestInputIterator extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
assertNull(wrapper.next());
|
assertNull(wrapper.next());
|
||||||
|
|
||||||
|
// test the sorted iterator wrapper with contexts
|
||||||
|
wrapper = new SortedInputIterator(new InputArrayIterator(unsortedWithContexts), comparator);
|
||||||
|
Iterator<Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>>> actualEntries = sortedWithContext.entrySet().iterator();
|
||||||
|
while (actualEntries.hasNext()) {
|
||||||
|
Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>> entry = actualEntries.next();
|
||||||
|
assertEquals(entry.getKey(), wrapper.next());
|
||||||
|
assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
|
||||||
|
Set<BytesRef> actualCtxs = entry.getValue().getValue();
|
||||||
|
assertEquals(actualCtxs, wrapper.contexts());
|
||||||
|
}
|
||||||
|
assertNull(wrapper.next());
|
||||||
|
|
||||||
// test the unsorted iterator wrapper with payloads
|
// test the unsorted iterator wrapper with payloads
|
||||||
wrapper = new UnsortedInputIterator(new InputArrayIterator(unsorted));
|
wrapper = new UnsortedInputIterator(new InputArrayIterator(unsorted));
|
||||||
TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> actual = new TreeMap<>();
|
TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> actual = new TreeMap<>();
|
||||||
|
|
|
@ -372,8 +372,8 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
||||||
AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
|
AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
|
||||||
suggester.build(new InputArrayIterator(new Input[0]));
|
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("a penny saved is a penny earned"), null, 10, new BytesRef("foobaz"));
|
||||||
suggester.add(new BytesRef("lend me your ear"), 8, new BytesRef("foobar"));
|
suggester.add(new BytesRef("lend me your ear"), null, 8, new BytesRef("foobar"));
|
||||||
suggester.refresh();
|
suggester.refresh();
|
||||||
List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
|
List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
|
||||||
assertEquals(2, results.size());
|
assertEquals(2, results.size());
|
||||||
|
@ -410,7 +410,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
||||||
AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
|
AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
|
||||||
suggester.build(new InputArrayIterator(new Input[0]));
|
suggester.build(new InputArrayIterator(new Input[0]));
|
||||||
suggester.add(new BytesRef("the pen is pretty"), 10, new BytesRef("foobaz"));
|
suggester.add(new BytesRef("the pen is pretty"), null, 10, new BytesRef("foobaz"));
|
||||||
suggester.refresh();
|
suggester.refresh();
|
||||||
|
|
||||||
List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("pen p", random()), 10, true, true);
|
List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("pen p", random()), 10, true, true);
|
||||||
|
@ -529,7 +529,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
System.out.println("TEST: iter=" + iter + " update input=" + input.term.utf8ToString() + "/" + weight);
|
System.out.println("TEST: iter=" + iter + " update input=" + input.term.utf8ToString() + "/" + weight);
|
||||||
}
|
}
|
||||||
suggester.update(input.term, weight, input.term);
|
suggester.update(input.term, null, weight, input.term);
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
// Add a new suggestion
|
// Add a new suggestion
|
||||||
|
@ -538,7 +538,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
System.out.println("TEST: iter=" + iter + " add input=" + text + "/" + weight);
|
System.out.println("TEST: iter=" + iter + " add input=" + text + "/" + weight);
|
||||||
}
|
}
|
||||||
BytesRef br = new BytesRef(text);
|
BytesRef br = new BytesRef(text);
|
||||||
suggester.add(br, weight, br);
|
suggester.add(br, null, weight, br);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (random().nextInt(15) == 7) {
|
if (random().nextInt(15) == 7) {
|
||||||
|
@ -743,7 +743,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
assertEquals(new BytesRef("foobar"), results.get(0).payload);
|
assertEquals(new BytesRef("foobar"), results.get(0).payload);
|
||||||
|
|
||||||
// Add a new suggestion:
|
// Add a new suggestion:
|
||||||
suggester.add(new BytesRef("a penny saved is a penny earned"), 10, new BytesRef("foobaz"));
|
suggester.add(new BytesRef("a penny saved is a penny earned"), null, 10, new BytesRef("foobaz"));
|
||||||
|
|
||||||
// Must refresh to see any newly added suggestions:
|
// Must refresh to see any newly added suggestions:
|
||||||
suggester.refresh();
|
suggester.refresh();
|
||||||
|
@ -777,7 +777,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
assertEquals(new BytesRef("foobaz"), results.get(0).payload);
|
assertEquals(new BytesRef("foobaz"), results.get(0).payload);
|
||||||
|
|
||||||
// Change the weight:
|
// Change the weight:
|
||||||
suggester.update(new BytesRef("lend me your ear"), 12, new BytesRef("foobox"));
|
suggester.update(new BytesRef("lend me your ear"), null, 12, new BytesRef("foobox"));
|
||||||
|
|
||||||
// Must refresh to see any newly added suggestions:
|
// Must refresh to see any newly added suggestions:
|
||||||
suggester.refresh();
|
suggester.refresh();
|
||||||
|
@ -792,4 +792,128 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
||||||
assertEquals(new BytesRef("foobaz"), results.get(1).payload);
|
assertEquals(new BytesRef("foobaz"), results.get(1).payload);
|
||||||
suggester.close();
|
suggester.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Set<BytesRef> asSet(String... values) {
|
||||||
|
HashSet<BytesRef> result = new HashSet<>();
|
||||||
|
for(String value : values) {
|
||||||
|
result.add(new BytesRef(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-5528
|
||||||
|
public void testBasicContext() throws Exception {
|
||||||
|
Input keys[] = new Input[] {
|
||||||
|
new Input("lend me your ear", 8, new BytesRef("foobar"), asSet("foo", "bar")),
|
||||||
|
new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz"), asSet("foo", "baz"))
|
||||||
|
};
|
||||||
|
|
||||||
|
File tempDir = TestUtil.getTempDir("analyzingInfixContext");
|
||||||
|
|
||||||
|
for(int iter=0;iter<2;iter++) {
|
||||||
|
AnalyzingInfixSuggester suggester;
|
||||||
|
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
||||||
|
if (iter == 0) {
|
||||||
|
suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
|
||||||
|
suggester.build(new InputArrayIterator(keys));
|
||||||
|
} else {
|
||||||
|
// Test again, after close/reopen:
|
||||||
|
suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
|
||||||
|
}
|
||||||
|
|
||||||
|
// No context provided, all results returned
|
||||||
|
List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
|
||||||
|
assertEquals(2, results.size());
|
||||||
|
LookupResult result = results.get(0);
|
||||||
|
assertEquals("a penny saved is a penny <b>ear</b>ned", result.key);
|
||||||
|
assertEquals(10, result.value);
|
||||||
|
assertEquals(new BytesRef("foobaz"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("baz")));
|
||||||
|
|
||||||
|
result = results.get(1);
|
||||||
|
assertEquals("lend me your <b>ear</b>", result.key);
|
||||||
|
assertEquals(8, result.value);
|
||||||
|
assertEquals(new BytesRef("foobar"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("bar")));
|
||||||
|
|
||||||
|
// Both suggestions have "foo" context:
|
||||||
|
results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), asSet("foo"), 10, true, true);
|
||||||
|
assertEquals(2, results.size());
|
||||||
|
|
||||||
|
result = results.get(0);
|
||||||
|
assertEquals("a penny saved is a penny <b>ear</b>ned", result.key);
|
||||||
|
assertEquals(10, result.value);
|
||||||
|
assertEquals(new BytesRef("foobaz"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("baz")));
|
||||||
|
|
||||||
|
result = results.get(1);
|
||||||
|
assertEquals("lend me your <b>ear</b>", result.key);
|
||||||
|
assertEquals(8, result.value);
|
||||||
|
assertEquals(new BytesRef("foobar"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("bar")));
|
||||||
|
|
||||||
|
// Only one has "bar" context:
|
||||||
|
results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), asSet("bar"), 10, true, true);
|
||||||
|
assertEquals(1, results.size());
|
||||||
|
|
||||||
|
result = results.get(0);
|
||||||
|
assertEquals("lend me your <b>ear</b>", result.key);
|
||||||
|
assertEquals(8, result.value);
|
||||||
|
assertEquals(new BytesRef("foobar"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("bar")));
|
||||||
|
|
||||||
|
// Only one has "baz" context:
|
||||||
|
results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), asSet("baz"), 10, true, true);
|
||||||
|
assertEquals(1, results.size());
|
||||||
|
|
||||||
|
result = results.get(0);
|
||||||
|
assertEquals("a penny saved is a penny <b>ear</b>ned", result.key);
|
||||||
|
assertEquals(10, result.value);
|
||||||
|
assertEquals(new BytesRef("foobaz"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("baz")));
|
||||||
|
|
||||||
|
// Both have foo or bar:
|
||||||
|
results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), asSet("foo", "bar"), 10, true, true);
|
||||||
|
assertEquals(2, results.size());
|
||||||
|
|
||||||
|
result = results.get(0);
|
||||||
|
assertEquals("a penny saved is a penny <b>ear</b>ned", result.key);
|
||||||
|
assertEquals(10, result.value);
|
||||||
|
assertEquals(new BytesRef("foobaz"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("baz")));
|
||||||
|
|
||||||
|
result = results.get(1);
|
||||||
|
assertEquals("lend me your <b>ear</b>", result.key);
|
||||||
|
assertEquals(8, result.value);
|
||||||
|
assertEquals(new BytesRef("foobar"), result.payload);
|
||||||
|
assertNotNull(result.contexts);
|
||||||
|
assertEquals(2, result.contexts.size());
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("foo")));
|
||||||
|
assertTrue(result.contexts.contains(new BytesRef("bar")));
|
||||||
|
|
||||||
|
suggester.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -136,12 +136,12 @@ public class BlendedInfixSuggesterTest extends LuceneTestCase {
|
||||||
|
|
||||||
|
|
||||||
// we don't find it for in the 2 first
|
// we don't find it for in the 2 first
|
||||||
assertEquals(2, suggester.lookup("the", 2, true, false).size());
|
assertEquals(2, suggester.lookup("the", null, 2, true, false).size());
|
||||||
long w0 = getInResults(suggester, "the", ret, 2);
|
long w0 = getInResults(suggester, "the", ret, 2);
|
||||||
assertTrue(w0 < 0);
|
assertTrue(w0 < 0);
|
||||||
|
|
||||||
// but it's there if we search for 3 elements
|
// but it's there if we search for 3 elements
|
||||||
assertEquals(3, suggester.lookup("the", 3, true, false).size());
|
assertEquals(3, suggester.lookup("the", null, 3, true, false).size());
|
||||||
long w1 = getInResults(suggester, "the", ret, 3);
|
long w1 = getInResults(suggester, "the", ret, 3);
|
||||||
assertTrue(w1 > 0);
|
assertTrue(w1 > 0);
|
||||||
|
|
||||||
|
@ -185,7 +185,7 @@ public class BlendedInfixSuggesterTest extends LuceneTestCase {
|
||||||
suggester.build(new InputArrayIterator(keys));
|
suggester.build(new InputArrayIterator(keys));
|
||||||
|
|
||||||
|
|
||||||
List<Lookup.LookupResult> responses = suggester.lookup("the", 4, true, false);
|
List<Lookup.LookupResult> responses = suggester.lookup("the", null, 4, true, false);
|
||||||
|
|
||||||
for (Lookup.LookupResult response : responses) {
|
for (Lookup.LookupResult response : responses) {
|
||||||
System.out.println(response);
|
System.out.println(response);
|
||||||
|
@ -196,7 +196,7 @@ public class BlendedInfixSuggesterTest extends LuceneTestCase {
|
||||||
|
|
||||||
private static long getInResults(BlendedInfixSuggester suggester, String prefix, BytesRef payload, int num) throws IOException {
|
private static long getInResults(BlendedInfixSuggester suggester, String prefix, BytesRef payload, int num) throws IOException {
|
||||||
|
|
||||||
List<Lookup.LookupResult> responses = suggester.lookup(prefix, num, true, false);
|
List<Lookup.LookupResult> responses = suggester.lookup(prefix, null, num, true, false);
|
||||||
|
|
||||||
for (Lookup.LookupResult response : responses) {
|
for (Lookup.LookupResult response : responses) {
|
||||||
if (response.payload.equals(payload)) {
|
if (response.payload.equals(payload)) {
|
||||||
|
|
|
@ -173,6 +173,15 @@ public class TestFreeTextSuggester extends LuceneTestCase {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
});
|
});
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
System.out.println(sug.sizeInBytes() + " bytes");
|
System.out.println(sug.sizeInBytes() + " bytes");
|
||||||
|
@ -364,6 +373,16 @@ public class TestFreeTextSuggester extends LuceneTestCase {
|
||||||
public boolean hasPayloads() {
|
public boolean hasPayloads() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BytesRef> contexts() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasContexts() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
// Build inefficient but hopefully correct model:
|
// Build inefficient but hopefully correct model:
|
||||||
|
|
Loading…
Reference in New Issue