LUCENE-5214: add FreeTextSuggester

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1528517 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2013-10-02 15:23:50 +00:00
parent 691ec53476
commit b50a2edbb2
5 changed files with 1363 additions and 8 deletions

View File

@ -80,6 +80,11 @@ New Features
on best effort which was not user-friendly.
(Uwe Schindler, Robert Muir)
* LUCENE-5214: Add new FreeTextSuggester, to predict the next word
using a simple ngram language model. This is useful for the "long
tail" suggestions, when a primary suggester fails to find a
suggestion. (Mike McCandless)
Bug Fixes
* LUCENE-4998: Fixed a few places to pass IOContext.READONCE instead

View File

@ -383,7 +383,7 @@ public final class BytesRefHash {
return ids[findHash(bytes, code)];
}
private final int findHash(BytesRef bytes, int code) {
private int findHash(BytesRef bytes, int code) {
assert bytesStart != null : "bytesStart is null - not initialized";
// final position
int hashPos = code & hashMask;
@ -578,7 +578,7 @@ public final class BytesRefHash {
}
/** A simple {@link BytesStartArray} that tracks
* memory allocation using a private {@link AtomicLong}
* memory allocation using a private {@link Counter}
* instance. */
public static class DirectBytesStartArray extends BytesStartArray {
// TODO: can't we just merge this w/

View File

@ -238,11 +238,16 @@ public final class Util {
}
}
private static class FSTPath<T> {
/** Represents a path in TopNSearcher.
*
* @lucene.experimental
*/
public static class FSTPath<T> {
public FST.Arc<T> arc;
public T cost;
public final IntsRef input;
/** Sole constructor */
public FSTPath(T cost, FST.Arc<T> arc, IntsRef input) {
this.arc = new FST.Arc<T>().copyFrom(arc);
this.cost = cost;
@ -300,7 +305,7 @@ public final class Util {
}
// If back plus this arc is competitive then add to queue:
private void addIfCompetitive(FSTPath<T> path) {
protected void addIfCompetitive(FSTPath<T> path) {
assert queue != null;
@ -399,6 +404,7 @@ public final class Util {
if (queue == null) {
// Ran out of paths
//System.out.println(" break queue=null");
break;
}
@ -408,6 +414,7 @@ public final class Util {
if (path == null) {
// There were less than topN paths available:
//System.out.println(" break no more paths");
break;
}
@ -478,6 +485,7 @@ public final class Util {
//System.out.println(" done!: " + path);
T finalOutput = fst.outputs.add(path.cost, path.arc.output);
if (acceptResult(path.input, finalOutput)) {
//System.out.println(" add result: " + path);
results.add(new MinResult<T>(path.input, finalOutput));
} else {
rejectCount++;
@ -761,10 +769,10 @@ public final class Util {
* Ensures an arc's label is indeed printable (dot uses US-ASCII).
*/
private static String printableLabel(int label) {
if (label >= 0x20 && label <= 0x7d) {
if (label != 0x22 && label != 0x5c) { // " OR \
return Character.toString((char) label);
}
// Any ordinary ascii character, except for " or \, are
// printed as the character; else, as a hex string:
if (label >= 0x20 && label <= 0x7d && label != 0x22 && label != 0x5c) { // " OR \
return Character.toString((char) label);
}
return "0x" + Integer.toHexString(label);
}

View File

@ -0,0 +1,766 @@
package org.apache.lucene.search.suggest.analyzing;
/*
* 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.
*/
// TODO
// - test w/ syns
// - add pruning of low-freq ngrams?
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
//import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
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.shingle.ShingleFilter;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.spell.TermFreqIterator;
import org.apache.lucene.search.spell.TermFreqPayloadIterator;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.Sort;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util.MinResult;
import org.apache.lucene.util.fst.Util;
/**
* Builds an ngram model from the text sent to {@link
* #build} and predicts based on the last grams-1 tokens in
* the request sent to {@link #lookup}. This tries to
* handle the "long tail" of suggestions for when the
* incoming query is a never before seen query string.
*
* <p>Likely this suggester would only be used as a
* fallback, when the primary suggester fails to find
* any suggestions.
*
* <p>Note that the weight for each suggestion is unused,
* and the suggestions are the analyzed forms (so your
* analysis process should normally be very "light").
*
* <p>This uses the stupid backoff language model to smooth
* scores across ngram models; see
* "Large language models in machine translation",
* http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.76.1126
* for details.
*
* <p> From {@link #lookup}, the key of each result is the
* ngram token; the value is Long.MAX_VALUE * score (fixed
* point, cast to long). Divide by Long.MAX_VALUE to get
* the score back, which ranges from 0.0 to 1.0.
*
* onlyMorePopular is unused.
*
* @lucene.experimental
*/
public class FreeTextSuggester extends Lookup {
/** Codec name used in the header for the saved model. */
public final static String CODEC_NAME = "freetextsuggest";
/** Initial version of the the saved model file format. */
public final static int VERSION_START = 0;
/** Current version of the the saved model file format. */
public final static int VERSION_CURRENT = VERSION_START;
/** By default we use a bigram model. */
public static final int DEFAULT_GRAMS = 2;
// In general this could vary with gram, but the
// original paper seems to use this constant:
/** The constant used for backoff smoothing; during
* lookup, this means that if a given trigram did not
* occur, and we backoff to the bigram, the overall score
* will be 0.4 times what the bigram model would have
* assigned. */
public final static double ALPHA = 0.4;
/** Holds 1gram, 2gram, 3gram models as a single FST. */
private FST<Long> fst;
/**
* Analyzer that will be used for analyzing suggestions at
* index time.
*/
private final Analyzer indexAnalyzer;
private long totTokens;
/**
* Analyzer that will be used for analyzing suggestions at
* query time.
*/
private final Analyzer queryAnalyzer;
// 2 = bigram, 3 = trigram
private final int grams;
private final byte separator;
/** The default character used to join multiple tokens
* into a single ngram token. The input tokens produced
* by the analyzer must not contain this character. */
public static final byte DEFAULT_SEPARATOR = 0x1e;
/** Instantiate, using the provided analyzer for both
* indexing and lookup, using bigram model by default. */
public FreeTextSuggester(Analyzer analyzer) {
this(analyzer, analyzer, DEFAULT_GRAMS);
}
/** Instantiate, using the provided indexing and lookup
* analyzers, using bigram model by default. */
public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
this(indexAnalyzer, queryAnalyzer, DEFAULT_GRAMS);
}
/** Instantiate, using the provided indexing and lookup
* analyzers, with the specified model (2
* = bigram, 3 = trigram, etc.). */
public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int grams) {
this(indexAnalyzer, queryAnalyzer, grams, DEFAULT_SEPARATOR);
}
/** Instantiate, using the provided indexing and lookup
* analyzers, and specified model (2 = bigram, 3 =
* trigram ,etc.). The separator is passed to {@link
* ShingleFilter#setTokenSeparator} to join multiple
* tokens into a single ngram token; it must be an ascii
* (7-bit-clean) byte. No input tokens should have this
* byte, otherwise {@code IllegalArgumentException} is
* thrown. */
public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int grams, byte separator) {
this.grams = grams;
this.indexAnalyzer = addShingles(indexAnalyzer);
this.queryAnalyzer = addShingles(queryAnalyzer);
if (grams < 1) {
throw new IllegalArgumentException("grams must be >= 1");
}
if ((separator & 0x80) != 0) {
throw new IllegalArgumentException("separator must be simple ascii character");
}
this.separator = separator;
}
/** Returns byte size of the underlying FST. */
public long sizeInBytes() {
if (fst == null) {
return 0;
}
return fst.sizeInBytes();
}
private static class AnalyzingComparator implements Comparator<BytesRef> {
private final ByteArrayDataInput readerA = new ByteArrayDataInput();
private final ByteArrayDataInput readerB = new ByteArrayDataInput();
private final BytesRef scratchA = new BytesRef();
private final BytesRef scratchB = new BytesRef();
@Override
public int compare(BytesRef a, BytesRef b) {
readerA.reset(a.bytes, a.offset, a.length);
readerB.reset(b.bytes, b.offset, b.length);
// By token:
scratchA.length = readerA.readShort();
scratchA.bytes = a.bytes;
scratchA.offset = readerA.getPosition();
scratchB.bytes = b.bytes;
scratchB.length = readerB.readShort();
scratchB.offset = readerB.getPosition();
int cmp = scratchA.compareTo(scratchB);
if (cmp != 0) {
return cmp;
}
readerA.skipBytes(scratchA.length);
readerB.skipBytes(scratchB.length);
// By length (smaller surface forms sorted first):
cmp = a.length - b.length;
if (cmp != 0) {
return cmp;
}
// By surface form:
scratchA.offset = readerA.getPosition();
scratchA.length = a.length - scratchA.offset;
scratchB.offset = readerB.getPosition();
scratchB.length = b.length - scratchB.offset;
return scratchA.compareTo(scratchB);
}
}
private Analyzer addShingles(final Analyzer other) {
if (grams == 1) {
return other;
} else {
// TODO: use ShingleAnalyzerWrapper?
// Tack on ShingleFilter to the end, to generate token ngrams:
return new AnalyzerWrapper(other.getReuseStrategy()) {
@Override
protected Analyzer getWrappedAnalyzer(String fieldName) {
return other;
}
@Override
protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
ShingleFilter shingles = new ShingleFilter(components.getTokenStream(), 2, grams);
shingles.setTokenSeparator(Character.toString((char) separator));
return new TokenStreamComponents(components.getTokenizer(), shingles);
}
};
}
}
@Override
public void build(TermFreqIterator iterator) throws IOException {
build(iterator, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
}
/** Build the suggest index, using up to the specified
* amount of temporary RAM while building. Note that
* the weights for the suggestions are ignored. */
public void build(TermFreqIterator iterator, double ramBufferSizeMB) throws IOException {
if (iterator instanceof TermFreqPayloadIterator) {
throw new IllegalArgumentException("payloads are not supported");
}
String prefix = getClass().getSimpleName();
File directory = Sort.defaultTempDir();
// TODO: messy ... java7 has Files.createTempDirectory
// ... but 4.x is java6:
File tempIndexPath = null;
Random random = new Random();
while (true) {
tempIndexPath = new File(directory, prefix + ".index." + random.nextInt(Integer.MAX_VALUE));
if (tempIndexPath.mkdir()) {
break;
}
}
Directory dir = FSDirectory.open(tempIndexPath);
IndexWriterConfig iwc = new IndexWriterConfig(Version.LUCENE_46, indexAnalyzer);
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
iwc.setRAMBufferSizeMB(ramBufferSizeMB);
IndexWriter writer = new IndexWriter(dir, iwc);
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
// TODO: if only we had IndexOptions.TERMS_ONLY...
ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
ft.setOmitNorms(true);
ft.freeze();
Document doc = new Document();
Field field = new Field("body", "", ft);
doc.add(field);
totTokens = 0;
IndexReader reader = null;
boolean success = false;
try {
while (true) {
BytesRef surfaceForm = iterator.next();
if (surfaceForm == null) {
break;
}
field.setStringValue(surfaceForm.utf8ToString());
writer.addDocument(doc);
}
reader = DirectoryReader.open(writer, false);
Terms terms = MultiFields.getTerms(reader, "body");
if (terms == null) {
throw new IllegalArgumentException("need at least one suggestion");
}
// Move all ngrams into an FST:
TermsEnum termsEnum = terms.iterator(null);
Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
IntsRef scratchInts = new IntsRef();
while (true) {
BytesRef term = termsEnum.next();
if (term == null) {
break;
}
int ngramCount = countGrams(term);
if (ngramCount > grams) {
throw new IllegalArgumentException("tokens must not contain separator byte; got token=" + term + " but gramCount=" + ngramCount + ", which is greater than expected max ngram size=" + grams);
}
if (ngramCount == 1) {
totTokens += termsEnum.totalTermFreq();
}
builder.add(Util.toIntsRef(term, scratchInts), encodeWeight(termsEnum.totalTermFreq()));
}
fst = builder.finish();
if (fst == null) {
throw new IllegalArgumentException("need at least one suggestion");
}
//System.out.println("FST: " + fst.getNodeCount() + " nodes");
/*
PrintWriter pw = new PrintWriter("/x/tmp/out.dot");
Util.toDot(fst, pw, true, true);
pw.close();
*/
success = true;
} finally {
try {
if (success) {
IOUtils.close(writer, reader);
} else {
IOUtils.closeWhileHandlingException(writer, reader);
}
} finally {
for(String file : dir.listAll()) {
File path = new File(tempIndexPath, file);
if (path.delete() == false) {
throw new IllegalStateException("failed to remove " + path);
}
}
if (tempIndexPath.delete() == false) {
throw new IllegalStateException("failed to remove " + tempIndexPath);
}
dir.close();
}
}
}
@Override
public boolean store(OutputStream output) throws IOException {
DataOutput out = new OutputStreamDataOutput(output);
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeByte(separator);
out.writeVInt(grams);
out.writeVLong(totTokens);
fst.save(out);
return true;
}
@Override
public boolean load(InputStream input) throws IOException {
DataInput in = new InputStreamDataInput(input);
CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
byte separatorOrig = in.readByte();
if (separatorOrig != separator) {
throw new IllegalStateException("separator=" + separator + " is incorrect: original model was built with separator=" + separatorOrig);
}
int gramsOrig = in.readVInt();
if (gramsOrig != grams) {
throw new IllegalStateException("grams=" + grams + " is incorrect: original model was built with grams=" + gramsOrig);
}
totTokens = in.readVLong();
fst = new FST<Long>(in, PositiveIntOutputs.getSingleton());
return true;
}
@Override
public List<LookupResult> lookup(final CharSequence key, /* ignored */ boolean onlyMorePopular, int num) {
try {
return lookup(key, num);
} catch (IOException ioe) {
// bogus:
throw new RuntimeException(ioe);
}
}
private int countGrams(BytesRef token) {
int count = 1;
for(int i=0;i<token.length;i++) {
if (token.bytes[token.offset + i] == separator) {
count++;
}
}
return count;
}
/** Retrieve suggestions. */
public List<LookupResult> lookup(final CharSequence key, int num) throws IOException {
TokenStream ts = queryAnalyzer.tokenStream("", key.toString());
TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
PositionLengthAttribute posLenAtt = ts.addAttribute(PositionLengthAttribute.class);
PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
ts.reset();
BytesRef[] lastTokens = new BytesRef[grams];
//System.out.println("lookup: key='" + key + "'");
// Run full analysis, but save only the
// last 1gram, last 2gram, etc.:
BytesRef tokenBytes = termBytesAtt.getBytesRef();
int maxEndOffset = -1;
boolean sawRealToken = false;
while(ts.incrementToken()) {
termBytesAtt.fillBytesRef();
sawRealToken |= tokenBytes.length > 0;
// TODO: this is somewhat iffy; today, ShingleFilter
// sets posLen to the gram count; maybe we should make
// a separate dedicated att for this?
int gramCount = posLenAtt.getPositionLength();
assert gramCount <= grams;
// Safety: make sure the recalculated count "agrees":
if (countGrams(tokenBytes) != gramCount) {
throw new IllegalArgumentException("tokens must not contain separator byte; got token=" + tokenBytes + " but gramCount=" + gramCount + " does not match recalculated count=" + countGrams(tokenBytes));
}
maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset());
lastTokens[gramCount-1] = BytesRef.deepCopyOf(tokenBytes);
}
ts.end();
if (!sawRealToken) {
throw new IllegalArgumentException("no tokens produced by analyzer, or the only tokens were empty strings");
}
// Carefully fill last tokens with _ tokens;
// ShingleFilter appraently won't emit "only hole"
// tokens:
int endPosInc = posIncAtt.getPositionIncrement();
// Note this will also be true if input is the empty
// string (in which case we saw no tokens and
// maxEndOffset is still -1), which in fact works out OK
// because we fill the unigram with an empty BytesRef
// below:
boolean lastTokenEnded = offsetAtt.endOffset() > maxEndOffset || endPosInc > 0;
ts.close();
//System.out.println("maxEndOffset=" + maxEndOffset + " vs " + offsetAtt.endOffset());
if (lastTokenEnded) {
//System.out.println(" lastTokenEnded");
// If user hit space after the last token, then
// "upgrade" all tokens. This way "foo " will suggest
// all bigrams starting w/ foo, and not any unigrams
// starting with "foo":
for(int i=grams-1;i>0;i--) {
BytesRef token = lastTokens[i-1];
if (token == null) {
continue;
}
token.grow(token.length+1);
token.bytes[token.length] = separator;
token.length++;
lastTokens[i] = token;
}
lastTokens[0] = new BytesRef();
}
Arc<Long> arc = new Arc<Long>();
BytesReader bytesReader = fst.getBytesReader();
// Try highest order models first, and if they return
// results, return that; else, fallback:
double backoff = 1.0;
List<LookupResult> results = new ArrayList<LookupResult>(num);
// We only add a given suffix once, from the highest
// order model that saw it; for subsequent lower order
// models we skip it:
final Set<BytesRef> seen = new HashSet<BytesRef>();
for(int gram=grams-1;gram>=0;gram--) {
BytesRef token = lastTokens[gram];
// Don't make unigram predictions from empty string:
if (token == null || (token.length == 0 && key.length() > 0)) {
// Input didn't have enough tokens:
//System.out.println(" gram=" + gram + ": skip: not enough input");
continue;
}
if (endPosInc > 0 && gram <= endPosInc) {
// Skip hole-only predictions; in theory we
// shouldn't have to do this, but we'd need to fix
// ShingleFilter to produce only-hole tokens:
//System.out.println(" break: only holes now");
break;
}
//System.out.println("try " + (gram+1) + " gram token=" + token.utf8ToString());
// TODO: we could add fuzziness here
// match the prefix portion exactly
//Pair<Long,BytesRef> prefixOutput = null;
Long prefixOutput = null;
try {
prefixOutput = lookupPrefix(fst, bytesReader, token, arc);
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
//System.out.println(" prefixOutput=" + prefixOutput);
if (prefixOutput == null) {
// This model never saw this prefix, e.g. the
// trigram model never saw context "purple mushroom"
backoff *= ALPHA;
continue;
}
// TODO: we could do this division at build time, and
// bake it into the FST?
// Denominator for computing scores from current
// model's predictions:
long contextCount = totTokens;
BytesRef lastTokenFragment = null;
for(int i=token.length-1;i>=0;i--) {
if (token.bytes[token.offset+i] == separator) {
BytesRef context = new BytesRef(token.bytes, token.offset, i);
Long output = Util.get(fst, Util.toIntsRef(context, new IntsRef()));
assert output != null;
contextCount = decodeWeight(output);
lastTokenFragment = new BytesRef(token.bytes, token.offset + i + 1, token.length - i - 1);
break;
}
}
final BytesRef finalLastToken;
if (lastTokenFragment == null) {
finalLastToken = BytesRef.deepCopyOf(token);
} else {
finalLastToken = BytesRef.deepCopyOf(lastTokenFragment);
}
assert finalLastToken.offset == 0;
CharsRef spare = new CharsRef();
// complete top-N
MinResult<Long> completions[] = null;
try {
// Because we store multiple models in one FST
// (1gram, 2gram, 3gram), we must restrict the
// search so that it only considers the current
// model. For highest order model, this is not
// necessary since all completions in the FST
// must be from this model, but for lower order
// models we have to filter out the higher order
// ones:
// Must do num+seen.size() for queue depth because we may
// reject up to seen.size() paths in acceptResult():
Util.TopNSearcher<Long> searcher = new Util.TopNSearcher<Long>(fst, num, num+seen.size(), weightComparator) {
BytesRef scratchBytes = new BytesRef();
@Override
protected void addIfCompetitive(Util.FSTPath<Long> path) {
if (path.arc.label != separator) {
//System.out.println(" keep path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
super.addIfCompetitive(path);
} else {
//System.out.println(" prevent path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
}
}
@Override
protected boolean acceptResult(IntsRef input, Long output) {
Util.toBytesRef(input, scratchBytes);
finalLastToken.grow(finalLastToken.length + scratchBytes.length);
int lenSav = finalLastToken.length;
finalLastToken.append(scratchBytes);
//System.out.println(" accept? input='" + scratchBytes.utf8ToString() + "'; lastToken='" + finalLastToken.utf8ToString() + "'; return " + (seen.contains(finalLastToken) == false));
boolean ret = seen.contains(finalLastToken) == false;
finalLastToken.length = lenSav;
return ret;
}
};
// since this search is initialized with a single start node
// it is okay to start with an empty input path here
searcher.addStartPaths(arc, prefixOutput, true, new IntsRef());
completions = searcher.search();
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
int prefixLength = token.length;
BytesRef suffix = new BytesRef(8);
//System.out.println(" " + completions.length + " completions");
nextCompletion:
for (MinResult<Long> completion : completions) {
token.length = prefixLength;
// append suffix
Util.toBytesRef(completion.input, suffix);
token.append(suffix);
//System.out.println(" completion " + token.utf8ToString());
// Skip this path if a higher-order model already
// saw/predicted its last token:
BytesRef lastToken = token;
for(int i=token.length-1;i>=0;i--) {
if (token.bytes[token.offset+i] == separator) {
assert token.length-i-1 > 0;
lastToken = new BytesRef(token.bytes, token.offset+i+1, token.length-i-1);
break;
}
}
if (seen.contains(lastToken)) {
//System.out.println(" skip dup " + lastToken.utf8ToString());
continue nextCompletion;
}
seen.add(BytesRef.deepCopyOf(lastToken));
spare.grow(token.length);
UnicodeUtil.UTF8toUTF16(token, spare);
LookupResult result = new LookupResult(spare.toString(), (long) (Long.MAX_VALUE * backoff * ((double) decodeWeight(completion.output)) / contextCount));
results.add(result);
assert results.size() == seen.size();
//System.out.println(" add result=" + result);
}
backoff *= ALPHA;
}
Collections.sort(results, new Comparator<LookupResult>() {
@Override
public int compare(LookupResult a, LookupResult b) {
if (a.value > b.value) {
return -1;
} else if (a.value < b.value) {
return 1;
} else {
// Tie break by UTF16 sort order:
return ((String) a.key).compareTo((String) b.key);
}
}
});
if (results.size() > num) {
results.subList(num, results.size()).clear();
}
return results;
}
/** weight -> cost */
private long encodeWeight(long ngramCount) {
return Long.MAX_VALUE - ngramCount;
}
/** cost -> weight */
//private long decodeWeight(Pair<Long,BytesRef> output) {
private long decodeWeight(Long output) {
assert output != null;
return (int)(Long.MAX_VALUE - output);
}
// NOTE: copied from WFSTCompletionLookup & tweaked
private Long lookupPrefix(FST<Long> fst, FST.BytesReader bytesReader,
BytesRef scratch, Arc<Long> arc) throws /*Bogus*/IOException {
Long output = fst.outputs.getNoOutput();
fst.getFirstArc(arc);
byte[] bytes = scratch.bytes;
int pos = scratch.offset;
int end = pos + scratch.length;
while (pos < end) {
if (fst.findTargetArc(bytes[pos++] & 0xff, arc, arc, bytesReader) == null) {
return null;
} else {
output = fst.outputs.add(output, arc.output);
}
}
return output;
}
static final Comparator<Long> weightComparator = new Comparator<Long> () {
@Override
public int compare(Long left, Long right) {
return left.compareTo(right);
}
};
/**
* Returns the weight associated with an input string,
* or null if it does not exist.
*/
public Object get(CharSequence key) {
throw new UnsupportedOperationException();
}
}

View File

@ -0,0 +1,576 @@
package org.apache.lucene.search.suggest.analyzing;
/*
* 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.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.Reader;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.core.StopFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.apache.lucene.document.Document;
import org.apache.lucene.search.spell.TermFreqIterator;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.search.suggest.TermFreq;
import org.apache.lucene.search.suggest.TermFreqArrayIterator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Ignore;
public class TestFreeTextSuggester extends LuceneTestCase {
public void testBasic() throws Exception {
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo bar baz blah", 50),
new TermFreq("boo foo bar foo bee", 20)
);
Analyzer a = new MockAnalyzer(random());
FreeTextSuggester sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
for(int i=0;i<2;i++) {
// Uses bigram model and unigram backoff:
assertEquals("foo bar/0.67 foo bee/0.33 baz/0.04 blah/0.04 boo/0.04",
toString(sug.lookup("foo b", 10)));
// Uses only bigram model:
assertEquals("foo bar/0.67 foo bee/0.33",
toString(sug.lookup("foo ", 10)));
// Uses only unigram model:
assertEquals("foo/0.33",
toString(sug.lookup("foo", 10)));
// Uses only unigram model:
assertEquals("bar/0.22 baz/0.11 bee/0.11 blah/0.11 boo/0.11",
toString(sug.lookup("b", 10)));
// Try again after save/load:
File tmpDir = _TestUtil.getTempDir("FreeTextSuggesterTest");
tmpDir.mkdir();
File path = new File(tmpDir, "suggester");
OutputStream os = new FileOutputStream(path);
sug.store(os);
os.close();
InputStream is = new FileInputStream(path);
sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
sug.load(is);
is.close();
}
}
public void testIllegalByteDuringBuild() throws Exception {
// Default separator is INFORMATION SEPARATOR TWO
// (0x1e), so no input token is allowed to contain it
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo\u001ebar baz", 50)
);
FreeTextSuggester sug = new FreeTextSuggester(new MockAnalyzer(random()));
try {
sug.build(new TermFreqArrayIterator(keys));
fail("did not hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
}
public void testIllegalByteDuringQuery() throws Exception {
// Default separator is INFORMATION SEPARATOR TWO
// (0x1e), so no input token is allowed to contain it
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo bar baz", 50)
);
FreeTextSuggester sug = new FreeTextSuggester(new MockAnalyzer(random()));
sug.build(new TermFreqArrayIterator(keys));
try {
sug.lookup("foo\u001eb", 10);
fail("did not hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
}
@Ignore
public void testWiki() throws Exception {
final LineFileDocs lfd = new LineFileDocs(null, "/lucenedata/enwiki/enwiki-20120502-lines-1k.txt", false);
// Skip header:
lfd.nextDoc();
FreeTextSuggester sug = new FreeTextSuggester(new MockAnalyzer(random()));
sug.build(new TermFreqIterator() {
private int count;
@Override
public long weight() {
return 1;
}
@Override
public BytesRef next() {
Document doc;
try {
doc = lfd.nextDoc();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
if (doc == null) {
return null;
}
if (count++ == 10000) {
return null;
}
return new BytesRef(doc.get("body"));
}
});
if (VERBOSE) {
System.out.println(sug.sizeInBytes() + " bytes");
List<LookupResult> results = sug.lookup("general r", 10);
System.out.println("results:");
for(LookupResult result : results) {
System.out.println(" " + result);
}
}
}
// Make sure you can suggest based only on unigram model:
public void testUnigrams() throws Exception {
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo bar baz blah boo foo bar foo bee", 50)
);
Analyzer a = new MockAnalyzer(random());
FreeTextSuggester sug = new FreeTextSuggester(a, a, 1, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
// Sorts first by count, descending, second by term, ascending
assertEquals("bar/0.22 baz/0.11 bee/0.11 blah/0.11 boo/0.11",
toString(sug.lookup("b", 10)));
}
// Make sure the last token is not duplicated
public void testNoDupsAcrossGrams() throws Exception {
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo bar bar bar bar", 50)
);
Analyzer a = new MockAnalyzer(random());
FreeTextSuggester sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
assertEquals("foo bar/1.00",
toString(sug.lookup("foo b", 10)));
}
// Lookup of just empty string produces unicode only matches:
public void testEmptyString() throws Exception {
Iterable<TermFreq> keys = shuffle(
new TermFreq("foo bar bar bar bar", 50)
);
Analyzer a = new MockAnalyzer(random());
FreeTextSuggester sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
try {
sug.lookup("", 10);
fail("did not hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
}
// With one ending hole, ShingleFilter produces "of _" and
// we should properly predict from that:
public void testEndingHole() throws Exception {
// Just deletes "of"
Analyzer a = new Analyzer() {
@Override
public TokenStreamComponents createComponents(String field, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader);
CharArraySet stopSet = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "of");
return new TokenStreamComponents(tokenizer, new StopFilter(TEST_VERSION_CURRENT, tokenizer, stopSet));
}
};
Iterable<TermFreq> keys = shuffle(
new TermFreq("wizard of oz", 50)
);
FreeTextSuggester sug = new FreeTextSuggester(a, a, 3, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
assertEquals("wizard _ oz/1.00",
toString(sug.lookup("wizard of", 10)));
// Falls back to unigram model, with backoff 0.4 times
// prop 0.5:
assertEquals("oz/0.20",
toString(sug.lookup("wizard o", 10)));
}
// If the number of ending holes exceeds the ngrams window
// then there are no predictions, because ShingleFilter
// does not produce e.g. a hole only "_ _" token:
public void testTwoEndingHoles() throws Exception {
// Just deletes "of"
Analyzer a = new Analyzer() {
@Override
public TokenStreamComponents createComponents(String field, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader);
CharArraySet stopSet = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "of");
return new TokenStreamComponents(tokenizer, new StopFilter(TEST_VERSION_CURRENT, tokenizer, stopSet));
}
};
Iterable<TermFreq> keys = shuffle(
new TermFreq("wizard of of oz", 50)
);
FreeTextSuggester sug = new FreeTextSuggester(a, a, 3, (byte) 0x20);
sug.build(new TermFreqArrayIterator(keys));
assertEquals("",
toString(sug.lookup("wizard of of", 10)));
}
private static Comparator<LookupResult> byScoreThenKey = new Comparator<LookupResult>() {
@Override
public int compare(LookupResult a, LookupResult b) {
if (a.value > b.value) {
return -1;
} else if (a.value < b.value) {
return 1;
} else {
// Tie break by UTF16 sort order:
return ((String) a.key).compareTo((String) b.key);
}
}
};
public void testRandom() throws IOException {
String[] terms = new String[_TestUtil.nextInt(random(), 2, 10)];
Set<String> seen = new HashSet<String>();
while (seen.size() < terms.length) {
String token = _TestUtil.randomSimpleString(random(), 1, 5);
if (!seen.contains(token)) {
terms[seen.size()] = token;
seen.add(token);
}
}
Analyzer a = new MockAnalyzer(random());
int numDocs = atLeast(10);
long totTokens = 0;
final String[][] docs = new String[numDocs][];
for(int i=0;i<numDocs;i++) {
docs[i] = new String[atLeast(100)];
if (VERBOSE) {
System.out.print(" doc " + i + ":");
}
for(int j=0;j<docs[i].length;j++) {
docs[i][j] = getZipfToken(terms);
if (VERBOSE) {
System.out.print(" " + docs[i][j]);
}
}
if (VERBOSE) {
System.out.println();
}
totTokens += docs[i].length;
}
int grams = _TestUtil.nextInt(random(), 1, 4);
if (VERBOSE) {
System.out.println("TEST: " + terms.length + " terms; " + numDocs + " docs; " + grams + " grams");
}
// Build suggester model:
FreeTextSuggester sug = new FreeTextSuggester(a, a, grams, (byte) 0x20);
sug.build(new TermFreqIterator() {
int upto;
@Override
public BytesRef next() {
if (upto == docs.length) {
return null;
} else {
StringBuilder b = new StringBuilder();
for(String token : docs[upto]) {
b.append(' ');
b.append(token);
}
upto++;
return new BytesRef(b.toString());
}
}
@Override
public long weight() {
return random().nextLong();
}
});
// Build inefficient but hopefully correct model:
List<Map<String,Integer>> gramCounts = new ArrayList<Map<String,Integer>>(grams);
for(int gram=0;gram<grams;gram++) {
if (VERBOSE) {
System.out.println("TEST: build model for gram=" + gram);
}
Map<String,Integer> model = new HashMap<String,Integer>();
gramCounts.add(model);
for(String[] doc : docs) {
for(int i=0;i<doc.length-gram;i++) {
StringBuilder b = new StringBuilder();
for(int j=i;j<=i+gram;j++) {
if (j > i) {
b.append(' ');
}
b.append(doc[j]);
}
String token = b.toString();
Integer curCount = model.get(token);
if (curCount == null) {
model.put(token, 1);
} else {
model.put(token, 1 + curCount);
}
if (VERBOSE) {
System.out.println(" add '" + token + "' -> count=" + model.get(token));
}
}
}
}
int lookups = atLeast(100);
for(int iter=0;iter<lookups;iter++) {
String[] tokens = new String[_TestUtil.nextInt(random(), 1, 5)];
for(int i=0;i<tokens.length;i++) {
tokens[i] = getZipfToken(terms);
}
// Maybe trim last token; be sure not to create the
// empty string:
int trimStart;
if (tokens.length == 1) {
trimStart = 1;
} else {
trimStart = 0;
}
int trimAt = _TestUtil.nextInt(random(), trimStart, tokens[tokens.length-1].length());
tokens[tokens.length-1] = tokens[tokens.length-1].substring(0, trimAt);
int num = _TestUtil.nextInt(random(), 1, 100);
StringBuilder b = new StringBuilder();
for(String token : tokens) {
b.append(' ');
b.append(token);
}
String query = b.toString();
query = query.substring(1);
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter + " query='" + query + "' num=" + num);
}
// Expected:
List<LookupResult> expected = new ArrayList<LookupResult>();
double backoff = 1.0;
seen = new HashSet<String>();
if (VERBOSE) {
System.out.println(" compute expected");
}
for(int i=grams-1;i>=0;i--) {
if (VERBOSE) {
System.out.println(" grams=" + i);
}
if (tokens.length < i+1) {
// Don't have enough tokens to use this model
if (VERBOSE) {
System.out.println(" skip");
}
continue;
}
if (i == 0 && tokens[tokens.length-1].length() == 0) {
// Never suggest unigrams from empty string:
if (VERBOSE) {
System.out.println(" skip unigram priors only");
}
continue;
}
// Build up "context" ngram:
b = new StringBuilder();
for(int j=tokens.length-i-1;j<tokens.length-1;j++) {
b.append(' ');
b.append(tokens[j]);
}
String context = b.toString();
if (context.length() > 0) {
context = context.substring(1);
}
if (VERBOSE) {
System.out.println(" context='" + context + "'");
}
long contextCount;
if (context.length() == 0) {
contextCount = totTokens;
} else {
Integer count = gramCounts.get(i-1).get(context);
if (count == null) {
// We never saw this context:
backoff *= FreeTextSuggester.ALPHA;
if (VERBOSE) {
System.out.println(" skip: never saw context");
}
continue;
}
contextCount = count;
}
if (VERBOSE) {
System.out.println(" contextCount=" + contextCount);
}
Map<String,Integer> model = gramCounts.get(i);
// First pass, gather all predictions for this model:
if (VERBOSE) {
System.out.println(" find terms w/ prefix=" + tokens[tokens.length-1]);
}
List<LookupResult> tmp = new ArrayList<LookupResult>();
for(String term : terms) {
if (term.startsWith(tokens[tokens.length-1])) {
if (VERBOSE) {
System.out.println(" term=" + term);
}
if (seen.contains(term)) {
if (VERBOSE) {
System.out.println(" skip seen");
}
continue;
}
String ngram = (context + " " + term).trim();
Integer count = model.get(ngram);
if (count != null) {
LookupResult lr = new LookupResult(ngram, (long) (Long.MAX_VALUE * (backoff * (double) count / contextCount)));
tmp.add(lr);
if (VERBOSE) {
System.out.println(" add tmp key='" + lr.key + "' score=" + lr.value);
}
}
}
}
// Second pass, trim to only top N, and fold those
// into overall suggestions:
Collections.sort(tmp, byScoreThenKey);
if (tmp.size() > num) {
tmp.subList(num, tmp.size()).clear();
}
for(LookupResult result : tmp) {
String key = result.key.toString();
int idx = key.lastIndexOf(' ');
String lastToken;
if (idx != -1) {
lastToken = key.substring(idx+1);
} else {
lastToken = key;
}
if (!seen.contains(lastToken)) {
seen.add(lastToken);
expected.add(result);
if (VERBOSE) {
System.out.println(" keep key='" + result.key + "' score=" + result.value);
}
}
}
backoff *= FreeTextSuggester.ALPHA;
}
Collections.sort(expected, byScoreThenKey);
if (expected.size() > num) {
expected.subList(num, expected.size()).clear();
}
// Actual:
List<LookupResult> actual = sug.lookup(query, num);
if (VERBOSE) {
System.out.println(" expected: " + expected);
System.out.println(" actual: " + actual);
}
assertEquals(expected.toString(), actual.toString());
}
}
private static String getZipfToken(String[] tokens) {
// Zipf-like distribution:
for(int k=0;k<tokens.length;k++) {
if (random().nextBoolean() || k == tokens.length-1) {
return tokens[k];
}
}
assert false;
return null;
}
private static String toString(List<LookupResult> results) {
StringBuilder b = new StringBuilder();
for(LookupResult result : results) {
b.append(' ');
b.append(result.key);
b.append('/');
b.append(String.format(Locale.ROOT, "%.2f", ((double) result.value)/Long.MAX_VALUE));
}
return b.toString().trim();
}
@SafeVarargs
private final <T> Iterable<T> shuffle(T...values) {
final List<T> asList = new ArrayList<T>(values.length);
for (T value : values) {
asList.add(value);
}
Collections.shuffle(asList, random());
return asList;
}
}