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 2011/01/05 11:52:06 UTC

svn commit: r1055405 [3/3] - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/index/codecs/sep/ src/java/org/apache/lucene/index/codecs/simpletext/ src/java/org/apache...

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Wed Jan  5 10:52:04 2011
@@ -67,10 +67,15 @@ public class MockFixedIntBlockCodec exte
 
   // only for testing
   public IntStreamFactory getIntFactory() {
-    return new MockIntFactory();
+    return new MockIntFactory(blockSize);
   }
 
-  private class MockIntFactory extends IntStreamFactory {
+  public static class MockIntFactory extends IntStreamFactory {
+    private final int blockSize;
+
+    public MockIntFactory(int blockSize) {
+      this.blockSize = blockSize;
+    }
 
     @Override
     public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
@@ -96,6 +101,7 @@ public class MockFixedIntBlockCodec exte
         @Override
         protected void flushBlock() throws IOException {
           for(int i=0;i<buffer.length;i++) {
+            assert buffer[i] >= 0;
             out.writeVInt(buffer[i]);
           }
         }
@@ -105,7 +111,7 @@ public class MockFixedIntBlockCodec exte
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
+    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(blockSize));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -139,7 +145,7 @@ public class MockFixedIntBlockCodec exte
     PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
                                                                       state.segmentInfo,
                                                                       state.readBufferSize,
-                                                                      new MockIntFactory(), state.codecId);
+                                                                      new MockIntFactory(blockSize), state.codecId);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Wed Jan  5 10:52:04 2011
@@ -67,7 +67,13 @@ public class MockVariableIntBlockCodec e
     return name + "(baseBlockSize="+ baseBlockSize + ")";
   }
 
