You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/09/29 00:49:00 UTC

svn commit: r1391686 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/analysis/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/java/org/apache/lucene/util/automaton/ lucene/core/src/java/...

Author: mikemccand
Date: Fri Sep 28 22:48:59 2012
New Revision: 1391686

URL: http://svn.apache.org/viewvc?rev=1391686&view=rev
Log:
LUCENE-3842: add AnalyzingSuggester

Added:
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
      - copied unchanged from r1391683, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java
      - copied unchanged from r1391683, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/
      - copied from r1391683, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/
      - copied from r1391683, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java
      - copied unchanged from r1391683, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java
Removed:
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
    lucene/dev/branches/branch_4x/lucene/suggest/   (props changed)
    lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Fri Sep 28 22:48:59 2012
@@ -18,6 +18,15 @@ New Features
   output for a single input.  UpToTwoPositiveIntsOutputs was moved
   from lucene/core to lucene/misc.  (Mike McCandless)
 
+* LUCENE-3842: New AnalyzingCompletionLookup, for doing auto-suggest
+  using an analyzer.  This can create powerful suggesters: if the analyzer
+  remove stop words then "ghost chr..." could suggest "The Ghost of
+  Christmas Past"; if SynonymFilter is used to map wifi and wireless
+  network to hotspot, then "wirele..." could suggest "wifi router";
+  token normalization likes stemmers, accent removel, etc. would allow
+  the suggester to ignore such variations. (Robert Muir, Sudarshan
+  Gaikaiwari, Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-4411: when sampling is enabled for a FacetRequest, its depth

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java Fri Sep 28 22:48:59 2012
@@ -35,6 +35,8 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.fst.Util;
 
 /**
  * Special automata operations.
@@ -209,4 +211,60 @@ final public class SpecialOperations {
     a.clearNumberedStates();
     return accept;
   }
+
+  // TODO: this is a dangerous method ... Automaton could be
+  // huge ... and it's better in general for caller to
+  // enumerate & process in a single walk:
+
+  /**
+   * Returns the set of accepted strings, assuming that at most
+   * <code>limit</code> strings are accepted. If more than <code>limit</code> 
+   * strings are accepted, null is returned. If <code>limit</code>&lt;0, then 
+   * the limit is infinite.
+   */
+  public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
+    HashSet<IntsRef> strings = new HashSet<IntsRef>();
+    if (a.isSingleton()) {
+      if (limit > 0) {
+        strings.add(Util.toUTF32(a.singleton, new IntsRef()));
+      } else {
+        return null;
+      }
+    } else if (!getFiniteStrings(a.initial, new HashSet<State>(), strings, new IntsRef(), limit)) {
+      return null;
+    }
+    return strings;
+  }
+  
+  /**
+   * Returns the strings that can be produced from the given state, or
+   * false if more than <code>limit</code> strings are found. 
+   * <code>limit</code>&lt;0 means "infinite".
+   */
+  private static boolean getFiniteStrings(State s, HashSet<State> pathstates, 
+      HashSet<IntsRef> strings, IntsRef path, int limit) {
+    pathstates.add(s);
+    for (Transition t : s.getTransitions()) {
+      if (pathstates.contains(t.to)) {
+        return false;
+      }
+      for (int n = t.min; n <= t.max; n++) {
+        path.grow(path.length+1);
+        path.ints[path.length] = n;
+        path.length++;
+        if (t.to.accept) {
+          strings.add(IntsRef.deepCopyOf(path));
+          if (limit >= 0 && strings.size() > limit) {
+            return false;
+          }
+        }
+        if (!getFiniteStrings(t.to, pathstates, strings, path, limit)) {
+          return false;
+        }
+        path.length--;
+      }
+    }
+    pathstates.remove(s);
+    return true;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/State.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/State.java Fri Sep 28 22:48:59 2012
@@ -62,7 +62,7 @@ public class State implements Comparable
   /**
    * Resets transition set.
    */
-  final void resetTransitions() {
+  public final void resetTransitions() {
     transitionsArray = new Transition[0];
     numTransitions = 0;
   }
@@ -165,7 +165,11 @@ public class State implements Comparable
     }
   }
   
