LUCENE-3846: commit current patch

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3846@1397171 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-10-11 17:08:47 +00:00
parent 319368792a
commit 66b2c78d6b
11 changed files with 1283 additions and 69 deletions

View File

@ -818,11 +818,11 @@
<classpath refid="@{junit.classpath}"/> <classpath refid="@{junit.classpath}"/>
<classpath refid="clover.classpath" /> <classpath refid="clover.classpath" />
<!-- Assertions. --> <!-- Assertions.
<assertions> <assertions>
<enable package="org.apache.lucene"/> <enable package="org.apache.lucene"/>
<enable package="org.apache.solr"/> <enable package="org.apache.solr"/>
</assertions> </assertions> nocommit -->
<!-- JVM arguments and system properties. --> <!-- JVM arguments and system properties. -->
<jvmarg line="${args}"/> <jvmarg line="${args}"/>

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.OutputStreamWriter; import java.io.OutputStreamWriter;
import java.io.Writer; import java.io.Writer;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute; import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
@ -88,6 +89,7 @@ public class TokenStreamToAutomaton {
final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class); final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class); final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class); final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
final BytesRef term = termBytesAtt.getBytesRef(); final BytesRef term = termBytesAtt.getBytesRef();
in.reset(); in.reset();

View File

@ -241,6 +241,20 @@ final public class BasicAutomata {
return a; return a;
} }
public static Automaton makeString(int[] word, int offset, int length) {
Automaton a = new Automaton();
a.setDeterministic(true);
State s = new State();
a.initial = s;
for (int i = offset; i < offset+length; i++) {
State s2 = new State();
s.addTransition(new Transition(word[i], s2));
s = s2;
}
s.accept = true;
return a;
}
/** /**
* Returns a new (deterministic and minimal) automaton that accepts the union * Returns a new (deterministic and minimal) automaton that accepts the union
* of the given collection of {@link BytesRef}s representing UTF-8 encoded * of the given collection of {@link BytesRef}s representing UTF-8 encoded

View File

@ -33,12 +33,13 @@ public class LevenshteinAutomata {
/** @lucene.internal */ /** @lucene.internal */
public static final int MAXIMUM_SUPPORTED_DISTANCE = 2; public static final int MAXIMUM_SUPPORTED_DISTANCE = 2;
/* input word */ /* input word */
final String input;
final int word[]; final int word[];
/* the automata alphabet. */ /* the automata alphabet. */
final int alphabet[]; final int alphabet[];
/* the maximum symbol in the alphabet (e.g. 256 for UTF-8 or 10FFFF for UTF-32) */
final int alphaMax;
/* the unicode ranges outside of alphabet */ /* the ranges outside of alphabet */
final int rangeLower[]; final int rangeLower[];
final int rangeUpper[]; final int rangeUpper[];
int numRanges = 0; int numRanges = 0;
@ -50,13 +51,16 @@ public class LevenshteinAutomata {
* Optionally count transpositions as a primitive edit. * Optionally count transpositions as a primitive edit.
*/ */
public LevenshteinAutomata(String input, boolean withTranspositions) { public LevenshteinAutomata(String input, boolean withTranspositions) {
this.input = input; this(codePoints(input), Character.MAX_CODE_POINT, withTranspositions);
int length = Character.codePointCount(input, 0, input.length());
word = new int[length];
for (int i = 0, j = 0, cp = 0; i < input.length(); i += Character.charCount(cp)) {
word[j++] = cp = input.codePointAt(i);
} }
/**
* Expert: Don't use this!
*/
public LevenshteinAutomata(int[] word, int alphaMax, boolean withTranspositions) {
this.word = word;
this.alphaMax = alphaMax;
// calculate the alphabet // calculate the alphabet
SortedSet<Integer> set = new TreeSet<Integer>(); SortedSet<Integer> set = new TreeSet<Integer>();
for (int i = 0; i < word.length; i++) for (int i = 0; i < word.length; i++)
@ -81,9 +85,9 @@ public class LevenshteinAutomata {
lower = higher + 1; lower = higher + 1;
} }
/* add the final endpoint */ /* add the final endpoint */
if (lower <= Character.MAX_CODE_POINT) { if (lower <= alphaMax) {
rangeLower[numRanges] = lower; rangeLower[numRanges] = lower;
rangeUpper[numRanges] = Character.MAX_CODE_POINT; rangeUpper[numRanges] = alphaMax;
numRanges++; numRanges++;
} }
@ -94,6 +98,15 @@ public class LevenshteinAutomata {
}; };
} }
private static int[] codePoints(String input) {
int length = Character.codePointCount(input, 0, input.length());
int word[] = new int[length];
for (int i = 0, j = 0, cp = 0; i < input.length(); i += Character.charCount(cp)) {
word[j++] = cp = input.codePointAt(i);
}
return word;
}
/** /**
* Compute a DFA that accepts all strings within an edit distance of <code>n</code>. * Compute a DFA that accepts all strings within an edit distance of <code>n</code>.
* <p> * <p>
@ -106,8 +119,9 @@ public class LevenshteinAutomata {
* </p> * </p>
*/ */
public Automaton toAutomaton(int n) { public Automaton toAutomaton(int n) {
if (n == 0) if (n == 0) {
return BasicAutomata.makeString(input); return BasicAutomata.makeString(word, 0, word.length);
}
if (n >= descriptions.length) if (n >= descriptions.length)
return null; return null;

View File

@ -22,6 +22,8 @@ import java.util.*;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
/** Static helper methods. /** Static helper methods.
* *
@ -304,7 +306,10 @@ public final class Util {
path.input.ints[path.input.length++] = path.arc.label; path.input.ints[path.input.length++] = path.arc.label;
final int cmp = bottom.input.compareTo(path.input); final int cmp = bottom.input.compareTo(path.input);
path.input.length--; path.input.length--;
// We should never see dups:
assert cmp != 0; assert cmp != 0;
if (cmp < 0) { if (cmp < 0) {
// Doesn't compete // Doesn't compete
return; return;
@ -329,12 +334,20 @@ public final class Util {
//newPath.input.ints[path.input.length] = path.arc.label; //newPath.input.ints[path.input.length] = path.arc.label;
//newPath.input.length = path.input.length+1; //newPath.input.length = path.input.length+1;
//System.out.println(" add path=" + newPath); //System.out.println(" add path=" + newPath + (bottom == null ? "" : (" newPath.compareTo(bottom)=" + newPath.compareTo(bottom))) + " bottom=" + bottom + " topN=" + topN);
// We should never see dups:
assert bottom == null || newPath.compareTo(bottom) != 0;
queue.add(newPath); queue.add(newPath);
if (bottom != null) { if (bottom != null) {
final FSTPath<T> removed = queue.pollLast(); final FSTPath<T> removed = queue.pollLast();
assert removed == bottom; assert removed == bottom;
if (queue.size() == 0) {
bottom = null;
} else {
bottom = queue.last(); bottom = queue.last();
}
//System.out.println(" now re-set bottom: " + bottom + " queue=" + queue); //System.out.println(" now re-set bottom: " + bottom + " queue=" + queue);
} else if (queue.size() == topN) { } else if (queue.size() == topN) {
// Queue just filled up: // Queue just filled up:
@ -854,4 +867,92 @@ public final class Util {
w.close(); w.close();
} }
*/ */
/**
* Reads the first arc greater or equal that the given label into the provided
* arc in place and returns it iff found, otherwise return <code>null</code>.
*
* @param label the label to ceil on
* @param fst the fst to operate on
* @param follow the arc to follow reading the label from
* @param arc the arc to read into in place
* @param in the fst's {@link BytesReader}
*/
public static <T> Arc<T> readCeilArc(int label, FST<T> fst, Arc<T> follow,
Arc<T> arc, BytesReader in) throws IOException {
if (label == FST.END_LABEL) {
if (follow.isFinal()) {
if (follow.target <= 0) {
arc.flags = FST.BIT_LAST_ARC;
} else {
arc.flags = 0;
// NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
arc.node = follow.target;
}
arc.output = follow.nextFinalOutput;
arc.label = FST.END_LABEL;
return arc;
} else {
return null;
}
}
if (!FST.targetHasArcs(follow)) {
return null;
}
fst.readFirstTargetArc(follow, arc, in);
if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
// Arcs are fixed array -- use binary search to find
// the target.
int low = arc.arcIdx;
int high = arc.numArcs - 1;
int mid = 0;
// System.out.println("do arc array low=" + low + " high=" + high +
// " targetLabel=" + targetLabel);
while (low <= high) {
mid = (low + high) >>> 1;
in.pos = arc.posArcsStart;
in.skip(arc.bytesPerArc * mid + 1);
final int midLabel = fst.readLabel(in);
final int cmp = midLabel - label;
// System.out.println(" cycle low=" + low + " high=" + high + " mid=" +
// mid + " midLabel=" + midLabel + " cmp=" + cmp);
if (cmp < 0) {
low = mid + 1;
} else if (cmp > 0) {
high = mid - 1;
} else {
arc.arcIdx = mid-1;
return fst.readNextRealArc(arc, in);
}
}
if (low == arc.numArcs) {
// DEAD END!
return null;
}
arc.arcIdx = (low > high ? high : low);
return fst.readNextRealArc(arc, in);
}
// Linear scan
fst.readFirstRealTargetArc(follow.target, arc, in);
while (true) {
// System.out.println(" non-bs cycle");
// TODO: we should fix this code to not have to create
// object for the output of every arc we scan... only
// for the matching arc, if found
if (arc.label >= label) {
// System.out.println(" found!");
return arc;
} else if (arc.isLast()) {
return null;
} else {
fst.readNextRealArc(arc, in);
}
}
}
} }

