Index: lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java	(revision 1337224)
+++ lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java	(working copy)
@@ -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 @@
       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));
+  }
 }
Index: lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java	(revision 1337224)
+++ lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java	(working copy)
@@ -17,9 +17,15 @@
  * 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.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,177 @@
       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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton expected = BasicAutomata.makeString("abc");
+    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("/tmp/out.dot"));
+    w.write(s);
+    w.close();
+    System.out.println("TEST: saved to /tmp/out.dot");
+  }
+
+  private static final Automaton SEP_A = BasicAutomata.makeCharRange(TokenStreamToAutomaton.POS_SEP,
+                                                                     TokenStreamToAutomaton.POS_SEP);
+
+  private static final String SEP = "" + (char) TokenStreamToAutomaton.POS_SEP;
+
+  private static final String HOLE = "" + (char) TokenStreamToAutomaton.HOLE;
+
+  public void testTwoTokens() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 1, 1),
+      });
+    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton expected =  BasicAutomata.makeString("abc" + SEP + "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 = TokenStreamToAutomaton.toAutomaton(ts);
+
+    // def is orphan'd by the hole:
+    final Automaton expected = BasicAutomata.makeString("abc\u0100\u0101\u0100def");
+
+    //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 = 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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a2 = BasicAutomata.makeString("abc" + SEP + "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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton a1 = BasicOperations.union(
+                                               BasicAutomata.makeString("a" + SEP + HOLE),
+                                               BasicAutomata.makeString("X"));
+    final Automaton expected = BasicOperations.concatenate(a1,
+                                                           BasicAutomata.makeString(SEP + "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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton expected = BasicOperations.union(
+                                                     BasicAutomata.makeString("xyz" + SEP + HOLE + SEP + "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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a2 = BasicAutomata.makeString("abc" + SEP + "def" + SEP + "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 = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton expected = BasicAutomata.makeString(HOLE + SEP + "abc");
+    toDot(actual);
+    assertTrue(BasicOperations.sameLanguage(expected, actual));
+  }
+
+  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
 }
Index: lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java	(revision 0)
+++ lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java	(working copy)
@@ -0,0 +1,136 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// TODO: probably move this to core at some point (eg,
+// cutover kuromoji, synfilter, LookaheadTokenFilter)
+
+/** Acts like forever growing T[], but internally uses a
+ *  circular buffer to reuse instances of T.
+ * 
+ *  @lucene.internal */
+public abstract class RollingBuffer<T extends RollingBuffer.Resettable> {
+
+  public static interface Resettable {
+    public void reset();
+  }
+
+  @SuppressWarnings("unchecked") private T[] buffer = (T[]) new RollingBuffer.Resettable[8];
+
+  // Next array index to write to:
+  private int nextWrite;
+
+  // Next position to write:
+  private int nextPos;
+
+  // How many valid Position are held in the
+  // array:
+  private int count;
+
+  public RollingBuffer() {
+    for(int idx=0;idx<buffer.length;idx++) {
+      buffer[idx] = newInstance();
+    }
+  }
+
+  protected abstract T newInstance();
+
+  public void reset() {
+    nextWrite--;
+    while (count > 0) {
+      if (nextWrite == -1) {
+        nextWrite = buffer.length - 1;
+      }
+      buffer[nextWrite--].reset();
+      count--;
+    }
+    nextWrite = 0;
+    nextPos = 0;
+    count = 0;
+  }
+
+  // For assert:
+  private boolean inBounds(int pos) {
+    return pos < nextPos && pos >= nextPos - count;
+  }
+
+  private int getIndex(int pos) {
+    int index = nextWrite - (nextPos - pos);
+    if (index < 0) {
+      index += buffer.length;
+    }
+    return index;
+  }
+
+  /** Get T instance for this absolute position;
+   *  this is allowed to be arbitrarily far "in the
+   *  future" but cannot be before the last freeBefore. */
+  public T get(int pos) {
+    //System.out.println("RA.get pos=" + pos + " nextPos=" + nextPos + " nextWrite=" + nextWrite + " count=" + count);
+    while (pos >= nextPos) {
+      if (count == buffer.length) {
+        @SuppressWarnings("unchecked") T[] newBuffer = (T[]) new Resettable[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        //System.out.println("  grow length=" + newBuffer.length);
+        System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length-nextWrite);
+        System.arraycopy(buffer, 0, newBuffer, buffer.length-nextWrite, nextWrite);
+        for(int i=buffer.length;i<newBuffer.length;i++) {
+          newBuffer[i] = newInstance();
+        }
+        nextWrite = buffer.length;
+        buffer = newBuffer;
+      }
+      if (nextWrite == buffer.length) {
+        nextWrite = 0;
+      }
+      // Should have already been reset:
+      nextWrite++;
+      nextPos++;
+      count++;
+    }
+    assert inBounds(pos);
+    final int index = getIndex(pos);
+    //System.out.println("  pos=" + pos + " nextPos=" + nextPos + " -> index=" + index);
+    //assert buffer[index].pos == pos;
+    return buffer[index];
+  }
+
+  /** Returns the maximum position looked up, or -1 if no
+   *  position has been looked up sinc reset/init.  */
+  public int getMaxPos() {
+    return nextPos-1;
+  }
+
+  public void freeBefore(int pos) {
+    final int toFree = count - (nextPos - pos);
+    assert toFree >= 0;
+    assert toFree <= count: "toFree=" + toFree + " count=" + count;
+    int index = nextWrite - count;
+    if (index < 0) {
+      index += buffer.length;
+    }
+    for(int i=0;i<toFree;i++) {
+      if (index == buffer.length) {
+        index = 0;
+      }
+      //System.out.println("  fb idx=" + index);
+      buffer[index].reset();
+      index++;
+    }
+    count -= toFree;
+  }
+}

Property changes on: lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java	(revision 1337224)
+++ lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java	(working copy)
@@ -35,6 +35,8 @@
 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,57 @@
     a.clearNumberedStates();
     return accept;
   }