-  void addEpsilon(State to) {
+  /** Virtually adds an epsilon transition to the target
+   *  {@code to} state.  This is implemented by copying all
+   *  transitions from {@code to} to this state, and if {@code
+   *  to} is an accept state then set accept for this state. */
+  public void addEpsilon(State to) {
     if (to.accept) accept = true;
     for (Transition t : to.getTransitions())
       addTransition(t);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java Fri Sep 28 22:48:59 2012
@@ -118,7 +118,7 @@ public final class PositiveIntOutputs ex
 
   private boolean valid(Long o) {
     assert o != null;
-    assert o == NO_OUTPUT || o > 0;
+    assert o == NO_OUTPUT || o > 0: "o=" + o;
     return true;
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Fri Sep 28 22:48:59 2012
@@ -233,13 +233,14 @@ public final class Util {
   private static class FSTPath<T> implements Comparable<FSTPath<T>> {
     public FST.Arc<T> arc;
     public T cost;
-    public final IntsRef input = new IntsRef();
+    public final IntsRef input;
     final Comparator<T> comparator;
 
-    public FSTPath(T cost, FST.Arc<T> arc, Comparator<T> comparator) {
+    public FSTPath(T cost, FST.Arc<T> arc, Comparator<T> comparator, IntsRef input) {
       this.arc = new FST.Arc<T>().copyFrom(arc);
       this.cost = cost;
       this.comparator = comparator;
+      this.input = input;
     }
 
     @Override
@@ -258,11 +259,15 @@ public final class Util {
     }
   }
 
-  private static class TopNSearcher<T> {
+  /** Utility class to find top N shortest paths from start
+   *  point(s). */
+  public static class TopNSearcher<T> {
 
     private final FST<T> fst;
-    private final FST.Arc<T> fromNode;
+    private final FST.BytesReader bytesReader;
     private final int topN;
+
+    private final FST.Arc<T> scratchArc = new FST.Arc<T>();
     
     final Comparator<T> comparator;
 
@@ -271,11 +276,13 @@ public final class Util {
 
     TreeSet<FSTPath<T>> queue = null;
 
-    public TopNSearcher(FST<T> fst, FST.Arc<T> fromNode, int topN, Comparator<T> comparator) {
+    public TopNSearcher(FST<T> fst, int topN, Comparator<T> comparator) {
       this.fst = fst;
+      this.bytesReader = fst.getBytesReader(0);
       this.topN = topN;
-      this.fromNode = fromNode;
       this.comparator = comparator;
+
+      queue = new TreeSet<FSTPath<T>>();
     }
 
     // If back plus this arc is competitive then add to queue:
@@ -308,12 +315,19 @@ public final class Util {
         // Queue isn't full yet, so any path we hit competes:
       }
 
-      final FSTPath<T> newPath = new FSTPath<T>(cost, path.arc, comparator);
-
-      newPath.input.grow(path.input.length+1);
-      System.arraycopy(path.input.ints, 0, newPath.input.ints, 0, path.input.length);
-      newPath.input.ints[path.input.length] = path.arc.label;
-      newPath.input.length = path.input.length+1;
+      // copy over the current input to the new input
+      // and add the arc.label to the end
+      IntsRef newInput = new IntsRef(path.input.length+1);     
+      System.arraycopy(path.input.ints, 0, newInput.ints, 0, path.input.length);
+      newInput.ints[path.input.length] = path.arc.label;
+      newInput.length = path.input.length+1;
+      final FSTPath<T> newPath = new FSTPath<T>(cost, path.arc, comparator, newInput);
+
+      // this is pointless right?  we do it above already:
+      //newPath.input.grow(path.input.length+1);
+      //System.arraycopy(path.input.ints, 0, newPath.input.ints, 0, path.input.length);
+      //newPath.input.ints[path.input.length] = path.arc.label;
+      //newPath.input.length = path.input.length+1;
 
       //System.out.println("    add path=" + newPath);
       queue.add(newPath);
@@ -329,12 +343,38 @@ public final class Util {
       }
     }
 
+    /** Adds all leaving arcs, including 'finished' arc, if
+     *  the node is final, from this node into the queue.  */
+    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRef input) throws IOException {
+
+      // De-dup NO_OUTPUT since it must be a singleton:
+      if (startOutput.equals(fst.outputs.getNoOutput())) {
+        startOutput = fst.outputs.getNoOutput();
+      }
+
+      FSTPath<T> path = new FSTPath<T>(startOutput, node, comparator, input);
+      fst.readFirstTargetArc(node, path.arc, bytesReader);
+
+      //System.out.println("add start paths");
+
+      // Bootstrap: find the min starting arc
+      while (true) {
+        if (allowEmptyString || path.arc.label != FST.END_LABEL) {
+          addIfCompetitive(path);
+        }
+        if (path.arc.isLast()) {
+          break;
+        }
+        fst.readNextArc(path.arc, bytesReader);
+      }
+    }
+
     public MinResult<T>[] search() throws IOException {
-      //System.out.println("  search topN=" + topN);
-      final FST.Arc<T> scratchArc = new FST.Arc<T>();
 
       final List<MinResult<T>> results = new ArrayList<MinResult<T>>();
 
+      //System.out.println("search topN=" + topN);
+
       final FST.BytesReader fstReader = fst.getBytesReader(0);
       final T NO_OUTPUT = fst.outputs.getNoOutput();
 
@@ -352,69 +392,21 @@ public final class Util {
         FSTPath<T> path;
 
         if (queue == null) {
+          // Ran out of paths
+          break;
+        }
 
-          if (results.size() != 0) {
-            // Ran out of paths
-            break;
-          }
-
-          // First pass (top path): start from original fromNode
-          if (topN > 1) {
-            queue = new TreeSet<FSTPath<T>>();
-          }
-
-          T minArcCost = null;
-          FST.Arc<T> minArc = null;
-
-          path = new FSTPath<T>(NO_OUTPUT, fromNode, comparator);
-          fst.readFirstTargetArc(fromNode, path.arc, fstReader);
-
-          // Bootstrap: find the min starting arc
-          while (true) {
-            T arcScore = path.arc.output;
-            //System.out.println("  arc=" + (char) path.arc.label + " cost=" + arcScore);
-            if (minArcCost == null || comparator.compare(arcScore, minArcCost) < 0) {
-              minArcCost = arcScore;
-              minArc = scratchArc.copyFrom(path.arc);
-              //System.out.println("    **");
-            }
-            if (queue != null) {
-              addIfCompetitive(path);
-            }
-            if (path.arc.isLast()) {
-              break;
-            }
-            fst.readNextArc(path.arc, fstReader);
-          }
-
-          assert minArc != null;
-
-          if (queue != null) {
-            // Remove top path since we are now going to
-            // pursue it:
-            path = queue.pollFirst();
-            //System.out.println("  remove init path=" + path);
-            assert path.arc.label == minArc.label;
-            if (bottom != null && queue.size() == topN-1) {
-              bottom = queue.last();
-              //System.out.println("    set init bottom: " + bottom);
-            }
-          } else {
-            path.arc.copyFrom(minArc);
-            path.input.grow(1);
-            path.input.ints[0] = minArc.label;
-            path.input.length = 1;
-            path.cost = minArc.output;
-          }
-
-        } else {
-          path = queue.pollFirst();
-          if (path == null) {
-            // There were less than topN paths available:
-            break;
-          }
+        // Remove top path since we are now going to
+        // pursue it:
+        path = queue.pollFirst();
+
+        if (path == null) {
+          // There were less than topN paths available:
+          break;
         }
 
+        //System.out.println("  remove init path=" + path);
+
         if (path.arc.label == FST.END_LABEL) {
           //System.out.println("    empty string!  cost=" + path.cost);
           // Empty string!
@@ -480,7 +472,10 @@ public final class Util {
           if (path.arc.label == FST.END_LABEL) {
             // Add final output:
             //System.out.println("    done!: " + path);
-            results.add(new MinResult<T>(path.input, fst.outputs.add(path.cost, path.arc.output), comparator));
+            T finalOutput = fst.outputs.add(path.cost, path.arc.output);
+            if (acceptResult(path.input, finalOutput)) {
+              results.add(new MinResult<T>(path.input, finalOutput, comparator));
+            }
             break;
           } else {
             path.input.grow(1+path.input.length);
@@ -495,6 +490,10 @@ public final class Util {
         (MinResult<T>[]) new MinResult[results.size()];
       return results.toArray(arr);
     }
+
+    protected boolean acceptResult(IntsRef input, T output) {
+      return true;
+    }
   }
 
   /** Holds a single input (IntsRef) + output, returned by
@@ -521,14 +520,19 @@ public final class Util {
   }
 
   /** Starting from node, find the top N min cost 
-   * completions to a final node.
+   *  completions to a final node.
    *
    *  <p>NOTE: you must share the outputs when you build the
    *  FST (pass doShare=true to {@link
    *  PositiveIntOutputs#getSingleton}). */
-
-  public static <T> MinResult<T>[] shortestPaths(FST<T> fst, FST.Arc<T> fromNode, Comparator<T> comparator, int topN) throws IOException {
-    return new TopNSearcher<T>(fst, fromNode, topN, comparator).search();
+  public static <T> MinResult<T>[] shortestPaths(FST<T> fst, FST.Arc<T> fromNode, T startOutput, Comparator<T> comparator, int topN,
+                                                 boolean allowEmptyString) throws IOException {
+    TopNSearcher<T> searcher = new TopNSearcher<T>(fst, topN, comparator);
+
+    // since this search is initialized with a single start node 
+    // it is okay to start with an empty input path here
+    searcher.addStartPaths(fromNode, startOutput, allowEmptyString, new IntsRef());
+    return searcher.search();
   } 
 
   /**
@@ -832,9 +836,22 @@ public final class Util {
   public static BytesRef toBytesRef(IntsRef input, BytesRef scratch) {
     scratch.grow(input.length);
     for(int i=0;i<input.length;i++) {
-      scratch.bytes[i] = (byte) input.ints[i+input.offset];
+      int value = input.ints[i+input.offset];
+      // NOTE: we allow -128 to 255
+      assert value >= Byte.MIN_VALUE && value <= 255: "value " + value + " doesn't fit into byte";
+      scratch.bytes[i] = (byte) value;
     }
     scratch.length = input.length;
     return scratch;
   }
+
+  // Uncomment for debugging:
+
+  /*
+  public static <T> void dotToFile(FST<T> fst, String filePath) throws IOException {
+    Writer w = new OutputStreamWriter(new FileOutputStream(filePath));
+    toDot(fst, w, true, true);
+    w.close();
+  }
+  */
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Fri Sep 28 22:48:59 2012
@@ -17,9 +17,15 @@ package org.apache.lucene.analysis;
  * limitations under the License.
  */
 
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.io.Reader;
+import java.io.StringWriter;
+import java.io.PrintWriter;
+import java.io.Writer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 
@@ -27,6 +33,9 @@ import org.apache.lucene.analysis.tokena
 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.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.BasicOperations;
 
 public class TestGraphTokenizers extends BaseTokenStreamTestCase {
 
@@ -386,4 +395,229 @@ public class TestGraphTokenizers extends
       checkRandomData(random, a, 5, atLeast(1000));
     }
   }
+
+  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 Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
+    final Token t = new Token(term, startOffset, endOffset);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  public void testSingleToken() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected = BasicAutomata.makeString("abc");
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("b", 3, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testSynOverMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("x", 0, 3),
+        token("b", 3, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
+    final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
+    final Automaton expected = BasicOperations.union(a1, a2);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  // for debugging!
+  /*
+  private static void toDot(Automaton a) throws IOException {
+    final String s = a.toDot();
+    Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    w.write(s);
+    w.close();
+    System.out.println("TEST: saved to /x/tmp/out.dot");
+  }
+  */
+
+  private static final Automaton SEP_A = BasicAutomata.makeChar(TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton HOLE_A = BasicAutomata.makeChar(TokenStreamToAutomaton.HOLE);
+
+  private Automaton join(String ... strings) {
+    List<Automaton> as = new ArrayList<Automaton>();
+    for(String s : strings) {
+      as.add(BasicAutomata.makeString(s));
+      as.add(SEP_A);
+    }
+    as.remove(as.size()-1);
+    return BasicOperations.concatenate(as);
+  }
+
+  private Automaton join(Automaton ... as) {
+    return BasicOperations.concatenate(Arrays.asList(as));
+  }
+
+  private Automaton s2a(String s) {
+    return BasicAutomata.makeString(s);
+  }
+
+  public void testTwoTokens() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 1, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected =  join("abc", "def");
+
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 2, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+
+    final Automaton expected = join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"));
+
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testOverlappedTokensSausage() throws Exception {
+
+    // Two tokens on top of each other (sausage):
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 1)
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton a1 = BasicAutomata.makeString("abc");
+    final Automaton a2 = BasicAutomata.makeString("xyz");
+    final Automaton expected = BasicOperations.union(a1, a2);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testOverlappedTokensLattice() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 2),
+        token("def", 1, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a2 = join("abc", "def");
+                                                                   
+    final Automaton expected = BasicOperations.union(a1, a2);
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testSynOverHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 2),
+        token("b", 2, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton a1 = BasicOperations.union(
+                                               join(s2a("a"), SEP_A, HOLE_A),
+                                               BasicAutomata.makeString("X"));
+    final Automaton expected = BasicOperations.concatenate(a1,
+                                                           join(SEP_A, s2a("b")));
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testSynOverHole2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("xyz", 1, 1),
+        token("abc", 0, 3),
+        token("def", 2, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected = BasicOperations.union(
+                                                     join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")),
+                                                     BasicAutomata.makeString("abc"));
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testOverlappedTokensLattice2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 3),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a2 = join("abc", "def", "ghi");
+    final Automaton expected = BasicOperations.union(a1, a2);
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  public void testToDot() throws Exception {
+    final TokenStream ts = new CannedTokenStream(new Token[] {token("abc", 1, 1, 0, 4)});
+    StringWriter w = new StringWriter();
+    new TokenStreamToDot("abcd", ts, new PrintWriter(w)).toDot();
+    assertTrue(w.toString().indexOf("abc / abcd") != -1);
+  }
+
+  public void testStartsWithHole() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 2, 1),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected = join(HOLE_A, SEP_A, s2a("abc"));
+    //toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
+
+  public void testSynHangingOverEnd() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 10),
+      });
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
+    final Automaton expected = BasicOperations.union(BasicAutomata.makeString("a"),
+                                                     BasicAutomata.makeString("X"));
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Fri Sep 28 22:48:59 2012
@@ -21,9 +21,13 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.analysis.CannedBinaryTokenStream; // javadocs
 
 /**
- * a binary tokenstream that lets you index a BytesRef
+ * A binary tokenstream that lets you index a single
+ * binary token (BytesRef value).
+ *
+ * @see CannedBinaryTokenStream
  */
 public final class BinaryTokenStream extends TokenStream {
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
@@ -61,7 +65,7 @@ public final class BinaryTokenStream ext
     public BytesRef getBytesRef() {
       return bytes;
     }
-    
+
     public void setBytesRef(BytesRef bytes) {
       this.bytes = bytes;
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java Fri Sep 28 22:48:59 2012
@@ -1,6 +1,11 @@
 package org.apache.lucene.util.automaton;
 
+import java.util.Set;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.fst.Util;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -31,4 +36,20 @@ public class TestSpecialOperations exten
       assertEquals(AutomatonTestUtil.isFiniteSlow(a), SpecialOperations.isFinite(b));
     }
   }
+  
+  /**
+   * Basic test for getFiniteStrings
+   */
+  public void testFiniteStrings() {
+    Automaton a = BasicOperations.union(BasicAutomata.makeString("dog"), BasicAutomata.makeString("duck"));
+    MinimizationOperations.minimize(a);
+    Set<IntsRef> strings = SpecialOperations.getFiniteStrings(a, -1);
+    assertEquals(2, strings.size());
+    IntsRef dog = new IntsRef();
+    Util.toIntsRef(new BytesRef("dog"), dog);
+    assertTrue(strings.contains(dog));
+    IntsRef duck = new IntsRef();
+    Util.toIntsRef(new BytesRef("duck"), duck);
+    assertTrue(strings.contains(duck));
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Fri Sep 28 22:48:59 2012
@@ -1206,9 +1206,11 @@ public class TestFSTs extends LuceneTest
     //w.close();
 
     Util.MinResult<Long>[] r = Util.shortestPaths(fst,
-                                           fst.getFirstArc(new FST.Arc<Long>()),
-                                           minLongComparator,
-                                           3);
+                                                  fst.getFirstArc(new FST.Arc<Long>()),
+                                                  outputs.getNoOutput(),
+                                                  minLongComparator,
+                                                  3,
+                                                  true);
     assertEquals(3, r.length);
 
     assertEquals(Util.toIntsRef(new BytesRef("aac"), scratch), r[0].input);
@@ -1248,9 +1250,11 @@ public class TestFSTs extends LuceneTest
     //w.close();
 
     Util.MinResult<Pair<Long,Long>>[] r = Util.shortestPaths(fst,
-                                           fst.getFirstArc(new FST.Arc<Pair<Long,Long>>()),
-                                           minPairWeightComparator,
-                                           3);
+                                                             fst.getFirstArc(new FST.Arc<Pair<Long,Long>>()),
+                                                             outputs.getNoOutput(),
+                                                             minPairWeightComparator,
+                                                             3,
+                                                             true);
     assertEquals(3, r.length);
 
     assertEquals(Util.toIntsRef(new BytesRef("aac"), scratch), r[0].input);
@@ -1322,7 +1326,7 @@ public class TestFSTs extends LuceneTest
 
       final int topN = _TestUtil.nextInt(random, 1, 10);
 
-      Util.MinResult<Long>[] r = Util.shortestPaths(fst, arc, minLongComparator, topN);
+      Util.MinResult<Long>[] r = Util.shortestPaths(fst, arc, fst.outputs.getNoOutput(), minLongComparator, topN, true);
 
       // 2. go thru whole treemap (slowCompletor) and check its actually the best suggestion
       final List<Util.MinResult<Long>> matches = new ArrayList<Util.MinResult<Long>>();
@@ -1426,7 +1430,7 @@ public class TestFSTs extends LuceneTest
 
       final int topN = _TestUtil.nextInt(random, 1, 10);
 
-      Util.MinResult<Pair<Long,Long>>[] r = Util.shortestPaths(fst, arc, minPairWeightComparator, topN);
+      Util.MinResult<Pair<Long,Long>>[] r = Util.shortestPaths(fst, arc, fst.outputs.getNoOutput(), minPairWeightComparator, topN, true);
 
       // 2. go thru whole treemap (slowCompletor) and check its actually the best suggestion
       final List<Util.MinResult<Pair<Long,Long>>> matches = new ArrayList<Util.MinResult<Pair<Long,Long>>>();

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Fri Sep 28 22:48:59 2012
@@ -56,7 +56,6 @@ import org.apache.lucene.util.fst.Util.M
  * Input weights must be between 0 and {@link Integer#MAX_VALUE}, any
  * other values will be rejected.
  * 
- * @see Util#shortestPaths(FST, FST.Arc, Comparator, int)
  * @lucene.experimental
  */
 public class WFSTCompletionLookup extends Lookup {
@@ -168,12 +167,14 @@ public class WFSTCompletionLookup extend
         return results; // that was quick
       }
     }
-    
+
     // complete top-N
     MinResult<Long> completions[] = null;
     try {
-      completions = Util.shortestPaths(fst, arc, weightComparator, num);
-    } catch (IOException bogus) { throw new RuntimeException(bogus); }
+      completions = Util.shortestPaths(fst, arc, prefixOutput, weightComparator, num, !exactFirst);
+    } catch (IOException bogus) {
+      throw new RuntimeException(bogus);
+    }
     
     BytesRef suffix = new BytesRef(8);
     for (MinResult<Long> completion : completions) {
@@ -183,7 +184,7 @@ public class WFSTCompletionLookup extend
       scratch.append(suffix);
       spare.grow(scratch.length);
       UnicodeUtil.UTF8toUTF16(scratch, spare);
-      results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput + completion.output)));
+      results.add(new LookupResult(spare.toString(), decodeWeight(completion.output)));
     }
     return results;
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java Fri Sep 28 22:48:59 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest
 
 import java.io.BufferedReader;
 import java.io.InputStreamReader;
+import java.lang.reflect.Constructor;
 import java.net.URL;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
@@ -30,7 +31,11 @@ import java.util.Random;
 import java.util.concurrent.Callable;
 
 import org.apache.lucene.util.*;
-import org.apache.lucene.search.suggest.Lookup;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.search.suggest.Lookup; // javadocs
+import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
 import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
@@ -49,7 +54,8 @@ public class LookupBenchmarkTest extends
       JaspellLookup.class, 
       TSTLookup.class,
       FSTCompletionLookup.class,
-      WFSTCompletionLookup.class);
+      WFSTCompletionLookup.class,
+      AnalyzingSuggester.class);
 
   private final static int rounds = 15;
   private final static int warmup = 5;
@@ -133,10 +139,19 @@ public class LookupBenchmarkTest extends
     System.err.println("-- RAM consumption");
     for (Class<? extends Lookup> cls : benchmarkClasses) {
       Lookup lookup = buildLookup(cls, dictionaryInput);
+      long sizeInBytes;
+      if (lookup instanceof AnalyzingSuggester) {
+        // Just get size of FST: else we are also measuring
+        // size of MockAnalyzer which is non-trivial and
+        // varies depending on test seed:
+        sizeInBytes = ((AnalyzingSuggester) lookup).sizeInBytes();
+      } else {
+        sizeInBytes = RamUsageEstimator.sizeOf(lookup);
+      }
       System.err.println(
           String.format(Locale.ROOT, "%-15s size[B]:%,13d",
               lookup.getClass().getSimpleName(), 
-              RamUsageEstimator.sizeOf(lookup)));
+              sizeInBytes));
     }
   }
 
