You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/05/28 09:53:10 UTC
svn commit: r1682158 [1/3] - in /lucene/dev/trunk/lucene: ./
core/src/java/org/apache/lucene/util/fst/
suggest/src/java/org/apache/lucene/search/suggest/document/
suggest/src/test/org/apache/lucene/search/suggest/document/
Author: mikemccand
Date: Thu May 28 07:53:09 2015
New Revision: 1682158
URL: http://svn.apache.org/r1682158
Log:
LUCENE-6459: add common suggest API for document based NRT suggester
Added:
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestRegexCompletionQuery.java (with props)
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java (with props)
Removed:
lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/SuggestFieldTest.java
Modified:
lucene/dev/trunk/lucene/CHANGES.txt
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu May 28 07:53:09 2015
@@ -41,6 +41,12 @@ New Features
can be used with getMultiValueSeparator render whole field
values. (Luca Cavanna via Robert Muir)
+* LUCENE-6459: Add common suggest API that mirrors Lucene's
+ Query/IndexSearcher APIs for Document based suggester.
+ Adds PrefixCompletionQuery, RegexCompletionQuery,
+ FuzzyCompletionQuery and ContextQuery.
+ (Areek Zillur via Mike McCandless)
+
Bug fixes
* LUCENE-6500: ParallelCompositeReader did not always call
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Thu May 28 07:53:09 2015
@@ -251,17 +251,29 @@ public final class Util {
public FST.Arc<T> arc;
public T cost;
public final IntsRefBuilder input;
+ public final float boost;
+ public final CharSequence context;
/** Sole constructor */
public FSTPath(T cost, FST.Arc<T> arc, IntsRefBuilder input) {
+ this(cost, arc, input, 0, null);
+ }
+
+ public FSTPath(T cost, FST.Arc<T> arc, IntsRefBuilder input, float boost, CharSequence context) {
this.arc = new FST.Arc<T>().copyFrom(arc);
this.cost = cost;
this.input = input;
+ this.boost = boost;
+ this.context = context;
+ }
+
+ public FSTPath<T> newPath(T cost, IntsRefBuilder input) {
+ return new FSTPath<>(cost, this.arc, input, this.boost, this.context);
}
@Override
public String toString() {
- return "input=" + input + " cost=" + cost;
+ return "input=" + input + " cost=" + cost + "context=" + context + "boost=" + boost;
}
}
@@ -307,13 +319,18 @@ public final class Util {
* @param comparator the comparator to select the top N
*/
public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator) {
+ this(fst, topN, maxQueueDepth, comparator, new TieBreakByInputComparator<>(comparator));
+ }
+
+ public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator,
+ Comparator<FSTPath<T>> pathComparator) {
this.fst = fst;
this.bytesReader = fst.getBytesReader();
this.topN = topN;
this.maxQueueDepth = maxQueueDepth;
this.comparator = comparator;
- queue = new TreeSet<>(new TieBreakByInputComparator<>(comparator));
+ queue = new TreeSet<>(pathComparator);
}
// If back plus this arc is competitive then add to queue:
@@ -354,25 +371,29 @@ public final class Util {
IntsRefBuilder newInput = new IntsRefBuilder();
newInput.copyInts(path.input.get());
newInput.append(path.arc.label);
- final FSTPath<T> newPath = new FSTPath<>(cost, path.arc, newInput);
- queue.add(newPath);
+ queue.add(path.newPath(cost, newInput));
if (queue.size() == maxQueueDepth+1) {
queue.pollLast();
}
}
+ public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input) throws IOException {
+ addStartPaths(node, startOutput, allowEmptyString, input, 0, null);
+ }
+
/** Adds all leaving arcs, including 'finished' arc, if
* the node is final, from this node into the queue. */
- public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input) throws IOException {
+ public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input,
+ float boost, CharSequence context) throws IOException {
// De-dup NO_OUTPUT since it must be a singleton:
if (startOutput.equals(fst.outputs.getNoOutput())) {
startOutput = fst.outputs.getNoOutput();
}
- FSTPath<T> path = new FSTPath<>(startOutput, node, input);
+ FSTPath<T> path = new FSTPath<>(startOutput, node, input, boost, context);
fst.readFirstTargetArc(node, path.arc, bytesReader);
//System.out.println("add start paths");
@@ -493,10 +514,10 @@ public final class Util {
if (path.arc.label == FST.END_LABEL) {
// Add final output:
//System.out.println(" done!: " + path);
- T finalOutput = fst.outputs.add(path.cost, path.arc.output);
- if (acceptResult(path.input.get(), finalOutput)) {
+ path.cost = fst.outputs.add(path.cost, path.arc.output);
+ if (acceptResult(path)) {
//System.out.println(" add result: " + path);
- results.add(new Result<>(path.input.get(), finalOutput));
+ results.add(new Result<>(path.input.get(), path.cost));
} else {
rejectCount++;
}
@@ -510,6 +531,10 @@ public final class Util {
return new TopResults<>(rejectCount + topN <= maxQueueDepth, results);
}
+ protected boolean acceptResult(FSTPath<T> path) {
+ return acceptResult(path.input.get(), path.cost);
+ }
+
protected boolean acceptResult(IntsRef input, T output) {
return true;
}
Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java Thu May 28 07:53:09 2015
@@ -17,22 +17,10 @@ package org.apache.lucene.search.suggest
* limitations under the License.
*/
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.AnalyzerWrapper;
-import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.TokenStreamToAutomaton;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.Operations;
-import org.apache.lucene.util.automaton.Transition;
/**
* Wraps an {@link org.apache.lucene.analysis.Analyzer}
@@ -40,15 +28,15 @@ import org.apache.lucene.util.automaton.
* (e.g. preserving token separators, preserving position increments while converting
* a token stream to an automaton)
* <p>
- * Can be used to index {@link SuggestField}
- * and as a query analyzer to {@link SuggestIndexSearcher}
+ * Can be used to index {@link SuggestField} and {@link ContextSuggestField}
+ * and as a query analyzer to {@link PrefixCompletionQuery} amd {@link FuzzyCompletionQuery}
* <p>
- * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements}
- * and {@link #preserveSep}
+ * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements()}
+ * and {@link #preserveSep()}
*
* @lucene.experimental
*/
-public class CompletionAnalyzer extends AnalyzerWrapper {
+public final class CompletionAnalyzer extends AnalyzerWrapper {
/**
* Represents the separation between tokens, if
@@ -64,7 +52,7 @@ public class CompletionAnalyzer extends
*/
final static int HOLE_CHARACTER = TokenStreamToAutomaton.HOLE;
- final static int DEFAULT_MAX_GRAPH_EXPANSIONS = -1;
+ final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
final static boolean DEFAULT_PRESERVE_SEP = true;
final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
@@ -133,6 +121,22 @@ public class CompletionAnalyzer extends
this(analyzer, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, maxGraphExpansions);
}
+ /**
+ * Returns true if separation between tokens are preserved when converting
+ * the token stream to an automaton
+ */
+ public boolean preserveSep() {
+ return preserveSep;
+ }
+
+ /**
+ * Returns true if position increments are preserved when converting
+ * the token stream to an automaton
+ */
+ public boolean preservePositionIncrements() {
+ return preservePositionIncrements;
+ }
+
@Override
protected Analyzer getWrappedAnalyzer(String fieldName) {
return analyzer;
@@ -141,33 +145,7 @@ public class CompletionAnalyzer extends
@Override
protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
CompletionTokenStream tokenStream = new CompletionTokenStream(components.getTokenStream(),
- preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions);
+ preserveSep, preservePositionIncrements, maxGraphExpansions);
return new TokenStreamComponents(components.getTokenizer(), tokenStream);
}
-
- /**
- * Converts <code>key</code> to an automaton using
- * {@link #preservePositionIncrements}, {@link #preserveSep}
- * and {@link #maxGraphExpansions}
- */
- public Automaton toAutomaton(String field, CharSequence key) throws IOException {
- for (int i = 0; i < key.length(); i++) {
- switch (key.charAt(i)) {
- case HOLE_CHARACTER:
- throw new IllegalArgumentException("lookup key cannot contain HOLE character U+001E; this character is reserved");
- case SEP_LABEL:
- throw new IllegalArgumentException("lookup key cannot contain unit separator character U+001F; this character is reserved");
- default:
- break;
- }
- }
-
- try (TokenStream tokenStream = analyzer.tokenStream(field, key.toString())) {
- try(CompletionTokenStream stream = new CompletionTokenStream(tokenStream,
- preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions)) {
- return stream.toAutomaton(tokenStream);
- }
- }
- }
-
}
Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java Thu May 28 07:53:09 2015
@@ -58,7 +58,7 @@ import static org.apache.lucene.search.s
final class CompletionFieldsConsumer extends FieldsConsumer {
private final String delegatePostingsFormatName;
- private final Map<String, Long> seenFields = new HashMap<>();
+ private final Map<String, CompletionMetaData> seenFields = new HashMap<>();
private final SegmentWriteState state;
private IndexOutput dictOut;
private FieldsConsumer delegateFieldsConsumer;
@@ -98,7 +98,10 @@ final class CompletionFieldsConsumer ext
// store lookup, if needed
long filePointer = dictOut.getFilePointer();
if (termWriter.finish(dictOut)) {
- seenFields.put(field, filePointer);
+ seenFields.put(field, new CompletionMetaData(filePointer,
+ termWriter.minWeight,
+ termWriter.maxWeight,
+ termWriter.type));
}
}
}
@@ -124,10 +127,14 @@ final class CompletionFieldsConsumer ext
// write # of seen fields
indexOut.writeVInt(seenFields.size());
// write field numbers and dictOut offsets
- for (Map.Entry<String, Long> seenField : seenFields.entrySet()) {
+ for (Map.Entry<String, CompletionMetaData> seenField : seenFields.entrySet()) {
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(seenField.getKey());
indexOut.writeVInt(fieldInfo.number);
- indexOut.writeVLong(seenField.getValue());
+ CompletionMetaData metaData = seenField.getValue();
+ indexOut.writeVLong(metaData.filePointer);
+ indexOut.writeVLong(metaData.minWeight);
+ indexOut.writeVLong(metaData.maxWeight);
+ indexOut.writeByte(metaData.type);
}
CodecUtil.writeFooter(indexOut);
CodecUtil.writeFooter(dictOut);
@@ -140,17 +147,36 @@ final class CompletionFieldsConsumer ext
}
}
+ private static class CompletionMetaData {
+ private final long filePointer;
+ private final long minWeight;
+ private final long maxWeight;
+ private final byte type;
+
+ private CompletionMetaData(long filePointer, long minWeight, long maxWeight, byte type) {
+ this.filePointer = filePointer;
+ this.minWeight = minWeight;
+ this.maxWeight = maxWeight;
+ this.type = type;
+ }
+ }
+
// builds an FST based on the terms written
private static class CompletionTermWriter {
private PostingsEnum postingsEnum = null;
private int docCount = 0;
+ private long maxWeight = 0;
+ private long minWeight = Long.MAX_VALUE;
+ private byte type;
+ private boolean first;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final NRTSuggesterBuilder builder;
public CompletionTermWriter() {
builder = new NRTSuggesterBuilder();
+ first = true;
}
/**
@@ -160,6 +186,9 @@ final class CompletionFieldsConsumer ext
public boolean finish(IndexOutput output) throws IOException {
boolean stored = builder.store(output);
assert stored || docCount == 0 : "the FST is null but docCount is != 0 actual value: [" + docCount + "]";
+ if (docCount == 0) {
+ minWeight = 0;
+ }
return stored;
}
@@ -181,7 +210,17 @@ final class CompletionFieldsConsumer ext
scratch.grow(len);
scratch.setLength(len);
input.readBytes(scratch.bytes(), 0, scratch.length());
- builder.addEntry(docID, scratch.get(), input.readVLong() - 1);
+ long weight = input.readVInt() - 1;
+ maxWeight = Math.max(maxWeight, weight);
+ minWeight = Math.min(minWeight, weight);
+ byte type = input.readByte();
+ if (first) {
+ this.type = type;
+ first = false;
+ } else if (this.type != type) {
+ throw new IllegalArgumentException("single field name has mixed types");
+ }
+ builder.addEntry(docID, scratch.get(), weight);
}
docFreq++;
docCount = Math.max(docCount, docFreq + 1);
Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java Thu May 28 07:53:09 2015
@@ -30,7 +30,6 @@ import org.apache.lucene.codecs.CodecUti
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FilterLeafReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
@@ -98,9 +97,12 @@ final class CompletionFieldsProducer ext
for (int i = 0; i < numFields; i++) {
int fieldNumber = index.readVInt();
long offset = index.readVLong();
+ long minWeight = index.readVLong();
+ long maxWeight = index.readVLong();
+ byte type = index.readByte();
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNumber);
// we don't load the FST yet
- readers.put(fieldInfo.name, new CompletionsTermsReader(offset));
+ readers.put(fieldInfo.name, new CompletionsTermsReader(dictIn, offset, minWeight, maxWeight, type));
}
CodecUtil.checkFooter(index);
success = true;
@@ -161,7 +163,11 @@ final class CompletionFieldsProducer ext
@Override
public Terms terms(String field) throws IOException {
- return new CompletionTerms(delegateFieldsProducer.terms(field), readers.get(field));
+ Terms terms = delegateFieldsProducer.terms(field) ;
+ if (terms == null) {
+ return null;
+ }
+ return new CompletionTerms(terms, readers.get(field));
}
@Override
@@ -169,60 +175,4 @@ final class CompletionFieldsProducer ext
return readers.size();
}
- private class CompletionsTermsReader implements Accountable {
- private final long offset;
- private NRTSuggester suggester;
-
- public CompletionsTermsReader(long offset) throws IOException {
- assert offset >= 0l && offset < dictIn.length();
- this.offset = offset;
- }
-
- public synchronized NRTSuggester suggester() throws IOException {
- if (suggester == null) {
- try (IndexInput dictClone = dictIn.clone()) { // let multiple fields load concurrently
- dictClone.seek(offset);
- suggester = NRTSuggester.load(dictClone);
- }
- }
- return suggester;
- }
-
- @Override
- public long ramBytesUsed() {
- return (suggester != null) ? suggester.ramBytesUsed() : 0;
- }
-
- @Override
- public Collection<Accountable> getChildResources() {
- return Collections.emptyList();
- }
- }
-
- /**
- * Thin wrapper over {@link org.apache.lucene.index.Terms} with
- * a {@link NRTSuggester}
- */
- public static class CompletionTerms extends FilterLeafReader.FilterTerms {
-
- private final CompletionsTermsReader reader;
-
- public CompletionTerms(Terms in, CompletionsTermsReader reader) {
- super(in);
- this.reader = reader;
- }
-
- /**
- * Returns a {@link NRTSuggester} for the field
- * or <code>null</code> if no FST
- * was indexed for this field
- */
- public NRTSuggester suggester() throws IOException {
- if (reader == null) {
- return null;
- }
- return reader.suggester();
- }
- }
-
}
Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java Thu May 28 07:53:09 2015
@@ -68,9 +68,12 @@ import org.apache.lucene.util.fst.FST;
* <li>CompletionIndex (.cmp) --> Header, NumSuggestFields, Entry<sup>NumSuggestFields</sup>, Footer</li>
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>NumSuggestFields --> {@link DataOutput#writeVInt Uint32}</li>
- * <li>Entry --> FieldNumber, CompletionDictionaryOffset</li>
+ * <li>Entry --> FieldNumber, CompletionDictionaryOffset, MinWeight, MaxWeight, Type</li>
* <li>FieldNumber --> {@link DataOutput#writeVInt Uint32}</li>
* <li>CompletionDictionaryOffset --> {@link DataOutput#writeVLong Uint64}</li>
+ * <li>MinWeight --> {@link DataOutput#writeVLong Uint64}</li>
+ * <li>MaxWeight --> {@link DataOutput#writeVLong Uint64}</li>
+ * <li>Type --> {@link DataOutput#writeByte Byte}</li>
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* <p>Notes:</p>
@@ -80,6 +83,8 @@ import org.apache.lucene.util.fst.FST;
* <li>NumSuggestFields is the number of suggest fields indexed</li>
* <li>FieldNumber is the fields number from {@link FieldInfos}. (.fnm)</li>
* <li>CompletionDictionaryOffset is the file offset of a field's FST in CompletionDictionary (.lkp)</li>
+ * <li>MinWeight and MaxWeight are the global minimum and maximum weight for the field</li>
+ * <li>Type indicates if the suggester has context or not</li>
* </ul>
*
* @lucene.experimental
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,170 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+
+import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.HOLE_CHARACTER;
+import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.SEP_LABEL;
+
+/**
+ * Abstract {@link Query} that match documents containing terms with a specified prefix
+ * filtered by {@link Filter}. This should be used to query against any {@link SuggestField}s
+ * or {@link ContextSuggestField}s of documents.
+ * <p>
+ * Use {@link SuggestIndexSearcher#suggest(CompletionQuery, int)} to execute any query
+ * that provides a concrete implementation of this query. Example below shows using this query
+ * to retrieve the top 5 documents.
+ *
+ * <pre class="prettyprint">
+ * SuggestIndexSearcher searcher = new SuggestIndexSearcher(reader);
+ * TopSuggestDocs suggestDocs = searcher.suggest(query, 5);
+ * </pre>
+ * This query rewrites to an appropriate {@link CompletionQuery} depending on the
+ * type ({@link SuggestField} or {@link ContextSuggestField}) of the field the query is run against.
+ *
+ * @lucene.experimental
+ */
+public abstract class CompletionQuery extends Query {
+
+ /**
+ * Term to query against
+ */
+ private final Term term;
+
+ /**
+ * Filter for document scoping
+ */
+ private final Filter filter;
+
+ /**
+ * Creates a base Completion query against a <code>term</code>
+ * with a <code>filter</code> to scope the documents
+ */
+ protected CompletionQuery(Term term, Filter filter) {
+ validate(term.text());
+ this.term = term;
+ this.filter = filter;
+ }
+
+ /**
+ * Returns the filter for the query, used to
+ * suggest completions on a subset of indexed documents
+ */
+ public Filter getFilter() {
+ return filter;
+ }
+
+ /**
+ * Returns the field name this query should
+ * be run against
+ */
+ public String getField() {
+ return term.field();
+ }
+
+ /**
+ * Returns the term to be queried against
+ */
+ public Term getTerm() {
+ return term;
+ }
+
+ @Override
+ public Query rewrite(IndexReader reader) throws IOException {
+ byte type = 0;
+ boolean first = true;
+ Terms terms;
+ for (LeafReaderContext context : reader.leaves()) {
+ LeafReader leafReader = context.reader();
+ try {
+ if ((terms = leafReader.terms(getField())) == null) {
+ continue;
+ }
+ } catch (IOException e) {
+ continue;
+ }
+ if (terms instanceof CompletionTerms) {
+ CompletionTerms completionTerms = (CompletionTerms) terms;
+ byte t = completionTerms.getType();
+ if (first) {
+ type = t;
+ first = false;
+ } else if (type != t) {
+ throw new IllegalStateException(getField() + " has values of multiple types");
+ }
+ }
+ }
+
+ if (first == false) {
+ if (this instanceof ContextQuery) {
+ if (type == SuggestField.TYPE) {
+ throw new IllegalStateException(this.getClass().getSimpleName()
+ + " can not be executed against a non context-enabled SuggestField: "
+ + getField());
+ }
+ } else {
+ if (type == ContextSuggestField.TYPE) {
+ return new ContextQuery(this);
+ }
+ }
+ }
+ return this;
+ }
+
+ @Override
+ public String toString(String field) {
+ StringBuilder buffer = new StringBuilder();
+ if (!term.field().equals(field)) {
+ buffer.append(term.field());
+ buffer.append(":");
+ }
+ buffer.append(term.text());
+ buffer.append('*');
+ if (filter != null) {
+ buffer.append(",");
+ buffer.append("filter");
+ buffer.append(":");
+ buffer.append(filter.toString(field));
+ }
+ return buffer.toString();
+ }
+
+ private void validate(String termText) {
+ for (int i = 0; i < termText.length(); i++) {
+ switch (termText.charAt(i)) {
+ case HOLE_CHARACTER:
+ throw new IllegalArgumentException(
+ "Term text cannot contain HOLE character U+001E; this character is reserved");
+ case SEP_LABEL:
+ throw new IllegalArgumentException(
+ "Term text cannot contain unit separator character U+001F; this character is reserved");
+ default:
+ break;
+ }
+ }
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java Thu May 28 07:53:09 2015
@@ -0,0 +1,103 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.automaton.Automaton;
+
+/**
+ * Expert: Responsible for executing the query against an
+ * appropriate suggester and collecting the results
+ * via a collector.
+ *
+ * {@link #score(LeafCollector, int, int)} is called
+ * for each leaf reader.
+ *
+ * {@link #accept(int)} and {@link #score(float, float)}
+ * is called for every matched completion (i.e. document)
+ *
+ * @lucene.experimental
+ */
+public class CompletionScorer extends BulkScorer {
+ private final NRTSuggester suggester;
+ private final Bits acceptDocs;
+
+ // values accessed by suggester
+ /** weight that created this scorer */
+ protected final CompletionWeight weight;
+ final LeafReader reader;
+ final boolean filtered;
+ final Automaton automaton;
+
+ /**
+ * Creates a scorer for a field-specific <code>suggester</code> scoped by <code>acceptDocs</code>
+ */
+ protected CompletionScorer(final CompletionWeight weight, final NRTSuggester suggester,
+ final LeafReader reader, final Bits acceptDocs,
+ final boolean filtered, final Automaton automaton) throws IOException {
+ this.weight = weight;
+ this.suggester = suggester;
+ this.reader = reader;
+ this.automaton = automaton;
+ this.filtered = filtered;
+ this.acceptDocs = acceptDocs;
+ }
+
+ @Override
+ public int score(LeafCollector collector, int min, int max) throws IOException {
+ if (!(collector instanceof TopSuggestDocsCollector)) {
+ throw new IllegalArgumentException("collector is not of type TopSuggestDocsCollector");
+ }
+ suggester.lookup(this, ((TopSuggestDocsCollector) collector));
+ return max;
+ }
+
+ @Override
+ public long cost() {
+ return 0;
+ }
+
+ /**
+ * Returns true if a document with <code>docID</code> is accepted,
+ * false if the docID maps to a deleted
+ * document or has been filtered out
+ */
+ public final boolean accept(int docID) {
+ return acceptDocs == null || acceptDocs.get(docID);
+ }
+
+ /**
+ * Returns the score for a matched completion
+ * based on the query time boost and the
+ * index time weight.
+ */
+ public float score(float weight, float boost) {
+ if (boost == 0f) {
+ return weight;
+ }
+ if (weight == 0f) {
+ return boost;
+ }
+ return weight * boost;
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java Thu May 28 07:53:09 2015
@@ -0,0 +1,74 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.Terms;
+
+/**
+ * Wrapped {@link org.apache.lucene.index.Terms}
+ * used by {@link SuggestField} and {@link ContextSuggestField}
+ * to access corresponding suggester and their attributes
+ *
+ * @lucene.experimental
+ */
+public final class CompletionTerms extends FilterLeafReader.FilterTerms {
+
+ private final CompletionsTermsReader reader;
+
+ /**
+ * Creates a completionTerms based on {@link CompletionsTermsReader}
+ */
+ CompletionTerms(Terms in, CompletionsTermsReader reader) {
+ super(in);
+ this.reader = reader;
+ }
+
+ /**
+ * Returns the type of FST, either {@link SuggestField#TYPE} or
+ * {@link ContextSuggestField#TYPE}
+ */
+ public byte getType() {
+ return (reader != null) ? reader.type : SuggestField.TYPE;
+ }
+
+ /**
+ * Returns the minimum weight of all entries in the weighted FST
+ */
+ public long getMinWeight() {
+ return (reader != null) ? reader.minWeight : 0;
+ }
+
+ /**
+ * Returns the maximum weight of all entries in the weighted FST
+ */
+ public long getMaxWeight() {
+ return (reader != null) ? reader.maxWeight : 0;
+ }
+
+ /**
+ * Returns a {@link NRTSuggester} for the field
+ * or <code>null</code> if no FST
+ * was indexed for this field
+ */
+ public NRTSuggester suggester() throws IOException {
+ return (reader != null) ? reader.suggester() : null;
+ }
+}
Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java Thu May 28 07:53:09 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.search.suggest
*/
import java.io.IOException;
-import java.util.HashSet;
+import java.util.BitSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.Set;
@@ -52,18 +52,18 @@ import static org.apache.lucene.search.s
* The token stream uses a {@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute} to store
* a completion's payload (see {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)})
*
+ * @lucene.experimental
*/
-final class CompletionTokenStream extends TokenStream {
+public final class CompletionTokenStream extends TokenStream {
private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
private final TokenStream input;
- private final boolean preserveSep;
- private final boolean preservePositionIncrements;
- private final int sepLabel;
- private final int maxGraphExpansions;
+ final boolean preserveSep;
+ final boolean preservePositionIncrements;
+ final int maxGraphExpansions;
private BytesRef payload;
private Iterator<IntsRef> finiteStrings;
@@ -77,29 +77,20 @@ final class CompletionTokenStream extend
* The token stream <code>input</code> is converted to an automaton
* with the default settings of {@link org.apache.lucene.search.suggest.document.CompletionAnalyzer}
*/
- public CompletionTokenStream(TokenStream input) {
- this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, SEP_LABEL, DEFAULT_MAX_GRAPH_EXPANSIONS);
+ CompletionTokenStream(TokenStream input) {
+ this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
}
- CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int sepLabel, int maxGraphExpansions) {
+ CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
// Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
// the input stream entirely in toFiniteStrings(input)
this.input = input;
this.preserveSep = preserveSep;
this.preservePositionIncrements = preservePositionIncrements;
- this.sepLabel = sepLabel;
this.maxGraphExpansions = maxGraphExpansions;
}
/**
- * Returns a separator label that is reserved for the payload
- * in {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)}
- */
- public int sepLabel() {
- return sepLabel;
- }
-
- /**
* Sets a payload available throughout successive token stream enumeration
*/
public void setPayload(BytesRef payload) {
@@ -111,7 +102,7 @@ final class CompletionTokenStream extend
clearAttributes();
if (finiteStrings == null) {
//TODO: make this return a Iterator<IntsRef> instead?
- Automaton automaton = toAutomaton(input);
+ Automaton automaton = toAutomaton();
Set<IntsRef> strings = Operations.getFiniteStrings(automaton, maxGraphExpansions);
posInc = strings.size();
@@ -165,9 +156,17 @@ final class CompletionTokenStream extend
}
/**
- * Converts <code>tokenStream</code> to an automaton
+ * Converts the token stream to an automaton,
+ * treating the transition labels as utf-8
+ */
+ public Automaton toAutomaton() throws IOException {
+ return toAutomaton(false);
+ }
+
+ /**
+ * Converts the tokenStream to an automaton
*/
- public Automaton toAutomaton(TokenStream tokenStream) throws IOException {
+ public Automaton toAutomaton(boolean unicodeAware) throws IOException {
// TODO refactor this
// maybe we could hook up a modified automaton from TermAutomatonQuery here?
Automaton automaton = null;
@@ -184,10 +183,11 @@ final class CompletionTokenStream extend
tsta = new TokenStreamToAutomaton();
}
tsta.setPreservePositionIncrements(preservePositionIncrements);
+ tsta.setUnicodeArcs(unicodeAware);
- automaton = tsta.toAutomaton(tokenStream);
+ automaton = tsta.toAutomaton(input);
} finally {
- IOUtils.closeWhileHandlingException(tokenStream);
+ IOUtils.closeWhileHandlingException(input);
}
// TODO: we can optimize this somewhat by determinizing
@@ -281,11 +281,12 @@ final class CompletionTokenStream extend
}
private static int[] topoSortStates(Automaton a) {
- int[] states = new int[a.getNumStates()];
- final Set<Integer> visited = new HashSet<>();
+ int numStates = a.getNumStates();
+ int[] states = new int[numStates];
+ final BitSet visited = new BitSet(numStates);
final LinkedList<Integer> worklist = new LinkedList<>();
worklist.add(0);
- visited.add(0);
+ visited.set(0);
int upto = 0;
states[upto] = 0;
upto++;
@@ -293,10 +294,10 @@ final class CompletionTokenStream extend
while (worklist.size() > 0) {
int s = worklist.removeFirst();
int count = a.initTransition(s, t);
- for (int i = 0; i < count; i++) {
+ for (int i=0;i<count;i++) {
a.getNextTransition(t);
- if (!visited.contains(t.dest)) {
- visited.add(t.dest);
+ if (!visited.get(t.dest)) {
+ visited.set(t.dest);
worklist.add(t.dest);
states[upto++] = t.dest;
}
@@ -305,21 +306,37 @@ final class CompletionTokenStream extend
return states;
}
- public interface ByteTermAttribute extends TermToBytesRefAttribute {
+ /**
+ * Attribute providing access to the term builder and UTF-16 conversion
+ */
+ private interface ByteTermAttribute extends TermToBytesRefAttribute {
// marker interface
/**
- * Return the builder from which the term is derived.
+ * Returns the builder from which the term is derived.
*/
- public BytesRefBuilder builder();
+ BytesRefBuilder builder();
- public CharSequence toUTF16();
+ /**
+ * Returns the term represented as UTF-16
+ */
+ CharSequence toUTF16();
}
+ /**
+ * Custom attribute implementation for completion token stream
+ */
public static final class ByteTermAttributeImpl extends AttributeImpl implements ByteTermAttribute, TermToBytesRefAttribute {
private final BytesRefBuilder bytes = new BytesRefBuilder();
private CharsRefBuilder charsRef;
+ /**
+ * Sole constructor
+ * no-op
+ */
+ public ByteTermAttributeImpl() {
+ }
+
@Override
public void fillBytesRef() {
// does nothing - we change in place
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java Thu May 28 07:53:09 2015
@@ -0,0 +1,209 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.Automaton;
+
+/**
+ * Expert: the Weight for CompletionQuery, used to
+ * score and explain these queries.
+ *
+ * Subclasses can override {@link #setNextMatch(IntsRef)},
+ * {@link #boost()} and {@link #context()}
+ * to calculate the boost and extract the context of
+ * a matched path prefix.
+ *
+ * @lucene.experimental
+ */
+public class CompletionWeight extends Weight {
+ private final CompletionQuery completionQuery;
+ private final Automaton automaton;
+ private final long maxWeight;
+ private final long minWeight;
+
+ /**
+ * Creates a weight for <code>query</code> with an <code>automaton</code>,
+ * using the <code>reader</code> for index stats
+ */
+ public CompletionWeight(final IndexReader reader, final CompletionQuery query,
+ final Automaton automaton) throws IOException {
+ super(query);
+ this.completionQuery = query;
+ this.automaton = automaton;
+ this.minWeight = minWeight(query.getField(), reader);
+ this.maxWeight = maxWeight(query.getField(), reader);
+ }
+
+ /**
+ * Returns the automaton specified
+ * by the {@link CompletionQuery}
+ *
+ * @return query automaton
+ */
+ public Automaton getAutomaton() {
+ return automaton;
+ }
+
+ @Override
+ public BulkScorer bulkScorer(final LeafReaderContext context, Bits acceptDocs) throws IOException {
+ final LeafReader reader = context.reader();
+ final Terms terms;
+ final NRTSuggester suggester;
+ if ((terms = reader.terms(completionQuery.getField())) == null) {
+ return null;
+ }
+ if (terms instanceof CompletionTerms) {
+ CompletionTerms completionTerms = (CompletionTerms) terms;
+ if ((suggester = completionTerms.suggester()) == null) {
+ // a segment can have a null suggester
+ // i.e. no FST was built
+ return null;
+ }
+ } else {
+ throw new IllegalArgumentException(completionQuery.getField() + " is not a SuggestField");
+ }
+
+ DocIdSet docIdSet = null;
+ Filter filter = completionQuery.getFilter();
+ if (filter != null) {
+ docIdSet = filter.getDocIdSet(context, acceptDocs);
+ if (docIdSet == null || docIdSet.iterator() == null) {
+ // filter matches no docs in current leave
+ return null;
+ } else if (docIdSet.bits() == null) {
+ throw new IllegalArgumentException("DocIDSet does not provide random access interface");
+ }
+ }
+ Bits acceptDocBits = (docIdSet != null) ? docIdSet.bits() : acceptDocs;
+ return new CompletionScorer(this, suggester, reader, acceptDocBits, filter != null, automaton);
+ }
+
+ /**
+ * Set for every partial path in the index that matched the query
+ * automaton.
+ *
+ * Subclasses should override {@link #boost()} and {@link #context()}
+ * to return an appropriate value with respect to the current pathPrefix.
+ *
+ * @param pathPrefix the prefix of a matched path
+ */
+ protected void setNextMatch(IntsRef pathPrefix) {
+ }
+
+ /**
+ * Returns the boost of the partial path set by {@link #setNextMatch(IntsRef)}
+ *
+ * @return suggestion query-time boost
+ */
+ protected float boost() {
+ return 0;
+ }
+
+ /**
+ * Returns the context of the partial path set by {@link #setNextMatch(IntsRef)}
+ *
+ * @return suggestion context
+ */
+ protected CharSequence context() {
+ return null;
+ }
+
+ private static long minWeight(String field, IndexReader reader) {
+ long minWeight = Long.MAX_VALUE;
+ Terms terms;
+ for (LeafReaderContext context : reader.leaves()) {
+ LeafReader leafReader = context.reader();
+ try {
+ if ((terms = leafReader.terms(field)) == null) {
+ continue;
+ }
+ } catch (IOException e) {
+ continue;
+ }
+ if (terms instanceof CompletionTerms) {
+ CompletionTerms completionTerms = (CompletionTerms) terms;
+ minWeight = Math.min(completionTerms.getMinWeight(), minWeight);
+ }
+ }
+ if (minWeight == Long.MAX_VALUE) {
+ minWeight = 0;
+ }
+ return minWeight;
+ }
+
+ private static long maxWeight(String field, IndexReader reader) {
+ long maxWeight = 0;
+ Terms terms;
+ for (LeafReaderContext context : reader.leaves()) {
+ LeafReader leafReader = context.reader();
+ try {
+ if ((terms = leafReader.terms(field)) == null) {
+ continue;
+ }
+ } catch (IOException e) {
+ continue;
+ }
+ if (terms instanceof CompletionTerms) {
+ CompletionTerms completionTerms = (CompletionTerms) terms;
+ maxWeight = Math.max(completionTerms.getMaxWeight(), maxWeight);
+ }
+ }
+ return maxWeight;
+ }
+
+ @Override
+ public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void extractTerms(Set<Term> terms) {
+ // no-op
+ }
+
+ @Override
+ public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+ //TODO
+ return null;
+ }
+
+ @Override
+ public float getValueForNormalization() throws IOException {
+ return 0;
+ }
+
+ @Override
+ public void normalize(float norm, float topLevelBoost) {
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java Thu May 28 07:53:09 2015
@@ -0,0 +1,82 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+
+/**
+ * Holder for suggester and field-level info
+ * for a suggest field
+ *
+ * @lucene.experimental
+ */
+public final class CompletionsTermsReader implements Accountable {
+ /** Minimum entry weight for the suggester */
+ public final long minWeight;
+ /** Maximum entry weight for the suggester */
+ public final long maxWeight;
+ /** type of suggester (context-enabled or not) */
+ public final byte type;
+ private final IndexInput dictIn;
+ private final long offset;
+
+ private NRTSuggester suggester;
+
+ /**
+ * Creates a CompletionTermsReader to load a field-specific suggester
+ * from the index <code>dictIn</code> with <code>offset</code>
+ */
+ CompletionsTermsReader(IndexInput dictIn, long offset, long minWeight, long maxWeight, byte type) throws IOException {
+ assert minWeight <= maxWeight;
+ assert offset >= 0l && offset < dictIn.length();
+ this.dictIn = dictIn;
+ this.offset = offset;
+ this.minWeight = minWeight;
+ this.maxWeight = maxWeight;
+ this.type = type;
+ }
+
+ /**
+ * Returns the suggester for a field, if not loaded already, loads
+ * the appropriate suggester from CompletionDictionary
+ */
+ public synchronized NRTSuggester suggester() throws IOException {
+ if (suggester == null) {
+ try (IndexInput dictClone = dictIn.clone()) { // let multiple fields load concurrently
+ dictClone.seek(offset);
+ suggester = NRTSuggester.load(dictClone);
+ }
+ }
+ return suggester;
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return (suggester != null) ? suggester.ramBytesUsed() : 0;
+ }
+
+ @Override
+ public Collection<Accountable> getChildResources() {
+ return Collections.emptyList();
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,307 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeSet;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * A {@link CompletionQuery} that match documents specified by
+ * a wrapped {@link CompletionQuery} supporting boosting and/or filtering
+ * by specified contexts.
+ * <p>
+ * Use this query against {@link ContextSuggestField}
+ * <p>
+ * Example of using a {@link CompletionQuery} with boosted
+ * contexts:
+ * <pre class="prettyprint">
+ * CompletionQuery completionQuery = ...;
+ * ContextQuery query = new ContextQuery(completionQuery);
+ * query.addContext("context1", 2);
+ * query.addContext("context2", 1);
+ * </pre>
+ * <p>
+ * NOTE:
+ * <ul>
+ * <li>
+ * This query can be constructed with
+ * {@link PrefixCompletionQuery}, {@link RegexCompletionQuery}
+ * or {@link FuzzyCompletionQuery} query.
+ * </li>
+ * <li>
+ * To suggest across all contexts with the same boost,
+ * use '*' as the context in {@link #addContext(CharSequence)})}.
+ * This can be combined with specific contexts with different boosts.
+ * </li>
+ * <li>
+ * To apply the same boost to multiple contexts sharing the same prefix,
+ * Use {@link #addContext(CharSequence, float, boolean)} with the common
+ * context prefix, boost and set <code>exact</code> to false.
+ * <li>
+ * Using this query against a {@link SuggestField} (not context enabled),
+ * would yield results ignoring any context filtering/boosting
+ * </li>
+ * </ul>
+ *
+ * @lucene.experimental
+ */
+public class ContextQuery extends CompletionQuery {
+ private Map<CharSequence, ContextMetaData> contexts;
+ /** Inner completion query */
+ protected CompletionQuery query;
+
+ /**
+ * Constructs a context completion query that matches
+ * documents specified by <code>query</code>.
+ * <p>
+ * Use {@link #addContext(CharSequence, float, boolean)}
+ * to add context(s) with boost
+ */
+ public ContextQuery(CompletionQuery query) {
+ super(query.getTerm(), query.getFilter());
+ if (query instanceof ContextQuery) {
+ throw new IllegalArgumentException("'query' parameter must not be of type "
+ + this.getClass().getSimpleName());
+ }
+ this.query = query;
+ contexts = new HashMap<>();
+ }
+
+ /**
+ * Adds an exact context with default boost of 1
+ */
+ public void addContext(CharSequence context) {
+ addContext(context, 1f, true);
+ }
+
+ /**
+ * Adds an exact context with boost
+ */
+ public void addContext(CharSequence context, float boost) {
+ addContext(context, boost, true);
+ }
+
+ /**
+ * Adds a context with boost, set <code>exact</code> to false
+ * if the context is a prefix of any indexed contexts
+ */
+ public void addContext(CharSequence context, float boost, boolean exact) {
+ if (boost < 0f) {
+ throw new IllegalArgumentException("'boost' must be >= 0");
+ }
+ for (int i = 0; i < context.length(); i++) {
+ if (ContextSuggestField.CONTEXT_SEPARATOR == context.charAt(i)) {
+ throw new IllegalArgumentException("Illegal value [" + context + "] UTF-16 codepoint [0x"
+ + Integer.toHexString((int) context.charAt(i))+ "] at position " + i + " is a reserved character");
+ }
+ }
+ contexts.put(context, new ContextMetaData(boost, exact));
+ }
+
+ @Override
+ public String toString(String field) {
+ StringBuilder buffer = new StringBuilder();
+ for (CharSequence context : contexts.keySet()) {
+ if (buffer.length() != 0) {
+ buffer.append(",");
+ } else {
+ buffer.append("contexts");
+ buffer.append(":[");
+ }
+ buffer.append(context);
+ ContextMetaData metaData = contexts.get(context);
+ if (metaData.exact == false) {
+ buffer.append("*");
+ }
+ if (metaData.boost != 0) {
+ buffer.append("^");
+ buffer.append(Float.toString(metaData.boost));
+ }
+ }
+ if (buffer.length() != 0) {
+ buffer.append("]");
+ buffer.append(",");
+ }
+ return buffer.toString() + query.toString(field);
+ }
+
+ @Override
+ public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+ IntsRefBuilder scratch = new IntsRefBuilder();
+ final Map<IntsRef, Float> contextMap = new HashMap<>(contexts.size());
+ final TreeSet<Integer> contextLengths = new TreeSet<>();
+ final CompletionWeight innerWeight = ((CompletionWeight) query.createWeight(searcher, needsScores));
+ Automaton contextsAutomaton = null;
+ Automaton gap = Automata.makeChar(ContextSuggestField.CONTEXT_SEPARATOR);
+ // if separators are preserved the fst contains a SEP_LABEL
+ // behind each gap. To have a matching automaton, we need to
+ // include the SEP_LABEL in the query as well
+ gap = Operations.concatenate(gap, Operations.optional(Automata.makeChar(CompletionAnalyzer.SEP_LABEL)));
+ final Automaton prefixAutomaton = Operations.concatenate(gap, innerWeight.getAutomaton());
+ final Automaton matchAllAutomaton = new RegExp(".*").toAutomaton();
+ for (Map.Entry<CharSequence, ContextMetaData> entry : contexts.entrySet()) {
+ Automaton contextAutomaton;
+ if (entry.getKey().equals("*")) {
+ contextAutomaton = Operations.concatenate(matchAllAutomaton, prefixAutomaton);
+ } else {
+ BytesRef ref = new BytesRef(entry.getKey());
+ ContextMetaData contextMetaData = entry.getValue();
+ contextMap.put(IntsRef.deepCopyOf(Util.toIntsRef(ref, scratch)), contextMetaData.boost);
+ contextLengths.add(scratch.length());
+ contextAutomaton = Automata.makeString(entry.getKey().toString());
+ if (contextMetaData.exact) {
+ contextAutomaton = Operations.concatenate(contextAutomaton, prefixAutomaton);
+ } else {
+ contextAutomaton = Operations.concatenate(Arrays.asList(contextAutomaton,
+ matchAllAutomaton,
+ prefixAutomaton));
+ }
+ }
+ if (contextsAutomaton == null) {
+ contextsAutomaton = contextAutomaton;
+ } else {
+ contextsAutomaton = Operations.union(contextsAutomaton, contextAutomaton);
+ }
+ }
+ if (contexts.size() == 0) {
+ addContext("*");
+ contextsAutomaton = Operations.concatenate(matchAllAutomaton, prefixAutomaton);
+ }
+ contextsAutomaton = Operations.determinize(contextsAutomaton, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
+ int[] contextLengthArray = new int[contextLengths.size()];
+ final Iterator<Integer> iterator = contextLengths.descendingIterator();
+ for (int i = 0; iterator.hasNext(); i++) {
+ contextLengthArray[i] = iterator.next();
+ }
+ return new ContextCompletionWeight(searcher.getIndexReader(), this, contextsAutomaton,
+ innerWeight, contextMap, contextLengthArray);
+ }
+
+ private static class ContextMetaData {
+ private final float boost;
+ private final boolean exact;
+
+ private ContextMetaData(float boost, boolean exact) {
+ this.boost = boost;
+ this.exact = exact;
+ }
+ }
+
+ private class ContextCompletionWeight extends CompletionWeight {
+
+ private final Map<IntsRef, Float> contextMap;
+ private final int[] contextLengths;
+ private final CompletionWeight innerWeight;
+ private final BytesRefBuilder scratch = new BytesRefBuilder();
+
+ private float currentBoost;
+ private CharSequence currentContext;
+
+ public ContextCompletionWeight(IndexReader reader, CompletionQuery query,
+ Automaton automaton, CompletionWeight innerWeight,
+ Map<IntsRef, Float> contextMap,
+ int[] contextLengths) throws IOException {
+ super(reader, query, automaton);
+ this.contextMap = contextMap;
+ this.contextLengths = contextLengths;
+ this.innerWeight = innerWeight;
+ }
+
+ @Override
+ protected void setNextMatch(IntsRef pathPrefix) {
+ IntsRef ref = pathPrefix.clone();
+
+ // check if the pathPrefix matches any
+ // defined context, longer context first
+ for (int contextLength : contextLengths) {
+ if (contextLength > pathPrefix.length) {
+ continue;
+ }
+ ref.length = contextLength;
+ if (contextMap.containsKey(ref)) {
+ currentBoost = contextMap.get(ref);
+ ref.length = pathPrefix.length;
+ ref.offset = contextLength;
+ while (ref.ints[ref.offset] != ContextSuggestField.CONTEXT_SEPARATOR) {
+ ref.offset++;
+ assert ref.offset < ref.length;
+ }
+ assert ref.ints[ref.offset] == ContextSuggestField.CONTEXT_SEPARATOR : "expected CONTEXT_SEPARATOR at offset=" + ref.offset;
+ if (ref.offset > pathPrefix.offset) {
+ currentContext = Util.toBytesRef(new IntsRef(pathPrefix.ints, pathPrefix.offset, ref.offset), scratch).utf8ToString();
+ } else {
+ currentContext = null;
+ }
+ ref.offset++;
+ if (ref.ints[ref.offset] == CompletionAnalyzer.SEP_LABEL) {
+ ref.offset++;
+ }
+ innerWeight.setNextMatch(ref);
+ return;
+ }
+ }
+ // unknown context
+ ref.length = pathPrefix.length;
+ currentBoost = contexts.get("*").boost;
+ for (int i = pathPrefix.offset; i < pathPrefix.length; i++) {
+ if (pathPrefix.ints[i] == ContextSuggestField.CONTEXT_SEPARATOR) {
+ if (i > pathPrefix.offset) {
+ currentContext = Util.toBytesRef(new IntsRef(pathPrefix.ints, pathPrefix.offset, i), scratch).utf8ToString();
+ } else {
+ currentContext = null;
+ }
+ ref.offset = ++i;
+ assert ref.offset < ref.length : "input should not end with the context separator";
+ if (pathPrefix.ints[i] == CompletionAnalyzer.SEP_LABEL) {
+ ref.offset++;
+ assert ref.offset < ref.length : "input should not end with a context separator followed by SEP_LABEL";
+ }
+ ref.length -= ref.offset;
+ innerWeight.setNextMatch(ref);
+ }
+ }
+ }
+
+ @Override
+ protected CharSequence context() {
+ return currentContext;
+ }
+
+ @Override
+ protected float boost() {
+ return currentBoost + innerWeight.boost();
+ }
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java Thu May 28 07:53:09 2015
@@ -0,0 +1,167 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/**
+ * {@link SuggestField} which additionally takes in a set of
+ * contexts. Example usage of adding a suggestion with contexts is as follows:
+ *
+ * <pre class="prettyprint">
+ * document.add(
+ * new ContextSuggestField(name, "suggestion", Arrays.asList("context1", "context2"), 4));
+ * </pre>
+ *
+ * Use {@link ContextQuery} to boost and/or filter suggestions
+ * at query-time. Use {@link PrefixCompletionQuery}, {@link RegexCompletionQuery}
+ * or {@link FuzzyCompletionQuery} if context boost/filtering
+ * are not needed.
+ *
+ * @lucene.experimental
+ */
+public class ContextSuggestField extends SuggestField {
+
+ /**
+ * Separator used between context value and the suggest field value
+ */
+ public static final int CONTEXT_SEPARATOR = '\u001D';
+ static final byte TYPE = 1;
+
+ private final Set<CharSequence> contexts;
+
+ /**
+ * Creates a context-enabled suggest field
+ *
+ * @param name field name
+ * @param contexts associated contexts
+ * @param value field value to get suggestion on
+ * @param weight field weight
+ *
+ * @throws IllegalArgumentException if either the name or value is null,
+ * if value is an empty string, if the weight is negative, if value or
+ * contexts contains any reserved characters
+ */
+ public ContextSuggestField(String name, Collection<CharSequence> contexts, String value, int weight) {
+ super(name, value, weight);
+ validate(value);
+ if (contexts != null) {
+ for (CharSequence context : contexts) {
+ validate(context);
+ }
+ this.contexts = new HashSet<>(contexts);
+ } else {
+ this.contexts = new HashSet<>();
+ }
+ }
+
+ @Override
+ protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
+ CompletionTokenStream completionTokenStream;
+ if (stream instanceof CompletionTokenStream) {
+ completionTokenStream = (CompletionTokenStream) stream;
+ completionTokenStream = new CompletionTokenStream(
+ new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts),
+ completionTokenStream.preserveSep,
+ completionTokenStream.preservePositionIncrements,
+ completionTokenStream.maxGraphExpansions);
+ } else {
+ completionTokenStream = new CompletionTokenStream(
+ new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts));
+ }
+ return completionTokenStream;
+ }
+
+ @Override
+ protected byte type() {
+ return TYPE;
+ }
+
+ /**
+ * The {@link PrefixTokenFilter} wraps a {@link TokenStream} and adds a set
+ * prefixes ahead. The position attribute will not be incremented for the prefixes.
+ */
+ private static final class PrefixTokenFilter extends TokenFilter {
+
+ private final char separator;
+ private final CharTermAttribute termAttr = addAttribute(CharTermAttribute.class);
+ private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
+ private final Iterable<CharSequence> prefixes;
+
+ private Iterator<CharSequence> currentPrefix;
+
+ /**
+ * Create a new {@link PrefixTokenFilter}
+ *
+ * @param input {@link TokenStream} to wrap
+ * @param separator Character used separate prefixes from other tokens
+ * @param prefixes {@link Iterable} of {@link CharSequence} which keeps all prefixes
+ */
+ public PrefixTokenFilter(TokenStream input, char separator, Iterable<CharSequence> prefixes) {
+ super(input);
+ this.prefixes = prefixes;
+ this.currentPrefix = null;
+ this.separator = separator;
+ }
+
+ @Override
+ public boolean incrementToken() throws IOException {
+ if (currentPrefix != null) {
+ if (!currentPrefix.hasNext()) {
+ return input.incrementToken();
+ } else {
+ posAttr.setPositionIncrement(0);
+ }
+ } else {
+ currentPrefix = prefixes.iterator();
+ termAttr.setEmpty();
+ posAttr.setPositionIncrement(1);
+ }
+ termAttr.setEmpty();
+ if (currentPrefix.hasNext()) {
+ termAttr.append(currentPrefix.next());
+ }
+ termAttr.append(separator);
+ return true;
+ }
+
+ @Override
+ public void reset() throws IOException {
+ super.reset();
+ currentPrefix = null;
+ }
+ }
+
+ private void validate(final CharSequence value) {
+ for (int i = 0; i < value.length(); i++) {
+ if (CONTEXT_SEPARATOR == value.charAt(i)) {
+ throw new IllegalArgumentException("Illegal value [" + value + "] UTF-16 codepoint [0x"
+ + Integer.toHexString((int) value.charAt(i))+ "] at position " + i + " is a reserved character");
+ }
+ }
+ }
+}
Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,254 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
+
+/**
+ * A {@link CompletionQuery} that match documents containing terms
+ * within an edit distance of the specified prefix.
+ * <p>
+ * This query boost documents relative to how similar the indexed terms are to the
+ * provided prefix.
+ * <p>
+ * Example usage of querying an analyzed prefix within an edit distance of 1 of 'subg'
+ * against a field 'suggest_field' is as follows:
+ *
+ * <pre class="prettyprint">
+ * CompletionQuery query = new FuzzyCompletionQuery(analyzer, new Term("suggest_field", "subg"));
+ * </pre>
+ *
+ * @lucene.experimental
+ */
+public class FuzzyCompletionQuery extends PrefixCompletionQuery {
+
+ /**
+ * Measure maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix
+ * parameters in Unicode code points (actual letters)
+ * instead of bytes.
+ * */
+ public static final boolean DEFAULT_UNICODE_AWARE = false;
+
+ /**
+ * The default minimum length of the key before any edits are allowed.
+ */
+ public static final int DEFAULT_MIN_FUZZY_LENGTH = 3;
+
+ /**
+ * The default prefix length where edits are not allowed.
+ */
+ public static final int DEFAULT_NON_FUZZY_PREFIX = 1;
+
+ /**
+ * The default maximum number of edits for fuzzy
+ * suggestions.
+ */
+ public static final int DEFAULT_MAX_EDITS = 1;
+
+ /**
+ * The default transposition value passed to {@link LevenshteinAutomata}
+ */
+ public static final boolean DEFAULT_TRANSPOSITIONS = true;
+
+ private final int maxEdits;
+ private final boolean transpositions;
+ private final int nonFuzzyPrefix;
+ private final int minFuzzyLength;
+ private final boolean unicodeAware;
+ private final int maxDeterminizedStates;
+
+ /**
+ * Calls {@link FuzzyCompletionQuery#FuzzyCompletionQuery(Analyzer, Term, Filter)}
+ * with no filter
+ */
+ public FuzzyCompletionQuery(Analyzer analyzer, Term term) {
+ this(analyzer, term, null);
+ }
+
+ /**
+ * Calls {@link FuzzyCompletionQuery#FuzzyCompletionQuery(Analyzer, Term, Filter,
+ * int, boolean, int, int, boolean, int)}
+ * with defaults for <code>maxEdits</code>, <code>transpositions</code>,
+ * <code>nonFuzzyPrefix</code>, <code>minFuzzyLength</code>,
+ * <code>unicodeAware</code> and <code>maxDeterminizedStates</code>
+ *
+ * See {@link #DEFAULT_MAX_EDITS}, {@link #DEFAULT_TRANSPOSITIONS},
+ * {@link #DEFAULT_NON_FUZZY_PREFIX}, {@link #DEFAULT_MIN_FUZZY_LENGTH},
+ * {@link #DEFAULT_UNICODE_AWARE} and {@link Operations#DEFAULT_MAX_DETERMINIZED_STATES}
+ * for defaults
+ */
+ public FuzzyCompletionQuery(Analyzer analyzer, Term term, Filter filter) {
+ this(analyzer, term, filter, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS, DEFAULT_NON_FUZZY_PREFIX,
+ DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE, Operations.DEFAULT_MAX_DETERMINIZED_STATES
+ );
+ }
+
+ /**
+ * Constructs an analyzed fuzzy prefix completion query
+ *
+ * @param analyzer used to analyze the provided {@link Term#text()}
+ * @param term query is run against {@link Term#field()} and {@link Term#text()}
+ * is analyzed with <code>analyzer</code>
+ * @param filter used to query on a sub set of documents
+ * @param maxEdits maximum number of acceptable edits
+ * @param transpositions value passed to {@link LevenshteinAutomata}
+ * @param nonFuzzyPrefix prefix length where edits are not allowed
+ * @param minFuzzyLength minimum prefix length before any edits are allowed
+ * @param unicodeAware treat prefix as unicode rather than bytes
+ * @param maxDeterminizedStates maximum automaton states allowed for {@link LevenshteinAutomata}
+ */
+ public FuzzyCompletionQuery(Analyzer analyzer, Term term, Filter filter, int maxEdits,
+ boolean transpositions, int nonFuzzyPrefix, int minFuzzyLength,
+ boolean unicodeAware, int maxDeterminizedStates) {
+ super(analyzer, term, filter);
+ this.maxEdits = maxEdits;
+ this.transpositions = transpositions;
+ this.nonFuzzyPrefix = nonFuzzyPrefix;
+ this.minFuzzyLength = minFuzzyLength;
+ this.unicodeAware = unicodeAware;
+ this.maxDeterminizedStates = maxDeterminizedStates;
+ }
+
+ @Override
+ public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+ CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
+ Automaton a = stream.toAutomaton(unicodeAware);
+ final Set<IntsRef> refs = Operations.getFiniteStrings(a, -1);
+ assert refs.size() > 0;
+ Automaton automaton = toLevenshteinAutomata(refs);
+ if (unicodeAware) {
+ Automaton utf8automaton = new UTF32ToUTF8().convert(automaton);
+ utf8automaton = Operations.determinize(utf8automaton, maxDeterminizedStates);
+ automaton = utf8automaton;
+ }
+ return new FuzzyCompletionWeight(searcher.getIndexReader(), this, automaton, refs);
+ }
+
+ private Automaton toLevenshteinAutomata(Set<IntsRef> ref) {
+ Automaton subs[] = new Automaton[ref.size()];
+ int upto = 0;
+ for (IntsRef path : ref) {
+ if (path.length <= nonFuzzyPrefix || path.length < minFuzzyLength) {
+ subs[upto] = Automata.makeString(path.ints, path.offset, path.length);
+ upto++;
+ } else {
+ int ints[] = new int[path.length - nonFuzzyPrefix];
+ System.arraycopy(path.ints, path.offset + nonFuzzyPrefix, ints, 0, ints.length);
+ // TODO: maybe add alphaMin to LevenshteinAutomata,
+ // and pass 1 instead of 0? We probably don't want
+ // to allow the trailing dedup bytes to be
+ // edited... but then 0 byte is "in general" allowed
+ // on input (but not in UTF8).
+ LevenshteinAutomata lev = new LevenshteinAutomata(ints,
+ unicodeAware ? Character.MAX_CODE_POINT : 255,
+ transpositions);
+ subs[upto] = lev.toAutomaton(maxEdits,
+ UnicodeUtil.newString(path.ints, path.offset, nonFuzzyPrefix));
+ upto++;
+ }
+ }
+
+ if (subs.length == 0) {
+ // automaton is empty, there is no accepted paths through it
+ return Automata.makeEmpty(); // matches nothing
+ } else if (subs.length == 1) {
+ // no synonyms or anything: just a single path through the tokenstream
+ return subs[0];
+ } else {
+ // multiple paths: this is really scary! is it slow?
+ // maybe we should not do this and throw UOE?
+ Automaton a = Operations.union(Arrays.asList(subs));
+ // TODO: we could call toLevenshteinAutomata() before det?
+ // this only happens if you have multiple paths anyway (e.g. synonyms)
+ return Operations.determinize(a, maxDeterminizedStates);
+ }
+ }
+
+ @Override
+ public String toString(String field) {
+ StringBuilder buffer = new StringBuilder();
+ if (!getField().equals(field)) {
+ buffer.append(getField());
+ buffer.append(":");
+ }
+ buffer.append(getTerm().text());
+ buffer.append('*');
+ buffer.append('~');
+ buffer.append(Integer.toString(maxEdits));
+ if (getFilter() != null) {
+ buffer.append(",");
+ buffer.append("filter");
+ buffer.append(getFilter().toString(field));
+ }
+ return buffer.toString();
+ }
+
+ private static class FuzzyCompletionWeight extends CompletionWeight {
+ private final Set<IntsRef> refs;
+ int currentBoost = 0;
+
+ public FuzzyCompletionWeight(IndexReader reader, CompletionQuery query,
+ Automaton automaton, Set<IntsRef> refs) throws IOException {
+ super(reader, query, automaton);
+ this.refs = refs;
+ }
+
+ @Override
+ protected void setNextMatch(IntsRef pathPrefix) {
+ // NOTE: the last letter of the matched prefix for the exact
+ // match never makes it through here
+ // so an exact match and a match with only a edit at the
+ // end is boosted the same
+ int maxCount = 0;
+ for (IntsRef ref : refs) {
+ int minLength = Math.min(ref.length, pathPrefix.length);
+ int count = 0;
+ for (int i = 0; i < minLength; i++) {
+ if (ref.ints[i + ref.offset] == pathPrefix.ints[i + pathPrefix.offset]) {
+ count++;
+ } else {
+ break;
+ }
+ }
+ maxCount = Math.max(maxCount, count);
+ }
+ currentBoost = maxCount;
+ }
+
+ @Override
+ protected float boost() {
+ return currentBoost;
+ }
+ }
+}