+  
+  /**
+   * 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.
+   */
+  // nocommit: probably not efficient
+  public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
+    HashSet<IntsRef> strings = new HashSet<IntsRef>();
+    if (a.isSingleton()) {
+      if (limit > 0) {
+        // nocommit: yuck
+        IntsRef ref = new IntsRef();
+        Util.toIntsRef(new BytesRef(a.singleton), ref);
+        strings.add(ref);
+      } 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".
+   */
+  // nocommit: probably not efficient
+  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;
+  }
 }
Index: lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java	(revision 0)
+++ lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java	(working copy)
@@ -0,0 +1,205 @@
+package org.apache.lucene.analysis;
+
+/**
+ * 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.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RollingBuffer;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.State;
+import org.apache.lucene.util.automaton.Transition;
+
+// TODO: maybe also toFST?  then we can translate atts into FST outputs/weights
+// nocommit need option to not add pos sep
+
+/** Consumes a TokenStream and creates an {@link Automaton}
+ *  where the transition labels are bytes from the {@link
+ *  TermToBytesRefAttribute}.  */
+public class TokenStreamToAutomaton {
+
+  private static class Position implements RollingBuffer.Resettable {
+    // Any tokens that ended at our position arrive to this state:
+    State arriving;
+
+    // Any tokens that start at our position leave from this state:
+    State leaving;
+
+    @Override
+    public void reset() {
+      arriving = null;
+      leaving = null;
+    }
+  }
+
+  private static class Positions extends RollingBuffer<Position> {
+    @Override
+    protected Position newInstance() {
+      return new Position();
+    }
+  }
+
+  /** We create transition between two adjacent tokens. */
+  public static final int POS_SEP = 256;
+
+  /** We add this arc to represent a hole. */
+  public static final int HOLE = 257;
+
+  /** Pulls the graph (including {@link
+   *  PositionLengthAttribute}) from the provided {@link
+   *  TokenStream}, and creates the corresponding
+   *  automaton where arcs are bytes from each term. */
+  public static Automaton toAutomaton(TokenStream in) throws IOException {
+    final Automaton a = new Automaton();
+
+    final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
+    final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
+    final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
+    final BytesRef term = termBytesAtt.getBytesRef();
+
+    in.reset();
+
+    // Only temporarily holds states ahead of our current
+    // position:
+
+    final RollingBuffer<Position> positions = new Positions();
+
+    // nocommit verify hole as first token works!
+
+    int pos = -1;
+    Position posData = null;
+
+    while (in.incrementToken()) {
+      int posInc = posIncAtt.getPositionIncrement();
+      if (pos == -1 && posInc == 0) {
+        // TODO: hmm are TS's still allowed to do this...?
+        posInc = 1;
+      }
+
+      if (posInc > 0) {
+
+        // New node:
+        pos += posInc;
+
+        posData = positions.get(pos);
+        assert posData.leaving == null;
+
+        if (posData.arriving == null) {
+          // No token ever arrived to this position
+          if (pos == 0) {
+            // OK: this is the first token
+            posData.leaving = a.getInitialState();
+          } else {
+            // This means there's a hole (eg, StopFilter
+            // does this):
+            posData.leaving = new State();
+            addHoles(a.getInitialState(), positions, pos);
+          }
+        } else {
+          posData.leaving = new State();
+          posData.arriving.addTransition(new Transition(POS_SEP, posData.leaving));
+          if (posInc > 1) {
+            // A token spanned over a hole; add holes
+            // "under" it:
+            addHoles(a.getInitialState(), positions, pos);
+          }
+        }
+        positions.freeBefore(pos);
+      }
+
+      final int endPos = pos + posLengthAtt.getPositionLength();
+
+      termBytesAtt.fillBytesRef();
+      final Position endPosData = positions.get(endPos);
+      if (endPosData.arriving == null) {
+        endPosData.arriving = new State();
+      }
+
+      State state = posData.leaving;
+      for(int byteIDX=0;byteIDX<term.length;byteIDX++) {
+        final State nextState = byteIDX == term.length-1 ? endPosData.arriving : new State();
+        state.addTransition(new Transition(term.bytes[term.offset + byteIDX] & 0xff, nextState));
+        state = nextState;
+      }
+    }
+
+    // nocommit test syns hanging off end of token stream!
+    pos++;
+    while (pos <= positions.getMaxPos()) {
+      posData = positions.get(pos);
+      if (posData.arriving != null) {
+        posData.arriving.setAccept(true);
+      }
+      pos++;
+    }
+
+    toDot(a);
+
+    return a;
+  }
+
+  // for debugging!
+  private static void toDot(Automaton a) throws IOException {
+    final String s = a.toDot();
+    Writer w = new OutputStreamWriter(new FileOutputStream("/tmp/out.dot"));
+    w.write(s);
+    w.close();
+    System.out.println("TEST: saved to /tmp/out.dot");
+  }
+
+  // nocommit test syn over N deleted stop words
+
+  private static void addHoles(State startState, RollingBuffer<Position> positions, int pos) {
+    Position posData = positions.get(pos);
+    Position prevPosData = positions.get(pos-1);
+
+    System.out.println("ADD holes");
+    while(posData.arriving == null || prevPosData.leaving == null) {
+      System.out.println("pos=" + pos);
+      if (posData.arriving == null) {
+        posData.arriving = new State();
+        posData.arriving.addTransition(new Transition(POS_SEP, posData.leaving));
+      }
+      if (prevPosData.leaving == null) {
+        if (pos == 1) {
+          prevPosData.leaving = startState;
+        } else {
+          prevPosData.leaving = new State();
+        }
+        if (prevPosData.arriving != null) {
+          prevPosData.arriving.addTransition(new Transition(POS_SEP, prevPosData.leaving));
+        }
+      }
+      prevPosData.leaving.addTransition(new Transition(HOLE, posData.arriving));
+      pos--;
+      if (pos <= 0) {
+        break;
+      }
+      posData = prevPosData;
+      prevPosData = positions.get(pos-1);
+    }
+  }
+}
Index: lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java	(revision 1337224)
+++ lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java	(working copy)
@@ -1,130 +0,0 @@
-package org.apache.lucene.util;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-// TODO: probably move this to core at some point (eg,
-// cutover kuromoji, synfilter, LookaheadTokenFilter)
-
-/** Acts like forever growing T[], but internally uses a
- *  circular buffer to reuse instances of T.
- * 
- *  @lucene.internal */
-public abstract class RollingBuffer<T extends RollingBuffer.Resettable> {
-
-  public static interface Resettable {
-    public void reset();
-  }
-
-  @SuppressWarnings("unchecked") private T[] buffer = (T[]) new RollingBuffer.Resettable[8];
-
-  // Next array index to write to:
-  private int nextWrite;
-
-  // Next position to write:
-  private int nextPos;
-
-  // How many valid Position are held in the
-  // array:
-  private int count;
-
-  public RollingBuffer() {
-    for(int idx=0;idx<buffer.length;idx++) {
-      buffer[idx] = newInstance();
-    }
-  }
-
-  protected abstract T newInstance();
-
-  public void reset() {
-    nextWrite--;
-    while (count > 0) {
-      if (nextWrite == -1) {
-        nextWrite = buffer.length - 1;
-      }
-      buffer[nextWrite--].reset();
-      count--;
-    }
-    nextWrite = 0;
-    nextPos = 0;
-    count = 0;
-  }
-
-  // For assert:
-  private boolean inBounds(int pos) {
-    return pos < nextPos && pos >= nextPos - count;
-  }
-
-  private int getIndex(int pos) {
-    int index = nextWrite - (nextPos - pos);
-    if (index < 0) {
-      index += buffer.length;
-    }
-    return index;
-  }
-
-  /** Get T instance for this absolute position;
-   *  this is allowed to be arbitrarily far "in the
-   *  future" but cannot be before the last freeBefore. */
-  public T get(int pos) {
-    //System.out.println("RA.get pos=" + pos + " nextPos=" + nextPos + " nextWrite=" + nextWrite + " count=" + count);
-    while (pos >= nextPos) {
-      if (count == buffer.length) {
-        @SuppressWarnings("unchecked") T[] newBuffer = (T[]) new Resettable[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-        //System.out.println("  grow length=" + newBuffer.length);
-        System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length-nextWrite);
-        System.arraycopy(buffer, 0, newBuffer, buffer.length-nextWrite, nextWrite);
-        for(int i=buffer.length;i<newBuffer.length;i++) {
-          newBuffer[i] = newInstance();
-        }
-        nextWrite = buffer.length;
-        buffer = newBuffer;
-      }
-      if (nextWrite == buffer.length) {
-        nextWrite = 0;
-      }
-      // Should have already been reset:
-      nextWrite++;
-      nextPos++;
-      count++;
-    }
-    assert inBounds(pos);
-    final int index = getIndex(pos);
-    //System.out.println("  pos=" + pos + " nextPos=" + nextPos + " -> index=" + index);
-    //assert buffer[index].pos == pos;
-    return buffer[index];
-  }
-
-  public void freeBefore(int pos) {
-    final int toFree = count - (nextPos - pos);
-    assert toFree >= 0;
-    assert toFree <= count: "toFree=" + toFree + " count=" + count;
-    int index = nextWrite - count;
-    if (index < 0) {
-      index += buffer.length;
-    }
-    for(int i=0;i<toFree;i++) {
-      if (index == buffer.length) {
-        index = 0;
-      }
-      //System.out.println("  fb idx=" + index);
-      buffer[index].reset();
-      index++;
-    }
-    count -= toFree;
-  }
-}
Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java
===================================================================
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java	(revision 0)
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java	(working copy)
@@ -0,0 +1,178 @@
+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.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+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.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+public class AnalyzingCompletionTest 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)
+    };
+    
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
+    suggester.build(new TermFreqArrayIterator(keys));
+    
+    // top N of 2, but only foo is available
+    List<LookupResult> 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
+    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, true);
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(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);
+  }
+  
+  public void testRandom() throws Exception {
+    int numWords = atLeast(1000);
+    
+    final TreeMap<String,Long> slowCompletor = new TreeMap<String,Long>();
+    final TreeSet<String> allPrefixes = new TreeSet<String>();
+    
+    TermFreq[] keys = new TermFreq[numWords];
+    
+    for (int i = 0; i < numWords; i++) {
+      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 (!slowCompletor.containsKey(s)) {
+          break;
+        }
+      }
+      
+      for (int j = 1; j < s.length(); j++) {
+        allPrefixes.add(s.substring(0, j));
+      }
+      // we can probably do Integer.MAX_VALUE here, but why worry.
+      int weight = random().nextInt(1<<24);
+      slowCompletor.put(s, (long)weight);
+      keys[i] = new TermFreq(s, weight);
+    }
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false), false);
+    suggester.build(new TermFreqArrayIterator(keys));
+    
+    for (String prefix : allPrefixes) {
+    
+      final int topN = _TestUtil.nextInt(random(), 1, 10);
+      List<LookupResult> r = suggester.lookup(_TestUtil.stringToCharSequence(prefix, random()), false, topN);
+
+      // 2. go thru whole treemap (slowCompletor) and check its actually the best suggestion
+      final List<LookupResult> matches = new ArrayList<LookupResult>();
+
+      // TODO: could be faster... but its slowCompletor for a reason
+      for (Map.Entry<String,Long> e : slowCompletor.entrySet()) {
+        if (e.getKey().startsWith(prefix)) {
+          matches.add(new LookupResult(e.getKey(), e.getValue().longValue()));
+        }
+      }
+
+      assertTrue(matches.size() > 0);
+      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.subList(topN, matches.size()).clear();
+      }
+
+      assertEquals(matches.size(), r.size());
+
+      for(int hit=0;hit<r.size();hit++) {
+        //System.out.println("  check hit " + hit);
+        assertEquals(matches.get(hit).key.toString(), r.get(hit).key.toString());
+        assertEquals(matches.get(hit).value, r.get(hit).value, 0f);
+      }
+    }
+  }
+}
Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
===================================================================
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java	(revision 1337224)
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java	(working copy)
@@ -19,6 +19,7 @@
 
 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.concurrent.Callable;
 
 import org.apache.lucene.util.*;