-  private class MockIntFactory extends IntStreamFactory {
+  public static class MockIntFactory extends IntStreamFactory {
+
+    private final int baseBlockSize;
+
+    public MockIntFactory(int baseBlockSize) {
+      this.baseBlockSize = baseBlockSize;
+    }
 
     @Override
     public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
@@ -104,6 +110,7 @@ public class MockVariableIntBlockCodec e
 
         @Override
         protected int add(int value) throws IOException {
+          assert value >= 0;
           buffer[pendingCount++] = value;
           // silly variable block length int encoder: if
           // first value <= 3, we write N vints at once;
@@ -128,7 +135,7 @@ public class MockVariableIntBlockCodec e
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
+    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(baseBlockSize));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -162,7 +169,7 @@ public class MockVariableIntBlockCodec e
     PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
                                                                       state.segmentInfo,
                                                                       state.readBufferSize,
-                                                                      new MockIntFactory(), state.codecId);
+                                                                      new MockIntFactory(baseBlockSize), state.codecId);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1055405&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Wed Jan  5 10:52:04 2011
@@ -0,0 +1,324 @@
+package org.apache.lucene.index.codecs.mockrandom;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
+import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
+import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.TermsIndexReaderBase;
+import org.apache.lucene.index.codecs.TermsIndexWriterBase;
+import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
+import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
+import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
+import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReaderImpl;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
+import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Randomly combines terms index impl w/ postings impls.
+ */
+
+public class MockRandomCodec extends Codec {
+
+  private final Random seedRandom;
+  private final String SEED_EXT = "sd";
+
+  public MockRandomCodec(Random random) {
+    name = "MockRandom";
+    this.seedRandom = random;
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+
+    final long seed = seedRandom.nextLong();
+
+    final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
+    final IndexOutput out = state.directory.createOutput(seedFileName);
+    out.writeLong(seed);
+    out.close();
+    state.flushedFiles.add(seedFileName);
+
+    final Random random = new Random(seed);
+    PostingsWriterBase postingsWriter;
+    final int n = random.nextInt(4);
+
+    if (n == 0) {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing MockSep postings");
+      }
+      postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory());
+    } else if (n == 1) {
+      final int blockSize = _TestUtil.nextInt(random, 1, 2000);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing MockFixedIntBlock(" + blockSize + ") postings");
+      }
+      postingsWriter = new SepPostingsWriterImpl(state, new MockFixedIntBlockCodec.MockIntFactory(blockSize));
+    } else if (n == 2) {
+      final int baseBlockSize = _TestUtil.nextInt(random, 1, 127);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing MockVariableIntBlock(" + baseBlockSize + ") postings");
+      }
+      postingsWriter = new SepPostingsWriterImpl(state, new MockVariableIntBlockCodec.MockIntFactory(baseBlockSize));
+    } else {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing Standard postings");
+      }
+      postingsWriter = new StandardPostingsWriter(state);
+    }
+
+    if (random.nextBoolean()) {
+      final int freqCutoff = _TestUtil.nextInt(random, 1, 20);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: pulsing postings with freqCutoff=" + freqCutoff);
+      }
+      postingsWriter = new PulsingPostingsWriterImpl(freqCutoff, postingsWriter);
+    }
+
+    final TermsIndexWriterBase indexWriter;
+    boolean success = false;
+
+    try {
+      if (random.nextBoolean()) {
+        state.termIndexInterval = _TestUtil.nextInt(random, 1, 100);
+        if (LuceneTestCase.VERBOSE) {
+          System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")");
+        }
+        indexWriter = new FixedGapTermsIndexWriter(state);
+      } else {
+        final VariableGapTermsIndexWriter.IndexTermSelector selector;
+        final int n2 = random.nextInt(3);
+        if (n2 == 0) {
+          final int tii = _TestUtil.nextInt(random, 1, 100);
+          selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii);
+          if (LuceneTestCase.VERBOSE) {
+            System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")");
+          }
+        } else if (n2 == 1) {
+          final int docFreqThresh = _TestUtil.nextInt(random, 2, 100);
+          final int tii = _TestUtil.nextInt(random, 1, 100);
+          selector = new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThresh, tii);
+        } else {
+          final long seed2 = random.nextLong();
+          final int gap = _TestUtil.nextInt(random, 2, 40);
+          if (LuceneTestCase.VERBOSE) {
+            System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")");
+          }
+          selector = new VariableGapTermsIndexWriter.IndexTermSelector() {
+              final Random rand = new Random(seed2);
+
+              @Override
+                public boolean isIndexTerm(BytesRef term, int docFreq) {
+                return random.nextInt(gap) == 17;
+              }
+            };
+        }
+        indexWriter = new VariableGapTermsIndexWriter(state, selector);
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        postingsWriter.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsWriter.close();
+        } finally {
+          indexWriter.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+
+    final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, SEED_EXT);
+    final IndexInput in = state.dir.openInput(seedFileName);
+    final long seed = in.readLong();
+    in.close();
+
+    final Random random = new Random(seed);
+    PostingsReaderBase postingsReader;
+    final int n = random.nextInt(4);
+
+    if (n == 0) {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading MockSep postings");
+      }
+      postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
+                                                 state.readBufferSize, new MockSingleIntFactory(), state.codecId);
+    } else if (n == 1) {
+      final int blockSize = _TestUtil.nextInt(random, 1, 2000);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading MockFixedIntBlock(" + blockSize + ") postings");
+      }
+      postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
+                                                 state.readBufferSize, new MockFixedIntBlockCodec.MockIntFactory(blockSize), state.codecId);
+    } else if (n == 2) {
+      final int baseBlockSize = _TestUtil.nextInt(random, 1, 127);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading MockVariableIntBlock(" + baseBlockSize + ") postings");
+      }
+      postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
+                                                 state.readBufferSize, new MockVariableIntBlockCodec.MockIntFactory(baseBlockSize), state.codecId);
+    } else {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading Standard postings");
+      }
+      postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
+    }
+
+    if (random.nextBoolean()) {
+      final int freqCutoff = _TestUtil.nextInt(random, 1, 20);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading pulsing postings with freqCutoff=" + freqCutoff);
+      }
+      postingsReader = new PulsingPostingsReaderImpl(postingsReader);
+    }
+
+    final TermsIndexReaderBase indexReader;
+    boolean success = false;
+
+    try {
+      if (random.nextBoolean()) {
+        state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
+        if (LuceneTestCase.VERBOSE) {
+          System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
+        }
+        indexReader = new FixedGapTermsIndexReader(state.dir,
+                                                   state.fieldInfos,
+                                                   state.segmentInfo.name,
+                                                   state.termsIndexDivisor,
+                                                   BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                   state.codecId);
+      } else {
+        final int n2 = random.nextInt(3);
+        if (n2 == 1) {
+          random.nextInt();
+        } else if (n2 == 2) {
+          random.nextLong();
+        }
+        if (LuceneTestCase.VERBOSE) {
+          System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
+        }
+        state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
+        indexReader = new VariableGapTermsIndexReader(state.dir,
+                                                      state.fieldInfos,
+                                                      state.segmentInfo.name,
+                                                      state.termsIndexDivisor,
+                                                      state.codecId);
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        postingsReader.close();
+      }
+    }
+
+    final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024);
+
+    success = false;
+    try {
+      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
+                                                      state.dir,
+                                                      state.fieldInfos,
+                                                      state.segmentInfo.name,
+                                                      postingsReader,
+                                                      state.readBufferSize,
+                                                      BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                      termsCacheSize,
+                                                      state.codecId);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsReader.close();
+        } finally {
+          indexReader.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
+    final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);    
+    files.add(seedFileName);
+    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    StandardPostingsReader.files(dir, segmentInfo, codecId, files);
+    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
+    VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
+    
+    // hackish!
+    Iterator<String> it = files.iterator();
+    while(it.hasNext()) {
+      final String file = it.next();
+      if (!dir.fileExists(file)) {
+        it.remove();
+      }
+    }
+    //System.out.println("MockRandom.files return " + files);
+  }
+
+  @Override
+  public void getExtensions(Set<String> extensions) {
+    SepPostingsWriterImpl.getExtensions(extensions);
+    PrefixCodedTermsReader.getExtensions(extensions);
+    FixedGapTermsIndexReader.getIndexExtensions(extensions);
+    VariableGapTermsIndexReader.getIndexExtensions(extensions);
+    extensions.add(SEED_EXT);
+    //System.out.println("MockRandom.getExtensions return " + extensions);
+  }
+}

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java Wed Jan  5 10:52:04 2011
@@ -42,6 +42,7 @@ public class MockSingleIntIndexOutput ex
   /** Write an int to the primary file */
   @Override
   public void write(int v) throws IOException {
+    assert v >= 0;
     out.writeVInt(v);
   }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java Wed Jan  5 10:52:04 2011
