You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/17 19:38:57 UTC

svn commit: r1232529 - in /lucene/dev/branches/branch_3x/lucene: ./ contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/ src/java/org/apache/lucene/util/fst/ src/test/org/apache/lucene/util/fst/

Author: rmuir
Date: Tue Jan 17 18:38:56 2012
New Revision: 1232529

URL: http://svn.apache.org/viewvc?rev=1232529&view=rev
Log:
LUCENE-3701: sync up FST apis to trunk

Modified:
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Tue Jan 17 18:38:56 2012
@@ -50,6 +50,12 @@ Changes in backwards compatibility polic
 
 * LUCENE-3695: Move confusing add(X) methods out of FST.Builder into
   FST.Util.  (Robert Muir, Mike McCandless)
+
+* LUCENE-3701: Added an additional argument to the expert FST.Builder
+  ctor to take FreezeTail, which you can use to (very-expertly) customize
+  the FST construction process. Pass null if you want the default
+  behavior.  Added seekExact() to FSTEnum, and added FST.save/read 
+  from a File. (Mike McCandless, Dawid Weiss, Robert Muir)
   
 Security fixes
 

Modified: lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Tue Jan 17 18:38:56 2012
@@ -217,7 +217,7 @@ public class FSTCompletionBuilder {
     final Object empty = outputs.getNoOutput();
     final Builder<Object> builder = new Builder<Object>(
         FST.INPUT_TYPE.BYTE1, 0, 0, true, true, 
-        shareMaxTailLength, outputs);
+        shareMaxTailLength, outputs, null);
     
     BytesRef scratch = new BytesRef();
     final IntsRef scratchIntsRef = new IntsRef();

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java Tue Jan 17 18:38:56 2012
@@ -52,6 +52,8 @@ public class Builder<T> {
   private final FST<T> fst;
   private final T NO_OUTPUT;
 
+  // private static final boolean DEBUG = false;
+
   // simplistic pruning: we prune node (and all following
   // nodes) if less than this number of terms go through it:
   private final int minSuffixCount1;
@@ -72,13 +74,21 @@ public class Builder<T> {
   // current "frontier"
   private UnCompiledNode<T>[] frontier;
 
+  // Expert: you pass an instance of this if you want to do
+  // something "custom" as suffixes are "frozen":
+  public static abstract class FreezeTail<T> {
+    public abstract void freeze(final UnCompiledNode<T>[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException;
+  }
+
+  private final FreezeTail<T> freezeTail;
+
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
-   * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs)} with 
+   * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs, FreezeTail)} with
    * pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
   }
 
   /**
@@ -119,9 +129,11 @@ public class Builder<T> {
    *    singleton output object.
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
-                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs) {
+                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
+                 FreezeTail<T> freezeTail) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
+    this.freezeTail = freezeTail;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
     fst = new FST<T>(inputType, outputs);
@@ -178,104 +190,135 @@ public class Builder<T> {
     return fn;
   }
 
-  private void compilePrevTail(int prefixLenPlus1) throws IOException {
-    assert prefixLenPlus1 >= 1;
-    //System.out.println("  compileTail " + prefixLenPlus1);
-    for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
-      boolean doPrune = false;
-      boolean doCompile = false;
+  private void freezeTail(int prefixLenPlus1) throws IOException {
+    if (freezeTail != null) {
+      // Custom plugin:
+      freezeTail.freeze(frontier, prefixLenPlus1, lastInput);
+    } else {
+      //System.out.println("  compileTail " + prefixLenPlus1);
+      final int downTo = Math.max(1, prefixLenPlus1);
+      for(int idx=lastInput.length; idx >= downTo; idx--) {
 
-      final UnCompiledNode<T> node = frontier[idx];
-      final UnCompiledNode<T> parent = frontier[idx-1];
+        boolean doPrune = false;
+        boolean doCompile = false;
 
-      if (node.inputCount < minSuffixCount1) {
-        doPrune = true;
-        doCompile = true;
-      } else if (idx > prefixLenPlus1) {
-        // prune if parent's inputCount is less than suffixMinCount2
-        if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) {
-          // my parent, about to be compiled, doesn't make the cut, so
-          // I'm definitely pruned 
-
-          // if pruneCount2 is 1, we keep only up
-          // until the 'distinguished edge', ie we keep only the
-          // 'divergent' part of the FST. if my parent, about to be
-          // compiled, has inputCount 1 then we are already past the
-          // distinguished edge.  NOTE: this only works if
-          // the FST outputs are not "compressible" (simple
-          // ords ARE compressible).
+        final UnCompiledNode<T> node = frontier[idx];
+        final UnCompiledNode<T> parent = frontier[idx-1];
+
+        if (node.inputCount < minSuffixCount1) {
           doPrune = true;
+          doCompile = true;
+        } else if (idx > prefixLenPlus1) {
+          // prune if parent's inputCount is less than suffixMinCount2
+          if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
+            // my parent, about to be compiled, doesn't make the cut, so
+            // I'm definitely pruned 
+
+            // if minSuffixCount2 is 1, we keep only up
+            // until the 'distinguished edge', ie we keep only the
+            // 'divergent' part of the FST. if my parent, about to be
+            // compiled, has inputCount 1 then we are already past the
+            // distinguished edge.  NOTE: this only works if
+            // the FST outputs are not "compressible" (simple
+            // ords ARE compressible).
+            doPrune = true;
+          } else {
+            // my parent, about to be compiled, does make the cut, so
+            // I'm definitely not pruned 
+            doPrune = false;
+          }
+          doCompile = true;
         } else {
-          // my parent, about to be compiled, does make the cut, so
-          // I'm definitely not pruned 
-          doPrune = false;
+          // if pruning is disabled (count is 0) we can always
+          // compile current node
+          doCompile = minSuffixCount2 == 0;
         }
-        doCompile = true;
-      } else {
-        // if pruning is disabled (count is 0) we can always
-        // compile current node
-        doCompile = minSuffixCount2 == 0;
-      }
 
-      //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
-
-      if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) {
-        // drop all arcs
-        for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-          @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
-          target.clear();
-        }
-        node.numArcs = 0;
-      }
+        //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
 
-      if (doPrune) {
-        // this node doesn't make it -- deref it
-        node.clear();
-        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
-      } else {
-
-        if (minSuffixCount2 != 0) {
-          compileAllTargets(node, lastInput.length-idx);
+        if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
+          // drop all arcs
+          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+            @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
+            target.clear();
+          }
+          node.numArcs = 0;
         }
-        final T nextFinalOutput = node.output;
 
-        // We "fake" the node as being final if it has no
-        // outgoing arcs; in theory we could leave it
-        // as non-final (the FST can represent this), but
-        // FSTEnum, Util, etc., have trouble w/ non-final
-        // dead-end states:
-        final boolean isFinal = node.isFinal || node.numArcs == 0;
-
-        if (doCompile) {
-          // this node makes it and we now compile it.  first,
-          // compile any targets that were previously
-          // undecided:
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             compileNode(node, 1+lastInput.length-idx),
-                             nextFinalOutput,
-                             isFinal);
+        if (doPrune) {
+          // this node doesn't make it -- deref it
+          node.clear();
+          parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
         } else {
-          // replaceLast just to install
-          // nextFinalOutput/isFinal onto the arc
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             node,
-                             nextFinalOutput,
-                             isFinal);
-          // this node will stay in play for now, since we are
-          // undecided on whether to prune it.  later, it
-          // will be either compiled or pruned, so we must
-          // allocate a new node:
-          frontier[idx] = new UnCompiledNode<T>(this, idx);
+
+          if (minSuffixCount2 != 0) {
+            compileAllTargets(node, lastInput.length-idx);
+          }
+          final T nextFinalOutput = node.output;
+
+          // We "fake" the node as being final if it has no
+          // outgoing arcs; in theory we could leave it
+          // as non-final (the FST can represent this), but
+          // FSTEnum, Util, etc., have trouble w/ non-final
+          // dead-end states:
+          final boolean isFinal = node.isFinal || node.numArcs == 0;
+
+          if (doCompile) {
+            // this node makes it and we now compile it.  first,
+            // compile any targets that were previously
+            // undecided:
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               compileNode(node, 1+lastInput.length-idx),
+                               nextFinalOutput,
+                               isFinal);
+          } else {
+            // replaceLast just to install
+            // nextFinalOutput/isFinal onto the arc
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               node,
+                               nextFinalOutput,
+                               isFinal);
+            // this node will stay in play for now, since we are
+            // undecided on whether to prune it.  later, it
+            // will be either compiled or pruned, so we must
+            // allocate a new node:
+            frontier[idx] = new UnCompiledNode<T>(this, idx);
+          }
         }
       }
     }
   }
 
+  // for debugging
+  /*
+  private String toString(BytesRef b) {
+    try {
+      return b.utf8ToString() + " " + b;
+    } catch (Throwable t) {
+      return b.toString();
+    }
+  }
+  */
+
   /** It's OK to add the same input twice in a row with
    *  different outputs, as long as outputs impls the merge
    *  method. */
   public void add(IntsRef input, T output) throws IOException {
-    //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
+    /*
+    if (DEBUG) {
+      BytesRef b = new BytesRef(input.length);
+      for(int x=0;x<input.length;x++) {
+        b.bytes[x] = (byte) input.ints[x];
+      }
+      b.length = input.length;
+      if (output == NO_OUTPUT) {
+        System.out.println("\nFST ADD: input=" + toString(b) + " " + b);
+      } else {
+        System.out.println("\nFST ADD: input=" + toString(b) + " " + b + " output=" + fst.outputs.outputToString(output));
+      }
+    }
+    */
+
     assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
     assert validOutput(output);
 
@@ -297,8 +340,8 @@ public class Builder<T> {
     int pos2 = input.offset;
     final int pos1Stop = Math.min(lastInput.length, input.length);
     while(true) {
-      //System.out.println("  incr " + pos1);
       frontier[pos1].inputCount++;
+      //System.out.println("  incr " + pos1 + " ct=" + frontier[pos1].inputCount + " n=" + frontier[pos1]);
       if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
         break;
       }
@@ -319,13 +362,12 @@ public class Builder<T> {
 
     // minimize/compile states from previous input's
     // orphan'd suffix
-    compilePrevTail(prefixLenPlus1);
+    freezeTail(prefixLenPlus1);
 
     // init tail states for current input
     for(int idx=prefixLenPlus1;idx<=input.length;idx++) {
       frontier[idx-1].addArc(input.ints[input.offset + idx - 1],
                              frontier[idx]);
-      //System.out.println("  incr tail " + idx);
       frontier[idx].inputCount++;
     }
 
@@ -384,34 +426,25 @@ public class Builder<T> {
    *  nothing is accepted by the FST. */
   public FST<T> finish() throws IOException {
 
+    final UnCompiledNode<T> root = frontier[0];
+
     // minimize nodes in the last word's suffix
-    compilePrevTail(1);
-    //System.out.println("finish: inputCount=" + frontier[0].inputCount);
-    if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
+    freezeTail(0);
+    if (root.inputCount < minSuffixCount1 || root.inputCount < minSuffixCount2 || root.numArcs == 0) {
       if (fst.emptyOutput == null) {
         return null;
       } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
         // empty string got pruned
         return null;
-      } else {
-        fst.finish(compileNode(frontier[0], lastInput.length).address);
-        //System.out.println("compile addr = " + fst.getStartNode());
-        return fst;
       }
     } else {
       if (minSuffixCount2 != 0) {
-        compileAllTargets(frontier[0], lastInput.length);
+        compileAllTargets(root, lastInput.length);
       }
-      //System.out.println("NOW: " + frontier[0].numArcs);
-      fst.finish(compileNode(frontier[0], lastInput.length).address);
     }
+    //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
+    fst.finish(compileNode(root, lastInput.length).address);
 
-    /*
-    if (dedupHash != null) {
-      System.out.println("NH: " + dedupHash.count()); 
-    }
-    */
-    
     return fst;
   }
 
@@ -430,7 +463,7 @@ public class Builder<T> {
     }
   }
 
-  static class Arc<T> {
+  public static class Arc<T> {
     public int label;                             // really an "unsigned" byte
     public Node target;
     public boolean isFinal;
@@ -453,16 +486,20 @@ public class Builder<T> {
     }
   }
 
-  static final class UnCompiledNode<T> implements Node {
+  public static final class UnCompiledNode<T> implements Node {
     final Builder<T> owner;
-    int numArcs;
-    Arc<T>[] arcs;
-    T output;
-    boolean isFinal;
-    long inputCount;
+    public int numArcs;
+    public Arc<T>[] arcs;
+    // TODO: instead of recording isFinal/output on the
+    // node, maybe we should use -1 arc to mean "end" (like
+    // we do when reading the FST).  Would simplify much
+    // code here...
+    public T output;
+    public boolean isFinal;
+    public long inputCount;
 
     /** This node's depth, starting from the automaton root. */
-    final int depth;
+    public final int depth;
 
     /**
      * @param depth

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Tue Jan 17 18:38:56 2012
@@ -133,6 +133,6 @@ public final class ByteSequenceOutputs e
 
   @Override
   public String outputToString(BytesRef output) {
-    return output.utf8ToString();
+    return output.toString();
   }
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java Tue Jan 17 18:38:56 2012
@@ -23,7 +23,7 @@ import org.apache.lucene.util.BytesRef;
 
 /** Can next() and advance() through the terms in an FST
  *
- * @lucene.experimental
+  * @lucene.experimental
 */
 
 public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
@@ -71,6 +71,21 @@ public final class BytesRefFSTEnum<T> ex
     return setResult();
   }
 
+  /** Seeks to exactly this term, returning null if the term
+   *  doesn't exist.  This is faster than using {@link
+   *  #seekFloor} or {@link #seekCeil} because it
+   *  short-circuits as soon the match is not found. */
+  public InputOutput<T> seekExact(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    if (super.doSeekExact()) {
+      assert upto == 1+target.length;
+      return setResult();
+    } else {
+      return null;
+    }
+  }
+
   @Override
   protected int getTargetLabel() {
     if (upto-1 == target.length) {

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java Tue Jan 17 18:38:56 2012
@@ -368,7 +368,7 @@ public class FST<T> {
       }
     }
   }
-  
+
   /**
    * Reads an automaton from a file. 
    */
@@ -670,8 +670,7 @@ public class FST<T> {
     }
   }
 
-  // Not private because NodeHash needs access:
-  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+  public Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
 
     final BytesReader in = getBytesReader(address);
 
@@ -754,7 +753,9 @@ public class FST<T> {
     return readLabel(in);
   }
 
-  Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+  /** Never returns null, but you should never call this if
+   *  arc.isLast() is true. */
+  public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
     // this is a continuing arc in a fixed array
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
@@ -932,7 +933,7 @@ public class FST<T> {
   public int getArcWithOutputCount() {
     return arcWithOutputCount;
   }
-  
+
   public void setAllowArrayArcs(boolean v) {
     allowArrayArcs = v;
   }
@@ -986,7 +987,7 @@ public class FST<T> {
     }
   }
 
-  final BytesReader getBytesReader(int pos) {
+  public final BytesReader getBytesReader(int pos) {
     // TODO: maybe re-use via ThreadLocal?
     return new BytesReader(pos);
   }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java Tue Jan 17 18:38:56 2012
@@ -73,6 +73,7 @@ abstract class FSTEnum<T> {
       final int cmp = getCurrentLabel() - getTargetLabel();
       if (cmp < 0) {
         // seek forward
+        //System.out.println("    seek fwd");
         break;
       } else if (cmp > 0) {
         // seek backwards -- reset this arc to the first arc
@@ -83,6 +84,7 @@ abstract class FSTEnum<T> {
       }
       upto++;
     }
+    //System.out.println("  fall through upto=" + upto);
   }
 
   protected void doNext() throws IOException {
@@ -352,7 +354,7 @@ abstract class FSTEnum<T> {
           //System.out.println(" hasFloor arcIdx=" + (arc.arcIdx+1));
           fst.readNextRealArc(arc, in);
           assert arc.isLast() || fst.readNextArcLabel(arc) > targetLabel;
-          assert arc.label < targetLabel;
+          assert arc.label < targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel;
           pushLast();
           return;
         }        
@@ -410,6 +412,48 @@ abstract class FSTEnum<T> {
     }
   }
 
+  /** Seeks to exactly target term. */
+  protected boolean doSeekExact() throws IOException {
+
+    // TODO: possibly caller could/should provide common
+    // prefix length?  ie this work may be redundant if
+    // caller is in fact intersecting against its own
+    // automaton
+
+    //System.out.println("FE: seek exact upto=" + upto);
+
+    // Save time by starting at the end of the shared prefix
+    // b/w our current term & the target:
+    rewindPrefix();
+
+    //System.out.println("FE: after rewind upto=" + upto);
+    FST.Arc<T> arc = getArc(upto-1);
+    int targetLabel = getTargetLabel();
+
+    while(true) {
+      //System.out.println("  cycle target=" + (targetLabel == -1 ? "-1" : (char) targetLabel));
+      final FST.Arc<T> nextArc = fst.findTargetArc(targetLabel, arc, getArc(upto));
+      if (nextArc == null) {
+        // short circuit
+        //upto--;
+        //upto = 0;
+        fst.readFirstTargetArc(arc, getArc(upto));
+        //System.out.println("  no match upto=" + upto);
+        return false;
+      }
+      // Match -- recurse:
+      output[upto] = fst.outputs.add(output[upto-1], nextArc.output);
+      if (targetLabel == FST.END_LABEL) {
+        //System.out.println("  return found; upto=" + upto + " output=" + output[upto] + " nextArc=" + nextArc.isLast());
+        return true;
+      }
+      setCurrentLabel(targetLabel);
+      incr();
+      targetLabel = getTargetLabel();
+      arc = nextArc;
+    }
+  }
+
   private void incr() {
     upto++;
     grow();

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java Tue Jan 17 18:38:56 2012
@@ -71,6 +71,21 @@ public final class IntsRefFSTEnum<T> ext
     return setResult();
   }
 
+  /** Seeks to exactly this term, returning null if the term
+   *  doesn't exist.  This is faster than using {@link
+   *  #seekFloor} or {@link #seekCeil} because it
+   *  short-circuits as soon the match is not found. */
+  public InputOutput<T> seekExact(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    if (super.doSeekExact()) {
+      assert upto == 1+target.length;
+      return setResult();
+    } else {
+      return null;
+    }
+  }
+
   @Override
   protected int getTargetLabel() {
     if (upto-1 == target.length) {

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java Tue Jan 17 18:38:56 2012
@@ -25,7 +25,7 @@ import org.apache.lucene.store.DataOutpu
 /**
  * Holds one or two longs for each input term.  If it's a
  * single output, Long is returned; else, TwoLongs.  Order
- * is preseved in the TwoLongs case, ie .first is the first
+ * is preserved in the TwoLongs case, ie .first is the first
  * input/output added to Builder, and .second is the
  * second.  You cannot store 0 output with this (that's
  * reserved to mean "no output")!

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1232529&r1=1232528&r2=1232529&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java Tue Jan 17 18:38:56 2012
@@ -458,7 +458,7 @@ public class TestFSTs extends LuceneTest
                                                 prune1==0 && prune2==0,
                                                 allowRandomSuffixSharing ? random.nextBoolean() : true,
                                                 allowRandomSuffixSharing ? _TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
-                                                outputs);
+                                                outputs, null);
 
       for(InputOutput<T> pair : pairs) {
         if (pair.output instanceof UpToTwoPositiveIntOutputs.TwoLongs) {
@@ -1175,7 +1175,7 @@ public class TestFSTs extends LuceneTest
       this.inputMode = inputMode;
       this.outputs = outputs;
       
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs);
+      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -1700,7 +1700,7 @@ public class TestFSTs extends LuceneTest
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
 
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null);
     builder.add(Util.toUTF32("stat", new IntsRef()), outputs.get(17));
     builder.add(Util.toUTF32("station", new IntsRef()), outputs.get(10));
     final FST<Long> fst = builder.finish();
@@ -1715,7 +1715,7 @@ public class TestFSTs extends LuceneTest
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
 
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();