View File

@ -302,7 +302,7 @@ public class AnalyzingSuggester extends Lookup {
} }
} }
private TokenStreamToAutomaton getTokenStreamToAutomaton() { TokenStreamToAutomaton getTokenStreamToAutomaton() {
if (preserveSep) { if (preserveSep) {
return new EscapingTokenStreamToAutomaton(); return new EscapingTokenStreamToAutomaton();
} else { } else {
@ -324,6 +324,7 @@ public class AnalyzingSuggester extends Lookup {
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton(); TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
// analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short) // analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short)
boolean success = false; boolean success = false;
byte buffer[] = new byte[8]; byte buffer[] = new byte[8];
@ -331,29 +332,8 @@ public class AnalyzingSuggester extends Lookup {
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer); ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
BytesRef surfaceForm; BytesRef surfaceForm;
while ((surfaceForm = iterator.next()) != null) { while ((surfaceForm = iterator.next()) != null) {
Set<IntsRef> paths = toFiniteStrings(surfaceForm, ts2a);
// Analyze surface form:
TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
// Create corresponding automaton: labels are bytes
// from each analyzed token, with byte 0 used as
// separator between tokens:
Automaton automaton = ts2a.toAutomaton(ts);
ts.end();
ts.close();
replaceSep(automaton);
assert SpecialOperations.isFinite(automaton);
// Get all paths from the automaton (there can be
// more than one path, eg if the analyzer created a
// graph using SynFilter or WDF):
// TODO: we could walk & add simultaneously, so we
// don't have to alloc [possibly biggish]
// intermediate HashSet in RAM:
Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
for (IntsRef path : paths) { for (IntsRef path : paths) {
Util.toBytesRef(path, scratch); Util.toBytesRef(path, scratch);
@ -495,24 +475,7 @@ public class AnalyzingSuggester extends Lookup {
try { try {
// TODO: is there a Reader from a CharSequence? Automaton lookupAutomaton = toLookupAutomaton(key);
// Turn tokenstream into automaton:
TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
Automaton automaton = getTokenStreamToAutomaton().toAutomaton(ts);
ts.end();
ts.close();
// TODO: we could use the end offset to "guess"
// whether the final token was a partial token; this
// would only be a heuristic ... but maybe an OK one.
// This way we could eg differentiate "net" from "net ",
// which we can't today...
replaceSep(automaton);
// TODO: we can optimize this somewhat by determinizing
// while we convert
BasicOperations.determinize(automaton);
final CharsRef spare = new CharsRef(); final CharsRef spare = new CharsRef();
@ -520,8 +483,7 @@ public class AnalyzingSuggester extends Lookup {
// Intersect automaton w/ suggest wFST and get all // Intersect automaton w/ suggest wFST and get all
// prefix starting nodes & their outputs: // prefix starting nodes & their outputs:
final List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths; final PathIntersector intersector = getPathIntersector(lookupAutomaton, fst);
prefixPaths = FSTUtil.intersectPrefixPaths(automaton, fst);
//System.out.println(" prefixPaths: " + prefixPaths.size()); //System.out.println(" prefixPaths: " + prefixPaths.size());
@ -532,6 +494,7 @@ public class AnalyzingSuggester extends Lookup {
List<LookupResult> results = new ArrayList<LookupResult>(); List<LookupResult> results = new ArrayList<LookupResult>();
if (exactFirst) { if (exactFirst) {
final List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = intersector.intersectExact();
Util.TopNSearcher<Pair<Long,BytesRef>> searcher; Util.TopNSearcher<Pair<Long,BytesRef>> searcher;
searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst, num, weightComparator); searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst, num, weightComparator);
@ -617,8 +580,10 @@ public class AnalyzingSuggester extends Lookup {
} }
} }
}; };
final List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = intersector.intersectAll();
// System.out.println(key);
for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) { for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
// System.out.println(UnicodeUtil.newString(path.input.ints, path.input.offset, path.input.length));
searcher.addStartPaths(path.fstNode, path.output, true, path.input); searcher.addStartPaths(path.fstNode, path.output, true, path.input);
} }
@ -638,6 +603,55 @@ public class AnalyzingSuggester extends Lookup {
} }
} }
final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
// Analyze surface form:
TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
// Create corresponding automaton: labels are bytes
// from each analyzed token, with byte 0 used as
// separator between tokens:
Automaton automaton = ts2a.toAutomaton(ts);
ts.end();
ts.close();
replaceSep(automaton);
assert SpecialOperations.isFinite(automaton);
// Get all paths from the automaton (there can be
// more than one path, eg if the analyzer created a
// graph using SynFilter or WDF):
// TODO: we could walk & add simultaneously, so we
// don't have to alloc [possibly biggish]
// intermediate HashSet in RAM:
return SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
}
final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
// TODO: is there a Reader from a CharSequence?
// Turn tokenstream into automaton:
TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
ts.end();
ts.close();
// TODO: we could use the end offset to "guess"
// whether the final token was a partial token; this
// would only be a heuristic ... but maybe an OK one.
// This way we could eg differentiate "net" from "net ",
// which we can't today...
replaceSep(automaton);
// TODO: we can optimize this somewhat by determinizing
// while we convert
BasicOperations.determinize(automaton);
return automaton;
}
/** /**
* Returns the weight associated with an input string, * Returns the weight associated with an input string,
* or null if it does not exist. * or null if it does not exist.
@ -664,4 +678,25 @@ public class AnalyzingSuggester extends Lookup {
return left.output1.compareTo(right.output1); return left.output1.compareTo(right.output1);
} }
}; };
protected PathIntersector getPathIntersector(Automaton automaton, FST<Pair<Long,BytesRef>> fst) {
return new PathIntersector(automaton, fst);
}
protected static class PathIntersector {
protected List<FSTUtil.Path<Pair<Long,BytesRef>>> intersect;
protected final Automaton automaton;
protected final FST<Pair<Long,BytesRef>> fst;
public PathIntersector(Automaton automaton, FST<Pair<Long,BytesRef>> fst) {
this.automaton = automaton;
this.fst = fst;
}
public List<FSTUtil.Path<Pair<Long,BytesRef>>> intersectExact() throws IOException {
return intersect = FSTUtil.intersectPrefixPathsExact(automaton, fst);
}
public List<FSTUtil.Path<Pair<Long,BytesRef>>> intersectAll() throws IOException {
return intersect == null ? intersect = FSTUtil.intersectPrefixPathsExact(automaton, fst) : intersect;
}
}
} }

View File

@ -22,10 +22,12 @@ import java.util.List;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.State; import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.automaton.Transition; import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Util;
// TODO: move to core? nobody else uses it yet though... // TODO: move to core? nobody else uses it yet though...
@ -65,7 +67,7 @@ public class FSTUtil {
/** Enumerates all paths in the automaton that also /** Enumerates all paths in the automaton that also
* intersect the FST, accumulating the FST end node and * intersect the FST, accumulating the FST end node and
* output for each path. */ * output for each path. */
public static<T> List<Path<T>> intersectPrefixPaths(Automaton a, FST<T> fst) throws IOException { public static<T> List<Path<T>> intersectPrefixPathsExact(Automaton a, FST<T> fst) throws IOException {
final List<Path<T>> queue = new ArrayList<Path<T>>(); final List<Path<T>> queue = new ArrayList<Path<T>>();
final List<Path<T>> endNodes = new ArrayList<Path<T>>(); final List<Path<T>> endNodes = new ArrayList<Path<T>>();
@ -88,7 +90,6 @@ public class FSTUtil {
IntsRef currentInput = path.input; IntsRef currentInput = path.input;
for(Transition t : path.state.getTransitions()) { for(Transition t : path.state.getTransitions()) {
// TODO: we can fix this if necessary: // TODO: we can fix this if necessary:
if (t.getMin() != t.getMax()) { if (t.getMin() != t.getMax()) {
throw new IllegalStateException("can only handle Transitions that match one character"); throw new IllegalStateException("can only handle Transitions that match one character");
@ -115,4 +116,85 @@ public class FSTUtil {
return endNodes; return endNodes;
} }
/**
* nocommit javadoc
*/
public static <T> List<Path<T>> intersectPrefixPaths(Automaton a, FST<T> fst) throws IOException {
assert a.isDeterministic();
final List<Path<T>> queue = new ArrayList<Path<T>>();
final List<Path<T>> endNodes = new ArrayList<Path<T>>();
queue.add(new Path<T>(a.getInitialState(), fst
.getFirstArc(new FST.Arc<T>()), fst.outputs.getNoOutput(),
new IntsRef()));
final FST.Arc<T> scratchArc = new FST.Arc<T>();
final FST.BytesReader fstReader = fst.getBytesReader(0);
while (queue.size() != 0) {
final Path<T> path = queue.remove(queue.size() - 1);
if (path.state.isAccept()) {
endNodes.add(path);
continue;
}
// System.out.println(UnicodeUtil.newString(path.input.ints, path.input.offset, path.input.length));
IntsRef currentInput = path.input;
for (Transition t : path.state.getTransitions()) {
if (t.getMin() == t.getMax()) {
final FST.Arc<T> nextArc = fst.findTargetArc(t.getMin(),
path.fstNode, scratchArc, fstReader);
if (nextArc != null) {
final IntsRef newInput = new IntsRef(currentInput.length + 1);
newInput.copyInts(currentInput);
newInput.ints[currentInput.length] = t.getMin();
newInput.length = currentInput.length + 1;
// if (t.getDest().isAccept()) {
// System.out.println(UnicodeUtil.newString(newInput.ints, newInput.offset, newInput.length));
// }
queue.add(new Path<T>(t.getDest(), new FST.Arc<T>()
.copyFrom(nextArc), fst.outputs
.add(path.output, nextArc.output), newInput));
}
} else {
// TODO:
// if we accept the entire range possible in the FST (ie. 0 to 256)
// we can simply use the prefix as the accepted state instead of
// looking up all the
// ranges and terminate early here?
FST.Arc<T> nextArc = Util.readCeilArc(t.getMin(), fst, path.fstNode,
scratchArc, fstReader);
while (nextArc != null && nextArc.label <= t.getMax()) {
assert nextArc.label <= t.getMax();
assert nextArc.label >= t.getMin() : nextArc.label + " "
+ t.getMin();
final IntsRef newInput = new IntsRef(currentInput.length + 1);
newInput.copyInts(currentInput);
newInput.ints[currentInput.length] = nextArc.label;
newInput.length = currentInput.length + 1;
// if (t.getDest().isAccept()) {
// System.out.println(UnicodeUtil.newString(newInput.ints, newInput.offset, newInput.length));
// }
queue.add(new Path<T>(t.getDest(), new FST.Arc<T>()
.copyFrom(nextArc), fst.outputs
.add(path.output, nextArc.output), newInput));
final int label = nextArc.label; // used in assert
nextArc = nextArc.isLast() ? null : fst.readNextRealArc(nextArc,
fstReader);
assert nextArc == null || label < nextArc.label : "last: " + label
+ " next: " + nextArc.label;
}
}
}
}
//System.out.println();
for (Path<T> path2 : endNodes) {
if ("poales".equals(UnicodeUtil.newString(path2.input.ints, path2.input.offset, path2.input.length)))
System.out.println(UnicodeUtil.newString(path2.input.ints, path2.input.offset, path2.input.length));
}
return endNodes;
}
} }

View File

@ -0,0 +1,115 @@
package org.apache.lucene.search.suggest.analyzing;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester.PathIntersector;
import org.apache.lucene.search.suggest.analyzing.FSTUtil.Path;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LevenshteinAutomata;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs.Pair;
/*
* 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.
*/
public class FuzzySuggester extends AnalyzingSuggester {
private final int maxEdits;
private final boolean transpositions;
private final int minPrefix;
public FuzzySuggester(Analyzer analyzer) {
this(analyzer, analyzer);
}
public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, 1, true, 1);
}
// nocommit: probably want an option to like, require the first character or something :)
public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer,
int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions, int maxEdits, boolean transpositions, int minPrefix) {
super(indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions);
this.maxEdits = maxEdits;
this.transpositions = transpositions;
this.minPrefix = minPrefix;
}
@Override
protected PathIntersector getPathIntersector(Automaton automaton,
FST<Pair<Long,BytesRef>> fst) {
return new FuzzyPathIntersector(automaton, fst);
}
final Automaton toLevenshteinAutomata(Automaton automaton) {
// nocommit: how slow can this be :)
Set<IntsRef> ref = SpecialOperations.getFiniteStrings(automaton, -1);
Automaton subs[] = new Automaton[ref.size()];
int upto = 0;
for (IntsRef path : ref) {
if (path.length <= minPrefix) {
subs[upto] = BasicAutomata.makeString(path.ints, path.offset, path.length);
upto++;
} else {
Automaton prefix = BasicAutomata.makeString(path.ints, path.offset, minPrefix);
int ints[] = new int[path.length-minPrefix];
System.arraycopy(path.ints, path.offset+minPrefix, ints, 0, ints.length);
LevenshteinAutomata lev = new LevenshteinAutomata(ints, 256, transpositions);
Automaton levAutomaton = lev.toAutomaton(maxEdits);
Automaton combined = BasicOperations.concatenate(Arrays.asList(prefix, levAutomaton));
combined.setDeterministic(true); // its like the special case in concatenate itself, except we cloneExpanded already
subs[upto] = combined;
upto++;
}
}
if (subs.length == 0) {
return BasicAutomata.makeEmpty(); // matches nothing
} else if (subs.length == 1) {
return subs[0];
} else {
Automaton a = BasicOperations.union(Arrays.asList(subs));
// nocommit: we could call toLevenshteinAutomata() before det?
// this only happens if you have multiple paths anyway (e.g. synonyms)
BasicOperations.determinize(a);
return a;
}
}
private final class FuzzyPathIntersector extends PathIntersector {
public FuzzyPathIntersector(Automaton automaton,
FST<Pair<Long,BytesRef>> fst) {
super(automaton, fst);
}
@Override
public List<Path<Pair<Long,BytesRef>>> intersectAll() throws IOException {
return FSTUtil.intersectPrefixPaths(toLevenshteinAutomata(automaton),fst);
}
}
}

View File

@ -36,6 +36,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.search.suggest.Lookup; // javadocs import org.apache.lucene.search.suggest.Lookup; // javadocs
import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester; import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
import org.apache.lucene.search.suggest.analyzing.FuzzySuggester;
import org.apache.lucene.search.suggest.fst.FSTCompletionLookup; import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup; import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
import org.apache.lucene.search.suggest.jaspell.JaspellLookup; import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
@ -47,15 +48,18 @@ import org.junit.Ignore;
/** /**
* Benchmarks tests for implementations of {@link Lookup} interface. * Benchmarks tests for implementations of {@link Lookup} interface.
*/ */
@Ignore("COMMENT ME TO RUN BENCHMARKS!") //@Ignore("COMMENT ME TO RUN BENCHMARKS!")
public class LookupBenchmarkTest extends LuceneTestCase { public class LookupBenchmarkTest extends LuceneTestCase {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList( private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList(
FuzzySuggester.class,
AnalyzingSuggester.class,
JaspellLookup.class, JaspellLookup.class,
TSTLookup.class, TSTLookup.class,
FSTCompletionLookup.class, FSTCompletionLookup.class,
WFSTCompletionLookup.class, WFSTCompletionLookup.class
AnalyzingSuggester.class);
);
private final static int rounds = 15; private final static int rounds = 15;
private final static int warmup = 5; private final static int warmup = 5;
@ -212,8 +216,9 @@ public class LookupBenchmarkTest extends LuceneTestCase {
final List<String> input = new ArrayList<String>(benchmarkInput.size()); final List<String> input = new ArrayList<String>(benchmarkInput.size());
for (TermFreq tf : benchmarkInput) { for (TermFreq tf : benchmarkInput) {
String s = tf.term.utf8ToString(); String s = tf.term.utf8ToString();
input.add(s.substring(0, Math.min(s.length(), String sub = s.substring(0, Math.min(s.length(),
minPrefixLen + random.nextInt(maxPrefixLen - minPrefixLen + 1)))); minPrefixLen + random.nextInt(maxPrefixLen - minPrefixLen + 1)));
input.add(sub);
} }
BenchmarkResult result = measure(new Callable<Integer>() { BenchmarkResult result = measure(new Callable<Integer>() {
@ -250,7 +255,9 @@ public class LookupBenchmarkTest extends LuceneTestCase {
} }
return new BenchmarkResult(times, warmup, rounds); return new BenchmarkResult(times, warmup, rounds);
} catch (Exception e) { } catch (Exception e) {
e.printStackTrace();
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }

View File

@ -0,0 +1,843 @@
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.IOException;
import java.io.Reader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CannedBinaryTokenStream.BinaryToken;
import org.apache.lucene.analysis.CannedTokenStream;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenFilter;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.TokenStreamToAutomaton;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
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.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LevenshteinAutomata;
import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.fst.Util;
public class FuzzySuggesterTest extends LuceneTestCase {
/** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
public void testKeyword() throws Exception {
TermFreq keys[] = new TermFreq[] {
new TermFreq("foo", 50),
new TermFreq("bar", 10),
new TermFreq("barbar", 12),
new TermFreq("barbara", 6)
};
FuzzySuggester suggester = new FuzzySuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
suggester.build(new TermFreqArrayIterator(keys));
List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence("bariar", random()), false, 2);
assertEquals(2, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
results = suggester.lookup(_TestUtil.stringToCharSequence("barbr", random()), false, 2);
assertEquals(2, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
results = suggester.lookup(_TestUtil.stringToCharSequence("barbara", random()), false, 2);
assertEquals(2, results.size());
assertEquals("barbara", results.get(0).key.toString());
assertEquals(6, results.get(0).value, 0.01F);
results = suggester.lookup(_TestUtil.stringToCharSequence("barbar", random()), false, 2);
assertEquals(1, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
results = suggester.lookup(_TestUtil.stringToCharSequence("barbaa", random()), false, 2);
assertEquals(2, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
assertEquals("barbara", results.get(1).key.toString());
assertEquals(6, results.get(1).value, 0.01F);
String addRandomEdit = addRandomEdit("barbara", 1);
results = suggester.lookup(_TestUtil.stringToCharSequence(addRandomEdit, random()), false, 2);
assertEquals(addRandomEdit, 1, results.size());
assertEquals("barbara", results.get(0).key.toString());
assertEquals(6, results.get(0).value, 0.01F);
// top N of 2, but only foo is available
results = suggester.lookup(_TestUtil.stringToCharSequence("f", random()), false, 2);
assertEquals(1, results.size());
assertEquals("foo", results.get(0).key.toString());
assertEquals(50, results.get(0).value, 0.01F);
// top N of 1 for 'bar': we return this even though
// barbar is higher because exactFirst is enabled:
results = suggester.lookup(_TestUtil.stringToCharSequence("bar", random()), false, 1);
assertEquals(1, results.size());
assertEquals("bar", results.get(0).key.toString());
assertEquals(10, results.get(0).value, 0.01F);
// top N Of 2 for 'b'
results = suggester.lookup(_TestUtil.stringToCharSequence("b", random()), false, 2);
assertEquals(2, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
assertEquals("bar", results.get(1).key.toString());
assertEquals(10, results.get(1).value, 0.01F);
// top N of 3 for 'ba'
results = suggester.lookup(_TestUtil.stringToCharSequence("ba", random()), false, 3);
assertEquals(3, results.size());
assertEquals("barbar", results.get(0).key.toString());
assertEquals(12, results.get(0).value, 0.01F);
assertEquals("bar", results.get(1).key.toString());
assertEquals(10, results.get(1).value, 0.01F);
assertEquals("barbara", results.get(2).key.toString());
assertEquals(6, results.get(2).value, 0.01F);
}
// TODO: more tests
/**
* basic "standardanalyzer" test with stopword removal
*/
public void testStandard() throws Exception {
TermFreq keys[] = new TermFreq[] {
new TermFreq("the ghost of christmas past", 50),
};
Analyzer standard = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET, false);
FuzzySuggester suggester = new FuzzySuggester(standard);
suggester.build(new TermFreqArrayIterator(keys));
List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence("the ghost of chris", random()), false, 1);
assertEquals(1, results.size());
assertEquals("the ghost of christmas past", results.get(0).key.toString());
assertEquals(50, results.get(0).value, 0.01F);
// omit the 'the' since its a stopword, its suggested anyway
results = suggester.lookup(_TestUtil.stringToCharSequence("ghost of chris", random()), false, 1);
assertEquals(1, results.size());
assertEquals("the ghost of christmas past", results.get(0).key.toString());
assertEquals(50, results.get(0).value, 0.01F);
// omit the 'the' and 'of' since they are stopwords, its suggested anyway
results = suggester.lookup(_TestUtil.stringToCharSequence("ghost chris", random()), false, 1);
assertEquals(1, results.size());
assertEquals("the ghost of christmas past", results.get(0).key.toString());
assertEquals(50, results.get(0).value, 0.01F);
}
public void testNoSeps() throws Exception {
TermFreq[] keys = new TermFreq[] {
new TermFreq("ab cd", 0),
new TermFreq("abcd", 1),
};
int options = 0;
Analyzer a = new MockAnalyzer(random());
FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, 1, true, 1);
suggester.build(new TermFreqArrayIterator(keys));
// TODO: would be nice if "ab " would allow the test to
// pass, and more generally if the analyzer can know
// that the user's current query has ended at a word,
// but, analyzers don't produce SEP tokens!
List<LookupResult> r = suggester.lookup(_TestUtil.stringToCharSequence("ab c", random()), false, 2);
assertEquals(2, r.size());
// With no PRESERVE_SEPS specified, "ab c" should also
// complete to "abcd", which has higher weight so should
// appear first:
assertEquals("abcd", r.get(0).key.toString());
}
public void testGraphDups() throws Exception {
final Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
return new TokenStreamComponents(tokenizer) {
int tokenStreamCounter = 0;
final TokenStream[] tokenStreams = new TokenStream[] {
new CannedTokenStream(new Token[] {
token("wifi",1,1),
token("hotspot",0,2),
token("network",1,1),
token("is",1,1),
token("slow",1,1)
}),
new CannedTokenStream(new Token[] {
token("wi",1,1),
token("hotspot",0,3),
token("fi",1,1),
token("network",1,1),
token("is",1,1),
token("fast",1,1)
}),
new CannedTokenStream(new Token[] {
token("wifi",1,1),
token("hotspot",0,2),
token("network",1,1)
}),
};
@Override
public TokenStream getTokenStream() {
TokenStream result = tokenStreams[tokenStreamCounter];
tokenStreamCounter++;
return result;
}
@Override
protected void setReader(final Reader reader) throws IOException {
}
};
}
};
TermFreq keys[] = new TermFreq[] {
new TermFreq("wifi network is slow", 50),
new TermFreq("wi fi network is fast", 10),
};
FuzzySuggester suggester = new FuzzySuggester(analyzer);
suggester.build(new TermFreqArrayIterator(keys));
List<LookupResult> results = suggester.lookup("wifi network", false, 10);
if (VERBOSE) {
System.out.println("Results: " + results);
}
assertEquals(2, results.size());
assertEquals("wifi network is slow", results.get(0).key);
assertEquals(50, results.get(0).value);
assertEquals("wi fi network is fast", results.get(1).key);
assertEquals(10, results.get(1).value);
}
public void testInputPathRequired() throws Exception {
// SynonymMap.Builder b = new SynonymMap.Builder(false);
// b.add(new CharsRef("ab"), new CharsRef("ba"), true);
// final SynonymMap map = b.build();
// The Analyzer below mimics the functionality of the SynonymAnalyzer
// using the above map, so that the suggest module does not need a dependency on the
// synonym module
final Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
return new TokenStreamComponents(tokenizer) {
int tokenStreamCounter = 0;
final TokenStream[] tokenStreams = new TokenStream[] {
new CannedTokenStream(new Token[] {
token("ab",1,1),
token("ba",0,1),
token("xc",1,1)
}),
new CannedTokenStream(new Token[] {
token("ba",1,1),
token("xd",1,1)
}),
new CannedTokenStream(new Token[] {
token("ab",1,1),
token("ba",0,1),
token("x",1,1)
})
};
@Override
public TokenStream getTokenStream() {
TokenStream result = tokenStreams[tokenStreamCounter];
tokenStreamCounter++;
return result;
}
@Override
protected void setReader(final Reader reader) throws IOException {
}
};
}
};
TermFreq keys[] = new TermFreq[] {
new TermFreq("ab xc", 50),
new TermFreq("ba xd", 50),
};
FuzzySuggester suggester = new FuzzySuggester(analyzer);
suggester.build(new TermFreqArrayIterator(keys));
List<LookupResult> results = suggester.lookup("ab x", false, 1);
assertTrue(results.size() == 1);
}
private static Token token(String term, int posInc, int posLength) {
final Token t = new Token(term, 0, 0);
t.setPositionIncrement(posInc);
t.setPositionLength(posLength);
return t;
}
private static BinaryToken token(BytesRef term) {
return new BinaryToken(term);
}
/*
private void printTokens(final Analyzer analyzer, String input) throws IOException {
System.out.println("Tokens for " + input);
TokenStream ts = analyzer.tokenStream("", new StringReader(input));
ts.reset();
final TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
final PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
final PositionLengthAttribute posLengthAtt = ts.addAttribute(PositionLengthAttribute.class);
while(ts.incrementToken()) {
termBytesAtt.fillBytesRef();
System.out.println(String.format("%s,%s,%s", termBytesAtt.getBytesRef().utf8ToString(), posIncAtt.getPositionIncrement(), posLengthAtt.getPositionLength()));
}
ts.end();
ts.close();
}
*/
private final Analyzer getUnusualAnalyzer() {
return new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
return new TokenStreamComponents(tokenizer) {
int count;
@Override
public TokenStream getTokenStream() {
// 4th time we are called, return tokens a b,
// else just a:
if (count++ != 3) {
return new CannedTokenStream(new Token[] {
token("a", 1, 1),
});
} else {
// After that "a b":
return new CannedTokenStream(new Token[] {
token("a", 1, 1),
token("b", 1, 1),
});
}
}
@Override
protected void setReader(final Reader reader) throws IOException {
}
};
}
};
}
public void testExactFirst() throws Exception {
Analyzer a = getUnusualAnalyzer();
FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1);
suggester.build(new TermFreqArrayIterator(new TermFreq[] {
new TermFreq("x y", 1),
new TermFreq("x y z", 3),
new TermFreq("x", 2),
new TermFreq("z z z", 20),
}));
//System.out.println("ALL: " + suggester.lookup("x y", false, 6));
for(int topN=1;topN<6;topN++) {
List<LookupResult> results = suggester.lookup("x y", false, topN);
//System.out.println("topN=" + topN + " " + results);
assertEquals(Math.min(topN, 4), results.size());
assertEquals("x y", results.get(0).key);
assertEquals(1, results.get(0).value);
if (topN > 1) {
assertEquals("z z z", results.get(1).key);
assertEquals(20, results.get(1).value);
if (topN > 2) {
assertEquals("x y z", results.get(2).key);
assertEquals(3, results.get(2).value);
if (topN > 3) {
assertEquals("x", results.get(3).key);
assertEquals(2, results.get(3).value);
}
}
}
}
}
public void testNonExactFirst() throws Exception {
Analyzer a = getUnusualAnalyzer();
FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1);
suggester.build(new TermFreqArrayIterator(new TermFreq[] {
new TermFreq("x y", 1),
new TermFreq("x y z", 3),
new TermFreq("x", 2),
new TermFreq("z z z", 20),
}));
for(int topN=1;topN<6;topN++) {
List<LookupResult> results = suggester.lookup("p", false, topN);
assertEquals(Math.min(topN, 4), results.size());
assertEquals("z z z", results.get(0).key);
assertEquals(20, results.get(0).value);
if (topN > 1) {
assertEquals("x y z", results.get(1).key);
assertEquals(3, results.get(1).value);
if (topN > 2) {
assertEquals("x", results.get(2).key);
assertEquals(2, results.get(2).value);
if (topN > 3) {
assertEquals("x y", results.get(3).key);
assertEquals(1, results.get(3).value);
}
}
}
}
}
// Holds surface form seperately:
private static class TermFreq2 implements Comparable<TermFreq2> {
public final String surfaceForm;
public final String analyzedForm;
public final long weight;
public TermFreq2(String surfaceForm, String analyzedForm, long weight) {
this.surfaceForm = surfaceForm;
this.analyzedForm = analyzedForm;
this.weight = weight;
}
@Override
public int compareTo(TermFreq2 other) {
int cmp = analyzedForm.compareTo(other.analyzedForm);
if (cmp != 0) {
return cmp;
} else if (weight > other.weight) {
return -1;
} else if (weight < other.weight) {
return 1;
} else {
assert false;
return 0;
}
}
}
static boolean isStopChar(char ch, int numStopChars) {
//System.out.println("IS? " + ch + ": " + (ch - 'a') + ": " + ((ch - 'a') < numStopChars));
return (ch - 'a') < numStopChars;
}
// Like StopFilter:
private static class TokenEater extends TokenFilter {
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final int numStopChars;
private final boolean preserveHoles;
private boolean first;
public TokenEater(boolean preserveHoles, TokenStream in, int numStopChars) {
super(in);
this.preserveHoles = preserveHoles;
this.numStopChars = numStopChars;
}
@Override
public void reset() throws IOException {
super.reset();
first = true;
}
@Override
public final boolean incrementToken() throws IOException {
int skippedPositions = 0;
while (input.incrementToken()) {
if (termAtt.length() != 1 || !isStopChar(termAtt.charAt(0), numStopChars)) {
int posInc = posIncrAtt.getPositionIncrement() + skippedPositions;
if (first) {
if (posInc == 0) {
// first token having posinc=0 is illegal.
posInc = 1;
}
first = false;
}
posIncrAtt.setPositionIncrement(posInc);
//System.out.println("RETURN term=" + termAtt + " numStopChars=" + numStopChars);
return true;
}
if (preserveHoles) {
skippedPositions += posIncrAtt.getPositionIncrement();
}
}
return false;
}
}
private static class MockTokenEatingAnalyzer extends Analyzer {
private int numStopChars;
private boolean preserveHoles;
public MockTokenEatingAnalyzer(int numStopChars, boolean preserveHoles) {
this.preserveHoles = preserveHoles;
this.numStopChars = numStopChars;
}
@Override
public TokenStreamComponents createComponents(String fieldName, Reader reader) {
MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false, MockTokenizer.DEFAULT_MAX_TOKEN_LENGTH);
tokenizer.setEnableChecks(true);
TokenStream next;
if (numStopChars != 0) {
next = new TokenEater(preserveHoles, tokenizer, numStopChars);
} else {
next = tokenizer;
}
return new TokenStreamComponents(tokenizer, next);
}
}
public void testRandom() throws Exception {
int numQueries = atLeast(100);
final List<TermFreq2> slowCompletor = new ArrayList<TermFreq2>();
final TreeSet<String> allPrefixes = new TreeSet<String>();
final Set<String> seen = new HashSet<String>();
TermFreq[] keys = new TermFreq[numQueries];
boolean preserveSep = random().nextBoolean();
final int numStopChars = random().nextInt(10);
final boolean preserveHoles = random().nextBoolean();
if (VERBOSE) {
System.out.println("TEST: " + numQueries + " words; preserveSep=" + preserveSep + " numStopChars=" + numStopChars + " preserveHoles=" + preserveHoles);
}
for (int i = 0; i < numQueries; i++) {
int numTokens = _TestUtil.nextInt(random(), 1, 4);
String key;
String analyzedKey;
while(true) {
key = "";
analyzedKey = "";
for(int token=0;token < numTokens;token++) {
String s;
while (true) {
// TODO: would be nice to fix this slowCompletor/comparator to
// use full range, but we might lose some coverage too...
s = _TestUtil.randomSimpleString(random());
if (s.length() > 0) {
if (token > 0) {
key += " ";
}
if (preserveSep && analyzedKey.length() > 0 && analyzedKey.charAt(analyzedKey.length()-1) != ' ') {
analyzedKey += " ";
}
key += s;
if (s.length() == 1 && isStopChar(s.charAt(0), numStopChars)) {
if (preserveSep && preserveHoles) {
analyzedKey += '\u0000';
}
} else {
analyzedKey += s;
}
break;
}
}
}
analyzedKey = analyzedKey.replaceAll("(^| )\u0000$", "");
// Don't add same surface form more than once:
if (!seen.contains(key)) {
seen.add(key);
break;
}
}
for (int j = 1; j < key.length(); j++) {
allPrefixes.add(key.substring(0, j));
}
// we can probably do Integer.MAX_VALUE here, but why worry.
int weight = random().nextInt(1<<24);
keys[i] = new TermFreq(key, weight);
slowCompletor.add(new TermFreq2(key, analyzedKey, weight));
}
if (VERBOSE) {
// Don't just sort original list, to avoid VERBOSE
// altering the test:
List<TermFreq2> sorted = new ArrayList<TermFreq2>(slowCompletor);
Collections.sort(sorted);
for(TermFreq2 ent : sorted) {
System.out.println(" surface='" + ent.surfaceForm + " analyzed='" + ent.analyzedForm + "' weight=" + ent.weight);
}
}
Analyzer a = new MockTokenEatingAnalyzer(numStopChars, preserveHoles);
FuzzySuggester suggester = new FuzzySuggester(a, a,
preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, 1, false, 1);
suggester.build(new TermFreqArrayIterator(keys));
for (String prefix : allPrefixes) {
if (VERBOSE) {
System.out.println("\nTEST: prefix=" + prefix);
}
final int topN = _TestUtil.nextInt(random(), 1, 10);
List<LookupResult> r = suggester.lookup(_TestUtil.stringToCharSequence(prefix, random()), true, topN);
// 2. go thru whole set to find suggestions:
List<LookupResult> matches = new ArrayList<LookupResult>();
// "Analyze" the key:
String[] tokens = prefix.split(" ");
StringBuilder builder = new StringBuilder();
for(int i=0;i<tokens.length;i++) {
String token = tokens[i];
if (preserveSep && builder.length() > 0 && !builder.toString().endsWith(" ")) {
builder.append(' ');
}
if (token.length() == 1 && isStopChar(token.charAt(0), numStopChars)) {
if (preserveSep && preserveHoles) {
builder.append("\u0000");
}
} else {
builder.append(token);
}
}
String analyzedKey = builder.toString();
// Remove trailing sep/holes (TokenStream.end() does
// not tell us any trailing holes, yet ... there is an
// issue open for this):
while (true) {
String s = analyzedKey.replaceAll("(^| )\u0000$", "");
s = s.replaceAll("\\s+$", "");
if (s.equals(analyzedKey)) {
break;
}
analyzedKey = s;
}
if (analyzedKey.length() == 0) {
// Currently suggester can't suggest from the empty
// string! You get no results, not all results...
continue;
}
if (VERBOSE) {
System.out.println(" analyzed: " + analyzedKey);
}
TokenStreamToAutomaton tokenStreamToAutomaton = suggester.getTokenStreamToAutomaton();
Automaton automaton = suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey));
assertTrue(automaton.isDeterministic());
// TODO: could be faster... but its slowCompletor for a reason
BytesRef spare = new BytesRef();
for (TermFreq2 e : slowCompletor) {
spare.copyChars(e.analyzedForm);
Set<IntsRef> finiteStrings = suggester.toFiniteStrings(spare, tokenStreamToAutomaton);
for (IntsRef intsRef : finiteStrings) {
State p = automaton.getInitialState();
BytesRef ref = Util.toBytesRef(intsRef, spare);
boolean added = false;
for (int i = ref.offset; i < ref.length; i++) {
State q = p.step(ref.bytes[i] & 0xff);
if (q == null) {
break;
} else if (q.isAccept()) {
matches.add(new LookupResult(e.surfaceForm, e.weight));
added = true;
break;
}
p = q;
}
if (!added && p.isAccept()) {
matches.add(new LookupResult(e.surfaceForm, e.weight));
}
}
}
assertTrue(numStopChars > 0 || matches.size() > 0);
if (matches.size() > 1) {
Collections.sort(matches, new Comparator<LookupResult>() {
public int compare(LookupResult left, LookupResult right) {
int cmp = Float.compare(right.value, left.value);
if (cmp == 0) {
return left.compareTo(right);
} else {
return cmp;
}
}
});
}
if (matches.size() > topN) {
matches = matches.subList(0, topN);
}
if (VERBOSE) {
System.out.println(" expected:");
for(LookupResult lr : matches) {
System.out.println(" key=" + lr.key + " weight=" + lr.value);
}
System.out.println(" actual:");
for(LookupResult lr : r) {
System.out.println(" key=" + lr.key + " weight=" + lr.value);
}
}
assertEquals(prefix + " " + topN, matches.size(), r.size());
for(int hit=0;hit<r.size();hit++) {
//System.out.println(" check hit " + hit);
assertEquals(prefix + " " + topN, matches.get(hit).key.toString(), r.get(hit).key.toString());
assertEquals(matches.get(hit).value, r.get(hit).value, 0f);
}
}
}
public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
Analyzer a = new MockAnalyzer(random());
FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, 1, true, 1);
List<TermFreq> keys = Arrays.asList(new TermFreq[] {
new TermFreq("a", 40),
new TermFreq("a ", 50),
new TermFreq(" a", 60),
});
Collections.shuffle(keys, random());
suggester.build(new TermFreqArrayIterator(keys));
List<LookupResult> results = suggester.lookup("a", false, 5);
assertEquals(2, results.size());
assertEquals(" a", results.get(0).key);
assertEquals(60, results.get(0).value);
assertEquals("a ", results.get(1).key);
assertEquals(50, results.get(1).value);
}
public String addRandomEdit(String string, int prefixLenght) {
char[] charArray = string.toCharArray();
StringBuilder builder = new StringBuilder();
for (int i = 0; i < charArray.length; i++) {
if (i >= prefixLenght && random().nextBoolean() && i < charArray.length-1) {
switch(random().nextInt(3)){
case 2:
for (int j = i+1; j < charArray.length; j++) {
builder.append(charArray[j]);
}
return builder.toString();
case 1:
if (i+1<charArray.length) {
builder.append(charArray[i+1]);
builder.append(charArray[i++]);
i++;
}
for (int j = i; j < charArray.length; j++) {
builder.append(charArray[j]);
}
return builder.toString();
case 0:
int x = random().nextInt(128);
builder.append((char) x);
for (int j = i; j < charArray.length; j++) {
builder.append(charArray[j]);
}
return builder.toString();
}
}
builder.append(charArray[i]);
}
return builder.toString();
}
public Automaton getAutomaton(String string) {
IntsRef path = new IntsRef();
Util.toUTF32(string, path);
if (path.length <= 1) {
return BasicAutomata.makeString(path.ints, path.offset, path.length);
} else {
Automaton prefix = BasicAutomata.makeString(path.ints, path.offset, 1);
int ints[] = new int[path.length-1-1];
System.arraycopy(path.ints, path.offset+1, ints, 0, ints.length);
LevenshteinAutomata lev = new LevenshteinAutomata(ints, 256, true);
Automaton levAutomaton = lev.toAutomaton(1);
Automaton suffix = BasicAutomata.makeString(path.ints, path.length-1, 1);
Automaton combined = BasicOperations.concatenate(Arrays.asList(prefix, levAutomaton, suffix, BasicAutomata.makeAnyString()));
combined.setDeterministic(true); // its like the special case in concatenate itself, except we cloneExpanded already
return combined;
}
}
}

View File

@ -35,7 +35,8 @@ public class TestRuleAssertionsRequired implements TestRule {
String msg = "Test class requires enabled assertions, enable globally (-ea)" + String msg = "Test class requires enabled assertions, enable globally (-ea)" +
" or for Solr/Lucene subpackages only: " + description.getClassName(); " or for Solr/Lucene subpackages only: " + description.getClassName();
System.err.println(msg); System.err.println(msg);
throw new Exception(msg); // nocommit put back:
//throw new Exception(msg);
} catch (AssertionError e) { } catch (AssertionError e) {
// Ok, enabled. // Ok, enabled.
} }