@@ -42,6 +42,7 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
 import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
+import org.apache.lucene.index.codecs.mockrandom.MockRandomCodec;
 import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
 import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
 import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
@@ -199,7 +200,7 @@ public abstract class LuceneTestCase ext
   
   private static Map<MockDirectoryWrapper,StackTraceElement[]> stores;
   
-  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock"};
+  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"};
 
   private static void swapCodec(Codec c, CodecProvider cp) {
     Codec prior = null;
@@ -252,6 +253,7 @@ public abstract class LuceneTestCase ext
     swapCodec(new MockFixedIntBlockCodec(codecHasParam && "MockFixedIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 2000)), cp);
     // baseBlockSize cannot be over 127:
     swapCodec(new MockVariableIntBlockCodec(codecHasParam && "MockVariableIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 127)), cp);
+    swapCodec(new MockRandomCodec(random), cp);
 
     return cp.lookup(codec);
   }
@@ -268,9 +270,9 @@ public abstract class LuceneTestCase ext
     cp.unregister(cp.lookup("MockSep"));
     cp.unregister(cp.lookup("MockFixedIntBlock"));
     cp.unregister(cp.lookup("MockVariableIntBlock"));
+    cp.unregister(cp.lookup("MockRandom"));
     swapCodec(new PulsingCodec(1), cp);
     cp.setDefaultFieldCodec(savedDefaultCodec);
-
   }
 
   // randomly picks from core and test codecs

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java Wed Jan  5 10:52:04 2011
@@ -20,13 +20,13 @@ package org.apache.lucene.util.automaton
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.InputStreamReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -122,8 +122,10 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testBasicFSA() throws IOException {
-    String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"};
+    String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation", "stat"};
+    String[] strings2 = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"};
     IntsRef[] terms = new IntsRef[strings.length];
+    IntsRef[] terms2 = new IntsRef[strings2.length];
     for(int inputMode=0;inputMode<2;inputMode++) {
       if (VERBOSE) {
         System.out.println("TEST: inputMode=" + inputModeToString(inputMode));
@@ -132,6 +134,10 @@ public class TestFSTs extends LuceneTest
       for(int idx=0;idx<strings.length;idx++) {
         terms[idx] = toIntsRef(strings[idx], inputMode);
       }
+      for(int idx=0;idx<strings2.length;idx++) {
+        terms2[idx] = toIntsRef(strings2[idx], inputMode);
+      }
+      Arrays.sort(terms2);
 
       doTest(inputMode, terms);
     
@@ -141,8 +147,8 @@ public class TestFSTs extends LuceneTest
       {
         final Outputs<Object> outputs = NoOutputs.getSingleton();
         final Object NO_OUTPUT = outputs.getNoOutput();      
-        final List<FSTTester.InputOutput<Object>> pairs = new ArrayList<FSTTester.InputOutput<Object>>(terms.length);
-        for(IntsRef term : terms) {
+        final List<FSTTester.InputOutput<Object>> pairs = new ArrayList<FSTTester.InputOutput<Object>>(terms2.length);
+        for(IntsRef term : terms2) {
           pairs.add(new FSTTester.InputOutput<Object>(term, NO_OUTPUT));
         }
         FST<Object> fst = new FSTTester<Object>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
@@ -154,9 +160,9 @@ public class TestFSTs extends LuceneTest
       // FST ord pos int
       {
         final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-        final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
-        for(int idx=0;idx<terms.length;idx++) {
-          pairs.add(new FSTTester.InputOutput<Long>(terms[idx], outputs.get(idx)));
+        final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms2.length);
+        for(int idx=0;idx<terms2.length;idx++) {
+          pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], outputs.get(idx)));
         }
         final FST<Long> fst = new FSTTester<Long>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
         assertNotNull(fst);
@@ -168,10 +174,10 @@ public class TestFSTs extends LuceneTest
       {
         final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
         final BytesRef NO_OUTPUT = outputs.getNoOutput();      
-        final List<FSTTester.InputOutput<BytesRef>> pairs = new ArrayList<FSTTester.InputOutput<BytesRef>>(terms.length);
-        for(int idx=0;idx<terms.length;idx++) {
+        final List<FSTTester.InputOutput<BytesRef>> pairs = new ArrayList<FSTTester.InputOutput<BytesRef>>(terms2.length);
+        for(int idx=0;idx<terms2.length;idx++) {
           final BytesRef output = random.nextInt(30) == 17 ? NO_OUTPUT : new BytesRef(Integer.toString(idx));
-          pairs.add(new FSTTester.InputOutput<BytesRef>(terms[idx], output));
+          pairs.add(new FSTTester.InputOutput<BytesRef>(terms2[idx], output));
         }
         final FST<BytesRef> fst = new FSTTester<BytesRef>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
         assertNotNull(fst);
@@ -322,19 +328,6 @@ public class TestFSTs extends LuceneTest
       }
     }
 
-    private String getRandomString() {
-      final String term;
-      if (random.nextBoolean()) {
-        term = _TestUtil.randomRealisticUnicodeString(random);
-      } else {
-        // we want to mix in limited-alphabet symbols so
-        // we get more sharing of the nodes given how few
-        // terms we are testing...
-        term = simpleRandomString(random);
-      }
-      return term;
-    }
-
     public void doTest() throws IOException {
       // no pruning
       doTest(0, 0);
@@ -346,156 +339,83 @@ public class TestFSTs extends LuceneTest
       doTest(0, _TestUtil.nextInt(random, 1, 1+pairs.size()));
     }
 
-    // NOTE: only copies the stuff this test needs!!
-    private FST.Arc<T> copyArc(FST.Arc<T> arc) {
-      final FST.Arc<T> copy = new FST.Arc<T>();
-      copy.label = arc.label;
-      copy.target = arc.target;
-      copy.output = arc.output;
-      copy.nextFinalOutput = arc.nextFinalOutput;
-      return arc;
-    }
-
     // runs the term, returning the output, or null if term
-    // isn't accepted.  if stopNode is non-null it must be
-    // length 2 int array; stopNode[0] will be the last
-    // matching node (-1 if the term is accepted)
-    // and stopNode[1] will be the length of the
-    // term prefix that matches
-    private T run(FST<T> fst, IntsRef term, int[] stopNode) throws IOException {
-      if (term.length == 0) {
-        final T output = fst.getEmptyOutput();
-        if (stopNode != null) {
-          stopNode[1] = 0;
-          if (output != null) {
-            // accepted
-            stopNode[0] = -1;
-          } else {
-            stopNode[0] = fst.getStartNode();
-          }
+    // isn't accepted.  if prefixLength is non-null it must be
+    // length 1 int array; prefixLength[0] is set to the length
+    // of the term prefix that matches
+    private T run(FST<T> fst, IntsRef term, int[] prefixLength) throws IOException {
+      assert prefixLength == null || prefixLength.length == 1;
+      final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+      final T NO_OUTPUT = fst.outputs.getNoOutput();
+      T output = NO_OUTPUT;
+
+      for(int i=0;i<=term.length;i++) {
+        final int label;
+        if (i == term.length) {
+          label = FST.END_LABEL;
+        } else {
+          label = term.ints[term.offset+i];
         }
-        return output;
-      }
-
-      final FST.Arc<T> arc = new FST.Arc<T>();
-      int node = fst.getStartNode();
-      int lastNode = -1;
-      T output = fst.outputs.getNoOutput();
-      //System.out.println("match?");
-      for(int i=0;i<term.length;i++) {
-        //System.out.println("  int=" + term.ints[i]);
-        if (!fst.hasArcs(node)) {
-          //System.out.println("    no arcs!");
-          // hit end node before term's end
-          if (stopNode != null) {
-            stopNode[0] = lastNode;
-            stopNode[1] = i-1;
+        //System.out.println("   loop i=" + i + " label=" + label + " output=" + fst.outputs.outputToString(output) + " curArc: target=" + arc.target + " isFinal?=" + arc.isFinal());
+        if (fst.findTargetArc(label, arc, arc) == null) {
+          if (prefixLength != null) {
+            prefixLength[0] = i;
             return output;
           } else {
             return null;
           }
         }
-
-        if (fst.findArc(node, term.ints[term.offset + i], arc) != null) {
-          node = arc.target;
-          //System.out.println("    match final?=" + arc.isFinal());
-          if (arc.output != fst.outputs.getNoOutput()) {
-            output = fst.outputs.add(output, arc.output);
-          }
-        } else if (stopNode != null) {
-          stopNode[0] = node;
-          stopNode[1] = i;
-          return output;
-        } else {
-          //System.out.println("    no match");
-          return null;
-        }
-
-        lastNode = node;
-      }
-
-      if (!arc.isFinal()) {
-        // hit term's end before end node
-        if (stopNode != null) {
-          stopNode[0] = node;
-          stopNode[1] = term.length;
-          return output;
-        } else {
-          return null;
-        }
+        output = fst.outputs.add(output, arc.output);
       }
 
-      if (arc.nextFinalOutput != fst.outputs.getNoOutput()) {
-        output = fst.outputs.add(output, arc.nextFinalOutput);
+      if (prefixLength != null) {
+        prefixLength[0] = term.length;
       }
 
-      if (stopNode != null) {
-        stopNode[0] = -1;
-        stopNode[1] = term.length;
-      }
       return output;
     }
 
     private T randomAcceptedWord(FST<T> fst, IntsRef in) throws IOException {
-      int node = fst.getStartNode();
+      FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
 
-      if (fst.noNodes()) {
-        // degenerate FST: only accepts the empty string
-        assertTrue(fst.getEmptyOutput() != null);
-        in.length = 0;
-        return fst.getEmptyOutput();
-      }
       final List<FST.Arc<T>> arcs = new ArrayList<FST.Arc<T>>();
       in.length = 0;
       in.offset = 0;
-      T output = fst.outputs.getNoOutput();
-      //System.out.println("get random");
+      final T NO_OUTPUT = fst.outputs.getNoOutput();
+      T output = NO_OUTPUT;
+
       while(true) {
         // read all arcs:
-        //System.out.println("  n=" + node);
-        int arcAddress = node;
-        FST.Arc<T> arc = new FST.Arc<T>();
-        fst.readFirstArc(arcAddress, arc);
-        arcs.add(copyArc(arc));
+        fst.readFirstTargetArc(arc, arc);
+        arcs.add(new FST.Arc<T>().copyFrom(arc));
         while(!arc.isLast()) {
           fst.readNextArc(arc);
-          arcs.add(copyArc(arc));
+          arcs.add(new FST.Arc<T>().copyFrom(arc));
         }
       
         // pick one
         arc = arcs.get(random.nextInt(arcs.size()));
-
         arcs.clear();
 
+        // accumulate output
+        output = fst.outputs.add(output, arc.output);
+
         // append label
+        if (arc.label == FST.END_LABEL) {
+          break;
+        }
+
         if (in.ints.length == in.length) {
           in.grow(1+in.length);
         }
         in.ints[in.length++] = arc.label;
-
-        output = fst.outputs.add(output, arc.output);
-
-        // maybe stop
-        if (arc.isFinal()) {
-          if (fst.hasArcs(arc.target)) {
-            // final state but it also has outgoing edges
-            if (random.nextBoolean()) {
-              output = fst.outputs.add(output, arc.nextFinalOutput);
-              break;
-            }
-          } else {
-            break;
-          }
-        }
-
-        node = arc.target;
       }
 
       return output;
     }
 
 
-    private FST<T> doTest(int prune1, int prune2) throws IOException {
+    FST<T> doTest(int prune1, int prune2) throws IOException {
       if (VERBOSE) {
         System.out.println("TEST: prune1=" + prune1 + " prune2=" + prune2);
       }
@@ -524,7 +444,7 @@ public class TestFSTs extends LuceneTest
 
       if (VERBOSE && pairs.size() <= 20 && fst != null) {
         PrintStream ps = new PrintStream("out.dot");
-        fst.toDot(ps);
+        Util.toDot(fst, ps);
         ps.close();
         System.out.println("SAVED out.dot");
       }
@@ -566,11 +486,19 @@ public class TestFSTs extends LuceneTest
 
       assertNotNull(fst);
 
-      // make sure all words are accepted
+      // visit valid paris in order -- make sure all words
+      // are accepted, and FSTEnum's next() steps through
+      // them correctly
+      if (VERBOSE) {
+        System.out.println("TEST: check valid terms/next()");
+      }
       {
         IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
         for(InputOutput<T> pair : pairs) {
           IntsRef term = pair.input;
+          if (VERBOSE) {
+            System.out.println("TEST: check term=" + inputToString(inputMode, term) + " output=" + fst.outputs.outputToString(pair.output));
+          }
           Object output = run(fst, term, null);
 
           assertNotNull("term " + inputToString(inputMode, term) + " is not accepted", output);
@@ -578,8 +506,8 @@ public class TestFSTs extends LuceneTest
 
           // verify enum's next
           IntsRefFSTEnum.InputOutput<T> t = fstEnum.next();
-
-          assertEquals(term, t.input);
+          assertNotNull(t);
+          assertEquals("expected input=" + inputToString(inputMode, term) + " but fstEnum returned " + inputToString(inputMode, t.input), term, t.input);
           assertEquals(pair.output, t.output);
         }
         assertNull(fstEnum.next());
@@ -591,6 +519,9 @@ public class TestFSTs extends LuceneTest
       }
 
       // find random matching word and make sure it's valid
+      if (VERBOSE) {
+        System.out.println("TEST: verify random accepted terms");
+      }
       final IntsRef scratch = new IntsRef(10);
       for(int iter=0;iter<500*RANDOM_MULTIPLIER;iter++) {
         T output = randomAcceptedWord(fst, scratch);
@@ -598,10 +529,15 @@ public class TestFSTs extends LuceneTest
         assertEquals(termsMap.get(scratch), output);
       }
     
-      // test single IntsRefFSTEnum.advance:
-      //System.out.println("TEST: verify advance");
+      // test IntsRefFSTEnum.seek:
+      if (VERBOSE) {
+        System.out.println("TEST: verify seek");
+      }
+      IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
       for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-        final IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
+        if (VERBOSE) {
+          System.out.println("TEST: iter=" + iter);
+        }
         if (random.nextBoolean()) {
           // seek to term that doesn't exist:
           while(true) {
@@ -611,15 +547,35 @@ public class TestFSTs extends LuceneTest
               pos = -(pos+1);
               // ok doesn't exist
               //System.out.println("  seek " + inputToString(inputMode, term));
-              final IntsRefFSTEnum.InputOutput<T> seekResult = fstEnum.advance(term);
-              if (pos < pairs.size()) {
+              final IntsRefFSTEnum.InputOutput<T> seekResult;
+              if (random.nextBoolean()) {
+                if (VERBOSE) {
+                  System.out.println("  do non-exist seekFloor term=" + inputToString(inputMode, term));
+                }
+                seekResult = fstEnum.seekFloor(term);
+                pos--;
+              } else {
+                if (VERBOSE) {
+                  System.out.println("  do non-exist seekCeil term=" + inputToString(inputMode, term));
+                }
+                seekResult = fstEnum.seekCeil(term);
+              }
+
+              if (pos != -1 && pos < pairs.size()) {
                 //System.out.println("    got " + inputToString(inputMode,seekResult.input) + " output=" + fst.outputs.outputToString(seekResult.output));
-                assertEquals(pairs.get(pos).input, seekResult.input);
+                assertNotNull("got null but expected term=" + inputToString(inputMode, pairs.get(pos).input), seekResult);
+                if (VERBOSE) {
+                  System.out.println("    got " + inputToString(inputMode, seekResult.input));
+                }
+                assertEquals("expected " + inputToString(inputMode, pairs.get(pos).input) + " but got " + inputToString(inputMode, seekResult.input), pairs.get(pos).input, seekResult.input);
                 assertEquals(pairs.get(pos).output, seekResult.output);
               } else {
-                // seeked beyond end
+                // seeked before start or beyond end
                 //System.out.println("seek=" + seekTerm);
                 assertNull("expected null but got " + (seekResult==null ? "null" : inputToString(inputMode, seekResult.input)), seekResult);
+                if (VERBOSE) {
+                  System.out.println("    got null");
+                }
               }
 
               break;
@@ -627,24 +583,36 @@ public class TestFSTs extends LuceneTest
           }
         } else {
           // seek to term that does exist:
-          InputOutput pair = pairs.get(random.nextInt(pairs.size()));
-          //System.out.println("  seek " + inputToString(inputMode, pair.input));
-          final IntsRefFSTEnum.InputOutput<T> seekResult = fstEnum.advance(pair.input);
-          assertEquals(pair.input, seekResult.input);
+          InputOutput<T> pair = pairs.get(random.nextInt(pairs.size()));
+          final IntsRefFSTEnum.InputOutput<T> seekResult;
+          if (random.nextBoolean()) {
+            if (VERBOSE) {
+              System.out.println("  do exists seekFloor " + inputToString(inputMode, pair.input));
+            }
+            seekResult = fstEnum.seekFloor(pair.input);
+          } else {
+            if (VERBOSE) {
+              System.out.println("  do exists seekCeil " + inputToString(inputMode, pair.input));
+            }
+            seekResult = fstEnum.seekCeil(pair.input);
+          }
+          assertNotNull(seekResult);
+          assertEquals("got " + inputToString(inputMode, seekResult.input) + " but expected " + inputToString(inputMode, pair.input), pair.input, seekResult.input);
           assertEquals(pair.output, seekResult.output);
         }
       }
 
       if (VERBOSE) {
-        System.out.println("TEST: mixed next/advance");
+        System.out.println("TEST: mixed next/seek");
       }
 
-      // test mixed next/advance
+      // test mixed next/seek
       for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
         if (VERBOSE) {
           System.out.println("TEST: iter " + iter);
         }
-        final IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
+        // reset:
+        fstEnum = new IntsRefFSTEnum<T>(fst);
         int upto = -1;
         while(true) {
           boolean isDone = false;
@@ -660,13 +628,24 @@ public class TestFSTs extends LuceneTest
             for(;attempt<10;attempt++) {
               IntsRef term = toIntsRef(getRandomString(), inputMode);
               if (!termsMap.containsKey(term) && term.compareTo(pairs.get(upto).input) > 0) {
-                if (VERBOSE) {
-                  System.out.println("  do non-exist advance(" + inputToString(inputMode, term) + "]");
-                }
                 int pos = Collections.binarySearch(pairs, new InputOutput<T>(term, null));
                 assert pos < 0;
                 upto = -(pos+1);
-                isDone = fstEnum.advance(term) == null;
+
+                if (random.nextBoolean()) {
+                  upto--;
+                  assertTrue(upto != -1);
+                  if (VERBOSE) {
+                    System.out.println("  do non-exist seekFloor(" + inputToString(inputMode, term) + ")");
+                  }
+                  isDone = fstEnum.seekFloor(term) == null;
+                } else {
+                  if (VERBOSE) {
+                    System.out.println("  do non-exist seekCeil(" + inputToString(inputMode, term) + ")");
+                  }
+                  isDone = fstEnum.seekCeil(term) == null;
+                }
+
                 break;
               }
             }
@@ -681,10 +660,17 @@ public class TestFSTs extends LuceneTest
               upto = 0;
             }
 
-            if (VERBOSE) {
-              System.out.println("  do advance(" + inputToString(inputMode, pairs.get(upto).input) + "]");
+            if (random.nextBoolean()) {
+              if (VERBOSE) {
+                System.out.println("  do advanceCeil(" + inputToString(inputMode, pairs.get(upto).input) + ")");
+              }
+              isDone = fstEnum.seekCeil(pairs.get(upto).input) == null;
+            } else {
+              if (VERBOSE) {
+                System.out.println("  do advanceFloor(" + inputToString(inputMode, pairs.get(upto).input) + ")");
+              }
+              isDone = fstEnum.seekFloor(pairs.get(upto).input) == null;
             }
-            isDone = fstEnum.advance(pairs.get(upto).input) == null;
           }
           if (VERBOSE) {
             if (!isDone) {
@@ -701,6 +687,24 @@ public class TestFSTs extends LuceneTest
             assertFalse(isDone);
             assertEquals(pairs.get(upto).input, fstEnum.current().input);
             assertEquals(pairs.get(upto).output, fstEnum.current().output);
+
+            /*
+            if (upto < pairs.size()-1) {
+              int tryCount = 0;
+              while(tryCount < 10) {
+                final IntsRef t = toIntsRef(getRandomString(), inputMode);
+                if (pairs.get(upto).input.compareTo(t) < 0) {
+                  final boolean expected = t.compareTo(pairs.get(upto+1).input) < 0;
+                  if (VERBOSE) {
+                    System.out.println("TEST: call beforeNext(" + inputToString(inputMode, t) + "); current=" + inputToString(inputMode, pairs.get(upto).input) + " next=" + inputToString(inputMode, pairs.get(upto+1).input) + " expected=" + expected);
+                  }
+                  assertEquals(expected, fstEnum.beforeNext(t));
+                  break;
+                }
+                tryCount++;
+              }
+            }
+            */
           }
         }
       }
@@ -757,7 +761,9 @@ public class TestFSTs extends LuceneTest
         }
       }
 
-      //System.out.println("TEST: now prune");
+      if (VERBOSE) {
+        System.out.println("TEST: now prune");
+      }
 
       // prune 'em
       final Iterator<Map.Entry<IntsRef,CountMinOutput<T>>> it = prefixes.entrySet().iterator();
@@ -765,7 +771,9 @@ public class TestFSTs extends LuceneTest
         Map.Entry<IntsRef,CountMinOutput<T>> ent = it.next();
         final IntsRef prefix = ent.getKey();
         final CountMinOutput<T> cmo = ent.getValue();
-        //System.out.println("  term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf);
+        if (VERBOSE) {
+          System.out.println("  term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf + " output=" + outputs.outputToString(cmo.output) + " isFinal=" + cmo.isFinal);
+        }
         final boolean keep;
         if (prune1 > 0) {
           keep = cmo.count >= prune1;
@@ -824,14 +832,20 @@ public class TestFSTs extends LuceneTest
       assertNotNull(fst);
 
       // make sure FST only enums valid prefixes
+      if (VERBOSE) {
+        System.out.println("TEST: check pruned enum");
+      }
       IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
-      IntsRefFSTEnum.InputOutput current;
+      IntsRefFSTEnum.InputOutput<T> current;
       while((current = fstEnum.next()) != null) {
-        //System.out.println("  fst enum term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output));
+        if (VERBOSE) {
+          System.out.println("  fstEnum.next term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output));
+        }
         final CountMinOutput cmo = prefixes.get(current.input);
         assertNotNull(cmo);
         assertTrue(cmo.isLeaf || cmo.isFinal);
-        if (cmo.isFinal && !cmo.isLeaf) {
+        //if (cmo.isFinal && !cmo.isLeaf) {
+        if (cmo.isFinal) {
           assertEquals(cmo.finalOutput, current.output);
         } else {
           assertEquals(cmo.output, current.output);
@@ -839,19 +853,24 @@ public class TestFSTs extends LuceneTest
       }
 
       // make sure all non-pruned prefixes are present in the FST
-      final int[] stopNode = new int[2];
+      if (VERBOSE) {
+        System.out.println("TEST: verify all prefixes");
+      }
+      final int[] stopNode = new int[1];
       for(Map.Entry<IntsRef,CountMinOutput<T>> ent : prefixes.entrySet()) {
         if (ent.getKey().length > 0) {
           final CountMinOutput<T> cmo = ent.getValue();
           final T output = run(fst, ent.getKey(), stopNode);
-          //System.out.println("  term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output));
+          if (VERBOSE) {
+            System.out.println("TEST: verify term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output));
+          }
           // if (cmo.isFinal && !cmo.isLeaf) {
           if (cmo.isFinal) {
             assertEquals(cmo.finalOutput, output);
           } else {
             assertEquals(cmo.output, output);
           }
-          assertEquals(ent.getKey().length, stopNode[1]);
+          assertEquals(ent.getKey().length, stopNode[0]);
         }
       }
     }
@@ -859,7 +878,7 @@ public class TestFSTs extends LuceneTest
 
   public void testRandomWords() throws IOException {
     testRandomWords(1000, 5 * RANDOM_MULTIPLIER);
-    //testRandomWords(10, 100);
+    //testRandomWords(20, 100);
   }
 
   private String inputModeToString(int mode) {
@@ -888,7 +907,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  private String getRandomString() {
+  static String getRandomString() {
     final String term;
     if (random.nextBoolean()) {
       term = _TestUtil.randomRealisticUnicodeString(random);
@@ -909,10 +928,10 @@ public class TestFSTs extends LuceneTest
   private static String inputToString(int inputMode, IntsRef term) {
     if (inputMode == 0) {
       // utf8
-      return toBytesRef(term).utf8ToString();
+      return toBytesRef(term).utf8ToString() + " " + term;
     } else {
       // utf32
-      return UnicodeUtil.newString(term.ints, term.offset, term.length);
+      return UnicodeUtil.newString(term.ints, term.offset, term.length) + " " + term;
     }
   }
 
@@ -931,6 +950,7 @@ public class TestFSTs extends LuceneTest
     final File tempDir = _TestUtil.getTempDir("fstlines");
     final MockDirectoryWrapper dir = new MockDirectoryWrapper(random, FSDirectory.open(tempDir));
     final IndexWriter writer = new IndexWriter(dir, conf);
+    writer.setInfoStream(VERBOSE ? System.out : null);
     final long stopTime = System.currentTimeMillis() + RUN_TIME_SEC * 1000;
     Document doc;
     int docCount = 0;
@@ -986,18 +1006,17 @@ public class TestFSTs extends LuceneTest
         // same:
         final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
         for(int iter=0;iter<1000*RANDOM_MULTIPLIER;iter++) {
-          fstEnum.reset();
           final BytesRef randomTerm = new BytesRef(getRandomString());
         
-          final TermsEnum.SeekStatus seekResult = termsEnum.seek(randomTerm);
-          final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.advance(randomTerm);
-
           if (VERBOSE) {
-            System.out.println("TEST: seek " + randomTerm.utf8ToString());
+            System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm);
           }
 
+          final TermsEnum.SeekStatus seekResult = termsEnum.seek(randomTerm);
+          final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.seekCeil(randomTerm);
+
           if (seekResult == TermsEnum.SeekStatus.END) {
-            assertNull(fstSeekResult);
+            assertNull("got " + (fstSeekResult == null ? "null" : fstSeekResult.input.utf8ToString()) + " but expected null", fstSeekResult);
           } else {
             assertSame(termsEnum, fstEnum, storeOrd);
             for(int nextIter=0;nextIter<10;nextIter++) {
@@ -1011,6 +1030,9 @@ public class TestFSTs extends LuceneTest
                 assertNotNull(fstEnum.next());
                 assertSame(termsEnum, fstEnum, storeOrd);
               } else {
+                if (VERBOSE) {
+                  System.out.println("  end!");
+                }
                 BytesRefFSTEnum.InputOutput<Long> nextResult = fstEnum.next();
                 if (nextResult != null) {
                   System.out.println("expected null but got: input=" + nextResult.input.utf8ToString() + " output=" + outputs.outputToString(nextResult.output));
@@ -1032,7 +1054,8 @@ public class TestFSTs extends LuceneTest
     if (termsEnum.term() == null) {
       assertNull(fstEnum.current());
     } else {
-      assertEquals(termsEnum.term(), fstEnum.current().input);
+      assertNotNull(fstEnum.current());
+      assertEquals(termsEnum.term().utf8ToString() + " != " + fstEnum.current().input.utf8ToString(), termsEnum.term(), fstEnum.current().input);
       if (storeOrd) {
         // fst stored the ord
         assertEquals(termsEnum.ord(), ((Long) fstEnum.current().output).longValue());
@@ -1095,7 +1118,7 @@ public class TestFSTs extends LuceneTest
         System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes());
         if (fst.getNodeCount() < 100) {
           PrintStream ps = new PrintStream("out.dot");
-          fst.toDot(ps);
+          Util.toDot(fst, ps);
           ps.close();
           System.out.println("Wrote FST to out.dot");
         }
@@ -1121,7 +1144,7 @@ public class TestFSTs extends LuceneTest
           }
           toIntsRef(w, inputMode, intsRef);
           T expected = getOutput(intsRef, ord);
-          T actual = fst.get(intsRef);
+          T actual = Util.get(fst, intsRef);
           if (actual == null) {
             throw new RuntimeException("unexpected null output on input=" + w);
           }
@@ -1233,4 +1256,57 @@ public class TestFSTs extends LuceneTest
       }.run(limit);
     }
   }
+
+  public void testSingleString() throws Exception {
+    final Outputs<Object> outputs = NoOutputs.getSingleton();
+    final Builder<Object> b = new Builder<Object>(FST.INPUT_TYPE.BYTE1, 0, 0, true, outputs);
+    b.add(new BytesRef("foobar"), outputs.getNoOutput());
+    final BytesRefFSTEnum<Object> fstEnum = new BytesRefFSTEnum<Object>(b.finish());
+    assertNull(fstEnum.seekFloor(new BytesRef("foo")));
+    assertNull(fstEnum.seekCeil(new BytesRef("foobaz")));
+  }
+
+  public void testSimple() throws Exception {
+
+    // Get outputs -- passing true means FST will share
+    // (delta code) the outputs.  This should result in
+    // smaller FST if the outputs grow monotonically.  But
+    // if numbers are "random", false should give smaller
+    // final size:
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+
+    // Build an FST mapping BytesRef -> Long
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, outputs);
+
+    final BytesRef a = new BytesRef("a");
+    final BytesRef b = new BytesRef("b");
+    final BytesRef c = new BytesRef("c");
+
+    builder.add(a, outputs.get(17));
+    builder.add(b, outputs.get(42));
+    builder.add(c, outputs.get(13824324872317238L));
+
+    final FST<Long> fst = builder.finish();
+
+    assertEquals(13824324872317238L, (long) Util.get(fst, c));
+    assertEquals(42, (long) Util.get(fst, b));
+    assertEquals(17, (long) Util.get(fst, a));
+
+    BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
+    BytesRefFSTEnum.InputOutput<Long> seekResult;
+    seekResult = fstEnum.seekFloor(a);
+    assertNotNull(seekResult);
+    assertEquals(17, (long) seekResult.output);
+
+    // goes to a
+    seekResult = fstEnum.seekFloor(new BytesRef("aa"));
+    assertNotNull(seekResult);
+    assertEquals(17, (long) seekResult.output);
+
+    // goes to b
+    seekResult = fstEnum.seekCeil(new BytesRef("aa"));
+    assertNotNull(seekResult);
+    assertEquals(b, seekResult.input);
+    assertEquals(42, (long) seekResult.output);
+  }
 }