+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;
+import org.apache.lucene.search.suggest.analyzing.AnalyzingCompletionLookup;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
 import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
@@ -42,14 +47,15 @@
 /**
  * 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 {
   @SuppressWarnings("unchecked")
   private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList(
       JaspellLookup.class, 
       TSTLookup.class,
       FSTCompletionLookup.class,
-      WFSTCompletionLookup.class);
+      WFSTCompletionLookup.class,
+      AnalyzingCompletionLookup.class);
 
   private final static int rounds = 15;
   private final static int warmup = 5;
@@ -144,7 +150,13 @@
    * 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;
   }
Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java
===================================================================
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java	(revision 0)
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java	(working copy)
@@ -0,0 +1,383 @@
+package org.apache.lucene.search.suggest.analyzing;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.TokenStreamToAutomaton;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.search.spell.TermFreqIterator;
+import org.apache.lucene.search.suggest.Lookup;
+import org.apache.lucene.search.suggest.fst.Sort;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST.BytesReader;
+import org.apache.lucene.util.fst.PairOutputs;
+import org.apache.lucene.util.fst.PairOutputs.Pair;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.fst.Util.MinResult;
+
+/**
+ * Suggester based on a weighted FST: it first traverses the prefix, 
+ * then walks the <i>n</i> shortest paths to retrieve top-ranked
+ * suggestions.
+ * <p>
+ * <b>NOTE</b>: Although the {@link TermFreqIterator} API specifies
+ * floating point weights, input weights should be whole numbers.
+ * Input weights will be cast to a java integer, and any
+ * negative, infinite, or NaN values will be rejected.
+ * 
+ * @see Util#shortestPaths(FST, FST.Arc, Comparator, int)
+ * @lucene.experimental
+ */
+public class AnalyzingCompletionLookup extends Lookup {
+ 
+  /**
+   * FST<Weight,Surface>: 
+   *  input is the analyzed form, with a null byte between terms
+   *  weights are encoded as costs: (Integer.MAX_VALUE-weight)
+   *  surface is the original, unanalyzed form.
+   */
+  private FST<Pair<Long,BytesRef>> fst = null;
+  
+  /** 
+   * Analyzer that will be used for analyzing suggestions 
+   */
+  private final Analyzer analyzer;
+  
+  /** 
+   * True if exact match suggestions should always be returned first.
+   */
+  private final boolean exactFirst;
+  
+  /**
+   * Calls {@link #AnalyzingCompletionLookup(Analyzer,boolean) AnalyzingCompletionLookup(analyzer, true)}
+   */
+  public AnalyzingCompletionLookup(Analyzer analyzer) {
+    this(analyzer, true);
+  }
+  
+  /**
+   * Creates a new suggester.
+   * 
+   * @param analyzer Analyzer that will be used for analyzing suggestions.
+   * @param exactFirst <code>true</code> if suggestions that match the 
+   *        prefix exactly should always be returned first, regardless
+   *        of score. This has no performance impact, but could result
+   *        in low-quality suggestions.
+   */
+  public AnalyzingCompletionLookup(Analyzer analyzer, boolean exactFirst) {
+    this.analyzer = analyzer;
+    this.exactFirst = exactFirst;
+  }
+  
+  @Override
+  public void build(TermFreqIterator iterator) throws IOException {
+    String prefix = getClass().getSimpleName();
+    File directory = Sort.defaultTempDir();
+    File tempInput = File.createTempFile(prefix, ".input", directory);
+    File tempSorted = File.createTempFile(prefix, ".sorted", directory);
+    
+    Sort.ByteSequencesWriter writer = new Sort.ByteSequencesWriter(tempInput);
+    Sort.ByteSequencesReader reader = null;
+    BytesRef scratch = new BytesRef();
+    
+    BytesRef separator = new BytesRef(new byte[] { (byte)0 });
+    
+    // encoding:
+    // analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short) 
+    boolean success = false;
+    byte buffer[] = new byte[8];
+    try {
+      ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
+      BytesRef spare;
+      while ((spare = iterator.next()) != null) {
+        
+        TokenStream ts = analyzer.tokenStream("", new StringReader(spare.utf8ToString()));
+        Automaton automaton = TokenStreamToAutomaton.toAutomaton(ts);
+        ts.end();
+        ts.close();
+        assert SpecialOperations.isFinite(automaton);
+        // nocommit: we should probably not wire this param to -1 but have a reasonable limit?!
+        Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, -1);
+        for (IntsRef path : paths) {        
+          scratch.length = 0;
+          // nocommit: terrible.
+          BytesRef nocommitScratch = new BytesRef();
+          Util.toBytesRef(path, nocommitScratch);
+          scratch.append(separator);
+          scratch.append(nocommitScratch);
+          
+          // length of the analyzed text (FST input)
+          short analyzedLength = (short) scratch.length;
+          // compute the required length:
+          // analyzed sequence + 12 (separator) + weight (4) + surface + analyzedLength (short)
+          int requiredLength = analyzedLength + 2 + 4 + spare.length + 2;
+          
+          buffer = ArrayUtil.grow(buffer, requiredLength);
+          
+          output.reset(buffer);
+          output.writeBytes(scratch.bytes, scratch.offset, scratch.length);
+          output.writeByte((byte)0); // separator: not used, just for sort order
+          output.writeByte((byte)0); // separator: not used, just for sort order
+          output.writeInt((int)encodeWeight(iterator.weight()));
+          output.writeBytes(spare.bytes, spare.offset, spare.length);
+          output.writeShort(analyzedLength);
+          writer.write(buffer, 0, output.getPosition());
+        }
+      }
+      writer.close();
+      new Sort().sort(tempInput, tempSorted);
+      reader = new Sort.ByteSequencesReader(tempSorted);
+      
+      PairOutputs<Long,BytesRef> outputs = new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton());
+      Builder<Pair<Long,BytesRef>> builder = new Builder<Pair<Long,BytesRef>>(FST.INPUT_TYPE.BYTE1, outputs);
+      
+      BytesRef previous = null;
+      BytesRef analyzed = new BytesRef();
+      BytesRef surface = new BytesRef();
+      IntsRef scratchInts = new IntsRef();
+      ByteArrayDataInput input = new ByteArrayDataInput();
+      while (reader.read(scratch)) {
+        input.reset(scratch.bytes, scratch.offset, scratch.length);
+        input.setPosition(input.length()-2);
+        short analyzedLength = input.readShort();
+
+        analyzed.bytes = scratch.bytes;
+        analyzed.offset = scratch.offset;
+        analyzed.length = analyzedLength;
+        
+        input.setPosition(analyzedLength + 2); // analyzed sequence + separator
+        long cost = input.readInt();
+   
+        surface.bytes = scratch.bytes;
+        surface.offset = input.getPosition();
+        surface.length = input.length() - input.getPosition() - 2;
+        
+        if (previous == null) {
+          previous = new BytesRef();
+        } else if (analyzed.equals(previous)) {
+          continue; // nocommit: "extend" duplicates with useless increasing bytes (it wont matter)
+        }
+        Util.toIntsRef(analyzed, scratchInts);
+        // nocommit
+        // nocommit: why must i deep copy?!
+        builder.add(scratchInts, outputs.newPair(cost, BytesRef.deepCopyOf(surface)));
+        previous.copyBytes(analyzed);
+      }
+      fst = builder.finish();
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(reader, writer);
+      } else {
+        IOUtils.closeWhileHandlingException(reader, writer);
+      }
+      
+      tempInput.delete();
+      tempSorted.delete();
+    }
+  }
+
+  @Override
+  public boolean store(OutputStream output) throws IOException {
+    try {
+      fst.save(new OutputStreamDataOutput(output));
+    } finally {
+      IOUtils.close(output);
+    }
+    return true;
+  }
+
+  @Override
+  public boolean load(InputStream input) throws IOException {
+    try {
+      this.fst = new FST<Pair<Long,BytesRef>>(new InputStreamDataInput(input), new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton()));
+    } finally {
+      IOUtils.close(input);
+    }
+    return true;
+  }
+
+  @Override
+  public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) {
+    assert num > 0;
+    Arc<Pair<Long,BytesRef>> arc = new Arc<Pair<Long,BytesRef>>();
+    
+    // match the prefix portion exactly
+    Pair<Long,BytesRef> prefixOutput = null;
+    try {
+      prefixOutput = lookupPrefix(key, arc);
+    } catch (IOException bogus) { throw new RuntimeException(bogus); }
+    
+    if (prefixOutput == null) {
+      return Collections.<LookupResult>emptyList();
+    }
+    
+    BytesRef prefix = prefixOutput.output2;
+    int prefixLength = prefix.length;
+    
+    List<LookupResult> results = new ArrayList<LookupResult>(num);
+    CharsRef spare = new CharsRef();
+    if (exactFirst && arc.isFinal()) {
+      prefix.append(arc.nextFinalOutput.output2);
+      spare.grow(prefix.length);
+      UnicodeUtil.UTF8toUTF16(prefix, spare);
+      results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput.output1 + arc.nextFinalOutput.output1)));
+      if (--num == 0) {
+        return results; // that was quick
+      }
+    }
+    
+    // complete top-N
+    MinResult<Pair<Long,BytesRef>> completions[] = null;
+    try {
+      completions = Util.shortestPaths(fst, arc, weightComparator, num);
+    } catch (IOException bogus) { throw new RuntimeException(bogus); }
+    
+    for (MinResult<Pair<Long,BytesRef>> completion : completions) {
+      prefix.length = prefixLength;
+      // append suffix
+      prefix.append(completion.output.output2);
+      spare.grow(prefix.length);
+      UnicodeUtil.UTF8toUTF16(prefix, spare);
+      results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput.output1 + completion.output.output1)));
+    }
+    return results;
+  }
+  
+  private Pair<Long,BytesRef> lookupPrefix(CharSequence prefix, Arc<Pair<Long,BytesRef>> arc) throws /*Bogus*/IOException {
+    Pair<Long,BytesRef> output = fst.outputs.getNoOutput();
+    BytesReader bytesReader = fst.getBytesReader(0);
+    
+    fst.getFirstArc(arc);
+    
+    // nocommit: isnt there a charsequencereader somewhere?
+    TokenStream ts = analyzer.tokenStream("", new StringReader(prefix.toString()));
+    
+    // nocommit: respect posincs
+    TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
+    PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+    BytesRef scratch = termAtt.getBytesRef();
+    ts.reset();
+    
+    while (ts.incrementToken()) {
+      // nocommit: bogus
+      // instead here, we should build an Automaton out of the tokenstream,
+      // and intersect that with the FST, producing N outputs, and complete from those?
+      if (posIncAtt.getPositionIncrement() == 0) {
+        continue;
+      }
+      // separator byte
+      // nocommit: it should be a separator per posInc,
+      // and separators should be optional (e.g. japanese completion)
+      if (fst.findTargetArc(0, arc, arc, bytesReader) == null) {
+        // TODO: should we fully consume?
+        ts.close();
+        return null;
+      } else {
+        output = fst.outputs.add(output, arc.output);
+      } 
+      
+      termAtt.fillBytesRef();
+      byte[] bytes = scratch.bytes;
+      int pos = scratch.offset;
+      int end = pos + scratch.length;
+      while (pos < end) {
+        if (fst.findTargetArc(bytes[pos++] & 0xff, arc, arc, bytesReader) == null) {
+          System.out.println("FAIL @ pos=" + pos + " byte=" + (char) (bytes[pos-1]&0xff));
+          // TODO: should we fully consume?
+          // nocommit MockTokenizer angry about this when
+          // you don't fully match!:
+          ts.close();
+          return null;
+        } else {
+          output = fst.outputs.add(output, arc.output);
+        } 
+      }
+    }
+    
+    ts.end();
+    ts.close();
+    return output;
+  }
+  
+  /**
+   * Returns the weight associated with an input string,
+   * or null if it does not exist.
+   */
+  public Object get(CharSequence key) {
+    // TODO: analyze, or just nuke this method!?
+    Arc<Pair<Long,BytesRef>> arc = new Arc<Pair<Long,BytesRef>>();
+    Pair<Long,BytesRef> result = null;
+    try {
+      result = lookupPrefix(key, arc);
+    } catch (IOException bogus) { throw new RuntimeException(bogus); }
+    if (result == null || !arc.isFinal()) {
+      return null;
+    } else {
+      return Integer.valueOf(decodeWeight(result.output1 + arc.nextFinalOutput.output1));
+    }
+  }
+  
+  /** cost -> weight */
+  private static int decodeWeight(long encoded) {
+    return (int)(Integer.MAX_VALUE - encoded);
+  }
+  
+  /** weight -> cost */
+  private static int encodeWeight(long value) {
+    if (value < 0 || value > Integer.MAX_VALUE) {
+      throw new UnsupportedOperationException("cannot encode value: " + value);
+    }
+    return Integer.MAX_VALUE - (int)value;
+  }
+   
+  static final Comparator<Pair<Long,BytesRef>> weightComparator = new Comparator<Pair<Long,BytesRef>> () {
+    public int compare(Pair<Long,BytesRef> left, Pair<Long,BytesRef> right) {
+      return left.output1.compareTo(right.output1);
+    }  
+  };
+}