@@ -144,7 +159,13 @@ public class LookupBenchmarkTest extends
    * Create {@link Lookup} instance and populate it. 
    */
   private Lookup buildLookup(Class<? extends Lookup> cls, TermFreq[] input) throws Exception {
-    Lookup lookup = cls.newInstance();
+    Lookup lookup = null;
+    try {
+      lookup = cls.newInstance();
+    } catch (InstantiationException e) {
+      Constructor<? extends Lookup> ctor = cls.getConstructor(Analyzer.class);
+      lookup = ctor.newInstance(new MockAnalyzer(random, MockTokenizer.KEYWORD, false));
+    }
     lookup.build(new TermFreqArrayIterator(input));
     return lookup;
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java?rev=1391686&r1=1391685&r2=1391686&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java Fri Sep 28 22:48:59 2012
@@ -44,7 +44,13 @@ public class WFSTCompletionTest extends 
     assertEquals(1, results.size());
     assertEquals("foo", results.get(0).key.toString());
     assertEquals(50, results.get(0).value, 0.01F);
-    
+
+    // make sure we don't get a dup exact suggestion:
+    results = suggester.lookup(_TestUtil.stringToCharSequence("foo", random), true, 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
     results = suggester.lookup(_TestUtil.stringToCharSequence("bar", random), false, 1);
     assertEquals(1, results.size());
@@ -69,6 +75,54 @@ public class WFSTCompletionTest extends 
     assertEquals("barbara", results.get(2).key.toString());
     assertEquals(6, results.get(2).value, 0.01F);
   }
+
+  public void testExactFirst() throws Exception {
+
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(true);
+
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+          new TermFreq("x y", 20),
+          new TermFreq("x", 2),
+        }));
+
+    for(int topN=1;topN<4;topN++) {
+      List<LookupResult> results = suggester.lookup("x", false, topN);
+
+      assertEquals(Math.min(topN, 2), results.size());
+
+      assertEquals("x", results.get(0).key);
+      assertEquals(2, results.get(0).value);
+
+      if (topN > 1) {
+        assertEquals("x y", results.get(1).key);
+        assertEquals(20, results.get(1).value);
+      }
+    }
+  }
+
+  public void testNonExactFirst() throws Exception {
+
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+          new TermFreq("x y", 20),
+          new TermFreq("x", 2),
+        }));
+
+    for(int topN=1;topN<4;topN++) {
+      List<LookupResult> results = suggester.lookup("x", false, topN);
+
+      assertEquals(Math.min(topN, 2), results.size());
+
+      assertEquals("x y", results.get(0).key);
+      assertEquals(20, results.get(0).value);
+
+      if (topN > 1) {
+        assertEquals("x", results.get(1).key);
+        assertEquals(2, results.get(1).value);
+      }
+    }
+  }
   
   public void testRandom() throws Exception {
     int numWords = atLeast(1000);