Index: modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
===================================================================
--- modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java	(revision 1159650)
+++ modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java	(working copy)
@@ -400,7 +400,7 @@
     if (terms == null) {
       return Collections.emptyList();
     }
-    FuzzyTermsEnum e = new FuzzyTermsEnum(terms.iterator(), atts, term, editDistance, Math.max(minPrefix, editDistance-1));
+    FuzzyTermsEnum e = new FuzzyTermsEnum(terms, atts, term, editDistance, Math.max(minPrefix, editDistance-1));
     final PriorityQueue<ScoreTerm> stQueue = new PriorityQueue<ScoreTerm>();
     
     BytesRef queryTerm = new BytesRef(term.text());
Index: lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
===================================================================
--- lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java	(revision 1159650)
+++ lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java	(working copy)
@@ -208,7 +208,7 @@
                   AttributeSource atts = new AttributeSource();
                   MaxNonCompetitiveBoostAttribute maxBoostAtt =
                     atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
-                  FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()).iterator(), atts, startTerm, f.minSimilarity, f.prefixLength);
+                  FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()), atts, startTerm, f.minSimilarity, f.prefixLength);
                   //store the df so all variants use same idf
                   int df = reader.docFreq(startTerm);
                   int numVariants=0;
Index: lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java	(working copy)
@@ -75,6 +75,9 @@
   }
 
   public void assertFromTestData(int codePointTable[]) throws Exception {
+    if (VERBOSE) {
+      System.out.println("TEST: codePointTable=" + codePointTable);
+    }
     InputStream stream = getClass().getResourceAsStream("fuzzyTestData.txt");
     BufferedReader reader = new BufferedReader(new InputStreamReader(stream, "UTF-8"));
     
@@ -83,6 +86,8 @@
     
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random, dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
+
+    writer.w.setInfoStream(VERBOSE ? System.out : null);
     
     Document doc = new Document();
     Field field = newField("field", "", Field.Store.NO, Field.Index.ANALYZED);
@@ -95,6 +100,9 @@
     
     IndexReader r = writer.getReader();
     IndexSearcher searcher = newSearcher(r);
+    if (VERBOSE) {
+      System.out.println("TEST: searcher=" + searcher);
+    }
     writer.close();
     String line;
     while ((line = reader.readLine()) != null) {
Index: lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java	(working copy)
@@ -87,6 +87,7 @@
             printHits(getName(), h, s);
         }
         assertEquals("result count", expected, h.length);
+        //System.out.println("TEST: now check");
         QueryUtils.check(random, q,s);
     }
 
Index: lucene/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java	(working copy)
@@ -94,12 +94,18 @@
   public static Query csrq(String f, String l, String h, boolean il, boolean ih) {
     TermRangeQuery query = TermRangeQuery.newStringRange(f, l, h, il, ih);
     query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
+    if (VERBOSE) {
+      System.out.println("TEST: query=" + query);
+    }
     return query;
   }
 
   public static Query csrq(String f, String l, String h, boolean il, boolean ih, MultiTermQuery.RewriteMethod method) {
     TermRangeQuery query = TermRangeQuery.newStringRange(f, l, h, il, ih);
     query.setRewriteMethod(method);
+    if (VERBOSE) {
+      System.out.println("TEST: query=" + query + " method=" + method);
+    }
     return query;
   }
 
@@ -275,6 +281,10 @@
     IndexReader reader = signedIndexReader;
     IndexSearcher search = newSearcher(reader);
 
+    if (VERBOSE) {
+      System.out.println("TEST: reader=" + reader);
+    }
+
     int medId = ((maxId - minId) / 2);
 
     String minIP = pad(minId);
Index: lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java	(working copy)
@@ -63,6 +63,9 @@
     reader = writer.getReader();
     searcher = newSearcher(reader);
     writer.close();
+    if (VERBOSE) {
+      System.out.println("TEST: setUp searcher=" + searcher);
+    }
   }
   
   private char N() {
@@ -85,7 +88,11 @@
   
   private void assertPatternHits(String pattern, int numHits) throws Exception {
     // TODO: run with different rewrites
-    Query wq = new WildcardQuery(new Term("field", fillPattern(pattern)));
+    final String filledPattern = fillPattern(pattern);
+    if (VERBOSE) {
+      System.out.println("TEST: run wildcard pattern=" + pattern + " filled=" + filledPattern);
+    }
+    Query wq = new WildcardQuery(new Term("field", filledPattern));
     TopDocs docs = searcher.search(wq, 25);
     assertEquals("Incorrect hits for pattern: " + pattern, numHits, docs.totalHits);
   }
Index: lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java	(working copy)
@@ -152,11 +152,14 @@
     }
     for (int i=1;i<10;i++) {
       query = newPhraseQuery("field", English.intToEnglish(i)+" hundred", true, new AveragePayloadFunction());
+      if (VERBOSE) {
+        System.out.println("TEST: run query=" + query);
+      }
       // all should have score = 3 because adjacent terms have payloads of 2,4
       // and all the similarity factors are set to 1
       hits = searcher.search(query, null, 100);
       assertTrue("hits is null and it shouldn't be", hits != null);
-      assertTrue("should be 100 hits", hits.totalHits == 100);
+      assertEquals("should be 100 hits", 100, hits.totalHits);
       for (int j = 0; j < hits.scoreDocs.length; j++) {
         ScoreDoc doc = hits.scoreDocs[j];
         //				System.out.println("Doc: " + doc.toString());
Index: lucene/src/test/org/apache/lucene/search/TestWildcard.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestWildcard.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestWildcard.java	(working copy)
@@ -135,7 +135,7 @@
     wq = new WildcardQuery(new Term("field", "*"));
     assertMatches(searcher, wq, 2);
     assertFalse(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
-    assertFalse(wq.getTermsEnum(terms) instanceof AutomatonTermsEnum);
+    assertFalse(wq.getTermsEnum(terms).getClass().getSimpleName().contains("AutomatonTermsEnum"));
     searcher.close();
     indexStore.close();
   }
Index: lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java	(working copy)
@@ -154,9 +154,7 @@
     assertEquals(a1, a2);
     
     assertEquals(a1, a3);
-    
-    assertEquals(a1.toString(), a3.toString());
-    
+  
     // different class
     AutomatonQuery w1 = new WildcardQuery(newTerm("foobar"));
     // different class
Index: lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java	(working copy)
@@ -18,9 +18,9 @@
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.ArrayList;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -28,18 +28,18 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -143,6 +143,9 @@
     int num = CodecProvider.getDefault().getFieldCodec("field").equals("PreFlex") ? 100 * RANDOM_MULTIPLIER : atLeast(1000);
     for (int i = 0; i < num; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random);
+      if (VERBOSE) {
+        System.out.println("TEST: regexp=" + reg);
+      }
       assertSame(reg);
     }
   }
@@ -153,18 +156,7 @@
   protected void assertSame(String regexp) throws IOException {   
     RegexpQuery smart = new RegexpQuery(new Term("field", regexp), RegExp.NONE);
     DumbRegexpQuery dumb = new DumbRegexpQuery(new Term("field", regexp), RegExp.NONE);
-    
-    // we can't compare the two if automaton rewrites to a simpler enum.
-    // for example: "a\uda07\udcc7?.*?" gets rewritten to a simpler query:
-    // a\uda07* prefixquery. Prefixquery then does the "wrong" thing, which
-    // isn't really wrong as the query was undefined to begin with... but not
-    // automatically comparable.
-    
-    // TODO: does this check even matter anymore?!
-    Terms terms = MultiFields.getTerms(searcher1.getIndexReader(), "field");
-    if (!(smart.getTermsEnum(terms) instanceof AutomatonTermsEnum))
-      return;
-    
+   
     TopDocs smartDocs = searcher1.search(smart, 25);
     TopDocs dumbDocs = searcher2.search(dumb, 25);
 
Index: lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java	(working copy)
@@ -223,6 +223,7 @@
     return cfg;
   }
 
+  @SuppressWarnings("fallthrough")
   public void runTestNumerics(IndexWriterConfig cfg, boolean withDeletions)
       throws IOException {
     Directory d = newDirectory();
Index: lucene/src/test/org/apache/lucene/index/TestTermsEnum.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestTermsEnum.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestTermsEnum.java	(working copy)
@@ -17,15 +17,32 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericField;
+import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
 
 public class TestTermsEnum extends LuceneTestCase {
 
@@ -140,4 +157,564 @@
     r.close();
     d.close();
   }
+
+  private String randomString() {
+    //return _TestUtil.randomSimpleString(random);
+    return _TestUtil.randomRealisticUnicodeString(random);
+  }
+
+  private void addDoc(RandomIndexWriter w, Collection<String> terms, Map<BytesRef,Integer> termToID, int id) throws IOException {
+    Document doc = new Document();
+    doc.add(new NumericField("id").setIntValue(id));
+    if (VERBOSE) {
+      System.out.println("TEST: addDoc id:" + id + " terms=" + terms);
+    }
+    for (String s2 : terms) {
+      doc.add(newField("f", s2, Field.Index.NOT_ANALYZED));
+      termToID.put(new BytesRef(s2), id);
+    }
+    w.addDocument(doc);
+    terms.clear();
+  }
+
+  private boolean accepts(CompiledAutomaton c, BytesRef b) {
+    int state = c.runAutomaton.getInitialState();
+    for(int idx=0;idx<b.length;idx++) {
+      assertTrue(state != -1);
+      state = c.runAutomaton.step(state, b.bytes[b.offset+idx] & 0xff);
+    }
+    return c.runAutomaton.isAccept(state);
+  }
+
+  // Tests Terms.intersect
+  public void testIntersectRandom() throws IOException {
+
+    final Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random, dir);
+    
+    final int numTerms = atLeast(1000);
+
+    final Set<String> terms = new HashSet<String>();
+    final Collection<String> pendingTerms = new ArrayList<String>();
+    final Map<BytesRef,Integer> termToID = new HashMap<BytesRef,Integer>();
+    int id = 0;
+    while(terms.size() != numTerms) {
+      final String s = randomString();
+      if (!terms.contains(s)) {
+        terms.add(s);
+        pendingTerms.add(s);
+        if (random.nextInt(20) == 7) {
+          addDoc(w, pendingTerms, termToID, id++);
+        }
+      }
+    }
+    addDoc(w, pendingTerms, termToID, id++);
+
+    final BytesRef[] termsArray = new BytesRef[terms.size()];
+    final Set<BytesRef> termsSet = new HashSet<BytesRef>();
+    {
+      int upto = 0;
+      for(String s : terms) {
+        final BytesRef b = new BytesRef(s);
+        termsArray[upto++] = b;
+        termsSet.add(b);
+      }
+      Arrays.sort(termsArray);
+    }
+
+    if (VERBOSE) {
+      System.out.println("\nTEST: indexed terms (unicode order):");
+      for(BytesRef t : termsArray) {
+        System.out.println("  " + t.utf8ToString() + " -> id:" + termToID.get(t));
+      }
+    }
+
+    final IndexReader r = w.getReader();
+    w.close();
+
+    // NOTE: intentional insanity!!
+    final int[] docIDToID = FieldCache.DEFAULT.getInts(r, "id");
+
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      // TODO: can we also test infinite As here...?
+
+      // From the random terms, pick some ratio and compile an
+      // automaton:
+      final List<Automaton> as = new ArrayList<Automaton>();
+      final Set<String> acceptTerms = new HashSet<String>();
+      final TreeSet<BytesRef> sortedAcceptTerms = new TreeSet<BytesRef>();
+      final double keepPct = random.nextDouble();
+      Automaton a;
+      if (iter == 0) {
+        if (VERBOSE) {
+          System.out.println("\nTEST: empty automaton");
+        }
+        a = BasicAutomata.makeEmpty();
+      } else {
+        if (VERBOSE) {
+          System.out.println("\nTEST: keepPct=" + keepPct);
+        }
+        for (String s : terms) {
+          final String s2;
+          if (random.nextDouble() <= keepPct) {
+            s2 = s;
+          } else {
+            s2 = randomString();
+          }
+          acceptTerms.add(s2);
+          sortedAcceptTerms.add(new BytesRef(s2));
+        }
+        a = DaciukMihovAutomatonBuilder.build(sortedAcceptTerms);
+      }
+      final CompiledAutomaton c = new CompiledAutomaton(a, true, false);
+
+      final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];
+      final Set<BytesRef> acceptTermsSet = new HashSet<BytesRef>();
+      int upto = 0;
+      for(String s : acceptTerms) {
+        final BytesRef b = new BytesRef(s);
+        acceptTermsArray[upto++] = b;
+        acceptTermsSet.add(b);
+        assertTrue(accepts(c, b));
+      }
+      Arrays.sort(acceptTermsArray);
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: accept terms (unicode order):");
+        for(BytesRef t : acceptTermsArray) {
+          System.out.println("  " + t.utf8ToString() + (termsSet.contains(t) ? " (exists)" : ""));
+        }
+        System.out.println(a.toDot());
+      }
+
+      for(int iter2=0;iter2<100;iter2++) {
+        final BytesRef startTerm = acceptTermsArray.length == 0 || random.nextBoolean() ? null : acceptTermsArray[random.nextInt(acceptTermsArray.length)];
+
+        final TermsEnum te = MultiFields.getTerms(r, "f").intersect(c, startTerm);
+
+        if (VERBOSE) {
+          System.out.println("\nTEST: iter2=" + iter2 + " startTerm=" + (startTerm == null ? "<null>" : startTerm.utf8ToString()));
+        }
+
+        int loc;
+        if (startTerm == null) {
+          loc = 0;
+        } else {
+          loc = Arrays.binarySearch(termsArray, new BytesRef(startTerm));
+          if (loc < 0) {
+            loc = -(loc+1);
+          } else {
+            // startTerm exists in index
+            loc++;
+          }
+        }
+        while(loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc])) {
+          loc++;
+        }
+
+        DocsEnum docsEnum = null;
+        while (loc < termsArray.length) {
+          final BytesRef expected = termsArray[loc];
+          final BytesRef actual = te.next();
+          if (VERBOSE) {
+            System.out.println("TEST:   next() expected=" + expected.utf8ToString() + " actual=" + actual.utf8ToString());
+          }
+          assertEquals(expected, actual);
+          assertEquals(1, te.docFreq());
+          docsEnum = te.docs(null, docsEnum);
+          final int docID = docsEnum.nextDoc();
+          assertTrue(docID != DocsEnum.NO_MORE_DOCS);
+          assertEquals(docIDToID[docID], termToID.get(expected).intValue());
+          do {
+            loc++;
+          } while (loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc]));
+        }
+
+        assertNull(te.next());
+      }
+    }
+
+    r.close();
+    dir.close();
+  }
+
+  private Directory d;
+  private IndexReader r;
+
+  private final String FIELD = "field";
+
+  private IndexReader makeIndex(String... terms) throws Exception {
+    d = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
+
+    /*
+    CoreCodecProvider cp = new CoreCodecProvider();    
+    cp.unregister(cp.lookup("Standard"));
+    cp.register(new StandardCodec(minTermsInBlock, maxTermsInBlock));
+    cp.setDefaultFieldCodec("Standard");
+    iwc.setCodecProvider(cp);
+    */
+
+    final RandomIndexWriter w = new RandomIndexWriter(random, d, iwc);
+    w.w.setInfoStream(VERBOSE ? System.out : null);
+    for(String term : terms) {
+      Document doc = new Document();
+      Field f = newField(FIELD, term, Field.Index.NOT_ANALYZED_NO_NORMS);
+      doc.add(f);
+      w.addDocument(doc);
+    }
+    if (r != null) {
+      close();
+    }
+    r = w.getReader();
+    w.close();
+    return r;
+  }
+
+  private void close() throws Exception {
+    final Directory d = ((SegmentReader) r.getSequentialSubReaders()[0]).directory();
+    r.close();
+    d.close();
+  }
+
+  private int docFreq(IndexReader r, String term) throws Exception {
+    return r.docFreq(new Term(FIELD, term));
+  }
+
+  public void testEasy() throws Exception {
+    // No floor arcs:
+    r = makeIndex("aa0", "aa1", "aa2", "aa3", "bb0", "bb1", "bb2", "bb3", "aa");
+
+    // First term in block:
+    assertEquals(1, docFreq(r, "aa0"));
+
+    // Scan forward to another term in same block
+    assertEquals(1, docFreq(r, "aa2"));
+
+    assertEquals(1, docFreq(r, "aa"));
+
+    // Reset same block then scan forwards
+    assertEquals(1, docFreq(r, "aa1"));
+
+    // Not found, in same block
+    assertEquals(0, docFreq(r, "aa5"));
+
+    // Found, in same block
+    assertEquals(1, docFreq(r, "aa2"));
+
+    // Not found in index:
+    assertEquals(0, docFreq(r, "b0"));
+
+    // Found:
+    assertEquals(1, docFreq(r, "aa2"));
+
+    // Found, rewind:
+    assertEquals(1, docFreq(r, "aa0"));
+
+
+    // First term in block:
+    assertEquals(1, docFreq(r, "bb0"));
+
+    // Scan forward to another term in same block
+    assertEquals(1, docFreq(r, "bb2"));
+
+    // Reset same block then scan forwards
+    assertEquals(1, docFreq(r, "bb1"));
+
+    // Not found, in same block
+    assertEquals(0, docFreq(r, "bb5"));
+
+    // Found, in same block
+    assertEquals(1, docFreq(r, "bb2"));
+
+    // Not found in index:
+    assertEquals(0, docFreq(r, "b0"));
+
+    // Found:
+    assertEquals(1, docFreq(r, "bb2"));
+
+    // Found, rewind:
+    assertEquals(1, docFreq(r, "bb0"));
+
+    close();
+  }
+
+  // tests:
+  //   - test same prefix has non-floor block and floor block (ie, has 2 long outputs on same term prefix)
+  //   - term that's entirely in the index
+
+  public void testFloorBlocks() throws Exception {
+    final String[] terms = new String[] {"aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9", "aa", "xx"};
+    r = makeIndex(terms);
+    //r = makeIndex("aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9");
+
+    // First term in first block:
+    assertEquals(1, docFreq(r, "aa0"));
+    assertEquals(1, docFreq(r, "aa4"));
+
+    // No block
+    assertEquals(0, docFreq(r, "bb0"));
+
+    // Second block
+    assertEquals(1, docFreq(r, "aa4"));
+
+    // Backwards to prior floor block:
+    assertEquals(1, docFreq(r, "aa0"));
+
+    // Forwards to last floor block:
+    assertEquals(1, docFreq(r, "aa9"));
+
+    assertEquals(0, docFreq(r, "a"));
+    assertEquals(1, docFreq(r, "aa"));
+    assertEquals(0, docFreq(r, "a"));
+    assertEquals(1, docFreq(r, "aa"));
+
+    // Forwards to last floor block:
+    assertEquals(1, docFreq(r, "xx"));
+    assertEquals(1, docFreq(r, "aa1"));
+    assertEquals(0, docFreq(r, "yy"));
+
+    assertEquals(1, docFreq(r, "xx"));
+    assertEquals(1, docFreq(r, "aa9"));
+
+    assertEquals(1, docFreq(r, "xx"));
+    assertEquals(1, docFreq(r, "aa4"));
+
+    final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator();
+    while(te.next() != null) {
+      //System.out.println("TEST: next term=" + te.term().utf8ToString());
+    }
+
+    assertTrue(seekExact(te, "aa1"));
+    assertEquals("aa2", next(te));
+    assertTrue(seekExact(te, "aa8"));
+    assertEquals("aa9", next(te));
+    assertEquals("xx", next(te));
+
+    testRandomSeeks(r, terms);
+    close();
+  }
+
+  public void testZeroTerms() throws Exception {
+    d = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random, d);
+    w.w.setInfoStream(VERBOSE ? System.out : null);
+    Document doc = new Document();
+    doc.add(newField("field", "one two three", Field.Index.ANALYZED));
+    doc = new Document();
+    doc.add(newField("field2", "one two three", Field.Index.ANALYZED));
+    w.addDocument(doc);
+    w.commit();
+    w.deleteDocuments(new Term("field", "one"));
+    w.optimize();
+    IndexReader r = w.getReader();
+    w.close();
+    assertEquals(1, r.numDocs());
+    assertEquals(1, r.maxDoc());
+    Terms terms = MultiFields.getTerms(r, "field");
+    if (terms != null) {
+      assertNull(terms.iterator().next());
+    }
+    r.close();
+    d.close();
+  }
+
+  private String getRandomString() {
+    //return _TestUtil.randomSimpleString(random);
+    return _TestUtil.randomRealisticUnicodeString(random);
+  }
+
+  public void testRandomTerms() throws Exception {
+    final String[] terms = new String[_TestUtil.nextInt(random, 1, atLeast(1000))];
+    final Set<String> seen = new HashSet<String>();
+
+    final boolean allowEmptyString = random.nextBoolean();
+
+    if (random.nextInt(10) == 7 && terms.length > 2) {
+      // Sometimes add a bunch of terms sharing a longish common prefix:
+      final int numTermsSamePrefix = random.nextInt(terms.length/2);
+      if (numTermsSamePrefix > 0) {
+        String prefix;
+        while(true) {
+          prefix = getRandomString();
+          if (prefix.length() < 5) {
+            continue;
+          } else {
+            break;
+          }
+        }
+        while(seen.size() < numTermsSamePrefix) {
+          final String t = prefix + getRandomString();
+          if (!seen.contains(t)) {
+            terms[seen.size()] = t;
+            seen.add(t);
+          }
+        }
+      }
+    }
+
+    while(seen.size() < terms.length) {
+      final String t = getRandomString();
+      if (!seen.contains(t) && (allowEmptyString || t.length() != 0)) {
+        terms[seen.size()] = t;
+        seen.add(t);
+      }
+    }
+    r = makeIndex(terms);
+    testRandomSeeks(r, terms);
+    close();
+  }
+
+  // sugar
+  private boolean seekExact(TermsEnum te, String term) throws IOException {
+    return te.seekExact(new BytesRef(term), random.nextBoolean());
+  }
+
+  // sugar
+  private String next(TermsEnum te) throws IOException {
+    final BytesRef br = te.next();
+    if (br == null) {
+      return null;
+    } else {
+      return br.utf8ToString();
+    }
+  }
+
+  private BytesRef getNonExistTerm(BytesRef[] terms) {
+    BytesRef t = null;
+    while(true) {
+      final String ts = getRandomString();
+      t = new BytesRef(ts);
+      if (Arrays.binarySearch(terms, t) < 0) {
+        return t;
+      }
+    }
+  }
+
+  private static class TermAndState {
+    public final BytesRef term;
+    public final TermState state;
+
+    public TermAndState(BytesRef term, TermState state) {
+      this.term = term;
+      this.state = state;
+    }
+  }
+
+  private void testRandomSeeks(IndexReader r, String... validTermStrings) throws IOException {
+    final BytesRef[] validTerms = new BytesRef[validTermStrings.length];
+    for(int termIDX=0;termIDX<validTermStrings.length;termIDX++) {
+      validTerms[termIDX] = new BytesRef(validTermStrings[termIDX]);
+    }
+    Arrays.sort(validTerms);
+    if (VERBOSE) {
+      System.out.println("TEST: " + validTerms.length + " terms:");
+      for(BytesRef t : validTerms) {
+        System.out.println("  " + t.utf8ToString() + " " + t);
+      }
+    }
+    final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator();
+
+    final int END_LOC = -validTerms.length-1;
+    
+    final List<TermAndState> termStates = new ArrayList<TermAndState>();
+
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      final BytesRef t;
+      int loc;
+      final TermState termState;
+      if (random.nextInt(6) == 4) {
+        // pick term that doens't exist:
+        t = getNonExistTerm(validTerms);
+        termState = null;
+        if (VERBOSE) {
+          System.out.println("\nTEST: invalid term=" + t.utf8ToString());
+        }
+        loc = Arrays.binarySearch(validTerms, t);
+      } else if (termStates.size() != 0 && random.nextInt(4) == 1) {
+        final TermAndState ts = termStates.get(random.nextInt(termStates.size()));
+        t = ts.term;
+        loc = Arrays.binarySearch(validTerms, t);
+        assertTrue(loc >= 0);
+        termState = ts.state;
+        if (VERBOSE) {
+          System.out.println("\nTEST: valid termState term=" + t.utf8ToString());
+        }
+      } else {
+        // pick valid term
+        loc = random.nextInt(validTerms.length);
+        t = new BytesRef(validTerms[loc]);
+        termState = null;
+        if (VERBOSE) {
+          System.out.println("\nTEST: valid term=" + t.utf8ToString());
+        }
+      }
+
+      // seekCeil or seekExact:
+      final boolean doSeekExact = random.nextBoolean();
+      if (termState != null) {
+        if (VERBOSE) {
+          System.out.println("  seekExact termState");
+        }
+        te.seekExact(t, termState);
+      } else if (doSeekExact) {
+        if (VERBOSE) {
+          System.out.println("  seekExact");
+        }
+        assertEquals(loc >= 0, te.seekExact(t, random.nextBoolean()));
+      } else {
+        if (VERBOSE) {
+          System.out.println("  seekCeil");
+        }
+
+        final TermsEnum.SeekStatus result = te.seekCeil(t, random.nextBoolean());
+        if (VERBOSE) {
+          System.out.println("  got " + result);
+        }
+
+        if (loc >= 0) {
+          assertEquals(TermsEnum.SeekStatus.FOUND, result);
+        } else if (loc == END_LOC) {
+          assertEquals(TermsEnum.SeekStatus.END, result);
+        } else {
+          assert loc >= -validTerms.length;
+          assertEquals(TermsEnum.SeekStatus.NOT_FOUND, result);
+        }
+      }
+
+      if (loc >= 0) {
+        assertEquals(t, te.term());
+      } else if (doSeekExact) {
+        // TermsEnum is unpositioned if seekExact returns false
+        continue;
+      } else if (loc == END_LOC) {
+        continue;
+      } else {
+        loc = -loc-1;
+        assertEquals(validTerms[loc], te.term());
+      }
+
+      // Do a bunch of next's after the seek
+      final int numNext = random.nextInt(validTerms.length);
+
+      for(int nextCount=0;nextCount<numNext;nextCount++) {
+        if (VERBOSE) {
+          System.out.println("\nTEST: next loc=" + loc + " of " + validTerms.length);
+        }
+        final BytesRef t2 = te.next();
+        loc++;
+        if (loc == validTerms.length) {
+          assertNull(t2);
+          break;
+        } else {
+          assertEquals(validTerms[loc], t2);
+          if (random.nextInt(40) == 17 && termStates.size() < 100) {
+            termStates.add(new TermAndState(validTerms[loc], te.termState()));
+          }
+        }
+      }
+    }
+  }
 }
Index: lucene/src/test/org/apache/lucene/index/TestLongPostings.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestLongPostings.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestLongPostings.java	(working copy)
@@ -355,7 +355,7 @@
       }
 
       if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1);
+        System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1 + " term=" + term);
       }
         
       final DocsEnum postings = MultiFields.getTermDocsEnum(r, null, "field", new BytesRef(term));
@@ -426,7 +426,7 @@
           
           if (random.nextInt(6) == 3) {
             final int freq = postings.freq();
-            assertTrue(freq >=1 && freq <= 4);
+            assertTrue("got invalid freq=" + freq, freq >=1 && freq <= 4);
           }
         }
       }
Index: lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java	(working copy)
@@ -194,7 +194,8 @@
   public void testRandomDocs() throws IOException {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random, dir,
-        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+                                                     newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+    writer.w.setInfoStream(VERBOSE ? System.out : null);
     int numDocs = atLeast(49);
     int max = 15678;
     int term = random.nextInt(max);
@@ -290,7 +291,7 @@
       writer.addDocument(doc);
     }
 
-    // now do seaches
+    // now do searches
     IndexReader reader = writer.getReader();
     writer.close();
 
Index: lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(working copy)
@@ -169,7 +169,7 @@
     Set<String> dif = difFiles(files, files2);
     
     if (!Arrays.equals(files, files2)) {
-      fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n    " + asString(files) + "\n  actual files:\n    " + asString(files2)+"\ndif: "+dif);
+      fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n    " + asString(files) + "\n  actual files:\n    " + asString(files2)+"\ndiff: "+dif);
     }
   }
 
Index: lucene/src/test/org/apache/lucene/index/TestMultiFields.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestMultiFields.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestMultiFields.java	(working copy)
@@ -29,10 +29,15 @@
 
     int num = atLeast(2);
     for (int iter = 0; iter < num; iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter);
+      }
+
       Directory dir = newDirectory();
 
       IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
       _TestUtil.keepFullyDeletedSegments(w);
+      w.setInfoStream(VERBOSE ? System.out : null);
 
       Map<BytesRef,List<Integer>> docs = new HashMap<BytesRef,List<Integer>>();
       Set<Integer> deleted = new HashSet<Integer>();
@@ -46,6 +51,9 @@
       doc.add(id);
 
       boolean onlyUniqueTerms = random.nextBoolean();
+      if (VERBOSE) {
+        System.out.println("TEST: onlyUniqueTerms=" + onlyUniqueTerms + " numDocs=" + numDocs);
+      }
       Set<BytesRef> uniqueTerms = new HashSet<BytesRef>();
       for(int i=0;i<numDocs;i++) {
 
@@ -74,21 +82,33 @@
           int delID = random.nextInt(i);
           deleted.add(delID);
           w.deleteDocuments(new Term("id", ""+delID));
+          if (VERBOSE) {
+            System.out.println("TEST: delete " + delID);
+          }
         }
       }
 
       if (VERBOSE) {
         List<BytesRef> termsList = new ArrayList<BytesRef>(uniqueTerms);
         Collections.sort(termsList, BytesRef.getUTF8SortedAsUTF16Comparator());
-        System.out.println("UTF16 order:");
+        System.out.println("TEST: terms in UTF16 order:");
         for(BytesRef b : termsList) {
-          System.out.println("  " + UnicodeUtil.toHexString(b.utf8ToString()));
+          System.out.println("  " + UnicodeUtil.toHexString(b.utf8ToString()) + " " + b);
+          for(int docID : docs.get(b)) {
+            if (deleted.contains(docID)) {
+              System.out.println("    " + docID + " (deleted)");
+            } else {
+              System.out.println("    " + docID);
+            }
+          }
         }
       }
 
       IndexReader reader = w.getReader();
       w.close();
-      //System.out.println("TEST reader=" + reader);
+      if (VERBOSE) {
+        System.out.println("TEST: reader=" + reader);
+      }
 
       Bits liveDocs = MultiFields.getLiveDocs(reader);
       for(int delDoc : deleted) {
@@ -99,7 +119,7 @@
       for(int i=0;i<100;i++) {
         BytesRef term = terms.get(random.nextInt(terms.size()));
         if (VERBOSE) {
-          System.out.println("TEST: seek to term= "+ UnicodeUtil.toHexString(term.utf8ToString()));
+          System.out.println("TEST: seek term="+ UnicodeUtil.toHexString(term.utf8ToString()) + " " + term);
         }
         
         DocsEnum docsEnum = terms2.docs(liveDocs, term, null);
Index: lucene/src/test/org/apache/lucene/index/TestCodecs.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestCodecs.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestCodecs.java	(working copy)
@@ -504,6 +504,9 @@
         }
 
         // Test seek to non-existent terms:
+        if (VERBOSE) {
+          System.out.println("TEST: seek non-exist terms");
+        }
         for(int i=0;i<100;i++) {
           final String text2 = _TestUtil.randomUnicodeString(random) + ".";
           status = termsEnum.seekCeil(new BytesRef(text2));
@@ -512,6 +515,9 @@
         }
 
         // Seek to each term, backwards:
+        if (VERBOSE) {
+          System.out.println("TEST: seek terms backwards");
+        }
         for(int i=field.terms.length-1;i>=0;i--) {
           assertEquals(Thread.currentThread().getName() + ": field=" + field.fieldInfo.name + " term=" + field.terms[i].text2, TermsEnum.SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(field.terms[i].text2)));
           assertEquals(field.terms[i].docs.length, termsEnum.docFreq());
Index: lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java	(working copy)
@@ -462,7 +462,7 @@
     */
 
     if (VERBOSE) {
-      System.out.println("TEST: verify prefix=" + prefixRef.utf8ToString());
+      System.out.println("TEST: verify prefix=" + (prefixRef==null ? "null" : prefixRef.utf8ToString()));
       System.out.println("TEST: all TERMS:");
       TermsEnum allTE = MultiFields.getTerms(r, "field").iterator();
       int ord = 0;
Index: lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java	(working copy)
@@ -74,8 +74,15 @@
     Term term = new Term("city", "Amsterdam");
     int hitCount = getHitCount(dir, term);
     assertEquals(1, hitCount);
+    if (VERBOSE) {
+      System.out.println("\nTEST: now delete by term=" + term);
+    }
     modifier.deleteDocuments(term);
     modifier.commit();
+
+    if (VERBOSE) {
+      System.out.println("\nTEST: now getHitCount");
+    }
     hitCount = getHitCount(dir, term);
     assertEquals(0, hitCount);
 
Index: lucene/src/test/org/apache/lucene/index/TestTermsEnum2.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestTermsEnum2.java	(revision 0)
+++ lucene/src/test/org/apache/lucene/index/TestTermsEnum2.java	(revision 0)
@@ -0,0 +1,175 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.search.AutomatonQuery;
+import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
+import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.util.automaton.SpecialOperations;
+
+public class TestTermsEnum2 extends LuceneTestCase {
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+  private SortedSet<BytesRef> terms; // the terms we put in the index
+  private Automaton termsAutomaton; // automata of the same
+  int numIterations;
+
+  public void setUp() throws Exception {
+    super.setUp();
+    // we generate aweful regexps: good for testing.
+    // but for preflex codec, the test can be very slow, so use less iterations.
+    numIterations = CodecProvider.getDefault().getFieldCodec("field").equals("PreFlex") ? 10 * RANDOM_MULTIPLIER : atLeast(50);
+    dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random, dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT,
+            new MockAnalyzer(random, MockTokenizer.KEYWORD, false))
+            .setMaxBufferedDocs(_TestUtil.nextInt(random, 50, 1000)));
+    Document doc = new Document();
+    Field field = newField("field", "", Field.Store.YES, Field.Index.NOT_ANALYZED);
+    doc.add(field);
+    terms = new TreeSet<BytesRef>();
+ 
+    int num = atLeast(200);
+    for (int i = 0; i < num; i++) {
+      String s = _TestUtil.randomUnicodeString(random);
+      field.setValue(s);
+      terms.add(new BytesRef(s));
+      writer.addDocument(doc);
+    }
+    
+    termsAutomaton = DaciukMihovAutomatonBuilder.build(terms);
+    
+    reader = writer.getReader();
+    searcher = newSearcher(reader);
+    writer.close();
+  }
+  
+  public void tearDown() throws Exception {
+    searcher.close();
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+  
+  /** tests a pre-intersected automaton against the original */
+  public void testFiniteVersusInfinite() throws Exception {
+    for (int i = 0; i < numIterations; i++) {
+      String reg = AutomatonTestUtil.randomRegexp(random);
+      Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
+      final List<BytesRef> matchedTerms = new ArrayList<BytesRef>();
+      for(BytesRef t : terms) {
+        if (BasicOperations.run(automaton, t.utf8ToString())) {
+          matchedTerms.add(t);
+        }
+      }
+
+      Automaton alternate = DaciukMihovAutomatonBuilder.build(matchedTerms);
+      //System.out.println("match " + matchedTerms.size() + " " + alternate.getNumberOfStates() + " states, sigma=" + alternate.getStartPoints().length);
+      //AutomatonTestUtil.minimizeSimple(alternate);
+      //System.out.println("minmize done");
+      AutomatonQuery a1 = new AutomatonQuery(new Term("field", ""), automaton);
+      AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate);
+      CheckHits.checkEqual(a1, searcher.search(a1, 25).scoreDocs, searcher.search(a2, 25).scoreDocs);
+    }
+  }
+  
+  /** seeks to every term accepted by some automata */
+  public void testSeeking() throws Exception {
+    for (int i = 0; i < numIterations; i++) {
+      String reg = AutomatonTestUtil.randomRegexp(random);
+      Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
+      TermsEnum te = MultiFields.getTerms(reader, "field").iterator();
+      ArrayList<BytesRef> unsortedTerms = new ArrayList<BytesRef>(terms);
+      Collections.shuffle(unsortedTerms, random);
+
+      for (BytesRef term : unsortedTerms) {
+        if (BasicOperations.run(automaton, term.utf8ToString())) {
+          // term is accepted
+          if (random.nextBoolean()) {
+            // seek exact
+            assertTrue(te.seekExact(term, random.nextBoolean()));
+          } else {
+            // seek ceil
+            assertEquals(SeekStatus.FOUND, te.seekCeil(term, random.nextBoolean()));
+            assertEquals(term, te.term());
+          }
+        }
+      }
+    }
+  }
+  
+  /** mixes up seek and next for all terms */
+  public void testSeekingAndNexting() throws Exception {
+    for (int i = 0; i < numIterations; i++) {
+      TermsEnum te = MultiFields.getTerms(reader, "field").iterator();
+
+      for (BytesRef term : terms) {
+        int c = random.nextInt(3);
+        if (c == 0) {
+          assertEquals(term, te.next());
+        } else if (c == 1) {
+          assertEquals(SeekStatus.FOUND, te.seekCeil(term, random.nextBoolean()));
+          assertEquals(term, te.term());
+        } else {
+          assertTrue(te.seekExact(term, random.nextBoolean()));
+        }
+      }
+    }
+  }
+  
+  /** tests intersect: TODO start at a random term! */
+  public void testIntersect() throws Exception {
+    for (int i = 0; i < numIterations; i++) {
+      String reg = AutomatonTestUtil.randomRegexp(random);
+      Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
+      CompiledAutomaton ca = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton), false);
+      TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null);
+      Automaton expected = BasicOperations.intersection(termsAutomaton, automaton);
+      TreeSet<BytesRef> found = new TreeSet<BytesRef>();
+      while (te.next() != null) {
+        found.add(new BytesRef(te.term()));
+      }
+      
+      Automaton actual = DaciukMihovAutomatonBuilder.build(found);     
+      assertTrue(BasicOperations.sameLanguage(expected, actual));
+    }
+  }
+}

Property changes on: lucene/src/test/org/apache/lucene/index/TestTermsEnum2.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java	(working copy)
@@ -221,6 +221,8 @@
             setMergePolicy(newLogMergePolicy(50))
     );
 
+    writer.setInfoStream(VERBOSE ? System.out : null);
+
     Document document = new Document();
 
     document = new Document();
Index: lucene/src/test/org/apache/lucene/index/Test2BTerms.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/Test2BTerms.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/index/Test2BTerms.java	(working copy)
@@ -155,9 +155,9 @@
     List<BytesRef> savedTerms = null;
 
     MockDirectoryWrapper dir = newFSDirectory(_TestUtil.getTempDir("2BTerms"));
+    //MockDirectoryWrapper dir = newFSDirectory(new File("/p/lucene/indices/2bindex"));
     dir.setThrottling(MockDirectoryWrapper.Throttling.NEVER);
     dir.setCheckIndexOnClose(false); // don't double-checkindex
-    //Directory dir = newFSDirectory(new File("/p/lucene/indices/2bindex"));
 
     if (true) {
 
@@ -169,6 +169,7 @@
                                       .setMergePolicy(newLogMergePolicy(false, 10))
                                       .setOpenMode(IndexWriterConfig.OpenMode.CREATE));
 
+      w.setInfoStream(VERBOSE ? System.out : null);
       MergePolicy mp = w.getConfig().getMergePolicy();
       if (mp instanceof LogByteSizeMergePolicy) {
         // 1 petabyte:
Index: lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java
===================================================================
--- lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java	(working copy)
@@ -86,7 +86,7 @@
       }
       IndexWriter writer = new IndexWriter(directory, conf);
       if (VERBOSE) {
-        System.out.println("TEST: now build index");
+        System.out.println("TEST: now build index MAX_DOCS=" + MAX_DOCS);
         writer.setInfoStream(System.out);
       }
 
Index: lucene/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java
===================================================================
--- lucene/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java	(revision 0)
+++ lucene/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java	(revision 0)
@@ -0,0 +1,121 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+public class TestCompiledAutomaton extends LuceneTestCase {
+
+  private CompiledAutomaton build(String... strings) {
+    final List<Automaton> as = new ArrayList<Automaton>();
+    for(String s : strings) {
+      as.add(BasicAutomata.makeString(s));
+    }
+    Automaton a = BasicOperations.union(as);
+    a.determinize();
+    return new CompiledAutomaton(a, true, false);
+  }
+
+  private void testFloor(CompiledAutomaton c, String input, String expected) {
+    final BytesRef b = new BytesRef(input);
+    final BytesRef result = c.floor(b, b);
+    if (expected == null) {
+      assertNull(result);
+    } else {
+      assertNotNull(result);
+      assertEquals("actual=" + result.utf8ToString() + " vs expected=" + expected + " (input=" + input + ")",
+                   result, new BytesRef(expected));
+    }
+  }
+
+  private void testTerms(String[] terms) throws Exception {
+    final CompiledAutomaton c = build(terms);
+    final BytesRef[] termBytes = new BytesRef[terms.length];
+    for(int idx=0;idx<terms.length;idx++) {
+      termBytes[idx] = new BytesRef(terms[idx]);
+    }
+    Arrays.sort(termBytes);
+
+    if (VERBOSE) {
+      System.out.println("\nTEST: terms in unicode order");
+      for(BytesRef t : termBytes) {
+        System.out.println("  " + t.utf8ToString());
+      }
+      //System.out.println(c.utf8.toDot());
+    }
+
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+      final String s = random.nextInt(10) == 1 ? terms[random.nextInt(terms.length)] : randomString();
+      if (VERBOSE) {
+        System.out.println("\nTEST: floor(" + s + ")");
+      }
+      int loc = Arrays.binarySearch(termBytes, new BytesRef(s));
+      final String expected;
+      if (loc >= 0) {
+        expected = s;
+      } else {
+        // term doesn't exist
+        loc = -(loc+1);
+        if (loc == 0) {
+          expected = null;
+        } else {
+          expected = termBytes[loc-1].utf8ToString();
+        }
+      }
+      if (VERBOSE) {
+        System.out.println("  expected=" + expected);
+      }
+      testFloor(c, s, expected);
+    }
+  }
+
+  public void testRandom() throws Exception {
+    final int numTerms = atLeast(1000);
+    final Set<String> terms = new HashSet<String>();
+    while(terms.size() != numTerms) {
+      terms.add(randomString());
+    }
+    testTerms(terms.toArray(new String[terms.size()]));
+  }
+
+  private String randomString() {
+    // return _TestUtil.randomSimpleString(random);
+    return _TestUtil.randomRealisticUnicodeString(random);
+  }
+
+  public void testBasic() throws Exception {
+    CompiledAutomaton c = build("foo", "fob", "goo");
+    testFloor(c, "goo", "goo");
+    testFloor(c, "ga", "foo");
+    testFloor(c, "g", "foo");
+    testFloor(c, "foc", "fob");
+    testFloor(c, "foz", "foo");
+    testFloor(c, "f", null);
+    testFloor(c, "", null);
+    testFloor(c, "aa", null);
+    testFloor(c, "zzz", "goo");
+  }
+}

Property changes on: lucene/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java
===================================================================
--- lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java	(revision 1159650)
+++ lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java	(working copy)
@@ -24,19 +24,25 @@
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
+import java.io.StringWriter;
 import java.io.Writer;
 import java.util.*;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IndexInput;
@@ -456,7 +462,8 @@
                                                 prune1==0 && prune2==0,
                                                 allowRandomSuffixSharing ? random.nextBoolean() : true,
                                                 allowRandomSuffixSharing ? _TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
-                                                outputs);
+                                                outputs,
+                                                null);
 
       for(InputOutput<T> pair : pairs) {
         if (pair.output instanceof UpToTwoPositiveIntOutputs.TwoLongs) {
@@ -872,15 +879,15 @@
         }
       }
 
-      //System.out.println("TEST: after prune");
-      /*
-        for(Map.Entry<BytesRef,CountMinOutput> ent : prefixes.entrySet()) {
-        System.out.println("  " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal);
-        if (ent.getValue().isFinal) {
-        System.out.println("    finalOutput=" + outputs.outputToString(ent.getValue().finalOutput));
+      if (VERBOSE) {
+        System.out.println("TEST: after prune");
+        for(Map.Entry<IntsRef,CountMinOutput<T>> ent : prefixes.entrySet()) {
+          System.out.println("  " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal);
+          if (ent.getValue().isFinal) {
+            System.out.println("    finalOutput=" + outputs.outputToString(ent.getValue().finalOutput));
+          }
         }
-        }
-      */
+      }
 
       if (prefixes.size() <= 1) {
         assertNull(fst);
@@ -1081,7 +1088,7 @@
           final BytesRef randomTerm = new BytesRef(getRandomString());
         
           if (VERBOSE) {
-            System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm);
+            System.out.println("TEST: seek non-exist " + randomTerm.utf8ToString() + " " + randomTerm);
           }
 
           final TermsEnum.SeekStatus seekResult = termsEnum.seekCeil(randomTerm);
@@ -1133,10 +1140,10 @@
       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());
+        assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.ord(), ((Long) fstEnum.current().output).longValue());
       } else {
         // fst stored the docFreq
-        assertEquals(termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue()));
+        assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue()));
       }
     }
   }
@@ -1154,7 +1161,7 @@
       this.inputMode = inputMode;
       this.outputs = outputs;
       
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs);
+      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -1254,7 +1261,7 @@
     }
   }
 
-  // java -cp build/classes/test:build/classes/java:build/classes/test-framework:lib/junit-4.7.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
+  // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.7.jar org.apache.lucene.util.automaton.fst.TestFSTs /x/tmp/allTerms3.txt out
   public static void main(String[] args) throws IOException {
     int prune = 0;
     int limit = Integer.MAX_VALUE;
@@ -1411,6 +1418,198 @@
     assertEquals(42, (long) seekResult.output);
   }
 
+  public void testPrimaryKeys() throws Exception {
+    Directory dir = newDirectory();
+
+    for(int cycle=0;cycle<2;cycle++) {
+      if (VERBOSE) {
+        System.out.println("TEST: cycle=" + cycle);
+      }
+      RandomIndexWriter w = new RandomIndexWriter(random, dir,
+                                                  newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
+      Document doc = new Document();
+      Field idField = newField("id", "", Field.Index.NOT_ANALYZED);
+      doc.add(idField);
+      
+      final int NUM_IDS = (int) (1000*RANDOM_MULTIPLIER*(1.0+random.nextDouble()));
+      //final int NUM_IDS = (int) (377 * (1.0+random.nextDouble()));
+      if (VERBOSE) {
+        System.out.println("TEST: NUM_IDS=" + NUM_IDS);
+      }
+      final Set<String> allIDs = new HashSet<String>();
+      for(int id=0;id<NUM_IDS;id++) {
+        String idString;
+        if (cycle == 0) {
+          // PKs are assigned sequentially
+          idString = String.format("%07d", id);
+        } else {
+          while(true) {
+            final String s = Long.toString(random.nextLong());
+            if (!allIDs.contains(s)) {
+              idString = s;
+              break;
+            }
+          }
+        }
+        allIDs.add(idString);
+        idField.setValue(idString);
+        w.addDocument(doc);
+      }
+
+      //w.optimize();
+
+      // turn writer into reader:
+      final IndexReader r = w.getReader();
+      final IndexSearcher s = new IndexSearcher(r);
+      w.close();
+
+      final List<String> allIDsList = new ArrayList<String>(allIDs);
+      final List<String> sortedAllIDsList = new ArrayList<String>(allIDsList);
+      Collections.sort(sortedAllIDsList);
+
+      // Sprinkle in some non-existent PKs:
+      Set<String> outOfBounds = new HashSet<String>();
+      for(int idx=0;idx<NUM_IDS/10;idx++) {
+        String idString;
+        if (cycle == 0) {
+          idString = String.format("%07d", (NUM_IDS + idx));
+        } else {
+          while(true) {
+            idString = Long.toString(random.nextLong());
+            if (!allIDs.contains(idString)) {
+              break;
+            }
+          }
+        }
+        outOfBounds.add(idString);
+        allIDsList.add(idString);
+      }
+
+      // Verify w/ TermQuery
+      for(int iter=0;iter<2*NUM_IDS;iter++) {
+        final String id = allIDsList.get(random.nextInt(allIDsList.size()));
+        final boolean exists = !outOfBounds.contains(id);
+        if (VERBOSE) {
+          System.out.println("TEST: TermQuery " + (exists ? "" : "non-exist ") + " id=" + id);
+        }
+        assertEquals((exists ? "" : "non-exist ") + "id=" + id, exists ? 1 : 0, s.search(new TermQuery(new Term("id", id)), 1).totalHits);
+      }
+
+      // Verify w/ MultiTermsEnum
+      final TermsEnum termsEnum = MultiFields.getTerms(r, "id").iterator();
+      for(int iter=0;iter<2*NUM_IDS;iter++) {
+        final String id;
+        final String nextID;
+        final boolean exists;
+
+        if (random.nextBoolean()) {
+          id = allIDsList.get(random.nextInt(allIDsList.size()));
+          exists = !outOfBounds.contains(id);
+          nextID = null;
+          if (VERBOSE) {
+            System.out.println("TEST: exactOnly " + (exists ? "" : "non-exist ") + "id=" + id);
+          }
+        } else {
+          // Pick ID between two IDs:
+          exists = false;
+          final int idv = random.nextInt(NUM_IDS-1);
+          if (cycle == 0) {
+            id = String.format("%07da", idv);
+            nextID = String.format("%07d", idv+1);
+          } else {
+            id = sortedAllIDsList.get(idv) + "a";
+            nextID = sortedAllIDsList.get(idv+1);
+          }
+          if (VERBOSE) {
+            System.out.println("TEST: not exactOnly id=" + id + " nextID=" + nextID);
+          }
+        }
+
+        final boolean useCache = random.nextBoolean();
+        if (VERBOSE) {
+          System.out.println("  useCache=" + useCache);
+        }
+
+        final TermsEnum.SeekStatus status;
+        if (nextID == null) {
+          if (termsEnum.seekExact(new BytesRef(id), useCache)) {
+            status = TermsEnum.SeekStatus.FOUND;
+          } else {
+            status = TermsEnum.SeekStatus.NOT_FOUND;
+          }
+        } else {
+          status = termsEnum.seekCeil(new BytesRef(id), useCache);
+        }
+
+        if (nextID != null) {
+          assertEquals(TermsEnum.SeekStatus.NOT_FOUND, status);
+          assertEquals("expected=" + nextID + " actual=" + termsEnum.term().utf8ToString(), new BytesRef(nextID), termsEnum.term());
+        } else if (!exists) {
+          assertTrue(status == TermsEnum.SeekStatus.NOT_FOUND ||
+                     status == TermsEnum.SeekStatus.END);
+        } else {
+          assertEquals(TermsEnum.SeekStatus.FOUND, status);
+        }
+      }
+
+      r.close();
+    }
+    dir.close();
+  }
+
+  public void testRandomTermLookup() throws Exception {
+    Directory dir = newDirectory();
+
+    RandomIndexWriter w = new RandomIndexWriter(random, dir,
+                                                newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
+    w.w.setInfoStream(VERBOSE ? System.out : null);
+
+    Document doc = new Document();
+    Field f = newField("field", "", Field.Index.NOT_ANALYZED);
+    doc.add(f);
+      
+    final int NUM_TERMS = (int) (1000*RANDOM_MULTIPLIER * (1+random.nextDouble()));
+    if (VERBOSE) {
+      System.out.println("TEST: NUM_TERMS=" + NUM_TERMS);
+    }
+
+    final Set<String> allTerms = new HashSet<String>();
+    while(allTerms.size() < NUM_TERMS) {
+      allTerms.add(simpleRandomString(random));
+    }
+
+    for(String term : allTerms) {
+      f.setValue(term);
+      w.addDocument(doc);
+    }
+
+    // turn writer into reader:
+    if (VERBOSE) {
+      System.out.println("TEST: get reader");
+    }
+    IndexReader r = w.getReader();
+    if (VERBOSE) {
+      System.out.println("TEST: got reader=" + r);
+    }
+    IndexSearcher s = new IndexSearcher(r);
+    w.close();
+
+    final List<String> allTermsList = new ArrayList<String>(allTerms);
+    Collections.shuffle(allTermsList, random);
+
+    // verify exact lookup
+    for(String term : allTermsList) {
+      if (VERBOSE) {
+        System.out.println("TEST: term=" + term);
+      }
+      assertEquals("term=" + term, 1, s.search(new TermQuery(new Term("field", term)), 1).totalHits);
+    }
+
+    r.close();
+    dir.close();
+  }
+
+
   /**
    * Test state expansion (array format) on close-to-root states. Creates
    * synthetic input that has one expanded state on each level.
@@ -1492,6 +1691,36 @@
     s.verifyStateAndBelow(fst, arc, 1);
   }
 
+  public void testFinalOutputOnEndState() throws Exception {
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null);
+    builder.add("stat", outputs.get(17));
+    builder.add("station", outputs.get(10));
+    final FST<Long> fst = builder.finish();
+    //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    StringWriter w = new StringWriter();
+    Util.toDot(fst, w, false, false);
+    w.close();
+    //System.out.println(w.toString());
+    assertTrue(w.toString().indexOf("label=\"t/[7]\"") != -1);
+  }
+
+  public void testInternalFinalState() throws Exception {
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
+    builder.add(new BytesRef("stat"), outputs.getNoOutput());
+    builder.add(new BytesRef("station"), outputs.getNoOutput());
+    final FST<Long> fst = builder.finish();
+    StringWriter w = new StringWriter();
+    //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    Util.toDot(fst, w, false, false);
+    w.close();
+    //System.out.println(w.toString());
+    assertTrue(w.toString().indexOf("6 [shape=doublecircle") != -1);
+  }
+
   // Make sure raw FST can differentiate between final vs
   // non-final end nodes
   public void testNonFinalStopNodes() throws Exception {
Index: lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/FuzzyQuery.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/FuzzyQuery.java	(working copy)
@@ -137,12 +137,10 @@
 
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    TermsEnum tenum = terms.iterator();
-    
     if (!termLongEnough) {  // can only match if it's exact
-      return new SingleTermsEnum(tenum, term);
+      return new SingleTermsEnum(terms.iterator(), term.bytes());
     }
-    return new FuzzyTermsEnum(tenum, atts, getTerm(), minimumSimilarity, prefixLength);
+    return new FuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength);
   }
   
   /**
Index: lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java	(working copy)
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
@@ -34,9 +33,9 @@
 
   private final BytesRef prefixRef;
 
-  public PrefixTermsEnum(TermsEnum tenum, Term prefix) throws IOException {
+  public PrefixTermsEnum(TermsEnum tenum, BytesRef prefixText) throws IOException {
     super(tenum);
-    setInitialSeekTerm(prefixRef = prefix.bytes());
+    setInitialSeekTerm(this.prefixRef = prefixText);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java	(working copy)
@@ -28,6 +28,8 @@
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 
+// TODO: move this class to oal.index
+
 /**
  * Abstract class for enumerating a subset of all terms. 
  * 
@@ -42,7 +44,7 @@
 public abstract class FilteredTermsEnum extends TermsEnum {
 
   private BytesRef initialSeekTerm = null;
-  private boolean doSeek = true;        
+  private boolean doSeek;
   private BytesRef actualTerm = null;
 
   private final TermsEnum tenum;
@@ -64,8 +66,17 @@
    * @param tenum the terms enumeration to filter.
    */
   public FilteredTermsEnum(final TermsEnum tenum) {
+    this(tenum, true);
+  }
+
+  /**
+   * Creates a filtered {@link TermsEnum} on a terms enum.
+   * @param tenum the terms enumeration to filter.
+   */
+  public FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) {
     assert tenum != null;
     this.tenum = tenum;
+    doSeek = startWithSeek;
   }
 
   /**
@@ -190,18 +201,23 @@
   @SuppressWarnings("fallthrough")
   @Override
   public BytesRef next() throws IOException {
+    //System.out.println("FTE.next doSeek=" + doSeek);
+    //new Throwable().printStackTrace(System.out);
     for (;;) {
       // Seek or forward the iterator
       if (doSeek) {
         doSeek = false;
         final BytesRef t = nextSeekTerm(actualTerm);
+        //System.out.println("  seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum);
         // Make sure we always seek forward:
         assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t;
         if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) {
           // no more terms to seek to or enum exhausted
+          //System.out.println("  return null");
           return null;
         }
         actualTerm = tenum.term();
+        //System.out.println("  got term=" + actualTerm.utf8ToString());
       } else {
         actualTerm = tenum.next();
         if (actualTerm == null) {
Index: lucene/src/java/org/apache/lucene/search/PrefixQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PrefixQuery.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/PrefixQuery.java	(working copy)
@@ -51,7 +51,7 @@
       // no prefix -- match all terms for this field:
       return tenum;
     }
-    return new PrefixTermsEnum(tenum, prefix);
+    return new PrefixTermsEnum(tenum, prefix.bytes());
   }
 
   /** Prints a user-readable version of this query. */
Index: lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java	(working copy)
@@ -78,15 +78,38 @@
       public void setNextEnum(TermsEnum termsEnum) throws IOException {
         this.termsEnum = termsEnum;
         this.termComp = termsEnum.getComparator();
+        
+        assert compareToLastTerm(null);
+
         // lazy init the initial ScoreTerm because comparator is not known on ctor:
         if (st == null)
           st = new ScoreTerm(this.termComp, new TermContext(topReaderContext));
         boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
       }
     
+      // for assert:
+      private BytesRef lastTerm;
+      private boolean compareToLastTerm(BytesRef t) throws IOException {
+        if (lastTerm == null && t != null) {
+          lastTerm = new BytesRef(t);
+        } else if (t == null) {
+          lastTerm = null;
+        } else {
+          assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t;
+          lastTerm.copy(t);
+        }
+        return true;
+      }
+  
       @Override
       public boolean collect(BytesRef bytes) throws IOException {
         final float boost = boostAtt.getBoost();
+
+        // make sure within a single seg we always collect
+        // terms in order
+        assert compareToLastTerm(bytes);
+
+        //System.out.println("TTR.collect term=" + bytes.utf8ToString() + " boost=" + boost + " ord=" + readerContext.ord);
         // ignore uncompetitive hits
         if (stQueue.size() == maxSize) {
           final ScoreTerm t = stQueue.peek();
@@ -134,9 +157,10 @@
     final Q q = getTopLevelQuery();
     final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
     ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
+    
     for (final ScoreTerm st : scoreTerms) {
       final Term term = new Term(query.field, st.bytes);
-      assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();
+      assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq() + " term=" + term;
       addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
     }
     query.incTotalNumberOfTerms(scoreTerms.length);
Index: lucene/src/java/org/apache/lucene/search/TermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TermQuery.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/TermQuery.java	(working copy)
@@ -98,17 +98,18 @@
     TermsEnum getTermsEnum(AtomicReaderContext context) throws IOException {
       final TermState state = termStates.get(context.ord);
       if (state == null) { // term is not present in that reader
-        assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader";
+        assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader term=" + term;
         return null;
       }
-      final TermsEnum termsEnum = context.reader.terms(term.field())
-          .getThreadTermsEnum();
+      //System.out.println("LD=" + reader.getLiveDocs() + " set?=" + (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null"));
+      final TermsEnum termsEnum = context.reader.terms(term.field()).getThreadTermsEnum();
       termsEnum.seekExact(term.bytes(), state);
       return termsEnum;
     }
     
     private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException {
       // only called from assert
+      //System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString());
       final Terms terms = reader.terms(field);
       return terms == null || terms.docFreq(bytes) == 0;
     }
Index: lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java	(working copy)
@@ -1,343 +0,0 @@
-package org.apache.lucene.search;
-
-/**
- * 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.Comparator;
-
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.ByteRunAutomaton;
-import org.apache.lucene.util.automaton.SpecialOperations;
-import org.apache.lucene.util.automaton.Transition;
-import org.apache.lucene.util.automaton.UTF32ToUTF8;
-
-/**
- * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
- * DFA.
- * <p>
- * The algorithm is such:
- * <ol>
- *   <li>As long as matches are successful, keep reading sequentially.
- *   <li>When a match fails, skip to the next string in lexicographic order that
- * does not enter a reject state.
- * </ol>
- * <p>
- * The algorithm does not attempt to actually skip to the next string that is
- * completely accepted. This is not possible when the language accepted by the
- * FSM is not finite (i.e. * operator).
- * </p>
- * @lucene.experimental
- */
-public class AutomatonTermsEnum extends FilteredTermsEnum {
-  // a tableized array-based form of the DFA
-  private final ByteRunAutomaton runAutomaton;
-  // common suffix of the automaton
-  private final BytesRef commonSuffixRef;
-  // true if the automaton accepts a finite language
-  private final boolean finite;
-  // array of sorted transitions for each state, indexed by state number
-  private final Transition[][] allTransitions;
-  // for path tracking: each long records gen when we last
-  // visited the state; we use gens to avoid having to clear
-  private final long[] visited;
-  private long curGen;
-  // the reference used for seeking forwards through the term dictionary
-  private final BytesRef seekBytesRef = new BytesRef(10); 
-  // true if we are enumerating an infinite portion of the DFA.
-  // in this case it is faster to drive the query based on the terms dictionary.
-  // when this is true, linearUpperBound indicate the end of range
-  // of terms where we should simply do sequential reads instead.
-  private boolean linear = false;
-  private final BytesRef linearUpperBound = new BytesRef(10);
-  private final Comparator<BytesRef> termComp;
-
-  /**
-   * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied TermsEnum
-   * <p>
-   * @lucene.experimental 
-   * <p>
-   * @param compiled CompiledAutomaton
-   */
-  public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
-    super(tenum);
-    this.finite = compiled.finite;
-    this.runAutomaton = compiled.runAutomaton;
-    this.commonSuffixRef = compiled.commonSuffixRef;
-    this.allTransitions = compiled.sortedTransitions;
-
-    // used for path tracking, where each bit is a numbered state.
-    visited = new long[runAutomaton.getSize()];
-
-    termComp = getComparator();
-  }
-  
-  /**
-   * Returns true if the term matches the automaton. Also stashes away the term
-   * to assist with smart enumeration.
-   */
-  @Override
-  protected AcceptStatus accept(final BytesRef term) {
-    if (commonSuffixRef == null || term.endsWith(commonSuffixRef)) {
-      if (runAutomaton.run(term.bytes, term.offset, term.length))
-        return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
-      else
-        return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
-            AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
-    } else {
-      return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
-          AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
-    }
-  }
-  
-  @Override
-  protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
-    if (term == null) {
-      assert seekBytesRef.length == 0;
-      // return the empty term, as its valid
-      if (runAutomaton.isAccept(runAutomaton.getInitialState())) {   
-        return seekBytesRef;
-      }
-    } else {
-      seekBytesRef.copy(term);
-    }
-
-    // seek to the next possible string;
-    if (nextString()) {
-      return seekBytesRef;  // reposition
-    } else {
-      return null;          // no more possible strings can match
-    }
-  }
-
-  /**
-   * Sets the enum to operate in linear fashion, as we have found
-   * a looping transition at position: we set an upper bound and 
-   * act like a TermRangeQuery for this portion of the term space.
-   */
-  private void setLinear(int position) {
-    assert linear == false;
-    
-    int state = runAutomaton.getInitialState();
-    int maxInterval = 0xff;
-    for (int i = 0; i < position; i++) {
-      state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
-      assert state >= 0: "state=" + state;
-    }
-    for (int i = 0; i < allTransitions[state].length; i++) {
-      Transition t = allTransitions[state][i];
-      if (t.getMin() <= (seekBytesRef.bytes[position] & 0xff) && 
-          (seekBytesRef.bytes[position] & 0xff) <= t.getMax()) {
-        maxInterval = t.getMax();
-        break;
-      }
-    }
-    // 0xff terms don't get the optimization... not worth the trouble.
-    if (maxInterval != 0xff)
-      maxInterval++;
-    int length = position + 1; /* position + maxTransition */
-    if (linearUpperBound.bytes.length < length)
-      linearUpperBound.bytes = new byte[length];
-    System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
-    linearUpperBound.bytes[position] = (byte) maxInterval;
-    linearUpperBound.length = length;
-    
-    linear = true;
-  }
-
-  private final IntsRef savedStates = new IntsRef(10);
-  
-  /**
-   * Increments the byte buffer to the next String in binary order after s that will not put
-   * the machine into a reject state. If such a string does not exist, returns
-   * false.
-   * 
-   * The correctness of this method depends upon the automaton being deterministic,
-   * and having no transitions to dead states.
-   * 
-   * @return true if more possible solutions exist for the DFA
-   */
-  private boolean nextString() {
-    int state;
-    int pos = 0;
-    savedStates.grow(seekBytesRef.length+1);
-    final int[] states = savedStates.ints;
-    states[0] = runAutomaton.getInitialState();
-    
-    while (true) {
-      curGen++;
-      linear = false;
-      // walk the automaton until a character is rejected.
-      for (state = states[pos]; pos < seekBytesRef.length; pos++) {
-        visited[state] = curGen;
-        int nextState = runAutomaton.step(state, seekBytesRef.bytes[pos] & 0xff);
-        if (nextState == -1)
-          break;
-        states[pos+1] = nextState;
-        // we found a loop, record it for faster enumeration
-        if (!finite && !linear && visited[nextState] == curGen) {
-          setLinear(pos);
-        }
-        state = nextState;
-      }
-
-      // take the useful portion, and the last non-reject state, and attempt to
-      // append characters that will match.
-      if (nextString(state, pos)) {
-        return true;
-      } else { /* no more solutions exist from this useful portion, backtrack */
-        if ((pos = backtrack(pos)) < 0) /* no more solutions at all */
-          return false;
-        final int newState = runAutomaton.step(states[pos], seekBytesRef.bytes[pos] & 0xff);
-        if (newState >= 0 && runAutomaton.isAccept(newState))
-          /* String is good to go as-is */
-          return true;
-        /* else advance further */
-        // TODO: paranoia? if we backtrack thru an infinite DFA, the loop detection is important!
-        // for now, restart from scratch for all infinite DFAs 
-        if (!finite) pos = 0;
-      }
-    }
-  }
-  
-  /**
-   * Returns the next String in lexicographic order that will not put
-   * the machine into a reject state. 
-   * 
-   * This method traverses the DFA from the given position in the String,
-   * starting at the given state.
-   * 
-   * If this cannot satisfy the machine, returns false. This method will
-   * walk the minimal path, in lexicographic order, as long as possible.
-   * 
-   * If this method returns false, then there might still be more solutions,
-   * it is necessary to backtrack to find out.
-   * 
-   * @param state current non-reject state
-   * @param position useful portion of the string
-   * @return true if more possible solutions exist for the DFA from this
-   *         position
-   */
-  private boolean nextString(int state, int position) {
-    /* 
-     * the next lexicographic character must be greater than the existing
-     * character, if it exists.
-     */
-    int c = 0;
-    if (position < seekBytesRef.length) {
-      c = seekBytesRef.bytes[position] & 0xff;
-      // if the next byte is 0xff and is not part of the useful portion,
-      // then by definition it puts us in a reject state, and therefore this
-      // path is dead. there cannot be any higher transitions. backtrack.
-      if (c++ == 0xff)
-        return false;
-    }
-
-    seekBytesRef.length = position;
-    visited[state] = curGen;
-
-    Transition transitions[] = allTransitions[state];
-
-    // find the minimal path (lexicographic order) that is >= c
-    
-    for (int i = 0; i < transitions.length; i++) {
-      Transition transition = transitions[i];
-      if (transition.getMax() >= c) {
-        int nextChar = Math.max(c, transition.getMin());
-        // append either the next sequential char, or the minimum transition
-        seekBytesRef.grow(seekBytesRef.length + 1);
-        seekBytesRef.length++;
-        seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
-        state = transition.getDest().getNumber();
-        /* 
-         * as long as is possible, continue down the minimal path in
-         * lexicographic order. if a loop or accept state is encountered, stop.
-         */
-        while (visited[state] != curGen && !runAutomaton.isAccept(state)) {
-          visited[state] = curGen;
-          /* 
-           * Note: we work with a DFA with no transitions to dead states.
-           * so the below is ok, if it is not an accept state,
-           * then there MUST be at least one transition.
-           */
-          transition = allTransitions[state][0];
-          state = transition.getDest().getNumber();
-          
-          // append the minimum transition
-          seekBytesRef.grow(seekBytesRef.length + 1);
-          seekBytesRef.length++;
-          seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
-          
-          // we found a loop, record it for faster enumeration
-          if (!finite && !linear && visited[state] == curGen) {
-            setLinear(seekBytesRef.length-1);
-          }
-        }
-        return true;
-      }
-    }
-    return false;
-  }
-  
-  /**
-   * Attempts to backtrack thru the string after encountering a dead end
-   * at some given position. Returns false if no more possible strings 
-   * can match.
-   * 
-   * @param position current position in the input String
-   * @return position >=0 if more possible solutions exist for the DFA
-   */
-  private int backtrack(int position) {
-    while (position-- > 0) {
-      int nextChar = seekBytesRef.bytes[position] & 0xff;
-      // if a character is 0xff its a dead-end too,
-      // because there is no higher character in binary sort order.
-      if (nextChar++ != 0xff) {
-        seekBytesRef.bytes[position] = (byte) nextChar;
-        seekBytesRef.length = position+1;
-        return position;
-      }
-    }
-    return -1; /* all solutions exhausted */
-  }
-  
-  /**
-   * immutable class with everything this enum needs.
-   */
-  public static class CompiledAutomaton {
-    public final ByteRunAutomaton runAutomaton;
-    public final Transition[][] sortedTransitions;
-    public final BytesRef commonSuffixRef;
-    public final boolean finite;
-    
-    public CompiledAutomaton(Automaton automaton, boolean finite) {
-      Automaton utf8 = new UTF32ToUTF8().convert(automaton);
-      runAutomaton = new ByteRunAutomaton(utf8, true);
-      sortedTransitions = utf8.getSortedTransitions();
-      this.finite = finite;
-      if (finite) {
-        commonSuffixRef = null;
-      } else {
-        commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
-      }
-    }
-  }
-}
Index: lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java	(working copy)
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
@@ -39,10 +38,10 @@
    * After calling the constructor the enumeration is already pointing to the term,
    * if it exists.
    */
-  public SingleTermsEnum(TermsEnum tenum, Term singleTerm) throws IOException {
+  public SingleTermsEnum(TermsEnum tenum, BytesRef termText) throws IOException {
     super(tenum);
-    singleRef = singleTerm.bytes();
-    setInitialSeekTerm(singleRef);
+    singleRef = termText;
+    setInitialSeekTerm(termText);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/search/IndexSearcher.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/IndexSearcher.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/IndexSearcher.java	(working copy)
@@ -882,6 +882,6 @@
 
   @Override
   public String toString() {
-    return "IndexSearcher(" + reader + ")";
+    return "IndexSearcher(" + reader + "; executor=" + executor + ")";
   }
 }
Index: lucene/src/java/org/apache/lucene/search/TermScorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TermScorer.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/TermScorer.java	(working copy)
@@ -70,6 +70,7 @@
   public boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     while (doc < end) {                           // for docs in window
+      //System.out.println("TS: collect doc=" + doc);
       c.collect(doc);                      // collect score
       if (++pointer >= pointerMax) {
         refillBuffer();
Index: lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java	(working copy)
@@ -17,12 +17,17 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -34,13 +39,9 @@
 import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-
 /** Subclass of TermsEnum for enumerating all terms that are similar
  * to the specified filter term.
  *
@@ -72,7 +73,7 @@
   private int maxEdits;
   private final boolean raw;
 
-  private final TermsEnum tenum;
+  private final Terms terms;
   private final Term term;
   private final int termText[];
   private final int realPrefixLength;
@@ -94,7 +95,7 @@
    * @param prefixLength Length of required common prefix. Default value is 0.
    * @throws IOException
    */
-  public FuzzyTermsEnum(TermsEnum tenum, AttributeSource atts, Term term, 
+  public FuzzyTermsEnum(Terms terms, AttributeSource atts, Term term, 
       final float minSimilarity, final int prefixLength) throws IOException {
     if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
       throw new IllegalArgumentException("fractional edit distances are not allowed");
@@ -102,7 +103,7 @@
       throw new IllegalArgumentException("minimumSimilarity cannot be less than 0");
     if(prefixLength < 0)
       throw new IllegalArgumentException("prefixLength cannot be less than 0");
-    this.tenum = tenum;
+    this.terms = terms;
     this.term = term;
 
     // convert the string into a utf32 int[] representation for fast comparisons
@@ -143,8 +144,10 @@
       throws IOException {
     final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
-      return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
-          .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
+      //if (BlockTreeTermsWriter.DEBUG) System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
+      final CompiledAutomaton compiled = runAutomata.get(editDistance);
+      return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRef())),
+                                         runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
     } else {
       return null;
     }
@@ -153,6 +156,7 @@
   /** initialize levenshtein DFAs up to maxDistance, if possible */
   private List<CompiledAutomaton> initAutomata(int maxDistance) {
     final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
+    //System.out.println("cached automata size: " + runAutomata.size());
     if (runAutomata.size() <= maxDistance && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
@@ -160,13 +164,14 @@
 
       for (int i = runAutomata.size(); i <= maxDistance; i++) {
         Automaton a = builder.toAutomaton(i);
+        //System.out.println("compute automaton n=" + i);
         // constant prefix
         if (realPrefixLength > 0) {
           Automaton prefix = BasicAutomata.makeString(
             UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
-        runAutomata.add(new CompiledAutomaton(a, true));
+        runAutomata.add(new CompiledAutomaton(a, true, false));
       }
     }
     return runAutomata;
@@ -301,65 +306,65 @@
   public BytesRef term() throws IOException {
     return actualEnum.term();
   }
-  
+
   /**
-   * Implement fuzzy enumeration with automaton.
+   * Implement fuzzy enumeration with Terms.intersect.
    * <p>
    * This is the fastest method as opposed to LinearFuzzyTermsEnum:
    * as enumeration is logarithmic to the number of terms (instead of linear)
    * and comparison is linear to length of the term (rather than quadratic)
    */
-  private class AutomatonFuzzyTermsEnum extends AutomatonTermsEnum {
+  private class AutomatonFuzzyTermsEnum extends FilteredTermsEnum {
     private final ByteRunAutomaton matchers[];
     
     private final BytesRef termRef;
     
-    private final BytesRef lastTerm;
     private final BoostAttribute boostAtt =
       attributes().addAttribute(BoostAttribute.class);
     
-    public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], 
-        BytesRef lastTerm) throws IOException {
-      super(tenum, compiled[compiled.length - 1]);
+    public AutomatonFuzzyTermsEnum(TermsEnum tenum, CompiledAutomaton compiled[]) 
+      throws IOException {
+      super(tenum, false);
       this.matchers = new ByteRunAutomaton[compiled.length];
       for (int i = 0; i < compiled.length; i++)
         this.matchers[i] = compiled[i].runAutomaton;
-      this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
     }
-    
+
     /** finds the smallest Lev(n) DFA that accepts the term. */
     @Override
     protected AcceptStatus accept(BytesRef term) {    
+      //System.out.println("AFTE.accept term=" + term);
       int ed = matchers.length - 1;
       
-      if (matches(term, ed)) { // we match the outer dfa
-        // now compute exact edit distance
-        while (ed > 0) {
-          if (matches(term, ed - 1)) {
-            ed--;
-          } else {
-            break;
-          }
-        }
-        
-        // scale to a boost and return (if similarity > minSimilarity)
-        if (ed == 0) { // exact match
-          boostAtt.setBoost(1.0F);
-          return AcceptStatus.YES_AND_SEEK;
+      // we are wrapping either an intersect() TermsEnum or an AutomatonTermsENum,
+      // so we know the outer DFA always matches.
+      // now compute exact edit distance
+      while (ed > 0) {
+        if (matches(term, ed - 1)) {
+          ed--;
         } else {
-          final int codePointCount = UnicodeUtil.codePointCount(term);
-          final float similarity = 1.0f - ((float) ed / (float) 
-              (Math.min(codePointCount, termLength)));
-          if (similarity > minSimilarity) {
-            boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
-            return AcceptStatus.YES_AND_SEEK;
-          } else {
-            return AcceptStatus.NO_AND_SEEK;
-          }
+          break;
         }
+      }
+      //System.out.println("CHECK term=" + term.utf8ToString() + " ed=" + ed);
+      
+      // scale to a boost and return (if similarity > minSimilarity)
+      if (ed == 0) { // exact match
+        boostAtt.setBoost(1.0F);
+        //System.out.println("  yes");
+        return AcceptStatus.YES;
       } else {
-        return AcceptStatus.NO_AND_SEEK;
+        final int codePointCount = UnicodeUtil.codePointCount(term);
+        final float similarity = 1.0f - ((float) ed / (float) 
+            (Math.min(codePointCount, termLength)));
+        if (similarity > minSimilarity) {
+          boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
+          //System.out.println("  yes");
+          return AcceptStatus.YES;
+        } else {
+          return AcceptStatus.NO;
+        }
       }
     }
     
@@ -367,16 +372,8 @@
     final boolean matches(BytesRef term, int k) {
       return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
     }
-    
-    /** defers to superclass, except can start at an arbitrary location */
-    @Override
-    protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
-      if (term == null)
-        term = lastTerm;
-      return super.nextSeekTerm(term);
-    }
   }
-  
+
   /**
    * Implement fuzzy enumeration with linear brute force.
    */
@@ -408,7 +405,7 @@
      * @throws IOException
      */
     public LinearFuzzyTermsEnum() throws IOException {
-      super(tenum);
+      super(terms.iterator());
 
       this.text = new int[termLength - realPrefixLength];
       System.arraycopy(termText, realPrefixLength, text, 0, text.length);
Index: lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/AutomatonQuery.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/search/AutomatonQuery.java	(working copy)
@@ -22,14 +22,11 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
-import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.MinimizationOperations;
-import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A {@link Query} that will match terms against a finite-state machine.
@@ -41,7 +38,7 @@
  * the standard Lucene wildcard syntax with {@link WildcardQuery}.
  * </p>
  * <p>
- * When the query is executed, it will create an equivalent minimal DFA of the
+ * When the query is executed, it will create an equivalent DFA of the
  * finite-state machine, and will enumerate the term dictionary in an
  * intelligent way to reduce the number of comparisons. For example: the regular
  * expression of <code>[dl]og?</code> will make approximately four comparisons:
@@ -52,20 +49,10 @@
 public class AutomatonQuery extends MultiTermQuery {
   /** the automaton to match index terms against */
   protected final Automaton automaton;
+  protected final CompiledAutomaton compiled;
   /** term containing the field, and possibly some pattern structure */
   protected final Term term;
 
-  /** 
-   * abstraction for returning a termsenum:
-   * in the ctor the query computes one of these, the actual
-   * implementation depends upon the automaton's structure.
-   */
-  private abstract class TermsEnumFactory {
-    protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
-  }
-  
-  private final TermsEnumFactory factory;
-
   /**
    * Create a new AutomatonQuery from an {@link Automaton}.
    * 
@@ -78,73 +65,12 @@
     super(term.field());
     this.term = term;
     this.automaton = automaton;
-    MinimizationOperations.minimize(automaton);
-    
-    if (BasicOperations.isEmpty(automaton)) {
-      // matches nothing
-      factory = new TermsEnumFactory() {
-        @Override
-        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-          return TermsEnum.EMPTY;
-        }
-      };
-    } else if (BasicOperations.isTotal(automaton)) {
-      // matches all possible strings
-      factory = new TermsEnumFactory() {
-        @Override
-        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-          return terms.iterator();
-        }
-      };
-    } else {
-      final String singleton;
-      final String commonPrefix;
-      
-      if (automaton.getSingleton() == null) {
-        commonPrefix = SpecialOperations.getCommonPrefix(automaton);
-        if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
-          singleton = commonPrefix;
-        } else {
-          singleton = null;
-        }
-      } else {
-        commonPrefix = null;
-        singleton = automaton.getSingleton();
-      }
-      
-      if (singleton != null) {
-        // matches a fixed string in singleton or expanded representation
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new SingleTermsEnum(terms.iterator(), new Term(field, singleton));
-          }
-        };
-      } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
-          BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
-        // matches a constant prefix
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new PrefixTermsEnum(terms.iterator(), new Term(field, commonPrefix));
-          }
-        };
-      } else {
-        final AutomatonTermsEnum.CompiledAutomaton compiled = 
-          new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton));
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new AutomatonTermsEnum(terms.iterator(), compiled);
-          }
-        };
-      }
-    }
+    this.compiled = new CompiledAutomaton(automaton);
   }
 
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    return factory.getTermsEnum(terms, atts);
+    return compiled.getTermsEnum(terms);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/index/MultiTerms.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/MultiTerms.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/MultiTerms.java	(working copy)
@@ -17,14 +17,16 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.ReaderUtil;
-
 import java.io.IOException;
-import java.util.List;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.List;
 
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
+
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
 /**
  * Exposes flex API, merged from flex API of
  * sub-segments.
@@ -59,6 +61,23 @@
   }
 
   @Override
+  public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+    final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
+    for(int i=0;i<subs.length;i++) {
+      final TermsEnum termsEnum = subs[i].intersect(compiled, startTerm);
+      if (termsEnum != null) {
+        termsEnums.add(new MultiTermsEnum.TermsEnumIndex(termsEnum, i));
+      }
+    }
+
+    if (termsEnums.size() > 0) {
+      return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+    } else {
+      return TermsEnum.EMPTY;
+    }
+  }
+
+  @Override
   public TermsEnum iterator() throws IOException {
 
     final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
Index: lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java	(working copy)
@@ -398,6 +398,7 @@
 
       if (termsEnum.seekExact(term.bytes(), false)) {
         DocsEnum docsEnum = termsEnum.docs(reader.getLiveDocs(), docs);
+        //System.out.println("BDS: got docsEnum=" + docsEnum);
 
         if (docsEnum != null) {
           while (true) {
Index: lucene/src/java/org/apache/lucene/index/CheckIndex.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CheckIndex.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/CheckIndex.java	(working copy)
@@ -17,13 +17,6 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -31,16 +24,25 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.document.AbstractField;  // for javadocs
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.IndexDocValues;
 import org.apache.lucene.index.values.ValuesEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
@@ -237,6 +239,8 @@
 
       /** Exception thrown during term index test (null on success) */
       public Throwable error = null;
+
+      public Map<String,BlockTreeTermsReader.Stats> blockTreeStats = null;
     }
 
     /**
@@ -285,12 +289,21 @@
     infoStream = null;
   }
 
+  private boolean verbose;
+
   /** Set infoStream where messages should go.  If null, no
-   *  messages are printed */
-  public void setInfoStream(PrintStream out) {
+   *  messages are printed.  If verbose is true then more
+   *  details are printed. */
+  public void setInfoStream(PrintStream out, boolean verbose) {
     infoStream = out;
+    this.verbose = verbose;
   }
 
+  /** Set infoStream where messages should go. See {@link setInfoStream(PrintStream,boolean)}. */
+  public void setInfoStream(PrintStream out) {
+    setInfoStream(out, false);
+  }
+
   private void msg(String msg) {
     if (infoStream != null)
       infoStream.println(msg);
@@ -871,6 +884,16 @@
             }
           }
         }
+        
+        final Terms fieldTerms = fields.terms(field);
+        if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
+          final BlockTreeTermsReader.Stats stats = ((BlockTreeTermsReader.FieldReader) fieldTerms).computeStats();
+          assert stats != null;
+          if (status.blockTreeStats == null) {
+            status.blockTreeStats = new HashMap<String,BlockTreeTermsReader.Stats>();
+          }
+          status.blockTreeStats.put(field, stats);
+        }
 
         if (sumTotalTermFreq != 0) {
           final long v = fields.terms(field).getSumTotalTermFreq();
@@ -888,7 +911,7 @@
 
         // Test seek to last term:
         if (lastTerm != null) {
-          if (terms.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) {
+          if (terms.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) { 
             throw new RuntimeException("seek to last term " + lastTerm + " failed");
           }
 
@@ -951,6 +974,13 @@
 
       msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
 
+      if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
+        for(Map.Entry<String,BlockTreeTermsReader.Stats> ent : status.blockTreeStats.entrySet()) {
+          infoStream.println("      field \"" + ent.getKey() + "\":");
+          infoStream.println("      " + ent.getValue().toString().replace("\n", "\n      "));
+        }
+      }
+
     } catch (Throwable e) {
       msg("ERROR: " + e);
       status.error = e;
@@ -1131,7 +1161,7 @@
     <p>
     Run it like this:
     <pre>
-    java -ea:org.apache.lucene... org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]
+    java -ea:org.apache.lucene... org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-verbose] [-segment X] [-segment Y]
     </pre>
     <ul>
     <li><code>-fix</code>: actually write a new segments_N file, removing any problematic segments
@@ -1161,6 +1191,7 @@
   public static void main(String[] args) throws IOException, InterruptedException {
 
     boolean doFix = false;
+    boolean verbose = false;
     List<String> onlySegments = new ArrayList<String>();
     String indexPath = null;
     int i = 0;
@@ -1168,6 +1199,9 @@
       if (args[i].equals("-fix")) {
         doFix = true;
         i++;
+      } else if (args[i].equals("-verbose")) {
+        verbose = true;
+        i++;
       } else if (args[i].equals("-segment")) {
         if (i == args.length-1) {
           System.out.println("ERROR: missing name for -segment option");
@@ -1190,6 +1224,7 @@
       System.out.println("\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]\n" +
                          "\n" +
                          "  -fix: actually write a new segments_N file, removing any problematic segments\n" +
+                         "  -verbose: print additional details\n" +
                          "  -segment X: only check the specified segments.  This can be specified multiple\n" + 
                          "              times, to check more than one segment, eg '-segment _2 -segment _a'.\n" +
                          "              You can't use this with the -fix option\n" +
@@ -1231,7 +1266,7 @@
     }
 
     CheckIndex checker = new CheckIndex(dir);
-    checker.setInfoStream(System.out);
+    checker.setInfoStream(System.out, verbose);
 
     Status result = checker.checkIndex(onlySegments);
     if (result.missingSegments) {
Index: lucene/src/java/org/apache/lucene/index/Terms.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/Terms.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/Terms.java	(working copy)
@@ -19,9 +19,11 @@
 
 import java.io.IOException;
 import java.util.Comparator;
+
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * Access to the terms in a specific field.  See {@link Fields}.
@@ -37,7 +39,40 @@
   /** Returns an iterator that will step through all
    *  terms. This method will not return null.*/
   public abstract TermsEnum iterator() throws IOException;
-  
+
+  /** Returns a TermsEnum that iterates over all terms that
+   *  are accepted by the provided {@link
+   *  CompiledAutomaton}.  If the <code>startTerm</code> is
+   *  provided then the returned enum will only accept terms
+   *  > <code>startTerm</code>, but you still must call
+   *  next() first to get to the first term.  Note that the
+   *  provided <code>startTerm</code> must be accepted by
+   *  the automaton.
+   *
+   * <p><b>NOTE</b>: the returned TermsEnum cannot
+   * seek</p>. */
+  public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
+    // TODO: eventually we could support seekCeil/Exact on
+    // the returned enum, instead of only being able to seek
+    // at the start
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+    }
+    if (startTerm == null) {
+      return new AutomatonTermsEnum(iterator(), compiled);
+    } else {
+      return new AutomatonTermsEnum(iterator(), compiled) {
+        @Override
+        protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
+          if (term == null) {
+            term = startTerm;
+          }
+          return super.nextSeekTerm(term);
+        }
+      };
+    }
+  }
+
   /** Return the BytesRef Comparator used to sort terms
    *  provided by the iterator.  This method may return null
    *  if there are no terms.  This method may be invoked
Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(working copy)
@@ -562,12 +562,14 @@
     }
     codec = segmentWriteState.segmentCodecs.codec();
     final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
+    boolean success = false;
     try {
       consumer.merge(mergeState,
                      new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
                                      slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
+      success = true;
     } finally {
-      consumer.close();
+      IOUtils.closeSafely(!success, consumer);
     }
   }
 
Index: lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(working copy)
@@ -26,6 +26,7 @@
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.IOUtils;
 
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
@@ -58,6 +59,8 @@
 
     final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
 
+    boolean success = false;
+
     try {
       TermsHash termsHash = null;
       
@@ -100,8 +103,9 @@
       if (termsHash != null) {
         termsHash.reset();
       }
+      success = true;
     } finally {
-      consumer.close();
+      IOUtils.closeSafely(!success, consumer);
     }
   }
 
Index: lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java	(revision 0)
@@ -0,0 +1,321 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.Comparator;
+
+import org.apache.lucene.search.FilteredTermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Transition;
+
+/**
+ * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
+ * DFA.
+ * <p>
+ * The algorithm is such:
+ * <ol>
+ *   <li>As long as matches are successful, keep reading sequentially.
+ *   <li>When a match fails, skip to the next string in lexicographic order that
+ * does not enter a reject state.
+ * </ol>
+ * <p>
+ * The algorithm does not attempt to actually skip to the next string that is
+ * completely accepted. This is not possible when the language accepted by the
+ * FSM is not finite (i.e. * operator).
+ * </p>
+ * @lucene.experimental
+ */
+class AutomatonTermsEnum extends FilteredTermsEnum {
+  // a tableized array-based form of the DFA
+  private final ByteRunAutomaton runAutomaton;
+  // common suffix of the automaton
+  private final BytesRef commonSuffixRef;
+  // true if the automaton accepts a finite language
+  private final boolean finite;
+  // array of sorted transitions for each state, indexed by state number
+  private final Transition[][] allTransitions;
+  // for path tracking: each long records gen when we last
+  // visited the state; we use gens to avoid having to clear
+  private final long[] visited;
+  private long curGen;
+  // the reference used for seeking forwards through the term dictionary
+  private final BytesRef seekBytesRef = new BytesRef(10); 
+  // true if we are enumerating an infinite portion of the DFA.
+  // in this case it is faster to drive the query based on the terms dictionary.
+  // when this is true, linearUpperBound indicate the end of range
+  // of terms where we should simply do sequential reads instead.
+  private boolean linear = false;
+  private final BytesRef linearUpperBound = new BytesRef(10);
+  private final Comparator<BytesRef> termComp;
+
+  /**
+   * Construct an enumerator based upon an automaton, enumerating the specified
+   * field, working on a supplied TermsEnum
+   * <p>
+   * @lucene.experimental 
+   * <p>
+   * @param compiled CompiledAutomaton
+   */
+  public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
+    super(tenum);
+    this.finite = compiled.finite;
+    this.runAutomaton = compiled.runAutomaton;
+    assert this.runAutomaton != null;
+    this.commonSuffixRef = compiled.commonSuffixRef;
+    this.allTransitions = compiled.sortedTransitions;
+
+    // used for path tracking, where each bit is a numbered state.
+    visited = new long[runAutomaton.getSize()];
+
+    termComp = getComparator();
+  }
+  
+  /**
+   * Returns true if the term matches the automaton. Also stashes away the term
+   * to assist with smart enumeration.
+   */
+  @Override
+  protected AcceptStatus accept(final BytesRef term) {
+    if (commonSuffixRef == null || term.endsWith(commonSuffixRef)) {
+      if (runAutomaton.run(term.bytes, term.offset, term.length))
+        return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
+      else
+        return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
+            AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
+    } else {
+      return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
+          AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
+    }
+  }
+  
+  @Override
+  protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
+    //System.out.println("ATE.nextSeekTerm term=" + term);
+    if (term == null) {
+      assert seekBytesRef.length == 0;
+      // return the empty term, as its valid
+      if (runAutomaton.isAccept(runAutomaton.getInitialState())) {   
+        return seekBytesRef;
+      }
+    } else {
+      seekBytesRef.copy(term);
+    }
+
+    // seek to the next possible string;
+    if (nextString()) {
+      return seekBytesRef;  // reposition
+    } else {
+      return null;          // no more possible strings can match
+    }
+  }
+
+  /**
+   * Sets the enum to operate in linear fashion, as we have found
+   * a looping transition at position: we set an upper bound and 
+   * act like a TermRangeQuery for this portion of the term space.
+   */
+  private void setLinear(int position) {
+    assert linear == false;
+    
+    int state = runAutomaton.getInitialState();
+    int maxInterval = 0xff;
+    for (int i = 0; i < position; i++) {
+      state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
+      assert state >= 0: "state=" + state;
+    }
+    for (int i = 0; i < allTransitions[state].length; i++) {
+      Transition t = allTransitions[state][i];
+      if (t.getMin() <= (seekBytesRef.bytes[position] & 0xff) && 
+          (seekBytesRef.bytes[position] & 0xff) <= t.getMax()) {
+        maxInterval = t.getMax();
+        break;
+      }
+    }
+    // 0xff terms don't get the optimization... not worth the trouble.
+    if (maxInterval != 0xff)
+      maxInterval++;
+    int length = position + 1; /* position + maxTransition */
+    if (linearUpperBound.bytes.length < length)
+      linearUpperBound.bytes = new byte[length];
+    System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
+    linearUpperBound.bytes[position] = (byte) maxInterval;
+    linearUpperBound.length = length;
+    
+    linear = true;
+  }
+
+  private final IntsRef savedStates = new IntsRef(10);
+  
+  /**
+   * Increments the byte buffer to the next String in binary order after s that will not put
+   * the machine into a reject state. If such a string does not exist, returns
+   * false.
+   * 
+   * The correctness of this method depends upon the automaton being deterministic,
+   * and having no transitions to dead states.
+   * 
+   * @return true if more possible solutions exist for the DFA
+   */
+  private boolean nextString() {
+    int state;
+    int pos = 0;
+    savedStates.grow(seekBytesRef.length+1);
+    final int[] states = savedStates.ints;
+    states[0] = runAutomaton.getInitialState();
+    
+    while (true) {
+      curGen++;
+      linear = false;
+      // walk the automaton until a character is rejected.
+      for (state = states[pos]; pos < seekBytesRef.length; pos++) {
+        visited[state] = curGen;
+        int nextState = runAutomaton.step(state, seekBytesRef.bytes[pos] & 0xff);
+        if (nextState == -1)
+          break;
+        states[pos+1] = nextState;
+        // we found a loop, record it for faster enumeration
+        if (!finite && !linear && visited[nextState] == curGen) {
+          setLinear(pos);
+        }
+        state = nextState;
+      }
+
+      // take the useful portion, and the last non-reject state, and attempt to
+      // append characters that will match.
+      if (nextString(state, pos)) {
+        return true;
+      } else { /* no more solutions exist from this useful portion, backtrack */
+        if ((pos = backtrack(pos)) < 0) /* no more solutions at all */
+          return false;
+        final int newState = runAutomaton.step(states[pos], seekBytesRef.bytes[pos] & 0xff);
+        if (newState >= 0 && runAutomaton.isAccept(newState))
+          /* String is good to go as-is */
+          return true;
+        /* else advance further */
+        // TODO: paranoia? if we backtrack thru an infinite DFA, the loop detection is important!
+        // for now, restart from scratch for all infinite DFAs 
+        if (!finite) pos = 0;
+      }
+    }
+  }
+  
+  /**
+   * Returns the next String in lexicographic order that will not put
+   * the machine into a reject state. 
+   * 
+   * This method traverses the DFA from the given position in the String,
+   * starting at the given state.
+   * 
+   * If this cannot satisfy the machine, returns false. This method will
+   * walk the minimal path, in lexicographic order, as long as possible.
+   * 
+   * If this method returns false, then there might still be more solutions,
+   * it is necessary to backtrack to find out.
+   * 
+   * @param state current non-reject state
+   * @param position useful portion of the string
+   * @return true if more possible solutions exist for the DFA from this
+   *         position
+   */
+  private boolean nextString(int state, int position) {
+    /* 
+     * the next lexicographic character must be greater than the existing
+     * character, if it exists.
+     */
+    int c = 0;
+    if (position < seekBytesRef.length) {
+      c = seekBytesRef.bytes[position] & 0xff;
+      // if the next byte is 0xff and is not part of the useful portion,
+      // then by definition it puts us in a reject state, and therefore this
+      // path is dead. there cannot be any higher transitions. backtrack.
+      if (c++ == 0xff)
+        return false;
+    }
+
+    seekBytesRef.length = position;
+    visited[state] = curGen;
+
+    Transition transitions[] = allTransitions[state];
+
+    // find the minimal path (lexicographic order) that is >= c
+    
+    for (int i = 0; i < transitions.length; i++) {
+      Transition transition = transitions[i];
+      if (transition.getMax() >= c) {
+        int nextChar = Math.max(c, transition.getMin());
+        // append either the next sequential char, or the minimum transition
+        seekBytesRef.grow(seekBytesRef.length + 1);
+        seekBytesRef.length++;
+        seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
+        state = transition.getDest().getNumber();
+        /* 
+         * as long as is possible, continue down the minimal path in
+         * lexicographic order. if a loop or accept state is encountered, stop.
+         */
+        while (visited[state] != curGen && !runAutomaton.isAccept(state)) {
+          visited[state] = curGen;
+          /* 
+           * Note: we work with a DFA with no transitions to dead states.
+           * so the below is ok, if it is not an accept state,
+           * then there MUST be at least one transition.
+           */
+          transition = allTransitions[state][0];
+          state = transition.getDest().getNumber();
+          
+          // append the minimum transition
+          seekBytesRef.grow(seekBytesRef.length + 1);
+          seekBytesRef.length++;
+          seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
+          
+          // we found a loop, record it for faster enumeration
+          if (!finite && !linear && visited[state] == curGen) {
+            setLinear(seekBytesRef.length-1);
+          }
+        }
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * Attempts to backtrack thru the string after encountering a dead end
+   * at some given position. Returns false if no more possible strings 
+   * can match.
+   * 
+   * @param position current position in the input String
+   * @return position >=0 if more possible solutions exist for the DFA
+   */
+  private int backtrack(int position) {
+    while (position-- > 0) {
+      int nextChar = seekBytesRef.bytes[position] & 0xff;
+      // if a character is 0xff its a dead-end too,
+      // because there is no higher character in binary sort order.
+      if (nextChar++ != 0xff) {
+        seekBytesRef.bytes[position] = (byte) nextChar;
+        seekBytesRef.length = position+1;
+        return position;
+      }
+    }
+    return -1; /* all solutions exhausted */
+  }
+}

Property changes on: lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java	(working copy)
@@ -22,28 +22,23 @@
 
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.BlockTermsReader;
-import org.apache.lucene.index.codecs.BlockTermsWriter;
-import org.apache.lucene.index.codecs.TermsIndexReaderBase;
-import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
+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.util.IOUtils;
 
 /** This codec "inlines" the postings for terms that have
  *  low docFreq.  It wraps another codec, which is used for
@@ -56,64 +51,52 @@
 public class PulsingCodec extends Codec {
 
   private final int freqCutoff;
+  private final int minBlockSize;
+  private final int maxBlockSize;
 
-  /**
-   * Creates a {@link PulsingCodec} with <tt>freqCutoff = 1</tt>
-   * 
-   * @see PulsingCodec#PulsingCodec(int)
-   */
   public PulsingCodec() {
     this(1);
   }
   
-  /** @lucene.internal */
-  public int getFreqCutoff() {
-    return freqCutoff;
+  public PulsingCodec(int freqCutoff) {
+    this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
   /** Terms with freq <= freqCutoff are inlined into terms
    *  dict. */
-  public PulsingCodec(int freqCutoff) {
+  public PulsingCodec(int freqCutoff, int minBlockSize, int maxBlockSize) {
     super("Pulsing");
     this.freqCutoff = freqCutoff;
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
   }
 
   @Override
   public String toString() {
-    return name + "(freqCutoff=" + freqCutoff + ")";
+    return name + "(freqCutoff=" + freqCutoff + " minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
   }
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    // We wrap StandardPostingsWriter, but any StandardPostingsWriter
+    // We wrap StandardPostingsWriter, but any PostingsWriterBase
     // will work:
+
     PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
 
     // Terms that have <= freqCutoff number of docs are
     // "pulsed" (inlined):
     PostingsWriterBase pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
 
-    // Terms dict index
-    TermsIndexWriterBase indexWriter;
+    // Terms dict
     boolean success = false;
     try {
-      indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
       success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeSafely(true, pulsingWriter);
-      }
-    }
-
-    // Terms dict
-    success = false;
-    try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter);
-      success = true;
       return ret;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, pulsingWriter, indexWriter);
+        pulsingWriter.close();
       }
     }
   }
@@ -126,51 +109,32 @@
     PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
     PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
 
-    // Terms dict index reader
-    TermsIndexReaderBase indexReader;
-
     boolean success = false;
     try {
-      indexReader = new VariableGapTermsIndexReader(state.dir,
-                                                    state.fieldInfos,
-                                                    state.segmentInfo.name,
-                                                    state.termsIndexDivisor,
-                                                    state.codecId, state.context);
+      FieldsProducer ret = new BlockTreeTermsReader(
+                                                    state.dir, state.fieldInfos, state.segmentInfo.name,
+                                                    pulsingReader,
+                                                    state.context,
+                                                    state.codecId,
+                                                    state.termsIndexDivisor);
       success = true;
+      return ret;
     } finally {
       if (!success) {
         pulsingReader.close();
       }
     }
+  }
 
-    // Terms dict reader
-    success = false;
-    try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir, state.fieldInfos, state.segmentInfo.name,
-                                                pulsingReader,
-                                                state.context,
-                                                StandardCodec.TERMS_CACHE_SIZE,
-                                                state.codecId);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        try {
-          pulsingReader.close();
-        } finally {
-          indexReader.close();
-        }
-      }
-    }
+  public int getFreqCutoff() {
+    return freqCutoff;
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, id, files);
-    BlockTermsReader.files(dir, segmentInfo, id, files);
-    VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+  public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
+    StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
+    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
   }
 
   @Override
@@ -178,7 +142,7 @@
     StandardCodec.getStandardExtensions(extensions);
     DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
   }
-  
+
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
     return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
Index: lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java	(working copy)
@@ -18,6 +18,8 @@
  */
 
 import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -50,6 +52,15 @@
   private IndexOptions indexOptions;
   private boolean storePayloads;
 
+  private static class PendingTerm {
+    private final byte[] bytes;
+    public PendingTerm(byte[] bytes) {
+      this.bytes = bytes;
+    }
+  }
+
+  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
+
   // one entry per position
   private final Position[] pending;
   private int pendingCount = 0;                           // -1 once we've hit too many positions
@@ -92,7 +103,7 @@
 
   @Override
   public void startTerm() {
-    //System.out.println("PW   startTerm");
+    if (DEBUG) System.out.println("PW   startTerm");
     assert pendingCount == 0;
   }
 
@@ -104,19 +115,33 @@
   @Override
   public void setField(FieldInfo fieldInfo) {
     this.indexOptions = fieldInfo.indexOptions;
-    //System.out.println("PW field=" + fieldInfo.name + " omitTF=" + omitTF);
+    if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions);
     storePayloads = fieldInfo.storePayloads;
     wrappedPostingsWriter.setField(fieldInfo);
+    //DEBUG = BlockTreeTermsWriter.DEBUG;
   }
 
+  private boolean DEBUG;
+
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
     assert docID >= 0: "got docID=" + docID;
-    //System.out.println("PW     doc=" + docID);
 
+    /*
+    if (termID != -1) {
+      if (docID == 0) {
+        baseDocID = termID;
+      } else if (baseDocID + docID != termID) {
+        throw new RuntimeException("WRITE: baseDocID=" + baseDocID + " docID=" + docID + " termID=" + termID);
+      }
+    }
+    */
+
+    if (DEBUG) System.out.println("PW     doc=" + docID);
+
     if (pendingCount == pending.length) {
       push();
-      //System.out.println("PW: wrapped.finishDoc");
+      if (DEBUG) System.out.println("PW: wrapped.finishDoc");
       wrappedPostingsWriter.finishDoc();
     }
 
@@ -142,7 +167,7 @@
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
 
-    //System.out.println("PW       pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
+    if (DEBUG) System.out.println("PW       pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
     if (pendingCount == pending.length) {
       push();
     }
@@ -170,24 +195,28 @@
 
   @Override
   public void finishDoc() throws IOException {
-    //System.out.println("PW     finishDoc");
+    if (DEBUG) System.out.println("PW     finishDoc");
     if (pendingCount == -1) {
       wrappedPostingsWriter.finishDoc();
     }
   }
 
   private final RAMOutputStream buffer = new RAMOutputStream();
-  private final RAMOutputStream buffer2 = new RAMOutputStream();
 
+  // private int baseDocID;
+
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
-    //System.out.println("PW   finishTerm docCount=" + stats.docFreq);
+    if (DEBUG) System.out.println("PW   finishTerm docCount=" + stats.docFreq + " pendingCount=" + pendingCount + " pendingTerms.size()=" + pendingTerms.size());
 
     assert pendingCount > 0 || pendingCount == -1;
 
     if (pendingCount == -1) {
       wrappedPostingsWriter.finishTerm(stats);
+      // Must add null entry to record terms that our
+      // wrapped postings impl added
+      pendingTerms.add(null);
     } else {
 
       // There were few enough total occurrences for this
@@ -210,7 +239,7 @@
           final int delta = doc.docID - lastDocID;
           lastDocID = doc.docID;
 
-          //System.out.println("  write doc=" + doc.docID + " freq=" + doc.termFreq);
+          if (DEBUG) System.out.println("  write doc=" + doc.docID + " freq=" + doc.termFreq);
 
           if (doc.termFreq == 1) {
             buffer.writeVInt((delta<<1)|1);
@@ -225,7 +254,7 @@
             assert pos.docID == doc.docID;
             final int posDelta = pos.pos - lastPos;
             lastPos = pos.pos;
-            //System.out.println("    write pos=" + pos.pos);
+            if (DEBUG) System.out.println("    write pos=" + pos.pos);
             if (storePayloads) {
               final int payloadLength = pos.payload == null ? 0 : pos.payload.length;
               if (payloadLength != lastPayloadLength) {
@@ -248,6 +277,7 @@
         for(int posIDX=0;posIDX<pendingCount;posIDX++) {
           final Position doc = pending[posIDX];
           final int delta = doc.docID - lastDocID;
+          assert doc.termFreq != 0;
           if (doc.termFreq == 1) {
             buffer.writeVInt((delta<<1)|1);
           } else {
@@ -264,10 +294,10 @@
           lastDocID = doc.docID;
         }
       }
-      
-      //System.out.println("  bytes=" + buffer.getFilePointer());
-      buffer2.writeVInt((int) buffer.getFilePointer());
-      buffer.writeTo(buffer2);
+
+      final byte[] bytes = new byte[(int) buffer.getFilePointer()];
+      buffer.writeTo(bytes, 0);
+      pendingTerms.add(new PendingTerm(bytes));
       buffer.reset();
     }
 
@@ -280,20 +310,51 @@
   }
 
   @Override
-  public void flushTermsBlock() throws IOException {
-    termsOut.writeVInt((int) buffer2.getFilePointer());
-    buffer2.writeTo(termsOut);
-    buffer2.reset();
+  public void flushTermsBlock(int start, int count) throws IOException {
+    if (DEBUG) System.out.println("PW: flushTermsBlock start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size());
+    int wrappedCount = 0;
+    assert buffer.getFilePointer() == 0;
+    assert start >= count;
 
+    final int limit = pendingTerms.size() - start + count;
+
+    for(int idx=pendingTerms.size()-start; idx<limit; idx++) {
+      final PendingTerm term = pendingTerms.get(idx);
+      if (term == null) {
+        wrappedCount++;
+      } else {
+        buffer.writeVInt(term.bytes.length);
+        buffer.writeBytes(term.bytes, 0, term.bytes.length);
+      }
+    }
+
+    termsOut.writeVInt((int) buffer.getFilePointer());
+    buffer.writeTo(termsOut);
+    buffer.reset();
+
+    // TDOO: this could be somewhat costly since
+    // pendingTerms.size() could be biggish?
+    int futureWrappedCount = 0;
+    final int limit2 = pendingTerms.size();
+    for(int idx=limit;idx<limit2;idx++) {
+      if (pendingTerms.get(idx) == null) {
+        futureWrappedCount++;
+      }
+    }
+
+    // Remove the terms we just wrote:
+    pendingTerms.subList(pendingTerms.size()-start, limit).clear();
+
+    if (DEBUG) System.out.println("PW:   len=" + buffer.getFilePointer() + " fp=" + termsOut.getFilePointer() + " futureWrappedCount=" + futureWrappedCount + " wrappedCount=" + wrappedCount);
     // TODO: can we avoid calling this if all terms
     // were inlined...?  Eg for a "primary key" field, the
     // wrapped codec is never invoked...
-    wrappedPostingsWriter.flushTermsBlock();
+    wrappedPostingsWriter.flushTermsBlock(futureWrappedCount+wrappedCount, wrappedCount);
   }
 
   // Pushes pending positions to the wrapped codec
   private void push() throws IOException {
-    //System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
+    if (DEBUG) System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
     assert pendingCount == pending.length;
       
     wrappedPostingsWriter.startTerm();
@@ -304,17 +365,17 @@
       for(Position pos : pending) {
         if (doc == null) {
           doc = pos;
-          //System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+          if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
           wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
         } else if (doc.docID != pos.docID) {
           assert pos.docID > doc.docID;
-          //System.out.println("PW: wrapped.finishDoc");
+          if (DEBUG) System.out.println("PW: wrapped.finishDoc");
           wrappedPostingsWriter.finishDoc();
           doc = pos;
-          //System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+          if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
           wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
         }
-        //System.out.println("PW:   wrapped.addPos pos=" + pos.pos);
+        if (DEBUG) System.out.println("PW:   wrapped.addPos pos=" + pos.pos);
         wrappedPostingsWriter.addPosition(pos.pos, pos.payload);
       }
       //wrappedPostingsWriter.finishDoc();
Index: lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java	(working copy)
@@ -69,8 +69,15 @@
 
     @Override
     public Object clone() {
-      PulsingTermState clone = new PulsingTermState();
-      clone.copyFrom(this);
+      PulsingTermState clone;
+      clone = (PulsingTermState) super.clone();
+      if (postingsSize != -1) {
+        clone.postings = new byte[postingsSize];
+        System.arraycopy(postings, 0, clone.postings, 0, postingsSize);
+      } else {
+        assert wrappedTermState != null;
+        clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
+      }
       return clone;
     }
 
@@ -84,10 +91,8 @@
           postings = new byte[ArrayUtil.oversize(other.postingsSize, 1)];
         }
         System.arraycopy(other.postings, 0, postings, 0, other.postingsSize);
-      } else if (wrappedTermState != null) {
+      } else {
         wrappedTermState.copyFrom(other.wrappedTermState);
-      } else {
-        wrappedTermState = (BlockTermState) other.wrappedTermState.clone();
       }
 
       // NOTE: we do not copy the
@@ -108,18 +113,20 @@
 
   @Override
   public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    //System.out.println("PR.readTermsBlock state=" + _termState);
     final PulsingTermState termState = (PulsingTermState) _termState;
     if (termState.inlinedBytes == null) {
       termState.inlinedBytes = new byte[128];
       termState.inlinedBytesReader = new ByteArrayDataInput();
     }
     int len = termsIn.readVInt();
+    //System.out.println("  len=" + len + " fp=" + termsIn.getFilePointer());
     if (termState.inlinedBytes.length < len) {
       termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)];
     }
     termsIn.readBytes(termState.inlinedBytes, 0, len);
     termState.inlinedBytesReader.reset(termState.inlinedBytes);
-    termState.wrappedTermState.termCount = 0;
+    termState.wrappedTermState.termBlockOrd = 0;
     wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState);
   }
 
@@ -140,7 +147,6 @@
     //System.out.println("  count=" + count + " threshold=" + maxPositions);
 
     if (count <= maxPositions) {
-      //System.out.println("  inlined pos=" + termState.inlinedBytesReader.getPosition());
 
       // Inlined into terms dict -- just read the byte[] blob in,
       // but don't decode it now (we only decode when a DocsEnum
@@ -154,6 +160,7 @@
       // current term block) into another byte[] (just the
       // blob for this term)...
       termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize);
+      //System.out.println("  inlined bytes=" + termState.postingsSize);
     } else {
       //System.out.println("  not inlined");
       termState.postingsSize = -1;
@@ -161,7 +168,7 @@
       termState.wrappedTermState.docFreq = termState.docFreq;
       termState.wrappedTermState.totalTermFreq = termState.totalTermFreq;
       wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState);
-      termState.wrappedTermState.termCount++;
+      termState.wrappedTermState.termBlockOrd++;
     }
   }
 
@@ -223,6 +230,7 @@
   }
 
   private static class PulsingDocsEnum extends DocsEnum {
+    private byte[] postingsBytes;
     private final ByteArrayDataInput postings = new ByteArrayDataInput();
     private final IndexOptions indexOptions;
     private final boolean storePayloads;
@@ -239,9 +247,16 @@
     public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) {
       //System.out.println("PR docsEnum termState=" + termState + " docFreq=" + termState.docFreq);
       assert termState.postingsSize != -1;
-      final byte[] bytes = new byte[termState.postingsSize];
-      System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
-      postings.reset(bytes);
+
+      // Must make a copy of termState's byte[] so that if
+      // app does TermsEnum.next(), this DocsEnum is not affected
+      if (postingsBytes == null) {
+        postingsBytes = new byte[termState.postingsSize];
+      } else if (postingsBytes.length < termState.postingsSize) {
+        postingsBytes = ArrayUtil.grow(postingsBytes, termState.postingsSize);
+      }
+      System.arraycopy(termState.postings, 0, postingsBytes, 0, termState.postingsSize);
+      postings.reset(postingsBytes, 0, termState.postingsSize);
       docID = 0;
       payloadLength = 0;
       freq = 1;
@@ -263,6 +278,7 @@
         }
 
         final int code = postings.readVInt();
+        //System.out.println("  read code=" + code);
         if (indexOptions == IndexOptions.DOCS_ONLY) {
           docID += code;
         } else {
@@ -295,7 +311,6 @@
         }
 
         if (liveDocs == null || liveDocs.get(docID)) {
-          //System.out.println("  return docID=" + docID + " freq=" + freq);
           return docID;
         }
       }
@@ -323,6 +338,7 @@
   }
 
   private static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
+    private byte[] postingsBytes;
     private final ByteArrayDataInput postings = new ByteArrayDataInput();
     private final boolean storePayloads;
 
@@ -346,9 +362,13 @@
 
     public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) {
       assert termState.postingsSize != -1;
-      final byte[] bytes = new byte[termState.postingsSize];
-      System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
-      postings.reset(bytes);
+      if (postingsBytes == null) {
+        postingsBytes = new byte[termState.postingsSize];
+      } else if (postingsBytes.length < termState.postingsSize) {
+        postingsBytes = ArrayUtil.grow(postingsBytes, termState.postingsSize);
+      }
+      System.arraycopy(termState.postings, 0, postingsBytes, 0, termState.postingsSize);
+      postings.reset(postingsBytes, 0, termState.postingsSize);
       this.liveDocs = liveDocs;
       payloadLength = 0;
       posPending = 0;
@@ -359,7 +379,7 @@
 
     @Override
     public int nextDoc() throws IOException {
-      //System.out.println("PR.nextDoc this=" + this);
+      //System.out.println("PR d&p nextDoc this=" + this);
 
       while(true) {
         //System.out.println("  cycle skip posPending=" + posPending);
@@ -367,16 +387,15 @@
         skipPositions();
 
         if (postings.eof()) {
-          //System.out.println("  END");
+          //System.out.println("PR   END");
           return docID = NO_MORE_DOCS;
         }
-        //System.out.println("  read doc code");
+
         final int code = postings.readVInt();
         docID += code >>> 1;            // shift off low bit
         if ((code & 1) != 0) {          // if low bit is set
           freq = 1;                     // freq is one
         } else {
-          //System.out.println("  read freq");
           freq = postings.readVInt();     // else read freq
         }
         posPending = freq;
@@ -401,10 +420,8 @@
 
     @Override
     public int advance(int target) throws IOException {
-      //System.out.println("PR.advance target=" + target);
       int doc;
       while((doc=nextDoc()) != NO_MORE_DOCS) {
-        //System.out.println("  nextDoc got doc=" + doc);
         if (doc >= target) {
           return docID = doc;
         }
@@ -414,7 +431,7 @@
 
     @Override
     public int nextPosition() throws IOException {
-      //System.out.println("PR.nextPosition posPending=" + posPending + " vs freq=" + freq);
+      //System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq);
       
       assert posPending > 0;
       posPending--;
@@ -424,7 +441,6 @@
           //System.out.println("PR     skip payload=" + payloadLength);
           postings.skipBytes(payloadLength);
         }
-        //System.out.println("  read pos code");
         final int code = postings.readVInt();
         //System.out.println("PR     code=" + code);
         if ((code & 1) != 0) {
@@ -437,17 +453,16 @@
         position += postings.readVInt();
       }
 
-      //System.out.println("  return pos=" + position + " hasPayload=" + !payloadRetrieved + " posPending=" + posPending + " this=" + this);
+      //System.out.println("PR d&p nextPos return pos=" + position + " this=" + this);
       return position;
     }
 
     private void skipPositions() throws IOException {
-      //System.out.println("PR.skipPositions: posPending=" + posPending);
       while(posPending != 0) {
         nextPosition();
       }
       if (storePayloads && !payloadRetrieved) {
-        //System.out.println("  skip last payload len=" + payloadLength);
+        //System.out.println("  skip payload len=" + payloadLength);
         postings.skipBytes(payloadLength);
         payloadRetrieved = true;
       }
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java	(working copy)
@@ -66,7 +66,7 @@
   private final TermsIndexWriterBase termsIndexWriter;
   private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
 
-  //private final String segment;
+  // private final String segment;
 
   public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
       SegmentWriteState state, PostingsWriterBase postingsWriter)
@@ -80,7 +80,7 @@
       writeHeader(out);
       currentField = null;
       this.postingsWriter = postingsWriter;
-      //segment = state.segmentName;
+      // segment = state.segmentName;
       
       //System.out.println("BTW.init seg=" + state.segmentName);
       
@@ -188,7 +188,7 @@
 
     @Override
     public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      //System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
+      //System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
       postingsWriter.startTerm();
       return postingsWriter;
     }
@@ -199,7 +199,7 @@
     public void finishTerm(BytesRef text, TermStats stats) throws IOException {
 
       assert stats.docFreq > 0;
-      //System.out.println("BTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
+      //System.out.println("BTW: finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
 
       final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
 
@@ -308,7 +308,7 @@
       bytesWriter.writeTo(out);
       bytesWriter.reset();
 
-      postingsWriter.flushTermsBlock();
+      postingsWriter.flushTermsBlock(pendingCount, pendingCount);
       lastPrevTerm.copy(pendingTerms[pendingCount-1].term);
       pendingCount = 0;
     }
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java	(working copy)
@@ -21,6 +21,8 @@
  *  index file format */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
@@ -34,18 +36,19 @@
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
-import org.apache.lucene.util.IOUtils;
 
 /** @lucene.experimental */
 public final class StandardPostingsWriter extends PostingsWriterBase {
-  final static String CODEC = "StandardPostingsWriterImpl";
+  final static String CODEC = "StandardPostingsWriter";
+
+  //private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
   
   // Increment version to change it:
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
 
-  IndexOutput freqOut;
-  IndexOutput proxOut;
+  final IndexOutput freqOut;
+  final IndexOutput proxOut;
   final DefaultSkipListWriter skipListWriter;
   /** Expert: The fraction of TermDocs entries stored in skip tables,
    * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
@@ -70,52 +73,42 @@
   IndexOptions indexOptions;
   boolean storePayloads;
   // Starts a new term
-  long lastFreqStart;
   long freqStart;
-  long lastProxStart;
   long proxStart;
   FieldInfo fieldInfo;
   int lastPayloadLength;
   int lastPosition;
 
-  private int pendingCount;
+  // private String segment;
 
-  //private String segment;
-
-  private RAMOutputStream bytesWriter = new RAMOutputStream();
-
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
     this(state, DEFAULT_SKIP_INTERVAL);
   }
   
   public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
+    super();
     this.skipInterval = skipInterval;
     this.skipMinimum = skipInterval; /* set to the same for now */
-    //this.segment = state.segmentName;
+    // this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName, state.context);
-    boolean success = false;
-    try {
-      if (state.fieldInfos.hasProx()) {
-        // At least one field does not omit TF, so create the
-        // prox file
-        fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
-        proxOut = state.directory.createOutput(fileName, state.context);
-      } else {
-        // Every field omits TF so we will write no prox file
-        proxOut = null;
-      }
-      
-      totalNumDocs = state.numDocs;
-      
-      skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels,
-          state.numDocs, freqOut, proxOut);
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeSafely(true, freqOut, proxOut);
-      }
+    if (state.fieldInfos.hasProx()) {
+      // At least one field does not omit TF, so create the
+      // prox file
+      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
+      proxOut = state.directory.createOutput(fileName, state.context);
+    } else {
+      // Every field omits TF so we will write no prox file
+      proxOut = null;
     }
+
+    totalNumDocs = state.numDocs;
+
+    skipListWriter = new DefaultSkipListWriter(skipInterval,
+                                               maxSkipLevels,
+                                               state.numDocs,
+                                               freqOut,
+                                               proxOut);
   }
 
   @Override
@@ -129,8 +122,8 @@
 
   @Override
   public void startTerm() {
-    //System.out.println("StandardW: startTerm seg=" + segment + " pendingCount=" + pendingCount);
     freqStart = freqOut.getFilePointer();
+    //if (DEBUG) System.out.println("SPW: startTerm freqOut.fp=" + freqStart);
     if (proxOut != null) {
       proxStart = proxOut.getFilePointer();
       // force first payload to write its length
@@ -144,6 +137,13 @@
   @Override
   public void setField(FieldInfo fieldInfo) {
     //System.out.println("SPW: setField");
+    /*
+    if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
+      DEBUG = true;
+    } else {
+      DEBUG = false;
+    }
+    */
     this.fieldInfo = fieldInfo;
     indexOptions = fieldInfo.indexOptions;
     storePayloads = fieldInfo.storePayloads;
@@ -158,7 +158,7 @@
    *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
-    //System.out.println("StandardW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq);
+    // if (DEBUG) System.out.println("SPW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());
 
     final int delta = docID - lastDocID;
     
@@ -189,14 +189,14 @@
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
-    //System.out.println("StandardW:     addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
+    //if (DEBUG) System.out.println("SPW:     addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
     assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS: "invalid indexOptions: " + indexOptions;
     assert proxOut != null;
 
     final int delta = position - lastPosition;
+    
+    assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;            // not quite right (if pos=0 is repeated twice we don't catch it)
 
-    assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;
-
     lastPosition = position;
 
     if (storePayloads) {
@@ -222,57 +222,104 @@
   public void finishDoc() {
   }
 
+  private static class PendingTerm {
+    public final long freqStart;
+    public final long proxStart;
+    public final int skipOffset;
+
+    public PendingTerm(long freqStart, long proxStart, int skipOffset) {
+      this.freqStart = freqStart;
+      this.proxStart = proxStart;
+      this.skipOffset = skipOffset;
+    }
+  }
+
+  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
+
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
-    //System.out.println("StandardW.finishTerm seg=" + segment);
+
+    // if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
     assert stats.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
     // for this term) in two places?
     assert stats.docFreq == df;
 
-    final boolean isFirstTerm = pendingCount == 0;
-    //System.out.println("  isFirstTerm=" + isFirstTerm);
-
-    //System.out.println("  freqFP=" + freqStart);
-    if (isFirstTerm) {
-      bytesWriter.writeVLong(freqStart);
+    final int skipOffset;
+    if (df >= skipMinimum) {
+      skipOffset = (int) (skipListWriter.writeSkip(freqOut)-freqStart);
     } else {
-      bytesWriter.writeVLong(freqStart-lastFreqStart);
+      skipOffset = -1;
     }
-    lastFreqStart = freqStart;
 
-    if (df >= skipMinimum) {
-      bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
+    pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset));
+
+    lastDocID = 0;
+    df = 0;
+  }
+
+  private final RAMOutputStream bytesWriter = new RAMOutputStream();
+
+  @Override
+  public void flushTermsBlock(int start, int count) throws IOException {
+    //if (DEBUG) System.out.println("SPW: flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count) + " pendingTerms.size()=" + pendingTerms.size());
+
+    if (count == 0) {
+      termsOut.writeByte((byte) 0);
+      return;
     }
 
+    assert start <= pendingTerms.size();
+    assert count <= start;
+
+    final int limit = pendingTerms.size() - start + count;
+    final PendingTerm firstTerm = pendingTerms.get(limit - count);
+    // First term in block is abs coded:
+    bytesWriter.writeVLong(firstTerm.freqStart);
+
+    if (firstTerm.skipOffset != -1) {
+      assert firstTerm.skipOffset > 0;
+      bytesWriter.writeVInt(firstTerm.skipOffset);
+    }
     if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      //System.out.println("  proxFP=" + proxStart);
-      if (isFirstTerm) {
-        bytesWriter.writeVLong(proxStart);
-      } else {
-        bytesWriter.writeVLong(proxStart - lastProxStart);
+      bytesWriter.writeVLong(firstTerm.proxStart);
+    }
+    long lastFreqStart = firstTerm.freqStart;
+    long lastProxStart = firstTerm.proxStart;
+    for(int idx=limit-count+1; idx<limit; idx++) {
+      final PendingTerm term = pendingTerms.get(idx);
+      //if (DEBUG) System.out.println("  write term freqStart=" + term.freqStart);
+      // The rest of the terms term are delta coded:
+      bytesWriter.writeVLong(term.freqStart - lastFreqStart);
+      lastFreqStart = term.freqStart;
+      if (term.skipOffset != -1) {
+        assert term.skipOffset > 0;
+        bytesWriter.writeVInt(term.skipOffset);
       }
-      lastProxStart = proxStart;
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        bytesWriter.writeVLong(term.proxStart - lastProxStart);
+        lastProxStart = term.proxStart;
+      }
     }
-     
-    lastDocID = 0;
-    df = 0;
-    pendingCount++;
-  }
 
-  @Override
-  public void flushTermsBlock() throws IOException {
-    //System.out.println("SPW.flushBlock pendingCount=" + pendingCount);
     termsOut.writeVInt((int) bytesWriter.getFilePointer());
     bytesWriter.writeTo(termsOut);
     bytesWriter.reset();
-    pendingCount = 0;
+
+    // Remove the terms we just wrote:
+    pendingTerms.subList(limit-count, limit).clear();
   }
 
   @Override
   public void close() throws IOException {
-    IOUtils.closeSafely(false, freqOut, proxOut);
+    try {
+      freqOut.close();
+    } finally {
+      if (proxOut != null) {
+        proxOut.close();
+      }
+    }
   }
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(working copy)
@@ -27,8 +27,8 @@
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.BlockTermState;
-import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -46,22 +46,23 @@
 
   private final IndexInput freqIn;
   private final IndexInput proxIn;
+  // public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   int skipInterval;
   int maxSkipLevels;
   int skipMinimum;
 
-  //private String segment;
+  // private String segment;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, int codecId) throws IOException {
+  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
     freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
-                           context);
-    //this.segment = segmentInfo.name;
+                           ioContext);
+    // this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
         proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
-                               context);
+                               ioContext);
         success = true;
       } finally {
         if (!success) {
@@ -73,10 +74,10 @@
     }
   }
 
-  public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
+  public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.FREQ_EXTENSION));
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.PROX_EXTENSION));
     }
   }
 
@@ -100,7 +101,7 @@
 
     // Only used by the "primary" TermState -- clones don't
     // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;
+    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
     byte[] bytes;
 
     @Override
@@ -155,7 +156,8 @@
     final StandardTermState termState = (StandardTermState) _termState;
 
     final int len = termsIn.readVInt();
-    //System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
+
+    // if (DEBUG) System.out.println("  SPR.readTermsBlock bytes=" + len + " ts=" + _termState);
     if (termState.bytes == null) {
       termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
       termState.bytesReader = new ByteArrayDataInput();
@@ -171,21 +173,25 @@
   public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
     throws IOException {
     final StandardTermState termState = (StandardTermState) _termState;
-    //System.out.println("StandardR.nextTerm seg=" + segment);
-    final boolean isFirstTerm = termState.termCount == 0;
+    // if (DEBUG) System.out.println("SPR: nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition());
+    final boolean isFirstTerm = termState.termBlockOrd == 0;
 
     if (isFirstTerm) {
       termState.freqOffset = termState.bytesReader.readVLong();
     } else {
       termState.freqOffset += termState.bytesReader.readVLong();
     }
-    //System.out.println("  dF=" + termState.docFreq);
-    //System.out.println("  freqFP=" + termState.freqOffset);
+    /*
+    if (DEBUG) {
+      System.out.println("  dF=" + termState.docFreq);
+      System.out.println("  freqFP=" + termState.freqOffset);
+    }
+    */
     assert termState.freqOffset < freqIn.length();
 
     if (termState.docFreq >= skipMinimum) {
       termState.skipOffset = termState.bytesReader.readVInt();
-      //System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
+      // if (DEBUG) System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
     } else {
       // undefined
@@ -197,7 +203,7 @@
       } else {
         termState.proxOffset += termState.bytesReader.readVLong();
       }
-      //System.out.println("  proxFP=" + termState.proxOffset);
+      // if (DEBUG) System.out.println("  proxFP=" + termState.proxOffset);
     }
   }
     
@@ -215,6 +221,7 @@
         docsEnum = new SegmentDocsEnum(freqIn);
       }
     }
+    // if (DEBUG) System.out.println("SPR.docs ts=" + termState);
     return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
   }
 
@@ -300,7 +307,7 @@
       assert limit > 0;
       ord = 0;
       doc = 0;
-      //System.out.println("  sde limit=" + limit + " freqFP=" + freqOffset);
+      // if (DEBUG) System.out.println("  sde limit=" + limit + " freqFP=" + freqOffset);
 
       skipped = false;
 
@@ -309,8 +316,10 @@
 
     @Override
     public int nextDoc() throws IOException {
+      //if (DEBUG) System.out.println("    stpr.nextDoc seg=" + segment + " fp=" + freqIn.getFilePointer());
       while(true) {
         if (ord == limit) {
+          //if (DEBUG) System.out.println("      return doc=" + NO_MORE_DOCS);
           return doc = NO_MORE_DOCS;
         }
 
@@ -318,6 +327,7 @@
 
         // Decode next doc/freq pair
         final int code = freqIn.readVInt();
+        // if (DEBUG) System.out.println("      code=" + code);
         if (omitTF) {
           doc += code;
         } else {
@@ -334,6 +344,7 @@
         }
       }
 
+      //if (DEBUG) System.out.println("    stpr.nextDoc return doc=" + doc);
       return doc;
     }
 
@@ -480,16 +491,17 @@
       freqOffset = termState.freqOffset;
       proxOffset = termState.proxOffset;
       skipOffset = termState.skipOffset;
-      //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
+      // if (DEBUG) System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
 
       return this;
     }
 
     @Override
     public int nextDoc() throws IOException {
+      // if (DEBUG) System.out.println("SPR.nextDoc seg=" + segment + " freqIn.fp=" + freqIn.getFilePointer());
       while(true) {
         if (ord == limit) {
-          //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
+          // if (DEBUG) System.out.println("  return END");
           return doc = NO_MORE_DOCS;
         }
 
@@ -513,7 +525,7 @@
 
       position = 0;
 
-      //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
+      // if (DEBUG) System.out.println("  return doc=" + doc);
       return doc;
     }
 
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java	(working copy)
@@ -22,31 +22,37 @@
 
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
+import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.TermsIndexWriterBase;
-import org.apache.lucene.index.codecs.TermsIndexReaderBase;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.index.codecs.BlockTermsWriter;
-import org.apache.lucene.index.codecs.BlockTermsReader;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.store.Directory;
 
 /** Default codec. 
  *  @lucene.experimental */
 public class StandardCodec extends Codec {
 
+  private final int minBlockSize;
+  private final int maxBlockSize;
+
   public StandardCodec() {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  public StandardCodec(int minBlockSize, int maxBlockSize) {
     super("Standard");
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
   }
 
   @Override
@@ -57,31 +63,16 @@
     // pluggable?  Ie so that this codec would record which
     // index impl was used, and switch on loading?
     // Or... you must make a new Codec for this?
-    TermsIndexWriterBase indexWriter;
     boolean success = false;
     try {
-      indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
       success = true;
+      return ret;
     } finally {
       if (!success) {
         docs.close();
       }
     }
-
-    success = false;
-    try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        try {
-          docs.close();
-        } finally {
-          indexWriter.close();
-        }
-      }
-    }
   }
 
   public final static int TERMS_CACHE_SIZE = 1024;
@@ -89,43 +80,24 @@
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
-    TermsIndexReaderBase indexReader;
 
     boolean success = false;
     try {
-      indexReader = new VariableGapTermsIndexReader(state.dir,
+      FieldsProducer ret = new BlockTreeTermsReader(
+                                                    state.dir,
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,
-                                                    state.termsIndexDivisor,
-                                                    state.codecId, state.context);
+                                                    postings,
+                                                    state.context,
+                                                    state.codecId,
+                                                    state.termsIndexDivisor);
       success = true;
+      return ret;
     } finally {
       if (!success) {
         postings.close();
       }
     }
-
-    success = false;
-    try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
-                                                state.fieldInfos,
-                                                state.segmentInfo.name,
-                                                postings,
-                                                state.context,
-                                                TERMS_CACHE_SIZE,
-                                                state.codecId);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        try {
-          postings.close();
-        } finally {
-          indexReader.close();
-        }
-      }
-    }
   }
 
   /** Extension of freq postings file */
@@ -135,11 +107,10 @@
   static final String PROX_EXTENSION = "prx";
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, id, files);
-    BlockTermsReader.files(dir, segmentInfo, id, files);
-    VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+  public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
+    StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
+    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
   }
 
   @Override
@@ -151,11 +122,15 @@
   public static void getStandardExtensions(Set<String> extensions) {
     extensions.add(FREQ_EXTENSION);
     extensions.add(PROX_EXTENSION);
-    BlockTermsReader.getExtensions(extensions);
-    VariableGapTermsIndexReader.getIndexExtensions(extensions);
+    BlockTreeTermsReader.getExtensions(extensions);
   }
 
   @Override
+  public String toString() {
+    return name + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
+  }
+
+  @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
     return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
   }
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java	(working copy)
@@ -29,11 +29,9 @@
   public int docFreq;            // how many docs have this term
   public long totalTermFreq;     // total number of occurrences of this term
 
-  public int termCount;          // term ord are in the current block
-  public long blockFilePointer;  // fp into the terms dict primary file (_X.tib) that holds this term
+  public int termBlockOrd;          // the term's ord in the current block
+  public long blockFilePointer;  // fp into the terms dict primary file (_X.tim) that holds this term
 
-  public int blockTermCount;     // how many terms in current block
-
   @Override
   public void copyFrom(TermState _other) {
     assert _other instanceof BlockTermState : "can not copy from " + _other.getClass().getName();
@@ -41,7 +39,7 @@
     super.copyFrom(_other);
     docFreq = other.docFreq;
     totalTermFreq = other.totalTermFreq;
-    termCount = other.termCount;
+    termBlockOrd = other.termBlockOrd;
     blockFilePointer = other.blockFilePointer;
 
     // NOTE: don't copy blockTermCount;
@@ -51,6 +49,6 @@
 
   @Override
   public String toString() {
-    return "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
+    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
   }
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java	(working copy)
@@ -26,9 +26,8 @@
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; // javadocs
-
-/** BlockTermsReader interacts with a single instance
+/** The core terms dictionaries (BlockTermsReader,
+ *  BlockTreeTermsReader) interact with a single instance
  *  of this class to manage creation of {@link DocsEnum} and
  *  {@link DocsAndPositionsEnum} instances.  It provides an
  *  IndexInput (termsIn) where this class may read any
@@ -49,11 +48,11 @@
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsEnum reuse) throws IOException;
+  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException;
+  public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
 
   public abstract void close() throws IOException;
 
Index: lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java	(working copy)
@@ -77,10 +77,14 @@
     }
 
     @Override
-    public void set(IntIndexOutput.Index other) throws IOException {
+    public void copyFrom(IntIndexOutput.Index other, boolean copyLast) throws IOException {
       Index idx = (Index) other;
-      lastFP = fp = idx.fp;
-      lastUpto = upto = idx.upto;
+      fp = idx.fp;
+      upto = idx.upto;
+      if (copyLast) {
+        lastFP = fp;
+        lastUpto = upto;
+      }
     }
 
     @Override
Index: lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java	(working copy)
@@ -68,10 +68,14 @@
     }
 
     @Override
-    public void set(IntIndexOutput.Index other) throws IOException {
+    public void copyFrom(IntIndexOutput.Index other, boolean copyLast) throws IOException {
       Index idx = (Index) other;
-      lastFP = fp = idx.fp;
-      lastUpto = upto = idx.upto;
+      fp = idx.fp;
+      upto = idx.upto;
+      if (copyLast) {
+        lastFP = fp;
+        lastUpto = upto;
+      }
     }
 
     @Override
Index: lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java	(working copy)
@@ -33,7 +33,11 @@
 
   public abstract void startTerm() throws IOException;
 
-  public abstract void flushTermsBlock() throws IOException;
+  /** Flush count terms starting at start "backwards", as a
+   *  block. start is a negative offset from the end of the
+   *  terms stack, ie bigger start means further back in
+   *  the stack. */
+  public abstract void flushTermsBlock(int start, int count) throws IOException;
 
   /** Finishes the current term */
   public abstract void finishTerm(TermStats stats) throws IOException;
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java	(working copy)
@@ -44,7 +44,7 @@
     public abstract void mark() throws IOException;
 
     /** Copies index from other */
-    public abstract void set(Index other) throws IOException;
+    public abstract void copyFrom(Index other, boolean copyLast) throws IOException;
 
     /** Writes "location" of current output pointer of primary
      *  output to different output (out) */
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java	(working copy)
@@ -153,6 +153,10 @@
 
     // Only used for "primary" term state; these are never
     // copied on clone:
+    
+    // TODO: these should somehow be stored per-TermsEnum
+    // not per TermState; maybe somehow the terms dict
+    // should load/manage the byte[]/DataReader for us?
     byte[] bytes;
     ByteArrayDataInput bytesReader;
 
@@ -216,8 +220,9 @@
   @Override
   public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
+    //System.out.println("SEPR: readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
     final int len = termsIn.readVInt();
-    //System.out.println("SepR.readTermsBlock len=" + len);
+    //System.out.println("  numBytes=" + len);
     if (termState.bytes == null) {
       termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
       termState.bytesReader = new ByteArrayDataInput(termState.bytes);
@@ -231,30 +236,30 @@
   @Override
   public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
-    //System.out.println("SepR.nextTerm termCount=" + termState.termCount);
+    final boolean isFirstTerm = termState.termBlockOrd == 0;
+    //System.out.println("SEPR.nextTerm termCount=" + termState.termBlockOrd + " isFirstTerm=" + isFirstTerm + " bytesReader.pos=" + termState.bytesReader.getPosition());
     //System.out.println("  docFreq=" + termState.docFreq);
-    final boolean isFirstTerm = termState.termCount == 0;
     termState.docIndex.read(termState.bytesReader, isFirstTerm);
     //System.out.println("  docIndex=" + termState.docIndex);
     if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
       termState.freqIndex.read(termState.bytesReader, isFirstTerm);
-    }
-    
-    if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      //System.out.println("  freqIndex=" + termState.freqIndex);
-      termState.posIndex.read(termState.bytesReader, isFirstTerm);
-      //System.out.println("  posIndex=" + termState.posIndex);
-      if (fieldInfo.storePayloads) {
-        if (isFirstTerm) {
-          termState.payloadFP = termState.bytesReader.readVLong();
-        } else {
-          termState.payloadFP += termState.bytesReader.readVLong();
+      if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        //System.out.println("  freqIndex=" + termState.freqIndex);
+        termState.posIndex.read(termState.bytesReader, isFirstTerm);
+        //System.out.println("  posIndex=" + termState.posIndex);
+        if (fieldInfo.storePayloads) {
+          if (isFirstTerm) {
+            termState.payloadFP = termState.bytesReader.readVLong();
+          } else {
+            termState.payloadFP += termState.bytesReader.readVLong();
+          }
+          //System.out.println("  payloadFP=" + termState.payloadFP);
         }
-        //System.out.println("  payloadFP=" + termState.payloadFP);
       }
     }
+
     if (termState.docFreq >= skipMinimum) {
-      //System.out.println("   readSkip @ " + termState.bytesReader.pos);
+      //System.out.println("   readSkip @ " + termState.bytesReader.getPosition());
       if (isFirstTerm) {
         termState.skipFP = termState.bytesReader.readVLong();
       } else {
@@ -656,6 +661,7 @@
           // Skipper did move
           skipper.getFreqIndex().seek(freqReader);
           skipper.getDocIndex().seek(docReader);
+          //System.out.println("  doc seek'd to " + skipper.getDocIndex());
           // NOTE: don't seek pos here; do it lazily
           // instead.  Eg a PhraseQuery may skip to many
           // docs before finally asking for positions...
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java	(working copy)
@@ -124,12 +124,12 @@
     Arrays.fill(lastSkipDoc, 0);
     Arrays.fill(lastSkipPayloadLength, -1);  // we don't have to write the first length in the skip list
     for(int i=0;i<numberOfSkipLevels;i++) {
-      docIndex[i].set(topDocIndex);
+      docIndex[i].copyFrom(topDocIndex, true);
       if (freqOutput != null) {
-        freqIndex[i].set(topFreqIndex);
+        freqIndex[i].copyFrom(topFreqIndex, true);
       }
       if (posOutput != null) {
-        posIndex[i].set(topPosIndex);
+        posIndex[i].copyFrom(topPosIndex, true);
       }
     }
     if (payloadOutput != null) {
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java	(working copy)
@@ -18,6 +18,8 @@
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.index.CorruptIndexException;
@@ -89,17 +91,13 @@
   boolean storePayloads;
   IndexOptions indexOptions;
 
-  long lastSkipFP;
-
   FieldInfo fieldInfo;
 
   int lastPayloadLength;
   int lastPosition;
   long payloadStart;
-  long lastPayloadStart;
   int lastDocID;
   int df;
-  private int pendingTermCount;
 
   // Holds pending byte[] blob for the current terms block
   private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
@@ -171,6 +169,7 @@
   @Override
   public void startTerm() throws IOException {
     docIndex.mark();
+    //System.out.println("SEPW: startTerm docIndex=" + docIndex);
 
     if (indexOptions != IndexOptions.DOCS_ONLY) {
       freqIndex.mark();
@@ -201,7 +200,7 @@
   public void startDoc(int docID, int termDocFreq) throws IOException {
 
     final int delta = docID - lastDocID;
-    //System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
+    //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@@ -223,15 +222,6 @@
     }
   }
 
-  @Override
-  public void flushTermsBlock() throws IOException {
-    //System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
-    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
-    indexBytesWriter.writeTo(termsOut);
-    indexBytesWriter.reset();
-    pendingTermCount = 0;
-  }
-
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
@@ -269,6 +259,24 @@
     lastPosition = 0;
   }
 
+  private static class PendingTerm {
+    public final IntIndexOutput.Index docIndex;
+    public final IntIndexOutput.Index freqIndex;
+    public final IntIndexOutput.Index posIndex;
+    public final long payloadFP;
+    public final long skipFP;
+
+    public PendingTerm(IntIndexOutput.Index docIndex, IntIndexOutput.Index freqIndex, IntIndexOutput.Index posIndex, long payloadFP, long skipFP) {
+      this.docIndex = docIndex;
+      this.freqIndex = freqIndex;
+      this.posIndex = posIndex;
+      this.payloadFP = payloadFP;
+      this.skipFP = skipFP;
+    }
+  }
+
+  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
+
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
@@ -276,53 +284,110 @@
     assert stats.docFreq > 0;
     assert stats.docFreq == df;
 
-    final boolean isFirstTerm = pendingTermCount == 0;  
-    //System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
+    final IntIndexOutput.Index docIndexCopy = docOut.index();
+    docIndexCopy.copyFrom(docIndex, false);
 
-    docIndex.write(indexBytesWriter, isFirstTerm);
-    //System.out.println("  docIndex=" + docIndex);
-
+    final IntIndexOutput.Index freqIndexCopy;
+    final IntIndexOutput.Index posIndexCopy;
     if (indexOptions != IndexOptions.DOCS_ONLY) {
-      freqIndex.write(indexBytesWriter, isFirstTerm);
-      //System.out.println("  freqIndex=" + freqIndex);
-    }
-
-    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      posIndex.write(indexBytesWriter, isFirstTerm);
-      //System.out.println("  posIndex=" + posIndex);
-      if (storePayloads) {
-        if (isFirstTerm) {
-          indexBytesWriter.writeVLong(payloadStart);
-        } else {
-          indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
-        }
-        lastPayloadStart = payloadStart;
-        //System.out.println("  payloadFP=" + payloadStart);
+      freqIndexCopy = freqOut.index();
+      freqIndexCopy.copyFrom(freqIndex, false);
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        posIndexCopy = posOut.index();
+        posIndexCopy.copyFrom(posIndex, false);
+      } else {
+        posIndexCopy = null;
       }
+    } else {
+      freqIndexCopy = null;
+      posIndexCopy = null;
     }
 
+    final long skipFP;
     if (df >= skipMinimum) {
-      //System.out.println("  skipFP=" + skipStart);
-      final long skipFP = skipOut.getFilePointer();
+      skipFP = skipOut.getFilePointer();
+      //System.out.println("  skipFP=" + skipFP);
       skipListWriter.writeSkip(skipOut);
-      //System.out.println("   writeSkip @ " + indexBytesWriter.getFilePointer());
-      if (isFirstTerm) {
-        indexBytesWriter.writeVLong(skipFP);
-      } else {
-        indexBytesWriter.writeVLong(skipFP - lastSkipFP);
-      }
-      lastSkipFP = skipFP;
-    } else if (isFirstTerm) {
-      // lazily write an absolute delta if a term in this block requires skip data.
-      lastSkipFP = 0;
+      //System.out.println("    numBytes=" + (skipOut.getFilePointer()-skipFP));
+    } else {
+      skipFP = -1;
     }
 
     lastDocID = 0;
     df = 0;
-    pendingTermCount++;
+
+    pendingTerms.add(new PendingTerm(docIndexCopy,
+                                     freqIndexCopy,
+                                     posIndexCopy,
+                                     payloadStart,
+                                     skipFP));
   }
 
   @Override
+  public void flushTermsBlock(int start, int count) throws IOException {
+    //System.out.println("SEPW: flushTermsBlock: start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size() + " termsOut.fp=" + termsOut.getFilePointer());
+    assert indexBytesWriter.getFilePointer() == 0;
+    final int absStart = pendingTerms.size() - start;
+    final List<PendingTerm> slice = pendingTerms.subList(absStart, absStart+count);
+
+    long lastPayloadFP = 0;
+    long lastSkipFP = 0;
+
+    if (count == 0) {
+      termsOut.writeByte((byte) 0);
+      return;
+    }
+
+    final PendingTerm firstTerm = slice.get(0);
+    final IntIndexOutput.Index docIndexFlush = firstTerm.docIndex;
+    final IntIndexOutput.Index freqIndexFlush = firstTerm.freqIndex;
+    final IntIndexOutput.Index posIndexFlush = firstTerm.posIndex;
+
+    for(int idx=0;idx<slice.size();idx++) {
+      final boolean isFirstTerm = idx == 0;
+      final PendingTerm t = slice.get(idx);
+      //System.out.println("  write idx=" + idx + " docIndex=" + t.docIndex);
+      docIndexFlush.copyFrom(t.docIndex, false);
+      docIndexFlush.write(indexBytesWriter, isFirstTerm);
+      if (indexOptions != IndexOptions.DOCS_ONLY) {
+        freqIndexFlush.copyFrom(t.freqIndex, false);
+        freqIndexFlush.write(indexBytesWriter, isFirstTerm);
+        //System.out.println("    freqIndex=" + t.freqIndex);
+        if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+          posIndexFlush.copyFrom(t.posIndex, false);
+          posIndexFlush.write(indexBytesWriter, isFirstTerm);
+          //System.out.println("    posIndex=" + t.posIndex);
+          if (storePayloads) {
+            //System.out.println("    payloadFP=" + t.payloadFP);
+            if (isFirstTerm) {
+              indexBytesWriter.writeVLong(t.payloadFP);
+            } else {
+              indexBytesWriter.writeVLong(t.payloadFP - lastPayloadFP);
+            }
+            lastPayloadFP = t.payloadFP;
+          }
+        }
+      }
+
+      if (t.skipFP != -1) {
+        if (isFirstTerm) {
+          indexBytesWriter.writeVLong(t.skipFP);
+        } else {
+          indexBytesWriter.writeVLong(t.skipFP - lastSkipFP);
+        }
+        lastSkipFP = t.skipFP;
+        //System.out.println("    skipFP=" + t.skipFP);
+      }
+    }
+
+    //System.out.println("  numBytes=" + indexBytesWriter.getFilePointer());
+    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
+    indexBytesWriter.writeTo(termsOut);
+    indexBytesWriter.reset();
+    slice.clear();
+  }
+
+  @Override
   public void close() throws IOException {
     IOUtils.closeSafely(false, docOut, skipOut, freqOut, posOut, payloadOut);
   }
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java	(revision 0)
@@ -0,0 +1,2832 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.TreeMap;
+
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RunAutomaton;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Outputs;
+import org.apache.lucene.util.fst.Util;
+
+/** A block-based terms index and dictionary that assigns
+ *  terms to variable length blocks according to how they
+ *  share prefixes.  The terms index is a prefix trie
+ *  whose leaves are term blocks.  The advantage of this
+ *  approach is that {@link #seekExact} is often able to
+ *  determine a term cannot exist without doing any IO, and
+ *  intersection with Automata is very fast.  Note that this
+ *  terms dictionary has it's own fixed terms index (ie, it
+ *  does not support a pluggable terms index
+ *  implementation).
+ *
+ *  <p><b>NOTE</b>: this terms dictionary does not support
+ *  index divisor when opening an IndexReader.  Instead, you
+ *  can change the min/maxItemsPerBlock during indexing.</p>
+ *
+ *  <p>The data structure used by this implementation is very
+ *  similar to a burst trie
+ *  (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
+ *  but with added logic to break up too-large blocks of all
+ *  terms sharing a given prefix into smaller ones.</p>
+ *
+ *  <p>Use {@link CheckIndex} with the <code>-verbose</code>
+ *  option to see summary statistics on the blocks in the
+ *  dictionary.
+ *
+ *  See {@link BlockTreeTermsWriter}.
+ *
+ * @lucene.experimental
+ */
+
+public class BlockTreeTermsReader extends FieldsProducer {
+
+  // Open input to the main terms dict file (_X.tib)
+  private final IndexInput in;
+
+  //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+  // Reads the terms dict entries, to gather state to
+  // produce DocsEnum on demand
+  private final PostingsReaderBase postingsReader;
+
+  private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
+
+  // keeps the dirStart offset
+  protected long dirOffset;
+  protected long indexDirOffset;
+
+  private String segment;
+  
+  public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, String segment,
+                              PostingsReaderBase postingsReader, IOContext ioContext,
+                              int codecId, int indexDivisor)
+    throws IOException {
+    
+    this.postingsReader = postingsReader;
+
+    this.segment = segment;
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_EXTENSION),
+                       ioContext);
+
+    boolean success = false;
+    IndexInput indexIn = null;
+
+    try {
+      readHeader(in);
+      if (indexDivisor != -1) {
+        indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
+                                ioContext);
+        readIndexHeader(indexIn);
+      }
+
+      // Have PostingsReader init itself
+      postingsReader.init(in);
+
+      // Read per-field details
+      seekDir(in, dirOffset);
+      if (indexDivisor != -1) {
+        seekDir(indexIn, indexDirOffset);
+      }
+
+      final int numFields = in.readVInt();
+
+      for(int i=0;i<numFields;i++) {
+        final int field = in.readVInt();
+        final long numTerms = in.readVLong();
+        assert numTerms >= 0;
+        final int numBytes = in.readVInt();
+        final BytesRef rootCode = new BytesRef(new byte[numBytes]);
+        in.readBytes(rootCode.bytes, 0, numBytes);
+        rootCode.length = numBytes;
+        final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        assert fieldInfo != null: "field=" + field;
+        final long sumTotalTermFreq = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
+        final long sumDocFreq = in.readVLong();
+        final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
+        assert !fields.containsKey(fieldInfo.name);
+        fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, indexStartFP, indexIn));
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeSafely(true, indexIn, this);
+      } else if (indexDivisor != -1) {
+        indexIn.close();
+      }
+    }
+  }
+
+  protected void readHeader(IndexInput input) throws IOException {
+    CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
+                          BlockTreeTermsWriter.VERSION_START,
+                          BlockTreeTermsWriter.VERSION_CURRENT);
+    dirOffset = input.readLong();    
+  }
+
+  protected void readIndexHeader(IndexInput input) throws IOException {
+    CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
+                          BlockTreeTermsWriter.VERSION_START,
+                          BlockTreeTermsWriter.VERSION_CURRENT);
+    indexDirOffset = input.readLong();    
+  }
+  
+  protected void seekDir(IndexInput input, long dirOffset)
+      throws IOException {
+    input.seek(dirOffset);
+  }
+
+  // for debugging
+  // private static String toHex(int v) {
+  //   return "0x" + Integer.toHexString(v);
+  // }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.closeSafely(false, in, postingsReader);
+    } finally { 
+      for(FieldReader field : fields.values()) {
+        field.close();
+      }
+      // Clear so refs to terms index is GCable even if
+      // app hangs onto us:
+      fields.clear();
+    }
+  }
+
+  public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) {
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION));
+  }
+
+  public static void getExtensions(Collection<String> extensions) {
+    extensions.add(BlockTreeTermsWriter.TERMS_EXTENSION);
+    extensions.add(BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
+  }
+
+  @Override
+  public FieldsEnum iterator() {
+    return new TermFieldsEnum();
+  }
+
+  @Override
+  public Terms terms(String field) throws IOException {
+    return fields.get(field);
+  }
+
+  // Iterates through all fields
+  private class TermFieldsEnum extends FieldsEnum {
+    final Iterator<FieldReader> it;
+    FieldReader current;
+
+    TermFieldsEnum() {
+      it = fields.values().iterator();
+    }
+
+    @Override
+    public String next() {
+      if (it.hasNext()) {
+        current = it.next();
+        return current.fieldInfo.name;
+      } else {
+        current = null;
+        return null;
+      }
+    }
+    
+    @Override
+    public TermsEnum terms() throws IOException {
+      return current.iterator();
+    }
+  }
+
+  // for debugging
+  String brToString(BytesRef b) {
+    if (b == null) {
+      return "null";
+    } else {
+      try {
+        return b.utf8ToString() + " " + b;
+      } catch (Throwable t) {
+        // If BytesRef isn't actually UTF8, or it's eg a
+        // prefix of UTF8 that ends mid-unicode-char, we
+        // fallback to hex:
+        return b.toString();
+      }
+    }
+  }
+
+  public static class Stats {
+    public int indexNodeCount;
+    public int indexArcCount;
+    public int indexNumBytes;
+
+    public long totalTermCount;
+    public long totalTermBytes;
+
+
+    public int nonFloorBlockCount;
+    public int floorBlockCount;
+    public int floorSubBlockCount;
+    public int mixedBlockCount;
+    public int termsOnlyBlockCount;
+    public int subBlocksOnlyBlockCount;
+    public int totalBlockCount;
+
+    public int[] blockCountByPrefixLen = new int[10];
+    private int startBlockCount;
+    private int endBlockCount;
+    public long totalBlockSuffixBytes;
+    public long totalBlockStatsBytes;
+
+    // Postings impl plus the other few vInts stored in
+    // the frame:
+    public long totalBlockOtherBytes;
+
+    public final String segment;
+    public final String field;
+
+    public Stats(String segment, String field) {
+      this.segment = segment;
+      this.field = field;
+    }
+
+    void startBlock(FieldReader.SegmentTermsEnum.Frame frame, boolean isFloor) {
+      totalBlockCount++;
+      if (isFloor) {
+        if (frame.fp == frame.fpOrig) {
+          floorBlockCount++;
+        }
+        floorSubBlockCount++;
+      } else {
+        nonFloorBlockCount++;
+      }
+
+      if (blockCountByPrefixLen.length <= frame.prefix) {
+        blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1+frame.prefix);
+      }
+      blockCountByPrefixLen[frame.prefix]++;
+      startBlockCount++;
+      totalBlockSuffixBytes += frame.suffixesReader.length();
+      totalBlockStatsBytes += frame.statsReader.length();
+    }
+
+    void endBlock(FieldReader.SegmentTermsEnum.Frame frame) {
+      final int termCount = frame.isLeafBlock ? frame.entCount : frame.state.termBlockOrd;
+      final int subBlockCount = frame.entCount - termCount;
+      totalTermCount += termCount;
+      if (termCount != 0 && subBlockCount != 0) {
+        mixedBlockCount++;
+      } else if (termCount != 0) {
+        termsOnlyBlockCount++;
+      } else if (subBlockCount != 0) {
+        subBlocksOnlyBlockCount++;
+      } else {
+        throw new IllegalStateException();
+      }
+      endBlockCount++;
+      final long otherBytes = frame.fpEnd - frame.fp - frame.suffixesReader.length() - frame.statsReader.length();
+      assert otherBytes > 0 : "otherBytes=" + otherBytes + " frame.fp=" + frame.fp + " frame.fpEnd=" + frame.fpEnd;
+      totalBlockOtherBytes += otherBytes;
+    }
+
+    void term(BytesRef term) {
+      totalTermBytes += term.length;
+    }
+
+    void finish() {
+      assert startBlockCount == endBlockCount: "startBlockCount=" + startBlockCount + " endBlockCount=" + endBlockCount;
+      assert totalBlockCount == floorSubBlockCount + nonFloorBlockCount: "floorSubBlockCount=" + floorSubBlockCount + " nonFloorBlockCount=" + nonFloorBlockCount + " totalBlockCount=" + totalBlockCount;
+      assert totalBlockCount == mixedBlockCount + termsOnlyBlockCount + subBlocksOnlyBlockCount: "totalBlockCount=" + totalBlockCount + " mixedBlockCount=" + mixedBlockCount + " subBlocksOnlyBlockCount=" + subBlocksOnlyBlockCount + " termsOnlyBlockCount=" + termsOnlyBlockCount;
+    }
+
+    @Override
+    public String toString() {
+      final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+      final PrintStream out = new PrintStream(bos);
+      
+      out.println("  index FST:");
+      out.println("    " + indexNodeCount + " nodes");
+      out.println("    " + indexArcCount + " arcs");
+      out.println("    " + indexNumBytes + " bytes");
+      out.println("  terms:");
+      out.println("    " + totalTermCount + " terms");
+      out.println("    " + totalTermBytes + " bytes" + (totalTermCount != 0 ? " (" + String.format("%.1f", ((double) totalTermBytes)/totalTermCount) + " bytes/term)" : ""));
+      out.println("  blocks:");
+      out.println("    " + totalBlockCount + " blocks");
+      out.println("    " + termsOnlyBlockCount + " terms-only blocks");
+      out.println("    " + subBlocksOnlyBlockCount + " sub-block-only blocks");
+      out.println("    " + mixedBlockCount + " mixed blocks");
+      out.println("    " + floorBlockCount + " floor blocks");
+      out.println("    " + (totalBlockCount-floorSubBlockCount) + " non-floor blocks");
+      out.println("    " + floorSubBlockCount + " floor sub-blocks");
+      out.println("    " + totalBlockSuffixBytes + " term suffix bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : ""));
+      out.println("    " + totalBlockStatsBytes + " term stats bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockStatsBytes)/totalBlockCount) + " stats-bytes/block)" : ""));
+      out.println("    " + totalBlockOtherBytes + " other bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockOtherBytes)/totalBlockCount) + " other-bytes/block)" : ""));
+      if (totalBlockCount != 0) {
+        out.println("    by prefix length:");
+        int total = 0;
+        for(int prefix=0;prefix<blockCountByPrefixLen.length;prefix++) {
+          final int blockCount = blockCountByPrefixLen[prefix];
+          total += blockCount;
+          if (blockCount != 0) {
+            out.println("      " + String.format("%2d", prefix) + ": " + blockCount);
+          }
+        }
+        assert totalBlockCount == total;
+      }
+
+      return bos.toString();
+    }
+  }
+
+  final Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
+  final BytesRef NO_OUTPUT = fstOutputs.getNoOutput();
+
+  public final class FieldReader extends Terms implements Closeable {
+    final long numTerms;
+    final FieldInfo fieldInfo;
+    final long sumTotalTermFreq;
+    final long sumDocFreq;
+    final long indexStartFP;
+    final long rootBlockFP;
+    final BytesRef rootCode;
+    private FST<BytesRef> index;
+
+    //private boolean DEBUG;
+
+    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, long indexStartFP, IndexInput indexIn) throws IOException {
+      assert numTerms > 0;
+      this.fieldInfo = fieldInfo;
+      //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
+      this.numTerms = numTerms;
+      this.sumTotalTermFreq = sumTotalTermFreq; 
+      this.sumDocFreq = sumDocFreq; 
+      this.indexStartFP = indexStartFP;
+      this.rootCode = rootCode;
+      // if (DEBUG) {
+      //   System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
+      // }
+
+      rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+
+      if (indexIn != null) {
+        final IndexInput clone = (IndexInput) indexIn.clone();
+        //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name);
+        clone.seek(indexStartFP);
+        index = new FST<BytesRef>(clone, ByteSequenceOutputs.getSingleton());
+        
+        /*
+        if (false) {
+          final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+          Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+          Util.toDot(index, w, false, false);
+          System.out.println("FST INDEX: SAVED to " + dotFileName);
+          w.close();
+        }
+        */
+      }
+    }
+
+    /** For debugging -- used by CheckIndex too*/
+    // TODO: maybe push this into Terms?
+    public Stats computeStats() throws IOException {
+      return new SegmentTermsEnum().computeBlockStats();
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public void close() {
+      super.close();
+    }
+    
+    @Override
+    public TermsEnum iterator() throws IOException {
+      return new SegmentTermsEnum();
+    }
+
+    @Override
+    public long getUniqueTermCount() {
+      return numTerms;
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      return sumTotalTermFreq;
+    }
+
+    @Override
+    public long getSumDocFreq() {
+      return sumDocFreq;
+    }
+
+    @Override
+    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+      if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+        throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+      }
+      return new IntersectEnum(compiled, startTerm);
+    }
+    
+    // NOTE: cannot seek!
+    private final class IntersectEnum extends TermsEnum {
+      private final IndexInput in;
+
+      private Frame[] stack;
+      
+      @SuppressWarnings("unchecked") private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
+
+      private final RunAutomaton runAutomaton;
+      private final CompiledAutomaton compiledAutomaton;
+
+      private Frame currentFrame;
+
+      private final BytesRef term = new BytesRef();
+
+      // TODO: can we share this with the frame in STE?
+      private final class Frame {
+        final int ord;
+        long fp;
+        long fpOrig;
+        long fpEnd;
+        long lastSubFP;
+
+        // State in automaton
+        int state;
+
+        int metaDataUpto;
+
+        byte[] suffixBytes = new byte[128];
+        final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
+
+        byte[] statBytes = new byte[64];
+        final ByteArrayDataInput statsReader = new ByteArrayDataInput();
+
+        byte[] floorData = new byte[32];
+        final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
+
+        // Length of prefix shared by all terms in this block
+        int prefix;
+
+        // Number of entries (term or sub-block) in this block
+        int entCount;
+
+        // Which term we will next read
+        int nextEnt;
+
+        // True if this block is either not a floor block,
+        // or, it's the last sub-block of a floor block
+        boolean isLastInFloor;
+
+        // True if all entries are terms
+        boolean isLeafBlock;
+
+        int numFollowFloorBlocks;
+        int nextFloorLabel;
+        
+        Transition[] transitions;
+        int curTransitionMax;
+        int transitionIndex;
+
+        FST.Arc<BytesRef> arc;
+
+        final BlockTermState termState;
+
+        // Cumulative output so far
+        BytesRef outputPrefix;
+
+        private int startBytePos;
+        private int suffix;
+
+        public Frame(int ord) throws IOException {
+          this.ord = ord;
+          termState = postingsReader.newTermState();
+          termState.totalTermFreq = -1;
+        }
+
+        void loadNextFloorBlock() throws IOException {
+          assert numFollowFloorBlocks > 0;
+          //if (DEBUG) System.out.println("    loadNextFoorBlock trans=" + transitions[transitionIndex]);
+
+          do {
+            fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+            numFollowFloorBlocks--;
+            // if (DEBUG) System.out.println("    skip floor block2!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+            if (numFollowFloorBlocks != 0) {
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+            } else {
+              nextFloorLabel = 256;
+            }
+            // if (DEBUG) System.out.println("    nextFloorLabel=" + (char) nextFloorLabel);
+          } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin());
+
+          load(null);
+        }
+
+        public void setState(int state) {
+          this.state = state;
+          transitionIndex = 0;
+          transitions = compiledAutomaton.sortedTransitions[state];
+          if (transitions.length != 0) {
+            curTransitionMax = transitions[0].getMax();
+          } else {
+            curTransitionMax = -1;
+          }
+        }
+
+        void load(BytesRef frameIndexData) throws IOException {
+
+          // if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
+
+          if (frameIndexData != null && transitions.length != 0) {
+            // Floor frame
+            if (floorData.length < frameIndexData.length) {
+              this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
+            }
+            System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
+            floorDataReader.reset(floorData, 0, frameIndexData.length);
+            // Skip first long -- has redundant fp, hasTerms
+            // flag, isFloor flag
+            final long code = floorDataReader.readVLong();
+            if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
+              numFollowFloorBlocks = floorDataReader.readVInt();
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+              // if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+
+              // If current state is accept, we must process
+              // first block in case it has empty suffix:
+              if (!runAutomaton.isAccept(state)) {
+                // Maybe skip floor blocks:
+                while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
+                  fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+                  numFollowFloorBlocks--;
+                  // if (DEBUG) System.out.println("    skip floor block!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+                  if (numFollowFloorBlocks != 0) {
+                    nextFloorLabel = floorDataReader.readByte() & 0xff;
+                  } else {
+                    nextFloorLabel = 256;
+                  }
+                }
+              }
+            }
+          }
+
+          in.seek(fp);
+          int code = in.readVInt();
+          entCount = code >>> 1;
+          assert entCount > 0;
+          isLastInFloor = (code & 1) != 0;
+
+          // term suffixes:
+          code = in.readVInt();
+          isLeafBlock = (code & 1) != 0;
+          int numBytes = code >>> 1;
+          // if (DEBUG) System.out.println("      entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes);
+          if (suffixBytes.length < numBytes) {
+            suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(suffixBytes, 0, numBytes);
+          suffixesReader.reset(suffixBytes, 0, numBytes);
+
+          // stats
+          numBytes = in.readVInt();
+          if (statBytes.length < numBytes) {
+            statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(statBytes, 0, numBytes);
+          statsReader.reset(statBytes, 0, numBytes);
+          metaDataUpto = 0;
+
+          termState.termBlockOrd = 0;
+          nextEnt = 0;
+          
+          postingsReader.readTermsBlock(in, fieldInfo, termState);
+
+          if (!isLastInFloor) {
+            // Sub-blocks of a single floor block are always
+            // written one after another -- tail recurse:
+            fpEnd = in.getFilePointer();
+          }
+        }
+
+        // TODO: maybe add scanToLabel; should give perf boost
+
+        public boolean next() {
+          return isLeafBlock ? nextLeaf() : nextNonLeaf();
+        }
+
+        // Decodes next entry; returns true if it's a sub-block
+        public boolean nextLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          suffix = suffixesReader.readVInt();
+          startBytePos = suffixesReader.getPosition();
+          suffixesReader.skipBytes(suffix);
+          return false;
+        }
+
+        public boolean nextNonLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          final int code = suffixesReader.readVInt();
+          suffix = code >>> 1;
+          startBytePos = suffixesReader.getPosition();
+          suffixesReader.skipBytes(suffix);
+          if ((code & 1) == 0) {
+            // A normal term
+            termState.termBlockOrd++;
+            return false;
+          } else {
+            // A sub-block; make sub-FP absolute:
+            lastSubFP = fp - suffixesReader.readVLong();
+            return true;
+          }
+        }
+
+        public int getTermBlockOrd() {
+          return isLeafBlock ? nextEnt : termState.termBlockOrd;
+        }
+
+        public void decodeMetaData() throws IOException {
+
+          // lazily catch up on metadata decode:
+          final int limit = getTermBlockOrd();
+          assert limit > 0;
+
+          // We must set/incr state.termCount because
+          // postings impl can look at this
+          termState.termBlockOrd = metaDataUpto;
+      
+          // TODO: better API would be "jump straight to term=N"???
+          while (metaDataUpto < limit) {
+
+            // TODO: we could make "tiers" of metadata, ie,
+            // decode docFreq/totalTF but don't decode postings
+            // metadata; this way caller could get
+            // docFreq/totalTF w/o paying decode cost for
+            // postings
+
+            // TODO: if docFreq were bulk decoded we could
+            // just skipN here:
+            termState.docFreq = statsReader.readVInt();
+            //if (DEBUG) System.out.println("    dF=" + state.docFreq);
+            if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+              termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
+              //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
+            }
+
+            postingsReader.nextTerm(fieldInfo, termState);
+            metaDataUpto++;
+            termState.termBlockOrd++;
+          }
+        }
+      }
+
+      private final BytesRef savedStartTerm;
+      
+      // TODO: in some cases we can filter by length?  eg
+      // regexp foo*bar must be at least length 6 bytes
+      public IntersectEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+        // if (DEBUG) {
+        //   System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef));
+        // }
+        runAutomaton = compiled.runAutomaton;
+        compiledAutomaton = compiled;
+        in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+        stack = new Frame[5];
+        for(int idx=0;idx<stack.length;idx++) {
+          stack[idx] = new Frame(idx);
+        }
+        for(int arcIdx=0;arcIdx<arcs.length;arcIdx++) {
+          arcs[arcIdx] = new FST.Arc<BytesRef>();
+        }
+
+        // TODO: if the automaon is "smallish" we really
+        // should use the terms index to seek at least to
+        // the initial term and likely to subsequent terms
+        // (or, maybe just fallback to ATE for such cases).
+        // Else the seek cost of loading the frames will be
+        // too costly.
+
+        final FST.Arc<BytesRef> arc = index.getFirstArc(arcs[0]);
+        // Empty string prefix must have an output in the index!
+        assert arc.isFinal();
+
+        // Special pushFrame since it's the first one:
+        final Frame f = stack[0];
+        f.fp = f.fpOrig = rootBlockFP;
+        f.prefix = 0;
+        f.setState(runAutomaton.getInitialState());
+        f.arc = arc;
+        f.outputPrefix = arc.output;
+        f.load(rootCode);
+
+        // for assert:
+        savedStartTerm = startTerm == null ? null : new BytesRef(startTerm);
+
+        currentFrame = f;
+        if (startTerm != null) {
+          seekToStartTerm(startTerm);
+        }
+      }
+
+      @Override
+      public TermState termState() throws IOException {
+        currentFrame.decodeMetaData();
+        return (TermState) currentFrame.termState.clone();
+      }
+
+      private Frame getFrame(int ord) throws IOException {
+        if (ord >= stack.length) {
+          final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(stack, 0, next, 0, stack.length);
+          for(int stackOrd=stack.length;stackOrd<next.length;stackOrd++) {
+            next[stackOrd] = new Frame(stackOrd);
+          }
+          stack = next;
+        }
+        assert stack[ord].ord == ord;
+        return stack[ord];
+      }
+
+      private FST.Arc<BytesRef> getArc(int ord) {
+        if (ord >= arcs.length) {
+          @SuppressWarnings("unchecked") final FST.Arc<BytesRef>[] next = new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(arcs, 0, next, 0, arcs.length);
+          for(int arcOrd=arcs.length;arcOrd<next.length;arcOrd++) {
+            next[arcOrd] = new FST.Arc<BytesRef>();
+          }
+          arcs = next;
+        }
+        return arcs[ord];
+      }
+
+      private Frame pushFrame(int state) throws IOException {
+        final Frame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord);
+        
+        f.fp = f.fpOrig = currentFrame.lastSubFP;
+        f.prefix = currentFrame.prefix + currentFrame.suffix;
+        // if (DEBUG) System.out.println("    pushFrame state=" + state + " prefix=" + f.prefix);
+        f.setState(state);
+
+        // Walk the arc through the index -- we only
+        // "bother" with this so we can get the floor data
+        // from the index and skip floor blocks when
+        // possible:
+        FST.Arc<BytesRef> arc = currentFrame.arc;
+        int idx = currentFrame.prefix;
+        assert currentFrame.suffix > 0;
+        BytesRef output = currentFrame.outputPrefix;
+        while (idx < f.prefix) {
+          final int target = term.bytes[idx] & 0xff;
+          // TODO: we could be more efficient for the next()
+          // case by using current arc as starting point,
+          // passed to findTargetArc
+          arc = index.findTargetArc(target, arc, getArc(1+idx));
+          assert arc != null;
+          output = fstOutputs.add(output, arc.output);
+          idx++;
+        }
+
+        f.arc = arc;
+        f.outputPrefix = output;
+        assert arc.isFinal();
+        f.load(fstOutputs.add(output, arc.nextFinalOutput));
+        return f;
+      }
+
+      @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        //if (DEBUG) System.out.println("BTIR.docFreq");
+        currentFrame.decodeMetaData();
+        //if (DEBUG) System.out.println("  return " + currentFrame.termState.docFreq);
+        return currentFrame.termState.docFreq;
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        currentFrame.decodeMetaData();
+        return currentFrame.termState.totalTermFreq;
+      }
+
+      @Override
+      public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+        currentFrame.decodeMetaData();
+        return postingsReader.docs(fieldInfo, currentFrame.termState, skipDocs, reuse);
+      }
+
+      @Override
+      public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+        if (fieldInfo.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+          return null;
+        } else {
+          currentFrame.decodeMetaData();
+          return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse);
+        }
+      }
+
+      private int getState() {
+        int state = currentFrame.state;
+        for(int idx=0;idx<currentFrame.suffix;idx++) {
+          state = runAutomaton.step(state,  currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
+          assert state != -1;
+        }
+        return state;
+      }
+
+      // NOTE: specialized to only doing the first-time
+      // seek, but we could generalize it to allow
+      // arbitrary seekExact/Ceil.  Note that this is a
+      // seekFloor!
+      private void seekToStartTerm(BytesRef target) throws IOException {
+        //if (DEBUG) System.out.println("seek to startTerm=" + target.utf8ToString());
+        assert currentFrame.ord == 0;
+        if (term.length < target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, target.length);
+        }
+        FST.Arc<BytesRef> arc = arcs[0];
+        assert arc == currentFrame.arc;
+
+        for(int idx=0;idx<=target.length;idx++) {
+
+          boolean lastIsSubBlock = false;
+
+          while (true) {
+            final int savePos = currentFrame.suffixesReader.getPosition();
+            final int saveStartBytePos = currentFrame.startBytePos;
+            final int saveSuffix = currentFrame.suffix;
+            final long saveLastSubFP = currentFrame.lastSubFP;
+            final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
+
+            final boolean isSubBlock = currentFrame.next();
+
+            //if (DEBUG) System.out.println("    cycle ent=" + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") prefix=" + currentFrame.prefix + " suffix=" + currentFrame.suffix);
+            term.length = currentFrame.prefix + currentFrame.suffix;
+            if (term.bytes.length < term.length) {
+              term.bytes = ArrayUtil.grow(term.bytes, term.length);
+            }
+            System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+
+            if (isSubBlock && target.startsWith(term)) {
+              // Recurse
+              currentFrame = pushFrame(getState());
+              break;
+            } else {
+              final int cmp = term.compareTo(target);
+              if (cmp < 0) {
+                if (currentFrame.nextEnt == currentFrame.entCount) {
+                  if (!currentFrame.isLastInFloor) {
+                    //if (DEBUG) System.out.println("  load floorBlock");
+                    currentFrame.loadNextFloorBlock();
+                    continue;
+                  } else {
+                    //if (DEBUG) System.out.println("  return term=" + brToString(term));
+                    return;
+                  }
+                }
+                lastIsSubBlock = isSubBlock;
+                continue;
+              } else if (cmp == 0) {
+                //if (DEBUG) System.out.println("  return term=" + brToString(term));
+                return;
+              } else {
+                // Fallback to prior entry: the semantics of
+                // this method is that the first call to
+                // next() will return the term after the
+                // requested term
+                currentFrame.nextEnt--;
+                currentFrame.lastSubFP = saveLastSubFP;
+                currentFrame.startBytePos = saveStartBytePos;
+                currentFrame.suffix = saveSuffix;
+                currentFrame.suffixesReader.setPosition(savePos);
+                currentFrame.termState.termBlockOrd = saveTermBlockOrd;
+                System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+                term.length = currentFrame.prefix + currentFrame.suffix;
+                if (lastIsSubBlock) {
+                  // Recurse
+                  currentFrame = pushFrame(getState());
+                  break;
+                } else {
+                  //if (DEBUG) System.out.println("  fallback return term=" + brToString(term) + " curFrame.nextEnt=" + currentFrame.nextEnt);
+                  return;
+                }
+              }
+            }
+          }
+        }
+
+        assert false;
+      }
+
+      @Override
+      public BytesRef next() throws IOException {
+
+        // if (DEBUG) {
+        //   System.out.println("\nintEnum.next seg=" + segment);
+        //   System.out.println("  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+        // }
+
+        nextTerm:
+        while(true) {
+          // Pop finished frames
+          while (currentFrame.nextEnt == currentFrame.entCount) {
+            if (!currentFrame.isLastInFloor) {
+              //if (DEBUG) System.out.println("    next-floor-block");
+              currentFrame.loadNextFloorBlock();
+              //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+            } else {
+              //if (DEBUG) System.out.println("  pop frame");
+              if (currentFrame.ord == 0) {
+                return null;
+              }
+              final long lastFP = currentFrame.fpOrig;
+              currentFrame = stack[currentFrame.ord-1];
+              assert currentFrame.lastSubFP == lastFP;
+              //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+            }
+          }
+
+          final boolean isSubBlock = currentFrame.next();
+          // if (DEBUG) {
+          //   final BytesRef suffixRef = new BytesRef();
+          //   suffixRef.bytes = currentFrame.suffixBytes;
+          //   suffixRef.offset = currentFrame.startBytePos;
+          //   suffixRef.length = currentFrame.suffix;
+          //   System.out.println("    " + (isSubBlock ? "sub-block" : "term") + " " + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") suffix=" + brToString(suffixRef));
+          // }
+
+          if (currentFrame.suffix != 0) {
+            final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
+            while (label > currentFrame.curTransitionMax) {
+              if (currentFrame.transitionIndex >= currentFrame.transitions.length-1) {
+                // Stop processing this frame -- no further
+                // matches are possible because we've moved
+                // beyond what the max transition will allow
+                //if (DEBUG) System.out.println("      break: trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]));
+
+                // sneaky!  forces a pop above
+                currentFrame.isLastInFloor = true;
+                currentFrame.nextEnt = currentFrame.entCount;
+                continue nextTerm;
+              }
+              currentFrame.transitionIndex++;
+              currentFrame.curTransitionMax = currentFrame.transitions[currentFrame.transitionIndex].getMax();
+              //if (DEBUG) System.out.println("      next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
+            }
+          }
+
+          // First test the common suffix, if set:
+          if (compiledAutomaton.commonSuffixRef != null && !isSubBlock) {
+            final int termLen = currentFrame.prefix + currentFrame.suffix;
+            if (termLen < compiledAutomaton.commonSuffixRef.length) {
+              // No match
+              // if (DEBUG) {
+              //   System.out.println("      skip: common suffix length");
+              // }
+              continue nextTerm;
+            }
+
+            final byte[] suffixBytes = currentFrame.suffixBytes;
+            final byte[] commonSuffixBytes = compiledAutomaton.commonSuffixRef.bytes;
+
+            final int lenInPrefix = compiledAutomaton.commonSuffixRef.length - currentFrame.suffix;
+            assert compiledAutomaton.commonSuffixRef.offset == 0;
+            int suffixBytesPos;
+            int commonSuffixBytesPos = 0;
+
+            if (lenInPrefix > 0) {
+              // A prefix of the common suffix overlaps with
+              // the suffix of the block prefix so we first
+              // test whether the prefix part matches:
+              final byte[] termBytes = term.bytes;
+              int termBytesPos = currentFrame.prefix - lenInPrefix;
+              assert termBytesPos >= 0;
+              final int termBytesPosEnd = currentFrame.prefix;
+              while (termBytesPos < termBytesPosEnd) {
+                if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+                  // if (DEBUG) {
+                  //   System.out.println("      skip: common suffix mismatch (in prefix)");
+                  // }
+                  continue nextTerm;
+                }
+              }
+              suffixBytesPos = currentFrame.startBytePos;
+            } else {
+              suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length;
+            }
+
+            // Test overlapping suffix part:
+            final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length;
+            while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
+              if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+                // if (DEBUG) {
+                //   System.out.println("      skip: common suffix mismatch");
+                // }
+                continue nextTerm;
+              }
+            }
+          }
+
+          // TODO: maybe we should do the same linear test
+          // that AutomatonTermsEnum does, so that if we
+          // reach a part of the automaton where .* is
+          // "temporarily" accepted, we just blindly .next()
+          // until the limit
+
+          // See if the term prefix matches the automaton:
+          int state = currentFrame.state;
+          for (int idx=0;idx<currentFrame.suffix;idx++) {
+            state = runAutomaton.step(state,  currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
+            if (state == -1) {
+              // No match
+              //System.out.println("    no s=" + state);
+              continue nextTerm;
+            } else {
+              //System.out.println("    c s=" + state);
+            }
+          }
+
+          if (isSubBlock) {
+            // Match!  Recurse:
+            //if (DEBUG) System.out.println("      sub-block match to state=" + state + "; recurse fp=" + currentFrame.lastSubFP);
+            copyTerm();
+            currentFrame = pushFrame(state);
+            //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+          } else if (runAutomaton.isAccept(state)) {
+            copyTerm();
+            //if (DEBUG) System.out.println("      term match to state=" + state + "; return term=" + brToString(term));
+            assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
+            return term;
+          } else {
+            //System.out.println("    no s=" + state);
+          }
+        }
+      }
+
+      private void copyTerm() {
+        //System.out.println("      copyTerm cur.prefix=" + currentFrame.prefix + " cur.suffix=" + currentFrame.suffix + " first=" + (char) currentFrame.suffixBytes[currentFrame.startBytePos]);
+        final int len = currentFrame.prefix + currentFrame.suffix;
+        if (term.bytes.length < len) {
+          term.bytes = ArrayUtil.grow(term.bytes, len);
+        }
+        System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+        term.length = len;
+      }
+
+      @Override
+      public Comparator<BytesRef> getComparator() {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      }
+
+      @Override
+      public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void seekExact(long ord) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long ord() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    // Iterates through terms in this field
+    private final class SegmentTermsEnum extends TermsEnum {
+      private final IndexInput in;
+
+      private Frame[] stack;
+      private final Frame staticFrame;
+      private Frame currentFrame;
+      private boolean termExists;
+
+      private int targetBeforeCurrentLength;
+
+      private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
+
+      // What prefix of the current term was present in the index:
+      private int validIndexPrefix;
+
+      // assert only:
+      private boolean eof;
+
+      final BytesRef term = new BytesRef();
+
+      @SuppressWarnings("unchecked") private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];      
+
+      public SegmentTermsEnum() throws IOException {
+        //if (DEBUG) System.out.println("BTTR.init seg=" + segment);
+        in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+        stack = new Frame[5];
+        for(int stackOrd=0;stackOrd<stack.length;stackOrd++) {
+          stack[stackOrd] = new Frame(stackOrd);
+        }
+        // Used to hold seek by TermState, or cached seek
+        staticFrame = new Frame(-1);
+
+        for(int arcIdx=0;arcIdx<arcs.length;arcIdx++) {
+          arcs[arcIdx] = new FST.Arc<BytesRef>();
+        }
+
+        // Init w/ root block; don't use index since it may
+        // not (and need not) have been loaded
+        //final FST.Arc<BytesRef> arc = index.getFirstArc(arcs[0]);
+
+        // Empty string prefix must have an output in the index!
+        //assert arc.isFinal();
+
+        currentFrame = staticFrame;
+        final FST.Arc<BytesRef> arc;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+        currentFrame = pushFrame(arc, rootCode, 0);
+        currentFrame.loadBlock();
+        validIndexPrefix = 0;
+        // if (DEBUG) {
+        //   System.out.println("init frame state " + currentFrame.ord);
+        //   printSeekState();
+        // }
+
+        //System.out.println();
+        // computeBlockStats().print(System.out);
+      }
+
+      /** Runs next() through the entire terms dict,
+       *  computing aggregate statistics. */
+      public Stats computeBlockStats() throws IOException {
+
+        Stats stats = new Stats(segment, fieldInfo.name);
+        if (index != null) {
+          stats.indexNodeCount = index.getNodeCount();
+          stats.indexArcCount = index.getArcCount();
+          stats.indexNumBytes = index.sizeInBytes();
+        }
+        
+        currentFrame = staticFrame;
+        FST.Arc<BytesRef> arc;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+
+        // Empty string prefix must have an output in the
+        // index!
+        currentFrame = pushFrame(arc, rootCode, 0);
+        currentFrame.fpOrig = currentFrame.fp;
+        currentFrame.loadBlock();
+        validIndexPrefix = 0;
+
+        stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+
+        allTerms:
+        while (true) {
+
+          // Pop finished blocks
+          while (currentFrame.nextEnt == currentFrame.entCount) {
+            stats.endBlock(currentFrame);
+            if (!currentFrame.isLastInFloor) {
+              currentFrame.loadNextFloorBlock();
+              stats.startBlock(currentFrame, true);
+            } else {
+              if (currentFrame.ord == 0) {
+                break allTerms;
+              }
+              final long lastFP = currentFrame.fpOrig;
+              currentFrame = stack[currentFrame.ord-1];
+              assert lastFP == currentFrame.lastSubFP;
+              // if (DEBUG) {
+              //   System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
+              // }
+            }
+          }
+
+          while(true) {
+            if (currentFrame.next()) {
+              // Push to new block:
+              currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+              currentFrame.fpOrig = currentFrame.fp;
+              // This is a "next" frame -- even if it's
+              // floor'd we must pretend it isn't so we don't
+              // try to scan to the right floor frame:
+              currentFrame.isFloor = false;
+              //currentFrame.hasTerms = true;
+              currentFrame.loadBlock();
+              stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+            } else {
+              stats.term(term);
+              break;
+            }
+          }
+        }
+
+        stats.finish();
+
+        // Put root frame back:
+        currentFrame = staticFrame;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+        currentFrame = pushFrame(arc, rootCode, 0);
+        currentFrame.rewind();
+        currentFrame.loadBlock();
+        validIndexPrefix = 0;
+        term.length = 0;
+
+        return stats;
+      }
+
+      private Frame getFrame(int ord) throws IOException {
+        if (ord >= stack.length) {
+          final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(stack, 0, next, 0, stack.length);
+          for(int stackOrd=stack.length;stackOrd<next.length;stackOrd++) {
+            next[stackOrd] = new Frame(stackOrd);
+          }
+          stack = next;
+        }
+        assert stack[ord].ord == ord;
+        return stack[ord];
+      }
+
+      private FST.Arc<BytesRef> getArc(int ord) {
+        if (ord >= arcs.length) {
+          @SuppressWarnings("unchecked") final FST.Arc<BytesRef>[] next = new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(arcs, 0, next, 0, arcs.length);
+          for(int arcOrd=arcs.length;arcOrd<next.length;arcOrd++) {
+            next[arcOrd] = new FST.Arc<BytesRef>();
+          }
+          arcs = next;
+        }
+        return arcs[ord];
+      }
+
+      @Override
+      public Comparator<BytesRef> getComparator() {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      }
+
+      // Pushes a frame we seek'd to
+      Frame pushFrame(FST.Arc<BytesRef> arc, BytesRef frameData, int length) throws IOException {
+        scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
+        final long code = scratchReader.readVLong();
+        final long fpSeek = code >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+        final Frame f = getFrame(1+currentFrame.ord);
+        f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
+        f.hasTermsOrig = f.hasTerms;
+        f.isFloor = (code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
+        if (f.isFloor) {
+          f.setFloorData(scratchReader, frameData);
+        }
+        pushFrame(arc, fpSeek, length);
+
+        return f;
+      }
+
+      // Pushes next'd frame or seek'd frame; we later
+      // lazy-load the frame only when needed
+      Frame pushFrame(FST.Arc<BytesRef> arc, long fp, int length) throws IOException {
+        final Frame f = getFrame(1+currentFrame.ord);
+        f.arc = arc;
+        if (f.fpOrig == fp && f.nextEnt != -1) {
+          //if (DEBUG) System.out.println("      push reused frame ord=" + f.ord + " fp=" + f.fp + " isFloor?=" + f.isFloor + " hasTerms=" + f.hasTerms + " pref=" + term + " nextEnt=" + f.nextEnt + " targetBeforeCurrentLength=" + targetBeforeCurrentLength + " term.length=" + term.length + " vs prefix=" + f.prefix);
+          if (f.prefix > targetBeforeCurrentLength) {
+            f.rewind();
+          } else {
+            // if (DEBUG) {
+            //   System.out.println("        skip rewind!");
+            // }
+          }
+          assert length == f.prefix;
+        } else {
+          f.nextEnt = -1;
+          f.prefix = length;
+          f.state.termBlockOrd = 0;
+          f.fpOrig = f.fp = fp;
+          f.lastSubFP = -1;
+          // if (DEBUG) {
+          //   final int sav = term.length;
+          //   term.length = length;
+          //   System.out.println("      push new frame ord=" + f.ord + " fp=" + f.fp + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " pref=" + brToString(term));
+          //   term.length = sav;
+          // }
+        }
+
+        return f;
+      }
+
+      // asserts only
+      private boolean clearEOF() {
+        eof = false;
+        return true;
+      }
+
+      // asserts only
+      private boolean setEOF() {
+        eof = true;
+        return true;
+      }
+
+      @Override
+      public boolean seekExact(final BytesRef target, final boolean useCache) throws IOException {
+
+        if (index == null) {
+          throw new IllegalStateException("terms index was not loaded");
+        }
+
+        if (term.bytes.length <= target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+        }
+
+        assert clearEOF();
+
+        // if (DEBUG) {
+        //   System.out.println("\nBTTR.seekExact seg=" + segment + " target=" + fieldInfo.name + ":" + brToString(target) + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=" + validIndexPrefix);
+        //   printSeekState();
+        // }
+
+        FST.Arc<BytesRef> arc;
+        int targetUpto;
+        BytesRef output;
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        if (currentFrame != staticFrame) {
+
+          // We are already seek'd; find the common
+          // prefix of new seek term vs current term and
+          // re-use the corresponding seek state.  For
+          // example, if app first seeks to foobar, then
+          // seeks to foobaz, we can re-use the seek state
+          // for the first 5 bytes.
+
+          // if (DEBUG) {
+          //   System.out.println("  re-use current seek state validIndexPrefix=" + validIndexPrefix);
+          // }
+
+          arc = arcs[0];
+          assert arc.isFinal();
+          output = arc.output;
+          targetUpto = 0;
+          
+          Frame lastFrame = stack[0];
+          assert validIndexPrefix <= term.length;
+
+          final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+          int cmp = 0;
+
+          // TODO: reverse vLong byte order for better FST
+          // prefix output sharing
+
+          // First compare up to valid seek frames:
+          while (targetUpto < targetLimit) {
+            cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+            // if (DEBUG) {
+            //   System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
+            // }
+            if (cmp != 0) {
+              break;
+            }
+            arc = arcs[1+targetUpto];
+            //if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) {
+            //System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF));
+            //}
+            assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+            if (arc.isFinal()) {
+              lastFrame = stack[1+lastFrame.ord];
+            }
+            targetUpto++;
+          }
+
+          if (cmp == 0) {
+            final int targetUptoMid = targetUpto;
+
+            // Second compare the rest of the term, but
+            // don't save arc/output/frame; we only do this
+            // to find out if the target term is before,
+            // equal or after the current term
+            final int targetLimit2 = Math.min(target.length, term.length);
+            while (targetUpto < targetLimit2) {
+              cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+              // if (DEBUG) {
+              //   System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+              // }
+              if (cmp != 0) {
+                break;
+              }
+              targetUpto++;
+            }
+
+            if (cmp == 0) {
+              cmp = term.length - target.length;
+            }
+            targetUpto = targetUptoMid;
+          }
+
+          if (cmp < 0) {
+            // Common case: target term is after current
+            // term, ie, app is seeking multiple terms
+            // in sorted order
+            // if (DEBUG) {
+            //   System.out.println("  target is after current (shares prefixLen=" + targetUpto + "); frame.ord=" + lastFrame.ord);
+            // }
+            currentFrame = lastFrame;
+
+          } else if (cmp > 0) {
+            // Uncommon case: target term
+            // is before current term; this means we can
+            // keep the currentFrame but we must rewind it
+            // (so we scan from the start)
+            targetBeforeCurrentLength = 0;
+            // if (DEBUG) {
+            //   System.out.println("  target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+            // }
+            currentFrame = lastFrame;
+            currentFrame.rewind();
+          } else {
+            // Target is exactly the same as current term
+            assert term.length == target.length;
+            if (termExists) {
+              // if (DEBUG) {
+              //   System.out.println("  target is same as current; return true");
+              // }
+              return true;
+            } else {
+              // if (DEBUG) {
+              //   System.out.println("  target is same as current but term doesn't exist");
+              // }
+            }
+            //validIndexPrefix = currentFrame.depth;
+            //term.length = target.length;
+            //return termExists;
+          }
+
+        } else {
+
+          targetBeforeCurrentLength = -1;
+          arc = index.getFirstArc(arcs[0]);
+
+          // Empty string prefix must have an output (block) in the index!
+          assert arc.isFinal();
+          assert arc.output != null;
+
+          // if (DEBUG) {
+          //   System.out.println("    no seek state; push root frame");
+          // }
+
+          output = arc.output;
+
+          currentFrame = staticFrame;
+
+          //term.length = 0;
+          targetUpto = 0;
+          currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+        }
+
+        // if (DEBUG) {
+        //   System.out.println("  start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+        // }
+
+        while (targetUpto < target.length) {
+
+          final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+          final FST.Arc<BytesRef> nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto));
+
+          if (nextArc == null) {
+
+            // Index is exhausted
+            // if (DEBUG) {
+            //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+            // }
+            
+            validIndexPrefix = currentFrame.prefix;
+            //validIndexPrefix = targetUpto;
+
+            currentFrame.scanToFloorFrame(target);
+
+            if (!currentFrame.hasTerms) {
+              termExists = false;
+              term.bytes[targetUpto] = (byte) targetLabel;
+              term.length = 1+targetUpto;
+              // if (DEBUG) {
+              //   System.out.println("  FAST NOT_FOUND term=" + brToString(term));
+              // }
+              return false;
+            }
+
+            currentFrame.loadBlock();
+
+            final SeekStatus result = currentFrame.scanToTerm(target, true);            
+            if (result == SeekStatus.FOUND) {
+              // if (DEBUG) {
+              //   System.out.println("  return FOUND term=" + term.utf8ToString() + " " + term);
+              // }
+              return true;
+            } else {
+              // if (DEBUG) {
+              //   System.out.println("  got " + result + "; return NOT_FOUND term=" + brToString(term));
+              // }
+              return false;
+            }
+          } else {
+            // Follow this arc
+            arc = nextArc;
+            term.bytes[targetUpto] = (byte) targetLabel;
+            // Aggregate output as we go:
+            assert arc.output != null;
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+
+            // if (DEBUG) {
+            //   System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+            // }
+            targetUpto++;
+
+            if (arc.isFinal()) {
+              //if (DEBUG) System.out.println("    arc is final!");
+              currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+              //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+            }
+          }
+        }
+
+        //validIndexPrefix = targetUpto;
+        validIndexPrefix = currentFrame.prefix;
+
+        currentFrame.scanToFloorFrame(target);
+
+        // Target term is entirely contained in the index:
+        if (!currentFrame.hasTerms) {
+          termExists = false;
+          term.length = targetUpto;
+          // if (DEBUG) {
+          //   System.out.println("  FAST NOT_FOUND term=" + brToString(term));
+          // }
+          return false;
+        }
+
+        currentFrame.loadBlock();
+
+        final SeekStatus result = currentFrame.scanToTerm(target, true);            
+        if (result == SeekStatus.FOUND) {
+          // if (DEBUG) {
+          //   System.out.println("  return FOUND term=" + term.utf8ToString() + " " + term);
+          // }
+          return true;
+        } else {
+          // if (DEBUG) {
+          //   System.out.println("  got result " + result + "; return NOT_FOUND term=" + term.utf8ToString());
+          // }
+
+          return false;
+        }
+      }
+
+      @Override
+      public SeekStatus seekCeil(final BytesRef target, final boolean useCache) throws IOException {
+        if (index == null) {
+          throw new IllegalStateException("terms index was not loaded");
+        }
+   
+        if (term.bytes.length <= target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+        }
+
+        assert clearEOF();
+
+        //if (DEBUG) {
+        //System.out.println("\nBTTR.seekCeil seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=  " + validIndexPrefix);
+        //printSeekState();
+        //}
+
+        FST.Arc<BytesRef> arc;
+        int targetUpto;
+        BytesRef output;
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        if (currentFrame != staticFrame) {
+
+          // We are already seek'd; find the common
+          // prefix of new seek term vs current term and
+          // re-use the corresponding seek state.  For
+          // example, if app first seeks to foobar, then
+          // seeks to foobaz, we can re-use the seek state
+          // for the first 5 bytes.
+
+          //if (DEBUG) {
+          //System.out.println("  re-use current seek state validIndexPrefix=" + validIndexPrefix);
+          //}
+
+          arc = arcs[0];
+          assert arc.isFinal();
+          output = arc.output;
+          targetUpto = 0;
+          
+          Frame lastFrame = stack[0];
+          assert validIndexPrefix <= term.length;
+
+          final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+          int cmp = 0;
+
+          // TOOD: we should write our vLong backwards (MSB
+          // first) to get better sharing from the FST
+
+          // First compare up to valid seek frames:
+          while (targetUpto < targetLimit) {
+            cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+            //if (DEBUG) {
+            //System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
+            //}
+            if (cmp != 0) {
+              break;
+            }
+            arc = arcs[1+targetUpto];
+            assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+            // TOOD: we could save the outputs in local
+            // byte[][] instead of making new objs ever
+            // seek; but, often the FST doesn't have any
+            // shared bytes (but this could change if we
+            // reverse vLong byte order)
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+            if (arc.isFinal()) {
+              lastFrame = stack[1+lastFrame.ord];
+            }
+            targetUpto++;
+          }
+
+
+          if (cmp == 0) {
+            final int targetUptoMid = targetUpto;
+            // Second compare the rest of the term, but
+            // don't save arc/output/frame:
+            final int targetLimit2 = Math.min(target.length, term.length);
+            while (targetUpto < targetLimit2) {
+              cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+              //if (DEBUG) {
+              //System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+              //}
+              if (cmp != 0) {
+                break;
+              }
+              targetUpto++;
+            }
+
+            if (cmp == 0) {
+              cmp = term.length - target.length;
+            }
+            targetUpto = targetUptoMid;
+          }
+
+          if (cmp < 0) {
+            // Common case: target term is after current
+            // term, ie, app is seeking multiple terms
+            // in sorted order
+            //if (DEBUG) {
+            //System.out.println("  target is after current (shares prefixLen=" + targetUpto + "); clear frame.scanned ord=" + lastFrame.ord);
+            //}
+            currentFrame = lastFrame;
+
+          } else if (cmp > 0) {
+            // Uncommon case: target term
+            // is before current term; this means we can
+            // keep the currentFrame but we must rewind it
+            // (so we scan from the start)
+            targetBeforeCurrentLength = 0;
+            //if (DEBUG) {
+            //System.out.println("  target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+            //}
+            currentFrame = lastFrame;
+            currentFrame.rewind();
+          } else {
+            // Target is exactly the same as current term
+            assert term.length == target.length;
+            if (termExists) {
+              //if (DEBUG) {
+              //System.out.println("  target is same as current; return FOUND");
+              //}
+              return SeekStatus.FOUND;
+            } else {
+              //if (DEBUG) {
+              //System.out.println("  target is same as current but term doesn't exist");
+              //}
+            }
+          }
+
+        } else {
+
+          targetBeforeCurrentLength = -1;
+          arc = index.getFirstArc(arcs[0]);
+
+          // Empty string prefix must have an output (block) in the index!
+          assert arc.isFinal();
+          assert arc.output != null;
+
+          //if (DEBUG) {
+          //System.out.println("    no seek state; push root frame");
+          //}
+
+          output = arc.output;
+
+          currentFrame = staticFrame;
+
+          //term.length = 0;
+          targetUpto = 0;
+          currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+        }
+
+        //if (DEBUG) {
+        //System.out.println("  start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord+1=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+        //}
+
+        while (targetUpto < target.length) {
+
+          final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+          final FST.Arc<BytesRef> nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto));
+
+          if (nextArc == null) {
+
+            // Index is exhausted
+            // if (DEBUG) {
+            //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+            // }
+            
+            validIndexPrefix = currentFrame.prefix;
+            //validIndexPrefix = targetUpto;
+
+            currentFrame.scanToFloorFrame(target);
+
+            currentFrame.loadBlock();
+
+            final SeekStatus result = currentFrame.scanToTerm(target, false);
+            if (result == SeekStatus.END) {
+              term.copy(target);
+              termExists = false;
+
+              if (next() != null) {
+                //if (DEBUG) {
+                //System.out.println("  return NOT_FOUND term=" + brToString(term) + " " + term);
+                //}
+                return SeekStatus.NOT_FOUND;
+              } else {
+                //if (DEBUG) {
+                //System.out.println("  return END");
+                //}
+                return SeekStatus.END;
+              }
+            } else {
+              //if (DEBUG) {
+              //System.out.println("  return " + result + " term=" + brToString(term) + " " + term);
+              //}
+              return result;
+            }
+          } else {
+            // Follow this arc
+            term.bytes[targetUpto] = (byte) targetLabel;
+            arc = nextArc;
+            // Aggregate output as we go:
+            assert arc.output != null;
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+
+            //if (DEBUG) {
+            //System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+            //}
+            targetUpto++;
+
+            if (arc.isFinal()) {
+              //if (DEBUG) System.out.println("    arc is final!");
+              currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+              //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+            }
+          }
+        }
+
+        //validIndexPrefix = targetUpto;
+        validIndexPrefix = currentFrame.prefix;
+
+        currentFrame.scanToFloorFrame(target);
+
+        currentFrame.loadBlock();
+
+        final SeekStatus result = currentFrame.scanToTerm(target, false);
+
+        if (result == SeekStatus.END) {
+          term.copy(target);
+          termExists = false;
+          if (next() != null) {
+            //if (DEBUG) {
+            //System.out.println("  return NOT_FOUND term=" + term.utf8ToString() + " " + term);
+            //}
+            return SeekStatus.NOT_FOUND;
+          } else {
+            //if (DEBUG) {
+            //System.out.println("  return END");
+            //}
+            return SeekStatus.END;
+          }
+        } else {
+          return result;
+        }
+      }
+
+      private void printSeekState() throws IOException {
+        if (currentFrame == staticFrame) {
+          System.out.println("  no prior seek");
+        } else {
+          System.out.println("  prior seek state:");
+          int ord = 0;
+          boolean isSeekFrame = true;
+          while(true) {
+            Frame f = getFrame(ord);
+            assert f != null;
+            final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
+            if (f.nextEnt == -1) {
+              System.out.println("    frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+            } else {
+              System.out.println("    frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+            }
+            if (index != null) {
+              assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
+              if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.bytes[f.prefix-1]&0xFF)) {
+                System.out.println("      broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.bytes[f.prefix-1]&0xFF));
+                throw new RuntimeException("seek state is broken");
+              }
+              BytesRef output = Util.get(index, prefix);
+              if (output == null) {
+                System.out.println("      broken seek state: prefix is not final in index");
+                throw new RuntimeException("seek state is broken");
+              } else if (isSeekFrame && !f.isFloor) {
+                final ByteArrayDataInput reader = new ByteArrayDataInput(output.bytes, output.offset, output.length);
+                final long codeOrig = reader.readVLong();
+                final long code = (f.fp << BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0);
+                if (codeOrig != code) {
+                  System.out.println("      broken seek state: output code=" + codeOrig + " doesn't match frame code=" + code);
+                  throw new RuntimeException("seek state is broken");
+                }
+              }
+            }
+            if (f == currentFrame) {
+              break;
+            }
+            if (f.prefix == validIndexPrefix) {
+              isSeekFrame = false;
+            }
+            ord++;
+          }
+        }
+      }
+
+      /* Decodes only the term bytes of the next term.  If caller then asks for
+         metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
+         decode all metadata up to the current term. */
+      @Override
+      public BytesRef next() throws IOException {
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        assert !eof;
+        //if (DEBUG) {
+        //System.out.println("\nBTTR.next seg=" + segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix);
+        //printSeekState();
+        //}
+
+        if (currentFrame == staticFrame) {
+          // If seek was previously called and the term was
+          // cached, or seek(TermState) was called, usually
+          // caller is just going to pull a D/&PEnum or get
+          // docFreq, etc.  But, if they then call next(),
+          // this method catches up all internal state so next()
+          // works properly:
+          //if (DEBUG) System.out.println("  re-seek to pending term=" + term.utf8ToString() + " " + term);
+          final boolean result = seekExact(term, false);
+          assert result;
+        }
+
+        // Pop finished blocks
+        while (currentFrame.nextEnt == currentFrame.entCount) {
+          if (!currentFrame.isLastInFloor) {
+            currentFrame.loadNextFloorBlock();
+          } else {
+            //if (DEBUG) System.out.println("  pop frame");
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              assert setEOF();
+              term.length = 0;
+              validIndexPrefix = 0;
+              currentFrame.rewind();
+              termExists = false;
+              return null;
+            }
+            final long lastFP = currentFrame.fpOrig;
+            currentFrame = stack[currentFrame.ord-1];
+
+            if (currentFrame.nextEnt == -1 || currentFrame.lastSubFP != lastFP) {
+              // We popped into a frame that's not loaded
+              // yet or not scan'd to the right entry
+              currentFrame.scanToFloorFrame(term);
+              currentFrame.loadBlock();
+              currentFrame.scanToSubBlock(lastFP);
+            }
+
+            // Note that the seek state (last seek) has been
+            // invalidated beyond this depth
+            validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix);
+            //if (DEBUG) {
+            //System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
+            //}
+          }
+        }
+
+        while(true) {
+          if (currentFrame.next()) {
+            // Push to new block:
+            //if (DEBUG) System.out.println("  push frame");
+            currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+            // This is a "next" frame -- even if it's
+            // floor'd we must pretend it isn't so we don't
+            // try to scan to the right floor frame:
+            currentFrame.isFloor = false;
+            //currentFrame.hasTerms = true;
+            currentFrame.loadBlock();
+          } else {
+            //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
+            return term;
+          }
+        }
+      }
+
+      @Override
+      public BytesRef term() {
+        assert !eof;
+        return term;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        assert !eof;
+        //if (DEBUG) System.out.println("BTR.docFreq");
+        currentFrame.decodeMetaData();
+        //if (DEBUG) System.out.println("  return " + currentFrame.state.docFreq);
+        return currentFrame.state.docFreq;
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        assert !eof;
+        currentFrame.decodeMetaData();
+        return currentFrame.state.totalTermFreq;
+      }
+
+      @Override
+      public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+        assert !eof;
+        //if (DEBUG) {
+        //System.out.println("BTTR.docs seg=" + segment);
+        //}
+        currentFrame.decodeMetaData();
+        //if (DEBUG) {
+        //System.out.println("  state=" + currentFrame.state);
+        //}
+        final DocsEnum docsEnum = postingsReader.docs(fieldInfo, currentFrame.state, skipDocs, reuse);
+
+        assert docsEnum != null;
+        return docsEnum;
+      }
+
+      @Override
+      public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+        assert !eof;
+        //System.out.println("BTR.d&p this=" + this);
+        if (fieldInfo.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+          return null;
+        } else {
+          currentFrame.decodeMetaData();
+          DocsAndPositionsEnum dpe = postingsReader.docsAndPositions(fieldInfo, currentFrame.state, skipDocs, reuse);
+          //System.out.println("  return d&pe=" + dpe);
+          return dpe;
+        }
+      }
+
+      @Override
+      public void seekExact(BytesRef target, TermState otherState) throws IOException {
+        // if (DEBUG) {
+        //   System.out.println("BTTR.seekExact termState seg=" + segment + " target=" + target.utf8ToString() + " " + target + " state=" + otherState);
+        // }
+        assert clearEOF();
+        if (target.compareTo(term) != 0 || !termExists) {
+          assert otherState != null && otherState instanceof BlockTermState;
+          currentFrame = staticFrame;
+          currentFrame.state.copyFrom(otherState);
+          term.copy(target);
+          currentFrame.metaDataUpto = currentFrame.getTermBlockOrd();
+          assert currentFrame.metaDataUpto > 0;
+          validIndexPrefix = 0;
+        } else {
+          // if (DEBUG) {
+          //   System.out.println("  skip seek: already on target state=" + currentFrame.state);
+          // }
+        }
+      }
+      
+      @Override
+      public TermState termState() throws IOException {
+        assert !eof;
+        currentFrame.decodeMetaData();
+        TermState ts = (TermState) currentFrame.state.clone();
+        //if (DEBUG) System.out.println("BTTR.termState seg=" + segment + " state=" + ts);
+        return ts;
+      }
+
+      @Override
+      public void seekExact(long ord) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long ord() {
+        throw new UnsupportedOperationException();
+      }
+
+      // Not static -- references term, postingsReader,
+      // fieldInfo, in
+      private final class Frame {
+        // Our index in stack[]:
+        final int ord;
+
+        boolean hasTerms;
+        boolean hasTermsOrig;
+        boolean isFloor;
+
+        FST.Arc<BytesRef> arc;
+
+        // File pointer where this block was loaded from
+        long fp;
+        long fpOrig;
+        long fpEnd;
+
+        byte[] suffixBytes = new byte[128];
+        final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
+
+        byte[] statBytes = new byte[64];
+        final ByteArrayDataInput statsReader = new ByteArrayDataInput();
+
+        byte[] floorData = new byte[32];
+        final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
+
+        // Length of prefix shared by all terms in this block
+        int prefix;
+
+        // Number of entries (term or sub-block) in this block
+        int entCount;
+
+        // Which term we will next read, or -1 if the block
+        // isn't loaded yet
+        int nextEnt;
+
+        // True if this block is either not a floor block,
+        // or, it's the last sub-block of a floor block
+        boolean isLastInFloor;
+
+        // True if all entries are terms
+        boolean isLeafBlock;
+
+        long lastSubFP;
+
+        int nextFloorLabel;
+        int numFollowFloorBlocks;
+
+        // Next term to decode metaData; we decode metaData
+        // lazily so that scanning to find the matching term is
+        // fast and only if you find a match and app wants the
+        // stats or docs/positions enums, will we decode the
+        // metaData
+        int metaDataUpto;
+
+        final BlockTermState state;
+
+        public Frame(int ord) throws IOException {
+          this.ord = ord;
+          state = postingsReader.newTermState();
+          state.totalTermFreq = -1;
+        }
+
+        public void setFloorData(ByteArrayDataInput in, BytesRef source) {
+          final int numBytes = source.length - (in.getPosition() - source.offset);
+          if (numBytes > floorData.length) {
+            floorData = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          System.arraycopy(source.bytes, source.offset+in.getPosition(), floorData, 0, numBytes);
+          floorDataReader.reset(floorData, 0, numBytes);
+          numFollowFloorBlocks = floorDataReader.readVInt();
+          nextFloorLabel = floorDataReader.readByte() & 0xff;
+          //if (DEBUG) {
+          //System.out.println("    setFloorData fpOrig=" + fpOrig + " bytes=" + new BytesRef(source.bytes, source.offset + in.getPosition(), numBytes) + " numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + toHex(nextFloorLabel));
+          //}
+        }
+
+        public int getTermBlockOrd() {
+          return isLeafBlock ? nextEnt : state.termBlockOrd;
+        }
+
+        void loadNextFloorBlock() throws IOException {
+          //if (DEBUG) {
+          //System.out.println("    loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
+          //}
+          assert arc == null || isFloor: "arc=" + arc + " isFloor=" + isFloor;
+          fp = fpEnd;
+          nextEnt = -1;
+          loadBlock();
+        }
+
+        /* Does initial decode of next block of terms; this
+           doesn't actually decode the docFreq, totalTermFreq,
+           postings details (frq/prx offset, etc.) metadata;
+           it just loads them as byte[] blobs which are then      
+           decoded on-demand if the metadata is ever requested
+           for any term in this block.  This enables terms-only
+           intensive consumes (eg certain MTQs, respelling) to
+           not pay the price of decoding metadata they won't
+           use. */
+        void loadBlock() throws IOException {
+
+          if (nextEnt != -1) {
+            // Already loaded
+            return;
+          }
+          //System.out.println("blc=" + blockLoadCount);
+
+          in.seek(fp);
+          int code = in.readVInt();
+          entCount = code >>> 1;
+          assert entCount > 0;
+          isLastInFloor = (code & 1) != 0;
+          assert arc == null || (isLastInFloor || isFloor);
+
+          // TODO: if suffixes were stored in random-access
+          // array structure, then we could do binary search
+          // instead of linear scan to find target term; eg
+          // we could have simple array of offsets
+
+          // term suffixes:
+          code = in.readVInt();
+          isLeafBlock = (code & 1) != 0;
+          int numBytes = code >>> 1;
+          if (suffixBytes.length < numBytes) {
+            suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(suffixBytes, 0, numBytes);
+          suffixesReader.reset(suffixBytes, 0, numBytes);
+
+          /*if (DEBUG) {
+            if (arc == null) {
+              System.out.println("    loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+            } else {
+              System.out.println("    loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+            }
+            }*/
+
+          // stats
+          numBytes = in.readVInt();
+          if (statBytes.length < numBytes) {
+            statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(statBytes, 0, numBytes);
+          statsReader.reset(statBytes, 0, numBytes);
+          metaDataUpto = 0;
+
+          state.termBlockOrd = 0;
+          nextEnt = 0;
+          lastSubFP = -1;
+
+          // TODO: we could skip this if !hasTerms; but
+          // that's rare so won't help much
+          postingsReader.readTermsBlock(in, fieldInfo, state);
+
+          // Sub-blocks of a single floor block are always
+          // written one after another -- tail recurse:
+          fpEnd = in.getFilePointer();
+          // if (DEBUG) {
+          //   System.out.println("      fpEnd=" + fpEnd);
+          // }
+        }
+
+        void rewind() throws IOException {
+
+          // Force reload:
+          fp = fpOrig;
+          nextEnt = -1;
+          hasTerms = hasTermsOrig;
+          if (isFloor) {
+            floorDataReader.rewind();
+            numFollowFloorBlocks = floorDataReader.readVInt();
+            nextFloorLabel = floorDataReader.readByte() & 0xff;
+          }
+
+          /*
+          //System.out.println("rewind");
+          // Keeps the block loaded, but rewinds its state:
+          if (nextEnt > 0 || fp != fpOrig) {
+            if (DEBUG) {
+              System.out.println("      rewind frame ord=" + ord + " fpOrig=" + fpOrig + " fp=" + fp + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " nextEnt=" + nextEnt + " prefixLen=" + prefix);
+            }
+            if (fp != fpOrig) {
+              fp = fpOrig;
+              nextEnt = -1;
+            } else {
+              nextEnt = 0;
+            }
+            hasTerms = hasTermsOrig;
+            if (isFloor) {
+              floorDataReader.rewind();
+              numFollowFloorBlocks = floorDataReader.readVInt();
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+            }
+            assert suffixBytes != null;
+            suffixesReader.rewind();
+            assert statBytes != null;
+            statsReader.rewind();
+            metaDataUpto = 0;
+            state.termBlockOrd = 0;
+            // TODO: skip this if !hasTerms?  Then postings
+            // impl wouldn't have to write useless 0 byte
+            postingsReader.resetTermsBlock(fieldInfo, state);
+            lastSubFP = -1;
+          } else if (DEBUG) {
+            System.out.println("      skip rewind fp=" + fp + " fpOrig=" + fpOrig + " nextEnt=" + nextEnt + " ord=" + ord);
+          }
+          */
+        }
+
+        public boolean next() {
+          return isLeafBlock ? nextLeaf() : nextNonLeaf();
+        }
+
+        // Decodes next entry; returns true if it's a sub-block
+        public boolean nextLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          suffix = suffixesReader.readVInt();
+          startBytePos = suffixesReader.getPosition();
+          term.length = prefix + suffix;
+          if (term.bytes.length < term.length) {
+            term.grow(term.length);
+          }
+          suffixesReader.readBytes(term.bytes, prefix, suffix);
+          // A normal term
+          termExists = true;
+          return false;
+        }
+
+        public boolean nextNonLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          final int code = suffixesReader.readVInt();
+          suffix = code >>> 1;
+          startBytePos = suffixesReader.getPosition();
+          term.length = prefix + suffix;
+          if (term.bytes.length < term.length) {
+            term.grow(term.length);
+          }
+          suffixesReader.readBytes(term.bytes, prefix, suffix);
+          if ((code & 1) == 0) {
+            // A normal term
+            termExists = true;
+            state.termBlockOrd++;
+            return false;
+          } else {
+            // A sub-block; make sub-FP absolute:
+            termExists = false;
+            lastSubFP = fp - suffixesReader.readVLong();
+            //if (DEBUG) {
+            //System.out.println("    lastSubFP=" + lastSubFP);
+            //}
+            return true;
+          }
+        }
+        
+        // TODO: make this array'd so we can do bin search?
+        // likely not worth it?  need to measure how many
+        // floor blocks we "typically" get
+        public void scanToFloorFrame(BytesRef target) {
+
+          if (!isFloor || target.length <= prefix) {
+            // if (DEBUG) {
+            //   System.out.println("    scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" + target.length + " vs prefix=" + prefix);
+            // }
+            return;
+          }
+
+          final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
+
+          // if (DEBUG) {
+          //   System.out.println("    scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" + toHex(targetLabel) + " vs nextFloorLabel=" + toHex(nextFloorLabel) + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+          // }
+
+          if (targetLabel < nextFloorLabel) {
+            // if (DEBUG) {
+            //   System.out.println("      already on correct block");
+            // }
+            return;
+          }
+
+          assert numFollowFloorBlocks != 0;
+
+          long newFP = fpOrig;
+          while (true) {
+            final long code = floorDataReader.readVLong();
+            newFP = fpOrig + (code >>> 1);
+            hasTerms = (code & 1) != 0;
+            // if (DEBUG) {
+            //   System.out.println("      label=" + toHex(nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
+            // }
+            
+            isLastInFloor = numFollowFloorBlocks == 1;
+            numFollowFloorBlocks--;
+
+            if (isLastInFloor) {
+              nextFloorLabel = 256;
+              // if (DEBUG) {
+              //   System.out.println("        stop!  last block nextFloorLabel=" + toHex(nextFloorLabel));
+              // }
+              break;
+            } else {
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+              if (targetLabel < nextFloorLabel) {
+                // if (DEBUG) {
+                //   System.out.println("        stop!  nextFloorLabel=" + toHex(nextFloorLabel));
+                // }
+                break;
+              }
+            }
+          }
+
+          if (newFP != fp) {
+            // Force re-load of the block:
+            // if (DEBUG) {
+            //   System.out.println("      force switch to fp=" + newFP + " oldFP=" + fp);
+            // }
+            nextEnt = -1;
+            fp = newFP;
+          } else {
+            // if (DEBUG) {
+            //   System.out.println("      stay on same fp=" + newFP);
+            // }
+          }
+        }
+    
+        public void decodeMetaData() throws IOException {
+
+          //if (DEBUG) System.out.println("\nBTTR.decodeMetadata seg=" + segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd);
+
+          // lazily catch up on metadata decode:
+          final int limit = getTermBlockOrd();
+          assert limit > 0;
+
+          // We must set/incr state.termCount because
+          // postings impl can look at this
+          state.termBlockOrd = metaDataUpto;
+      
+          // TODO: better API would be "jump straight to term=N"???
+          while (metaDataUpto < limit) {
+
+            // TODO: we could make "tiers" of metadata, ie,
+            // decode docFreq/totalTF but don't decode postings
+            // metadata; this way caller could get
+            // docFreq/totalTF w/o paying decode cost for
+            // postings
+
+            // TODO: if docFreq were bulk decoded we could
+            // just skipN here:
+            state.docFreq = statsReader.readVInt();
+            //if (DEBUG) System.out.println("    dF=" + state.docFreq);
+            if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+              state.totalTermFreq = state.docFreq + statsReader.readVLong();
+              //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
+            }
+
+            postingsReader.nextTerm(fieldInfo, state);
+            metaDataUpto++;
+            state.termBlockOrd++;
+          }
+        }
+
+        // Used only by assert
+        private boolean prefixMatches(BytesRef target) {
+          for(int bytePos=0;bytePos<prefix;bytePos++) {
+            if (target.bytes[target.offset + bytePos] != term.bytes[bytePos]) {
+              return false;
+            }
+          }
+
+          return true;
+        }
+
+        // Scans to sub-block that has this target fp; only
+        // called by next(); NOTE: does not set
+        // startBytePos/suffix as a side effect
+        public void scanToSubBlock(long subFP) {
+          assert !isLeafBlock;
+          //if (DEBUG) System.out.println("  scanToSubBlock fp=" + fp + " subFP=" + subFP + " entCount=" + entCount + " lastSubFP=" + lastSubFP);
+          //assert nextEnt == 0;
+          if (lastSubFP == subFP) {
+            //if (DEBUG) System.out.println("    already positioned");
+            return;
+          }
+          assert subFP < fp : "fp=" + fp + " subFP=" + subFP;
+          final long targetSubCode = fp - subFP;
+          //if (DEBUG) System.out.println("    targetSubCode=" + targetSubCode);
+          while(true) {
+            assert nextEnt < entCount;
+            nextEnt++;
+            final int code = suffixesReader.readVInt();
+            suffixesReader.skipBytes(isLeafBlock ? code : code >>> 1);
+            //if (DEBUG) System.out.println("    " + nextEnt + " (of " + entCount + ") ent isSubBlock=" + ((code&1)==1));
+            if ((code & 1) != 0) {
+              final long subCode = suffixesReader.readVLong();
+              //if (DEBUG) System.out.println("      subCode=" + subCode);
+              if (targetSubCode == subCode) {
+                //if (DEBUG) System.out.println("        match!");
+                lastSubFP = subFP;
+                return;
+              }
+            } else {
+              state.termBlockOrd++;
+            }
+          }
+        }
+
+        // NOTE: sets startBytePos/suffix as a side effect
+        public SeekStatus scanToTerm(BytesRef target, boolean exactOnly) throws IOException {
+          return isLeafBlock ? scanToTermLeaf(target, exactOnly) : scanToTermNonLeaf(target, exactOnly);
+        }
+
+        private int startBytePos;
+        private int suffix;
+        private long subCode;
+
+        // Target's prefix matches this block's prefix; we
+        // scan the entries check if the suffix matches.
+        public SeekStatus scanToTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
+
+          // if (DEBUG) System.out.println("    scanToTermLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(term));
+
+          assert nextEnt != -1;
+
+          termExists = true;
+          subCode = 0;
+
+          if (nextEnt == entCount) {
+            if (exactOnly) {
+              fillTerm();
+            }
+            return SeekStatus.END;
+          }
+
+          assert prefixMatches(target);
+
+          // Loop over each entry (term or sub-block) in this block:
+          //nextTerm: while(nextEnt < entCount) {
+          nextTerm: while (true) {
+            nextEnt++;
+
+            suffix = suffixesReader.readVInt();
+
+            // if (DEBUG) {
+            //   BytesRef suffixBytesRef = new BytesRef();
+            //   suffixBytesRef.bytes = suffixBytes;
+            //   suffixBytesRef.offset = suffixesReader.getPosition();
+            //   suffixBytesRef.length = suffix;
+            //   System.out.println("      cycle: term " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
+            // }
+
+            final int termLen = prefix + suffix;
+            startBytePos = suffixesReader.getPosition();
+            suffixesReader.skipBytes(suffix);
+
+            final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
+            int targetPos = target.offset + prefix;
+
+            // Loop over bytes in the suffix, comparing to
+            // the target
+            int bytePos = startBytePos;
+            while(true) {
+              final int cmp;
+              final boolean stop;
+              if (targetPos < targetLimit) {
+                cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+                stop = false;
+              } else {
+                assert targetPos == targetLimit;
+                cmp = termLen - target.length;
+                stop = true;
+              }
+
+              if (cmp < 0) {
+                // Current entry is still before the target;
+                // keep scanning
+
+                if (nextEnt == entCount) {
+                  if (exactOnly) {
+                    fillTerm();
+                  }
+                  // We are done scanning this block
+                  break nextTerm;
+                } else {
+                  continue nextTerm;
+                }
+              } else if (cmp > 0) {
+
+                // Done!  Current entry is after target --
+                // return NOT_FOUND:
+                fillTerm();
+
+                if (!exactOnly && !termExists) {
+                  // We are on a sub-block, and caller wants
+                  // us to position to the next term after
+                  // the target, so we must recurse into the
+                  // sub-frame(s):
+                  currentFrame = pushFrame(null, currentFrame.lastSubFP, termLen);
+                  currentFrame.loadBlock();
+                  while (currentFrame.next()) {
+                    currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+                    currentFrame.loadBlock();
+                  }
+                }
+                
+                //if (DEBUG) System.out.println("        not found");
+                return SeekStatus.NOT_FOUND;
+              } else if (stop) {
+                // Exact match!
+
+                // This cannot be a sub-block because we
+                // would have followed the index to this
+                // sub-block from the start:
+
+                assert termExists;
+                fillTerm();
+                //if (DEBUG) System.out.println("        found!");
+                return SeekStatus.FOUND;
+              }
+            }
+          }
+
+          // It is possible (and OK) that terms index pointed us
+          // at this block, but, we scanned the entire block and
+          // did not find the term to position to.  This happens
+          // when the target is after the last term in the block
+          // (but, before the next term in the index).  EG
+          // target could be foozzz, and terms index pointed us
+          // to the foo* block, but the last term in this block
+          // was fooz (and, eg, first term in the next block will
+          // bee fop).
+          //if (DEBUG) System.out.println("      block end");
+          if (exactOnly) {
+            fillTerm();
+          }
+
+          // TODO: not consistent that in the
+          // not-exact case we don't next() into the next
+          // frame here
+          return SeekStatus.END;
+        }
+
+        // Target's prefix matches this block's prefix; we
+        // scan the entries check if the suffix matches.
+        public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
+
+          //if (DEBUG) System.out.println("    scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(term));
+
+          assert nextEnt != -1;
+
+          if (nextEnt == entCount) {
+            if (exactOnly) {
+              fillTerm();
+              termExists = subCode == 0;
+            }
+            return SeekStatus.END;
+          }
+
+          assert prefixMatches(target);
+
+          // Loop over each entry (term or sub-block) in this block:
+          //nextTerm: while(nextEnt < entCount) {
+          nextTerm: while (true) {
+            nextEnt++;
+
+            final int code = suffixesReader.readVInt();
+            suffix = code >>> 1;
+            // if (DEBUG) {
+            //   BytesRef suffixBytesRef = new BytesRef();
+            //   suffixBytesRef.bytes = suffixBytes;
+            //   suffixBytesRef.offset = suffixesReader.getPosition();
+            //   suffixBytesRef.length = suffix;
+            //   System.out.println("      cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
+            // }
+
+            termExists = (code & 1) == 0;
+            final int termLen = prefix + suffix;
+            startBytePos = suffixesReader.getPosition();
+            suffixesReader.skipBytes(suffix);
+            if (termExists) {
+              state.termBlockOrd++;
+              subCode = 0;
+            } else {
+              subCode = suffixesReader.readVLong();
+              lastSubFP = fp - subCode;
+            }
+
+            final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
+            int targetPos = target.offset + prefix;
+
+            // Loop over bytes in the suffix, comparing to
+            // the target
+            int bytePos = startBytePos;
+            while(true) {
+              final int cmp;
+              final boolean stop;
+              if (targetPos < targetLimit) {
+                cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+                stop = false;
+              } else {
+                assert targetPos == targetLimit;
+                cmp = termLen - target.length;
+                stop = true;
+              }
+
+              if (cmp < 0) {
+                // Current entry is still before the target;
+                // keep scanning
+
+                if (nextEnt == entCount) {
+                  if (exactOnly) {
+                    fillTerm();
+                    //termExists = true;
+                  }
+                  // We are done scanning this block
+                  break nextTerm;
+                } else {
+                  continue nextTerm;
+                }
+              } else if (cmp > 0) {
+
+                // Done!  Current entry is after target --
+                // return NOT_FOUND:
+                fillTerm();
+
+                if (!exactOnly && !termExists) {
+                  // We are on a sub-block, and caller wants
+                  // us to position to the next term after
+                  // the target, so we must recurse into the
+                  // sub-frame(s):
+                  currentFrame = pushFrame(null, currentFrame.lastSubFP, termLen);
+                  currentFrame.loadBlock();
+                  while (currentFrame.next()) {
+                    currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+                    currentFrame.loadBlock();
+                  }
+                }
+                
+                //if (DEBUG) System.out.println("        not found");
+                return SeekStatus.NOT_FOUND;
+              } else if (stop) {
+                // Exact match!
+
+                // This cannot be a sub-block because we
+                // would have followed the index to this
+                // sub-block from the start:
+
+                assert termExists;
+                fillTerm();
+                //if (DEBUG) System.out.println("        found!");
+                return SeekStatus.FOUND;
+              }
+            }
+          }
+
+          // It is possible (and OK) that terms index pointed us
+          // at this block, but, we scanned the entire block and
+          // did not find the term to position to.  This happens
+          // when the target is after the last term in the block
+          // (but, before the next term in the index).  EG
+          // target could be foozzz, and terms index pointed us
+          // to the foo* block, but the last term in this block
+          // was fooz (and, eg, first term in the next block will
+          // bee fop).
+          //if (DEBUG) System.out.println("      block end");
+          if (exactOnly) {
+            fillTerm();
+          }
+
+          // TODO: not consistent that in the
+          // not-exact case we don't next() into the next
+          // frame here
+          return SeekStatus.END;
+        }
+
+        private void fillTerm() {
+          final int termLength = prefix + suffix;
+          term.length = prefix + suffix;
+          if (term.bytes.length < termLength) {
+            term.grow(termLength);
+          }
+          System.arraycopy(suffixBytes, startBytePos, term.bytes, prefix, suffix);
+        }
+      }
+    }
+  }
+}

Property changes on: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java	(revision 0)
@@ -0,0 +1,943 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.NoOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/*
+  TODO:
+  
+    - Currently there is a one-to-one mapping of indexed
+      term to term block, but we could decouple the two, ie,
+      put more terms into the index than there are blocks.
+      The index would take up more RAM but then it'd be able
+      to avoid seeking more often and could make PK/FuzzyQ
+      faster if the additional indexed terms could store
+      the offset into the terms block.
+
+    - The blocks are not written in true depth-first
+      order, meaning if you just next() the file pointer will
+      sometimes jump backwards.  For example, block foo* will
+      be written before block f* because it finished before.
+      This could possibly hurt performance if the terms dict is
+      not hot, since OSs anticipate sequential file access.  We
+      could fix the writer to re-order the blocks as a 2nd
+      pass.
+
+    - Each block encodes the term suffixes packed
+      sequentially using a separate vInt per term, which is
+      1) wasteful and 2) slow (must linear scan to find a
+      particular suffix).  We should instead 1) make
+      random-access array so we can directly access the Nth
+      suffix, and 2) bulk-encode this array using bulk int[]
+      codecs; then at search time we can binary search when
+      we seek a particular term.
+
+/**
+ * Writes terms dict and index, block-encoding (column
+ * stride) each term's metadata for each set of terms
+ * between two index terms.
+ *
+ * @lucene.experimental
+ */
+
+/** See {@link BlockTreeTermsReader}.
+ *
+ * @lucene.experimental
+*/
+
+public class BlockTreeTermsWriter extends FieldsConsumer {
+
+  public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
+  public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
+
+  //public final static boolean DEBUG = false;
+  public final static boolean SAVE_DOT_FILES = false;
+
+  static final int OUTPUT_FLAGS_NUM_BITS = 2;
+  static final int OUTPUT_FLAGS_MASK = 0x3;
+  static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
+  static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
+
+  final static String CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
+
+  // Initial format
+  public static final int VERSION_START = 0;
+
+  public static final int VERSION_CURRENT = VERSION_START;
+
+  /** Extension of terms file */
+  static final String TERMS_EXTENSION = "tim";
+  static final String TERMS_INDEX_EXTENSION = "tip";
+
+  protected final IndexOutput out;
+  private final IndexOutput indexOut;
+  final int minItemsInBlock;
+  final int maxItemsInBlock;
+
+  final PostingsWriterBase postingsWriter;
+  final FieldInfos fieldInfos;
+  FieldInfo currentField;
+  private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
+  // private final String segment;
+
+  /** Create a new writer.  The number of items (terms or
+   *  sub-blocks) per block will aim to be between
+   *  minItemsPerBlock and maxItemsPerBlock, though in some
+   *  cases the blocks may be smaller than the min. */
+  public BlockTreeTermsWriter(
+                              SegmentWriteState state,
+                              PostingsWriterBase postingsWriter,
+                              int minItemsInBlock,
+                              int maxItemsInBlock)
+    throws IOException
+  {
+    if (minItemsInBlock <= 1) {
+      throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock);
+    }
+    if (maxItemsInBlock <= 0) {
+      throw new IllegalArgumentException("maxItemsInBlock must be >= 1; got " + maxItemsInBlock);
+    }
+    if (minItemsInBlock > maxItemsInBlock) {
+      throw new IllegalArgumentException("maxItemsInBlock must be >= minItemsInBlock; got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
+    }
+    if (2*(minItemsInBlock-1) > maxItemsInBlock) {
+      throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
+    }
+
+    final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
+    out = state.directory.createOutput(termsFileName, state.context);
+    boolean success = false;
+    IndexOutput indexOut = null;
+    try {
+      fieldInfos = state.fieldInfos;
+      this.minItemsInBlock = minItemsInBlock;
+      this.maxItemsInBlock = maxItemsInBlock;
+      writeHeader(out);
+
+      //DEBUG = state.segmentName.equals("_4a");
+
+      final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
+      indexOut = state.directory.createOutput(termsIndexFileName, state.context);
+      writeIndexHeader(indexOut);
+
+      currentField = null;
+      this.postingsWriter = postingsWriter;
+      // segment = state.segmentName;
+
+      // System.out.println("BTW.init seg=" + state.segmentName);
+
+      postingsWriter.start(out);                          // have consumer write its format/header
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeSafely(true, out, indexOut);
+      }
+    }
+    this.indexOut = indexOut;
+  }
+  
+  protected void writeHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); 
+    out.writeLong(0);                             // leave space for end index pointer    
+  }
+
+  protected void writeIndexHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); 
+    out.writeLong(0);                             // leave space for end index pointer    
+  }
+
+  protected void writeTrailer(long dirStart) throws IOException {
+    out.seek(CodecUtil.headerLength(CODEC_NAME));
+    out.writeLong(dirStart);    
+  }
+
+  protected void writeIndexTrailer(long dirStart) throws IOException {
+    indexOut.seek(CodecUtil.headerLength(CODEC_NAME));
+    indexOut.writeLong(dirStart);    
+  }
+  
+  @Override
+  public TermsConsumer addField(FieldInfo field) throws IOException {
+    //DEBUG = field.name.equals("id");
+    //if (DEBUG) System.out.println("\nBTTW.addField seg=" + segment + " field=" + field.name);
+    assert currentField == null || currentField.name.compareTo(field.name) < 0;
+    currentField = field;
+    final TermsWriter terms = new TermsWriter(field);
+    fields.add(terms);
+    return terms;
+  }
+
+  static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
+    assert fp < (1L << 62);
+    return (fp << 2) | (hasTerms ? OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? OUTPUT_FLAG_IS_FLOOR : 0);
+  }
+
+  private static class PendingEntry {
+    public final boolean isTerm;
+
+    protected PendingEntry(boolean isTerm) {
+      this.isTerm = isTerm;
+    }
+  }
+
+  private static final class PendingTerm extends PendingEntry {
+    public final BytesRef term;
+    public final TermStats stats;
+
+    public PendingTerm(BytesRef term, TermStats stats) {
+      super(true);
+      this.term = term;
+      this.stats = stats;
+    }
+
+    @Override
+    public String toString() {
+      return term.utf8ToString();
+    }
+  }
+
+  private static final class PendingBlock extends PendingEntry {
+    public final BytesRef prefix;
+    public final long fp;
+    public FST<BytesRef> index;
+    public List<FST<BytesRef>> subIndices;
+    public final boolean hasTerms;
+    public final boolean isFloor;
+    public final int floorLeadByte;
+
+    public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List<FST<BytesRef>> subIndices) {
+      super(false);
+      this.prefix = prefix;
+      this.fp = fp;
+      this.hasTerms = hasTerms;
+      this.isFloor = isFloor;
+      this.floorLeadByte = floorLeadByte;
+      this.subIndices = subIndices;
+    }
+
+    @Override
+    public String toString() {
+      return "BLOCK: " + prefix.utf8ToString();
+    }
+
+    public void compileIndex(List<PendingBlock> floorBlocks, RAMOutputStream scratchBytes) throws IOException {
+
+      assert (isFloor && floorBlocks != null && floorBlocks.size() != 0) || (!isFloor && floorBlocks == null): "isFloor=" + isFloor + " floorBlocks=" + floorBlocks;
+
+      assert scratchBytes.getFilePointer() == 0;
+
+      // TODO: try writing the leading vLong in MSB order
+      // (opposite of what Lucene does today), for better
+      // outputs sharing in the FST
+      scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
+      if (isFloor) {
+        scratchBytes.writeVInt(floorBlocks.size());
+        for (PendingBlock sub : floorBlocks) {
+          assert sub.floorLeadByte != -1;
+          //if (DEBUG) {
+          //  System.out.println("    write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
+          //}
+          scratchBytes.writeByte((byte) sub.floorLeadByte);
+          assert sub.fp > fp;
+          scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0));
+        }
+      }
+
+      final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+      final Builder<BytesRef> indexBuilder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1,
+                                                                   0, 0, true, false, Integer.MAX_VALUE,
+                                                                   outputs, null);
+      //if (DEBUG) {
+      //  System.out.println("  compile index for prefix=" + prefix);
+      //}
+      //indexBuilder.DEBUG = false;
+      final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+      assert bytes.length > 0;
+      scratchBytes.writeTo(bytes, 0);
+      indexBuilder.add(prefix, new BytesRef(bytes, 0, bytes.length));
+      scratchBytes.reset();
+
+      // Copy over index for all sub-blocks
+
+      if (subIndices != null) {
+        for(FST<BytesRef> subIndex : subIndices) {
+          append(indexBuilder, subIndex);
+        }
+      }
+
+      if (floorBlocks != null) {
+        for (PendingBlock sub : floorBlocks) {
+          if (sub.subIndices != null) {
+            for(FST<BytesRef> subIndex : sub.subIndices) {
+              append(indexBuilder, subIndex);
+            }
+          }
+          sub.subIndices = null;
+        }
+      }
+
+      index = indexBuilder.finish();
+      subIndices = null;
+
+      /*
+      Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
+      Util.toDot(index, w, false, false);
+      System.out.println("SAVED to out.dot");
+      w.close();
+      */
+    }
+
+    // TODO: maybe we could add bulk-add method to
+    // Builder?  Takes FST and unions it w/ current
+    // FST.
+    private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex) throws IOException {
+      final BytesRefFSTEnum<BytesRef> subIndexEnum = new BytesRefFSTEnum<BytesRef>(subIndex);
+      BytesRefFSTEnum.InputOutput<BytesRef> indexEnt;
+      while((indexEnt = subIndexEnum.next()) != null) {
+        //if (DEBUG) {
+        //  System.out.println("      add sub=" + indexEnt.input + " " + indexEnt.input + " output=" + indexEnt.output);
+        //}
+        builder.add(indexEnt.input, indexEnt.output);
+      }
+    }
+  }
+
+  final RAMOutputStream scratchBytes = new RAMOutputStream();
+
+  class TermsWriter extends TermsConsumer {
+    private final FieldInfo fieldInfo;
+    private long numTerms;
+    long sumTotalTermFreq;
+    long sumDocFreq;
+    long indexStartFP;
+
+    // Used only to partition terms into the block tree; we
+    // don't pull an FST from this builder:
+    private final NoOutputs noOutputs;
+    private final Builder<Object> blockBuilder;
+
+    // PendingTerm or PendingBlock:
+    private final List<PendingEntry> pending = new ArrayList<PendingEntry>();
+
+    // Index into pending of most recently written block
+    private int lastBlockIndex = -1;
+
+    // Re-used when segmenting a too-large block into floor
+    // blocks:
+    private int[] subBytes = new int[10];
+    private int[] subTermCounts = new int[10];
+    private int[] subTermCountSums = new int[10];
+    private int[] subSubCounts = new int[10];
+
+    // This class assigns terms to blocks "naturally", ie,
+    // according to the number of terms under a given prefix
+    // that we encounter:
+    private class FindBlocks extends Builder.FreezeTail<Object> {
+
+      @Override
+      public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException {
+
+        //if (DEBUG) System.out.println("  freeze prefixLenPlus1=" + prefixLenPlus1);
+
+        for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
+          final Builder.UnCompiledNode<Object> node = frontier[idx];
+
+          long totCount = 0;
+
+          if (node.isFinal) {
+            totCount++;
+          }
+
+          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+            @SuppressWarnings("unchecked") final Builder.UnCompiledNode<Object> target = (Builder.UnCompiledNode<Object>) node.arcs[arcIdx].target;
+            totCount += target.inputCount;
+            target.clear();
+            node.arcs[arcIdx].target = null;
+          }
+          node.numArcs = 0;
+
+          if (totCount >= minItemsInBlock || idx == 0) {
+            // We are on a prefix node that has enough
+            // entries (terms or sub-blocks) under it to let
+            // us write a new block or multiple blocks (main
+            // block + follow on floor blocks):
+            //if (DEBUG) {
+            //  if (totCount < minItemsInBlock && idx != 0) {
+            //    System.out.println("  force block has terms");
+            //  }
+            //}
+            writeBlocks(lastInput, idx, (int) totCount);
+            node.inputCount = 1;
+          } else {
+            // stragglers!  carry count upwards
+            node.inputCount = totCount;
+          }
+          frontier[idx] = new Builder.UnCompiledNode<Object>(blockBuilder, idx);
+        }
+      }
+    }
+
+    // Write the top count entries on the pending stack as
+    // one or more blocks.  Returns how many blocks were
+    // written.  If the entry count is <= maxItemsPerBlock
+    // we just write a single block; else we break into
+    // primary (initial) block and then one or more
+    // following floor blocks:
+
+    void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException {
+      if (prefixLength == 0 || count <= maxItemsInBlock) {
+        // Easy case: not floor block.  Eg, prefix is "foo",
+        // and we found 30 terms/sub-blocks starting w/ that
+        // prefix, and minItemsInBlock <= 30 <=
+        // maxItemsInBlock.
+        final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true);
+        nonFloorBlock.compileIndex(null, scratchBytes);
+        pending.add(nonFloorBlock);
+      } else {
+        // Floor block case.  Eg, prefix is "foo" but we
+        // have 100 terms/sub-blocks starting w/ that
+        // prefix.  We segment the entries into a primary
+        // block and following floor blocks using the first
+        // label in the suffix to assign to floor blocks.
+
+        // TODO: we could store min & max suffix start byte
+        // in each block, to make floor blocks authoritative
+
+        //if (DEBUG) {
+        //  final BytesRef prefix = new BytesRef(prefixLength);
+        //  for(int m=0;m<prefixLength;m++) {
+        //    prefix.bytes[m] = (byte) prevTerm.ints[m];
+        //  }
+        //  prefix.length = prefixLength;
+        //  //System.out.println("\nWBS count=" + count + " prefix=" + prefix.utf8ToString() + " " + prefix);
+        //  System.out.println("writeBlocks: prefix=" + prefix + " " + prefix + " count=" + count + " pending.size()=" + pending.size());
+        //}
+        //System.out.println("\nwbs count=" + count);
+
+        final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength];
+
+        // Count up how many items fall under
+        // each unique label after the prefix.
+        
+        // TODO: this is wasteful since the builder had
+        // already done this (partitioned these sub-terms
+        // according to their leading prefix byte)
+        
+        final List<PendingEntry> slice = pending.subList(pending.size()-count, pending.size());
+        int lastSuffixLeadLabel = -1;
+        int termCount = 0;
+        int subCount = 0;
+        int numSubs = 0;
+
+        for(PendingEntry ent : slice) {
+
+          // First byte in the suffix of this term
+          final int suffixLeadLabel;
+          if (ent.isTerm) {
+            PendingTerm term = (PendingTerm) ent;
+            if (term.term.length == prefixLength) {
+              // Suffix is 0, ie prefix 'foo' and term is
+              // 'foo' so the term has empty string suffix
+              // in this block
+              assert lastSuffixLeadLabel == -1;
+              assert numSubs == 0;
+              suffixLeadLabel = -1;
+            } else {
+              suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
+            }
+          } else {
+            PendingBlock block = (PendingBlock) ent;
+            assert block.prefix.length > prefixLength;
+            suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
+          }
+
+          if (suffixLeadLabel != lastSuffixLeadLabel && (termCount + subCount) != 0) {
+            if (subBytes.length == numSubs) {
+              subBytes = ArrayUtil.grow(subBytes);
+              subTermCounts = ArrayUtil.grow(subTermCounts);
+              subSubCounts = ArrayUtil.grow(subSubCounts);
+            }
+            subBytes[numSubs] = lastSuffixLeadLabel;
+            lastSuffixLeadLabel = suffixLeadLabel;
+            subTermCounts[numSubs] = termCount;
+            subSubCounts[numSubs] = subCount;
+            /*
+            if (suffixLeadLabel == -1) {
+              System.out.println("  sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
+            } else {
+              System.out.println("  sub " + Integer.toHexString(suffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
+            }
+            */
+            termCount = subCount = 0;
+            numSubs++;
+          }
+
+          if (ent.isTerm) {
+            termCount++;
+          } else {
+            subCount++;
+          }
+        }
+
+        if (subBytes.length == numSubs) {
+          subBytes = ArrayUtil.grow(subBytes);
+          subTermCounts = ArrayUtil.grow(subTermCounts);
+          subSubCounts = ArrayUtil.grow(subSubCounts);
+        }
+
+        subBytes[numSubs] = lastSuffixLeadLabel;
+        subTermCounts[numSubs] = termCount;
+        subSubCounts[numSubs] = subCount;
+        numSubs++;
+        /*
+        if (lastSuffixLeadLabel == -1) {
+          System.out.println("  sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
+        } else {
+          System.out.println("  sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
+        }
+        */
+
+        if (subTermCountSums.length < numSubs) {
+          subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs);
+        }
+
+        // Roll up (backwards) the termCounts; postings impl
+        // needs this to know where to pull the term slice
+        // from its pending terms stack:
+        int sum = 0;
+        for(int idx=numSubs-1;idx>=0;idx--) {
+          sum += subTermCounts[idx];
+          subTermCountSums[idx] = sum;
+        }
+
+        // TODO: make a better segmenter?  It'd have to
+        // absorb the too-small end blocks backwards into
+        // the previous blocks
+
+        // Naive greedy segmentation; this is not always
+        // best (it can produce a too-small block as the
+        // last block):
+        int pendingCount = 0;
+        int startLabel = subBytes[0];
+        int curStart = count;
+        subCount = 0;
+
+        final List<PendingBlock> floorBlocks = new ArrayList<PendingBlock>();
+        PendingBlock firstBlock = null;
+
+        for(int sub=0;sub<numSubs;sub++) {
+          pendingCount += subTermCounts[sub] + subSubCounts[sub];
+          //System.out.println("  " + (subTermCounts[sub] + subSubCounts[sub]));
+          subCount++;
+
+          // Greedily make a floor block as soon as we've
+          // crossed the min count
+          if (pendingCount >= minItemsInBlock) {
+            final int curPrefixLength;
+            if (startLabel == -1) {
+              curPrefixLength = prefixLength;
+            } else {
+              curPrefixLength = 1+prefixLength;
+              // floor term:
+              prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
+            }
+            //System.out.println("  " + subCount + " subs");
+            final PendingBlock floorBlock = writeBlock(prevTerm, prefixLength, curPrefixLength, curStart, pendingCount, subTermCountSums[1+sub], true, startLabel, curStart == pendingCount);
+            if (firstBlock == null) {
+              firstBlock = floorBlock;
+            } else {
+              floorBlocks.add(floorBlock);
+            }
+            curStart -= pendingCount;
+            //System.out.println("    = " + pendingCount);
+            pendingCount = 0;
+
+            assert minItemsInBlock == 1 || subCount > 1: "minItemsInBlock=" + minItemsInBlock + " subCount=" + subCount + " sub=" + sub + " of " + numSubs + " subTermCount=" + subTermCountSums[sub] + " subSubCount=" + subSubCounts[sub] + " depth=" + prefixLength;
+            subCount = 0;
+            startLabel = subBytes[sub+1];
+
+            if (curStart == 0) {
+              break;
+            }
+
+            if (curStart <= maxItemsInBlock) {
+              // remainder is small enough to fit into a
+              // block.  NOTE that this may be too small (<
+              // minItemsInBlock); need a true segmenter
+              // here
+              assert startLabel != -1;
+              assert firstBlock != null;
+              prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
+              //System.out.println("  final " + (numSubs-sub-1) + " subs");
+              /*
+              for(sub++;sub < numSubs;sub++) {
+                System.out.println("  " + (subTermCounts[sub] + subSubCounts[sub]));
+              }
+              System.out.println("    = " + curStart);
+              if (curStart < minItemsInBlock) {
+                System.out.println("      **");
+              }
+              */
+              floorBlocks.add(writeBlock(prevTerm, prefixLength, prefixLength+1, curStart, curStart, 0, true, startLabel, true));
+              break;
+            }
+          }
+        }
+
+        prevTerm.ints[prevTerm.offset + prefixLength] = savLabel;
+
+        assert firstBlock != null;
+        firstBlock.compileIndex(floorBlocks, scratchBytes);
+
+        pending.add(firstBlock);
+        //if (DEBUG) System.out.println("  done pending.size()=" + pending.size());
+      }
+      lastBlockIndex = pending.size()-1;
+    }
+
+    // for debugging
+    private String toString(BytesRef b) {
+      try {
+        return b.utf8ToString() + " " + b;
+      } catch (Throwable t) {
+        // If BytesRef isn't actually UTF8, or it's eg a
+        // prefix of UTF8 that ends mid-unicode-char, we
+        // fallback to hex:
+        return b.toString();
+      }
+    }
+
+    // Writes all entries in the pending slice as a single
+    // block: 
+    private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
+                                    int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
+
+      assert length > 0;
+
+      final int start = pending.size()-startBackwards;
+
+      assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length;
+
+      final List<PendingEntry> slice = pending.subList(start, start + length);
+
+      final long startFP = out.getFilePointer();
+
+      final BytesRef prefix = new BytesRef(indexPrefixLength);
+      for(int m=0;m<indexPrefixLength;m++) {
+        prefix.bytes[m] = (byte) prevTerm.ints[m];
+      }
+      prefix.length = indexPrefixLength;
+
+      // Write block header:
+      out.writeVInt((length<<1)|(isLastInFloor ? 1:0));
+
+      // if (DEBUG) {
+      //   System.out.println("  writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + toString(prefix) + " entCount=" + length + " startFP=" + startFP + " futureTermCount=" + futureTermCount + (isFloor ? (" floorLeadByte=" + Integer.toHexString(floorLeadByte&0xff)) : "") + " isLastInFloor=" + isLastInFloor);
+      // }
+
+      // 1st pass: pack term suffix bytes into byte[] blob
+      // TODO: cutover to bulk int codec... simple64?
+
+      final boolean isLeafBlock;
+      if (lastBlockIndex < start) {
+        // This block definitely does not contain sub-blocks:
+        isLeafBlock = true;
+        //System.out.println("no scan true isFloor=" + isFloor);
+      } else if (!isFloor) {
+        // This block definitely does contain at least one sub-block:
+        isLeafBlock = false;
+        //System.out.println("no scan false " + lastBlockIndex + " vs start=" + start + " len=" + length);
+      } else {
+        // Must scan up-front to see if there is a sub-block
+        boolean v = true;
+        //System.out.println("scan " + lastBlockIndex + " vs start=" + start + " len=" + length);
+        for (PendingEntry ent : slice) {
+          if (!ent.isTerm) {
+            v = false;
+            break;
+          }
+        }
+        isLeafBlock = v;
+      }
+
+      final List<FST<BytesRef>> subIndices;
+
+      int termCount;
+      if (isLeafBlock) {
+        subIndices = null;
+        for (PendingEntry ent : slice) {
+          assert ent.isTerm;
+          PendingTerm term = (PendingTerm) ent;
+          final int suffix = term.term.length - prefixLength;
+          // if (DEBUG) {
+          //   BytesRef suffixBytes = new BytesRef(suffix);
+          //   System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+          //   suffixBytes.length = suffix;
+          //   System.out.println("    write term suffix=" + suffixBytes);
+          // }
+          // For leaf block we write suffix straight
+          bytesWriter.writeVInt(suffix);
+          bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+
+          // Write term stats, to separate byte[] blob:
+          bytesWriter2.writeVInt(term.stats.docFreq);
+          if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+            assert term.stats.totalTermFreq >= term.stats.docFreq;
+            bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
+          }
+        }
+        termCount = length;
+      } else {
+        subIndices = new ArrayList<FST<BytesRef>>();
+        termCount = 0;
+        for (PendingEntry ent : slice) {
+          if (ent.isTerm) {
+            PendingTerm term = (PendingTerm) ent;
+            final int suffix = term.term.length - prefixLength;
+            // if (DEBUG) {
+            //   BytesRef suffixBytes = new BytesRef(suffix);
+            //   System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+            //   suffixBytes.length = suffix;
+            //   System.out.println("    write term suffix=" + suffixBytes);
+            // }
+            // For non-leaf block we borrow 1 bit to record
+            // if entry is term or sub-block
+            bytesWriter.writeVInt(suffix<<1);
+            bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+
+            // Write term stats, to separate byte[] blob:
+            bytesWriter2.writeVInt(term.stats.docFreq);
+            if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+              assert term.stats.totalTermFreq >= term.stats.docFreq;
+              bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
+            }
+
+            termCount++;
+          } else {
+            PendingBlock block = (PendingBlock) ent;
+            final int suffix = block.prefix.length - prefixLength;
+
+            assert suffix > 0;
+
+            // For non-leaf block we borrow 1 bit to record
+            // if entry is term or sub-block
+            bytesWriter.writeVInt((suffix<<1)|1);
+            bytesWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
+            assert block.fp < startFP;
+
+            // if (DEBUG) {
+            //   BytesRef suffixBytes = new BytesRef(suffix);
+            //   System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+            //   suffixBytes.length = suffix;
+            //   System.out.println("    write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
+            // }
+
+            bytesWriter.writeVLong(startFP - block.fp);
+            subIndices.add(block.index);
+          }
+        }
+
+        assert subIndices.size() != 0;
+      }
+
+      // TODO: we could block-write the term suffix pointers;
+      // this would take more space but would enable binary
+      // search on lookup
+
+      // Write suffixes byte[] blob to terms dict output:
+      out.writeVInt((int) (bytesWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
+      bytesWriter.writeTo(out);
+      bytesWriter.reset();
+
+      // Write term stats byte[] blob
+      out.writeVInt((int) bytesWriter2.getFilePointer());
+      bytesWriter2.writeTo(out);
+      bytesWriter2.reset();
+
+      // Have postings writer write block
+      postingsWriter.flushTermsBlock(futureTermCount+termCount, termCount);
+
+      // Remove slice replaced by block:
+      slice.clear();
+
+      if (lastBlockIndex >= start) {
+        if (lastBlockIndex < start+length) {
+          lastBlockIndex = start;
+        } else {
+          lastBlockIndex -= length;
+        }
+      }
+
+      // if (DEBUG) {
+      //   System.out.println("      fpEnd=" + out.getFilePointer());
+      // }
+
+      return new PendingBlock(prefix, startFP, termCount != 0, isFloor, floorLeadByte, subIndices);
+    }
+
+    TermsWriter(FieldInfo fieldInfo) {
+      this.fieldInfo = fieldInfo;
+
+      noOutputs = NoOutputs.getSingleton();
+
+      // This Builder is just used transiently to fragment
+      // terms into "good" blocks; we don't save the
+      // resulting FST:
+      blockBuilder = new Builder<Object>(FST.INPUT_TYPE.BYTE1,
+                                         0, 0, true,
+                                         true, Integer.MAX_VALUE,
+                                         noOutputs,
+                                         new FindBlocks());
+
+      postingsWriter.setField(fieldInfo);
+    }
+    
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public PostingsConsumer startTerm(BytesRef text) throws IOException {
+      //if (DEBUG) System.out.println("\nBTTW.startTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment);
+      postingsWriter.startTerm();
+      /*
+      if (fieldInfo.name.equals("id")) {
+        postingsWriter.termID = Integer.parseInt(text.utf8ToString());
+      } else {
+        postingsWriter.termID = -1;
+      }
+      */
+      return postingsWriter;
+    }
+
+    @Override
+    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+
+      assert stats.docFreq > 0;
+      //if (DEBUG) System.out.println("BTTW.finishTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment + " df=" + stats.docFreq);
+
+      blockBuilder.add(text, noOutputs.getNoOutput());
+      pending.add(new PendingTerm(new BytesRef(text), stats));
+      postingsWriter.finishTerm(stats);
+      numTerms++;
+    }
+
+    // Finishes all terms in this field
+    @Override
+    public void finish(long sumTotalTermFreq, long sumDocFreq) throws IOException {
+      if (numTerms > 0) {
+        blockBuilder.finish();
+
+        // We better have one final "root" block:
+        assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;
+        final PendingBlock root = (PendingBlock) pending.get(0);
+        assert root.prefix.length == 0;
+        assert root.index.getEmptyOutput() != null;
+
+        this.sumTotalTermFreq = sumTotalTermFreq;
+        this.sumDocFreq = sumDocFreq;
+
+        // Write FST to index
+        indexStartFP = indexOut.getFilePointer();
+        root.index.save(indexOut);
+        //System.out.println("  write FST " + indexStartFP + " field=" + fieldInfo.name);
+
+        // if (SAVE_DOT_FILES || DEBUG) {
+        //   final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+        //   Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+        //   Util.toDot(root.index, w, false, false);
+        //   System.out.println("SAVED to " + dotFileName);
+        //   w.close();
+        // }
+      }
+    }
+
+    private final RAMOutputStream bytesWriter = new RAMOutputStream();
+    private final RAMOutputStream bytesWriter2 = new RAMOutputStream();
+  }
+
+  @Override
+  public void close() throws IOException {
+
+    IOException ioe = null;
+    try {
+      
+      int nonZeroCount = 0;
+      for(TermsWriter field : fields) {
+        if (field.numTerms > 0) {
+          nonZeroCount++;
+        }
+      }
+
+      final long dirStart = out.getFilePointer();
+      final long indexDirStart = indexOut.getFilePointer();
+
+      out.writeVInt(nonZeroCount);
+      
+      for(TermsWriter field : fields) {
+        if (field.numTerms > 0) {
+          //System.out.println("  field " + field.fieldInfo.name + " " + field.numTerms + " terms");
+          out.writeVInt(field.fieldInfo.number);
+          out.writeVLong(field.numTerms);
+          final BytesRef rootCode = ((PendingBlock) field.pending.get(0)).index.getEmptyOutput();
+          assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms;
+          out.writeVInt(rootCode.length);
+          out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length);
+          if (field.fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+            out.writeVLong(field.sumTotalTermFreq);
+          }
+          out.writeVLong(field.sumDocFreq);
+          indexOut.writeVLong(field.indexStartFP);
+        }
+      }
+      writeTrailer(dirStart);
+      writeIndexTrailer(indexDirStart);
+    } catch (IOException ioe2) {
+      ioe = ioe2;
+    } finally {
+      IOUtils.closeSafely(ioe, out, indexOut, postingsWriter);
+    }
+  }
+}

Property changes on: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java	(working copy)
@@ -106,7 +106,7 @@
     }
   }
   
-  //private String segment;
+  // private String segment;
   
   public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
                           int termsCacheSize, int codecId)
@@ -115,7 +115,7 @@
     this.postingsReader = postingsReader;
     termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
 
-    //this.segment = segment;
+    // this.segment = segment;
     in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
                        context);
 
@@ -321,6 +321,9 @@
       /* Common prefix used for all terms in this block. */
       private int termBlockPrefix;
 
+      /* How many terms in current block */
+      private int blockTermCount;
+
       private byte[] docFreqBytes;
       private final ByteArrayDataInput freqReader = new ByteArrayDataInput();
       private int metaDataUpto;
@@ -358,16 +361,14 @@
           throw new IllegalStateException("terms index was not loaded");
         }
    
-        /*
-        System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this="  + this);
+        //System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this="  + this);
         if (didIndexNext) {
           if (nextIndexTerm == null) {
-            System.out.println("  nextIndexTerm=null");
+            //System.out.println("  nextIndexTerm=null");
           } else {
-            System.out.println("  nextIndexTerm=" + nextIndexTerm.utf8ToString());
+            //System.out.println("  nextIndexTerm=" + nextIndexTerm.utf8ToString());
           }
         }
-        */
 
         // Check cache
         if (useCache) {
@@ -444,7 +445,7 @@
           //System.out.println("  seek: term=" + term.utf8ToString());
         } else {
           //System.out.println("  skip seek");
-          if (state.termCount == state.blockTermCount && !nextBlock()) {
+          if (state.termBlockOrd == blockTermCount && !nextBlock()) {
             indexIsCurrent = false;
             return SeekStatus.END;
           }
@@ -480,9 +481,9 @@
               // but it could be in next block.  We
               // must scan to end-of-block to set common
               // prefix for next block:
-              if (state.termCount < state.blockTermCount) {
-                while(state.termCount < state.blockTermCount-1) {
-                  state.termCount++;
+              if (state.termBlockOrd < blockTermCount) {
+                while(state.termBlockOrd < blockTermCount-1) {
+                  state.termBlockOrd++;
                   state.ord++;
                   termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
                 }
@@ -505,7 +506,7 @@
               // Target's prefix is before the common prefix
               // of this block, so we position to start of
               // block and return NOT_FOUND:
-              assert state.termCount == 0;
+              assert state.termBlockOrd == 0;
 
               final int suffix = termSuffixesReader.readVInt();
               term.length = termBlockPrefix + suffix;
@@ -523,7 +524,7 @@
 
           // Test every term in this block
           while (true) {
-            state.termCount++;
+            state.termBlockOrd++;
             state.ord++;
 
             final int suffix = termSuffixesReader.readVInt();
@@ -581,7 +582,7 @@
               }
             }
 
-            if (state.termCount == state.blockTermCount) {
+            if (state.termBlockOrd == blockTermCount) {
               // Must pre-fill term for next block's common prefix
               term.length = termBlockPrefix + suffix;
               if (term.bytes.length < term.length) {
@@ -613,7 +614,7 @@
 
       @Override
       public BytesRef next() throws IOException {
-        //System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termCount);
+        //System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termBlockOrd);
 
         // If seek was previously called and the term was cached,
         // usually caller is just going to pull a D/&PEnum or get
@@ -623,7 +624,7 @@
         if (seekPending) {
           assert !indexIsCurrent;
           in.seek(state.blockFilePointer);
-          final int pendingSeekCount = state.termCount;
+          final int pendingSeekCount = state.termBlockOrd;
           boolean result = nextBlock();
 
           final long savOrd = state.ord;
@@ -633,7 +634,7 @@
           // on a real term:
           assert result;
 
-          while(state.termCount < pendingSeekCount) {
+          while(state.termBlockOrd < pendingSeekCount) {
             BytesRef nextResult = _next();
             assert nextResult != null;
           }
@@ -647,8 +648,8 @@
          metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
          decode all metadata up to the current term. */
       private BytesRef _next() throws IOException {
-        //System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
-        if (state.termCount == state.blockTermCount && !nextBlock()) {
+        //System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termBlockOrd + " (vs " + blockTermCount + ")");
+        if (state.termBlockOrd == blockTermCount && !nextBlock()) {
           //System.out.println("  eof");
           indexIsCurrent = false;
           return null;
@@ -663,12 +664,12 @@
           term.grow(term.length);
         }
         termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
-        state.termCount++;
+        state.termBlockOrd++;
 
         // NOTE: meaningless in the non-ord case
         state.ord++;
 
-        //System.out.println("  return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term);
+        //System.out.println("  return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term + " tbOrd=" + state.termBlockOrd);
         return term;
       }
 
@@ -695,9 +696,10 @@
       public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
         //System.out.println("BTR.docs this=" + this);
         decodeMetaData();
-        //System.out.println("  state.docFreq=" + state.docFreq);
+        //System.out.println("BTR.docs:  state.docFreq=" + state.docFreq);
         final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, liveDocs, reuse);
         assert docsEnum != null;
+        //System.out.println("BTR.docs:  return docsEnum=" + docsEnum);
         return docsEnum;
       }
 
@@ -716,7 +718,7 @@
 
       @Override
       public void seekExact(BytesRef target, TermState otherState) throws IOException {
-        //System.out.println("BTR.seek termState target=" + target.utf8ToString() + " " + target + " this=" + this);
+        //System.out.println("BTR.seekExact termState target=" + target.utf8ToString() + " " + target + " this=" + this);
         assert otherState != null && otherState instanceof BlockTermState;
         assert !doOrd || ((BlockTermState) otherState).ord < numTerms;
         state.copyFrom(otherState);
@@ -800,9 +802,9 @@
 
         //System.out.println("BTR.nextBlock() fp=" + in.getFilePointer() + " this=" + this);
         state.blockFilePointer = in.getFilePointer();
-        state.blockTermCount = in.readVInt();
-        //System.out.println("  blockTermCount=" + state.blockTermCount);
-        if (state.blockTermCount == 0) {
+        blockTermCount = in.readVInt();
+        //System.out.println("  blockTermCount=" + blockTermCount);
+        if (blockTermCount == 0) {
           return false;
         }
         termBlockPrefix = in.readVInt();
@@ -826,7 +828,7 @@
         freqReader.reset(docFreqBytes, 0, len);
         metaDataUpto = 0;
 
-        state.termCount = 0;
+        state.termBlockOrd = 0;
 
         postingsReader.readTermsBlock(in, fieldInfo, state);
 
@@ -838,7 +840,7 @@
       }
      
       private void decodeMetaData() throws IOException {
-        //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
+        //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termBlockOrd + " state=" + state);
         if (!seekPending) {
           // TODO: cutover to random-access API
           // here.... really stupid that we have to decode N
@@ -846,10 +848,10 @@
           // that we really need...
 
           // lazily catch up on metadata decode:
-          final int limit = state.termCount;
+          final int limit = state.termBlockOrd;
           // We must set/incr state.termCount because
           // postings impl can look at this
-          state.termCount = metaDataUpto;
+          state.termBlockOrd = metaDataUpto;
           // TODO: better API would be "jump straight to term=N"???
           while (metaDataUpto < limit) {
             //System.out.println("  decode mdUpto=" + metaDataUpto);
@@ -870,9 +872,9 @@
 
             postingsReader.nextTerm(fieldInfo, state);
             metaDataUpto++;
-            state.termCount++;
+            state.termBlockOrd++;
           }
-        //} else {
+        } else {
           //System.out.println("  skip! seekPending");
         }
       }
Index: lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java	(working copy)
@@ -84,7 +84,7 @@
   public synchronized Codec lookup(String name) {
     final Codec codec = codecs.get(name);
     if (codec == null) {
-      throw new IllegalArgumentException("required codec '" + name + "' not found");
+      throw new IllegalArgumentException("required codec '" + name + "' not found; known codecs: " + codecs.keySet());
     }
     return codec;
   }
Index: lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java	(working copy)
@@ -19,10 +19,6 @@
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.Lock;
 import org.apache.lucene.util.IOUtils;
 
 import java.util.Collection;
@@ -189,14 +185,14 @@
   }
   
   @Override
-  public synchronized IndexInput openInput(String id, IOContext context) throws IOException {
+  public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException {
     ensureOpen();
     assert !openForWrite;
-    id = IndexFileNames.stripSegmentName(id);
+    final String id = IndexFileNames.stripSegmentName(fileName);
     final FileEntry entry = entries.get(id);
-    if (entry == null)
-      throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
-    
+    if (entry == null) {
+      throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")");
+    }
     return openInputSlice(id, entry.offset, entry.length, readBufferSize);
   }
   
Index: lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java	(working copy)
@@ -43,16 +43,30 @@
     reset(bytes, 0, bytes.length);
   }
 
+  // NOTE: sets pos to 0, which is not right if you had
+  // called reset w/ non-zero offset!!
+  public void rewind() {
+    pos = 0;
+  }
+
   public int getPosition() {
     return pos;
   }
 
+  public void setPosition(int pos) {
+    this.pos = pos;
+  }
+
   public void reset(byte[] bytes, int offset, int len) {
     this.bytes = bytes;
     pos = offset;
     limit = offset + len;
   }
 
+  public int length() {
+    return limit;
+  }
+
   public boolean eof() {
     return pos == limit;
   }
Index: lucene/src/java/org/apache/lucene/store/FSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/FSDirectory.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/store/FSDirectory.java	(working copy)
@@ -448,6 +448,7 @@
     /** output methods: */
     @Override
     public void flushBuffer(byte[] b, int offset, int size) throws IOException {
+      assert isOpen;
       if (rateLimiter != null) {
         rateLimiter.pause(size);
       }
Index: lucene/src/java/org/apache/lucene/util/TermContext.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/TermContext.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/TermContext.java	(working copy)
@@ -21,14 +21,13 @@
 import java.util.Arrays;
 
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader.ReaderContext;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 
 /**
  * Maintains a {@link IndexReader} {@link TermState} view over
@@ -45,6 +44,9 @@
   private int docFreq;
   private long totalTermFreq;
 
+  //public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+
   /**
    * Creates an empty {@link TermContext} from a {@link ReaderContext}
    */
@@ -85,7 +87,9 @@
     final BytesRef bytes = term.bytes();
     final TermContext perReaderTermState = new TermContext(context);
     final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
+    //if (DEBUG) System.out.println("prts.build term=" + term);
     for (int i = 0; i < leaves.length; i++) {
+      //if (DEBUG) System.out.println("  r=" + leaves[i].reader);
       final Fields fields = leaves[i].reader.fields();
       if (fields != null) {
         final Terms terms = fields.terms(field);
@@ -93,6 +97,7 @@
           final TermsEnum termsEnum = terms.getThreadTermsEnum(); // thread-private don't share!
           if (termsEnum.seekExact(bytes, cache)) { 
             final TermState termState = termsEnum.termState();
+            //if (DEBUG) System.out.println("    found");
             perReaderTermState.register(termState, leaves[i].ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
           }
         }
Index: lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java	(revision 0)
@@ -0,0 +1,313 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.PrefixTermsEnum;
+import org.apache.lucene.search.SingleTermsEnum;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Immutable class holding compiled details for a given
+ * Automaton.  The Automaton is deterministic, must not have
+ * dead states but may not be minimal.
+ *
+ * @lucene.experimental
+ */
+public class CompiledAutomaton {
+  public enum AUTOMATON_TYPE {NONE, ALL, SINGLE, PREFIX, NORMAL};
+  public final AUTOMATON_TYPE type;
+
+  // For PREFIX, this is the prefix term; for SINGLE this is
+  // the singleton term:
+  public final BytesRef term;
+
+  // NOTE: the next 4 members are only non-null if type ==
+  // NORMAL:
+  public final ByteRunAutomaton runAutomaton;
+  // TODO: would be nice if these sortedTransitions had "int
+  // to;" instead of "State to;" somehow:
+  public final Transition[][] sortedTransitions;
+  public final BytesRef commonSuffixRef;
+  public final Boolean finite;
+
+  public CompiledAutomaton(Automaton automaton) {
+    this(automaton, null, true);
+  }
+
+  public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
+
+    if (simplify) {
+      // Test whether the automaton is a "simple" form and
+      // if so, don't create a runAutomaton.  Note that on a
+      // large automaton these tests could be costly:
+      if (BasicOperations.isEmpty(automaton)) {
+        // matches nothing
+        type = AUTOMATON_TYPE.NONE;
+        term = null;
+        commonSuffixRef = null;
+        runAutomaton = null;
+        sortedTransitions = null;
+        this.finite = null;
+        return;
+      } else if (BasicOperations.isTotal(automaton)) {
+        // matches all possible strings
+        type = AUTOMATON_TYPE.ALL;
+        term = null;
+        commonSuffixRef = null;
+        runAutomaton = null;
+        sortedTransitions = null;
+        this.finite = null;
+        return;
+      } else {
+        final String commonPrefix;
+        final String singleton;
+        if (automaton.getSingleton() == null) {
+          commonPrefix = SpecialOperations.getCommonPrefix(automaton);
+          if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
+            singleton = commonPrefix;
+          } else {
+            singleton = null;
+          }
+        } else {
+          commonPrefix = null;
+          singleton = automaton.getSingleton();
+        }
+      
+        if (singleton != null) {
+          // matches a fixed string in singleton or expanded
+          // representation
+          type = AUTOMATON_TYPE.SINGLE;
+          term = new BytesRef(singleton);
+          commonSuffixRef = null;
+          runAutomaton = null;
+          sortedTransitions = null;
+          this.finite = null;
+          return;
+        } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
+                                                                                       BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
+          // matches a constant prefix
+          type = AUTOMATON_TYPE.PREFIX;
+          term = new BytesRef(commonPrefix);
+          commonSuffixRef = null;
+          runAutomaton = null;
+          sortedTransitions = null;
+          this.finite = null;
+          return;
+        }
+      }
+    }
+
+    type = AUTOMATON_TYPE.NORMAL;
+    term = null;
+    if (finite == null) {
+      this.finite = SpecialOperations.isFinite(automaton);
+    } else {
+      this.finite = finite;
+    }
+    Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+    if (this.finite) {
+      commonSuffixRef = null;
+    } else {
+      commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+    }
+    runAutomaton = new ByteRunAutomaton(utf8, true);
+    sortedTransitions = utf8.getSortedTransitions();
+  }
+  
+  //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+  private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
+
+    // Find biggest transition that's < label
+    // TODO: use binary search here
+    Transition maxTransition = null;
+    for (Transition transition : sortedTransitions[state]) {
+      if (transition.min < leadLabel) {
+        maxTransition = transition;
+      }
+    }
+
+    assert maxTransition != null;
+
+    // Append floorLabel
+    final int floorLabel;
+    if (maxTransition.max > leadLabel-1) {
+      floorLabel = leadLabel-1;
+    } else {
+      floorLabel = maxTransition.max;
+    }
+    if (idx >= term.bytes.length) {
+      term.grow(1+idx);
+    }
+    //if (DEBUG) System.out.println("  add floorLabel=" + (char) floorLabel + " idx=" + idx);
+    term.bytes[idx] = (byte) floorLabel;
+
+    state = maxTransition.to.getNumber();
+    idx++;
+
+    // Push down to last accept state
+    while (true) {
+      Transition[] transitions = sortedTransitions[state];
+      if (transitions.length == 0) {
+        assert runAutomaton.isAccept(state);
+        term.length = idx;
+        //if (DEBUG) System.out.println("  return " + term.utf8ToString());
+        return term;
+      } else {
+        // We are pushing "top" -- so get last label of
+        // last transition:
+        assert transitions.length != 0;
+        Transition lastTransition = transitions[transitions.length-1];
+        if (idx >= term.bytes.length) {
+          term.grow(1+idx);
+        }
+        //if (DEBUG) System.out.println("  push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
+        term.bytes[idx] = (byte) lastTransition.max;
+        state = lastTransition.to.getNumber();
+        idx++;
+      }
+    }
+  }
+
+  // TODO: should this take startTerm too?  This way
+  // Terms.intersect could forward to this method if type !=
+  // NORMAL:
+  public TermsEnum getTermsEnum(Terms terms) throws IOException {
+    switch(type) {
+    case NONE:
+      return TermsEnum.EMPTY;
+    case ALL:
+      return terms.iterator();
+    case SINGLE:
+      return new SingleTermsEnum(terms.iterator(), term);
+    case PREFIX:
+      // TODO: this is very likely faster than .intersect,
+      // but we should test and maybe cutover
+      return new PrefixTermsEnum(terms.iterator(), term);
+    case NORMAL:
+      return terms.intersect(this, null);
+    default:
+      // unreachable
+      throw new RuntimeException("unhandled case");
+    }
+  }
+
+  /** Finds largest term accepted by this Automaton, that's
+   *  <= the provided input term.  The result is placed in
+   *  output; it's fine for output and input to point to
+   *  the same BytesRef.  The returned result is either the
+   *  provided output, or null if there is no floor term
+   *  (ie, the provided input term is before the first term
+   *  accepted by this Automaton). */
+  public BytesRef floor(BytesRef input, BytesRef output) {
+
+    output.offset = 0;
+    //if (DEBUG) System.out.println("CA.floor input=" + input.utf8ToString());
+
+    int state = runAutomaton.getInitialState();
+
+    // Special case empty string:
+    if (input.length == 0) {
+      if (runAutomaton.isAccept(state)) {
+        output.length = 0;
+        return output;
+      } else {
+        return null;
+      }
+    }
+
+    final List<Integer> stack = new ArrayList<Integer>();
+
+    int idx = 0;
+    while (true) {
+      int label = input.bytes[input.offset + idx] & 0xff;
+      int nextState = runAutomaton.step(state, label);
+      //if (DEBUG) System.out.println("  cycle label=" + (char) label + " nextState=" + nextState);
+
+      if (idx == input.length-1) {
+        if (nextState != -1 && runAutomaton.isAccept(nextState)) {
+          // Input string is accepted
+          if (idx >= output.bytes.length) {
+            output.grow(1+idx);
+          }
+          output.bytes[idx] = (byte) label;
+          output.length = input.length;
+          //if (DEBUG) System.out.println("  input is accepted; return term=" + output.utf8ToString());
+          return output;
+        } else {
+          nextState = -1;
+        }
+      }
+
+      if (nextState == -1) {
+
+        // Pop back to a state that has a transition
+        // <= our label:
+        while (true) {
+          Transition[] transitions = sortedTransitions[state];
+          if (transitions.length == 0) {
+            assert runAutomaton.isAccept(state);
+            output.length = idx;
+            //if (DEBUG) System.out.println("  return " + output.utf8ToString());
+            return output;
+          } else if (label-1 < transitions[0].min) {
+
+            if (runAutomaton.isAccept(state)) {
+              output.length = idx;
+              //if (DEBUG) System.out.println("  return " + output.utf8ToString());
+              return output;
+            }
+            // pop
+            if (stack.size() == 0) {
+              //if (DEBUG) System.out.println("  pop ord=" + idx + " return null");
+              return null;
+            } else {
+              state = stack.remove(stack.size()-1);
+              idx--;
+              //if (DEBUG) System.out.println("  pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
+              label = input.bytes[input.offset + idx] & 0xff;
+            }
+
+          } else {
+            //if (DEBUG) System.out.println("  stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
+            break;
+          }
+        }
+
+        //if (DEBUG) System.out.println("  label=" + (char) label + " idx=" + idx);
+
+        return addTail(state, output, idx, label);
+        
+      } else {
+        if (idx >= output.bytes.length) {
+          output.grow(1+idx);
+        }
+        output.bytes[idx] = (byte) label;
+        stack.add(state);
+        state = nextState;
+        idx++;
+      }
+    }
+  }
+}

Property changes on: lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java	(working copy)
@@ -133,6 +133,6 @@
 
   @Override
   public String outputToString(BytesRef output) {
-    return output.utf8ToString();
+    return output.toString();
   }
 }
Index: lucene/src/java/org/apache/lucene/util/fst/Util.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/fst/Util.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/fst/Util.java	(working copy)
@@ -213,6 +213,7 @@
 
     // Shape for states.
     final String stateShape = "circle";
+    final String finalStateShape = "doublecircle";
 
     // Emit DOT prologue.
     out.write("digraph FST {\n");
@@ -223,12 +224,34 @@
     }
 
     emitDotState(out, "initial", "point", "white", "");
-    emitDotState(out, Integer.toString(startArc.target), stateShape, 
-        fst.isExpandedTarget(startArc) ? expandedNodeColor : null, 
-        "");
+
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+
+    // final FST.Arc<T> scratchArc = new FST.Arc<T>();
+
+    {
+      final String stateColor;
+      if (fst.isExpandedTarget(startArc)) {
+        stateColor = expandedNodeColor;
+      } else {
+        stateColor = null;
+      }
+
+      final boolean isFinal;
+      final T finalOutput;
+      if (startArc.isFinal()) {
+        isFinal = true;
+        finalOutput = startArc.nextFinalOutput == NO_OUTPUT ? null : startArc.nextFinalOutput;
+      } else {
+        isFinal = false;
+        finalOutput = null;
+      }
+      
+      emitDotState(out, Integer.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
+    }
+
     out.write("  initial -> " + startArc.target + "\n");
 
-    final T NO_OUTPUT = fst.outputs.getNoOutput();
     int level = 0;
 
     while (!nextLevelQueue.isEmpty()) {
@@ -240,19 +263,48 @@
       out.write("\n  // Transitions and states at level: " + level + "\n");
       while (!thisLevelQueue.isEmpty()) {
         final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
-        
         if (fst.targetHasArcs(arc)) {
           // scan all arcs
           final int node = arc.target;
           fst.readFirstTargetArc(arc, arc);
-          
+
+          if (arc.label == FST.END_LABEL) {
+            // Skip it -- prior recursion took this into account already
+            assert !arc.isLast();
+            fst.readNextArc(arc);
+          }
+
           while (true) {
+
             // Emit the unseen state and add it to the queue for the next level.
             if (arc.target >= 0 && !seen.get(arc.target)) {
-              final boolean isExpanded = fst.isExpandedTarget(arc);
-              emitDotState(out, Integer.toString(arc.target), stateShape, 
-                  isExpanded ?  expandedNodeColor : null, 
-                  labelStates ? Integer.toString(arc.target) : ""); 
+
+              /*
+              boolean isFinal = false;
+              T finalOutput = null;
+              fst.readFirstTargetArc(arc, scratchArc);
+              if (scratchArc.isFinal() && fst.targetHasArcs(scratchArc)) {
+                // target is final
+                isFinal = true;
+                finalOutput = scratchArc.output == NO_OUTPUT ? null : scratchArc.output;
+                System.out.println("dot hit final label=" + (char) scratchArc.label);
+              }
+              */
+              final String stateColor;
+              if (fst.isExpandedTarget(arc)) {
+                stateColor = expandedNodeColor;
+              } else {
+               stateColor = null;
+              }
+
+              final String finalOutput;
+              if (arc.nextFinalOutput != null && arc.nextFinalOutput != NO_OUTPUT) {
+                finalOutput = fst.outputs.outputToString(arc.nextFinalOutput);
+              } else {
+                finalOutput = "";
+              }
+
+              emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput);
               seen.set(arc.target);
               nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
               sameLevelStates.add(arc.target);
@@ -265,15 +317,19 @@
               outs = "";
             }
 
-            final String cl;
-            if (arc.label == FST.END_LABEL) {
-              cl = "~";
-            } else {
-              cl = printableLabel(arc.label);
+            if (!fst.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
+              // Tricky special case: sometimes, due to
+              // pruning, the builder can [sillily] produce
+              // an FST with an arc into the final end state
+              // (-1) but also with a next final output; in
+              // this case we pull that output up onto this
+              // arc
+              outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
             }
 
-            out.write("  " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]\n");
-            
+            assert arc.label != FST.END_LABEL;
+            out.write("  " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n");
+                   
             // Break the loop if we're on the last arc of this state.
             if (arc.isLast()) {
               break;
@@ -295,7 +351,7 @@
     }
 
     // Emit terminating state (always there anyway).
-    out.write("  -1 [style=filled, color=black, shape=circle, label=\"\"]\n\n");
+    out.write("  -1 [style=filled, color=black, shape=doublecircle, label=\"\"]\n\n");
     out.write("  {rank=sink; -1 }\n");
     
     out.write("}\n");
Index: lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java	(working copy)
@@ -39,8 +39,8 @@
 public final class UpToTwoPositiveIntOutputs extends Outputs<Object> {
 
   public final static class TwoLongs {
-    final long first;
-    final long second;
+    public final long first;
+    public final long second;
 
     public TwoLongs(long first, long second) {
       this.first = first;
Index: lucene/src/java/org/apache/lucene/util/fst/Builder.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/fst/Builder.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/fst/Builder.java	(working copy)
@@ -53,6 +53,8 @@
   private final FST<T> fst;
   private final T NO_OUTPUT;
 
+  // private static final boolean DEBUG = false;
+
   // simplistic pruning: we prune node (and all following
   // nodes) if less than this number of terms go through it:
   private final int minSuffixCount1;
@@ -73,13 +75,21 @@
   // current "frontier"
   private UnCompiledNode<T>[] frontier;
 
+  // Expert: you pass an instance of this if you want to do
+  // something "custom" as suffixes are "frozen":
+  public static abstract class FreezeTail<T> {
+    public abstract void freeze(final UnCompiledNode<T>[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException;
+  }
+
+  private final FreezeTail<T> freezeTail;
+
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs)} with 
    * pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
   }
 
   /**
@@ -120,9 +130,11 @@
    *    singleton output object.
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
-                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs) {
+                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
+                 FreezeTail<T> freezeTail) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
+    this.freezeTail = freezeTail;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
     fst = new FST<T>(inputType, outputs);
@@ -179,94 +191,100 @@
     return fn;
   }
 
-  private void compilePrevTail(int prefixLenPlus1) throws IOException {
-    assert prefixLenPlus1 >= 1;
-    //System.out.println("  compileTail " + prefixLenPlus1);
-    for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
-      boolean doPrune = false;
-      boolean doCompile = false;
+  private void freezeTail(int prefixLenPlus1) throws IOException {
+    if (freezeTail != null) {
+      // Custom plugin:
+      freezeTail.freeze(frontier, prefixLenPlus1, lastInput);
+    } else {
+      //System.out.println("  compileTail " + prefixLenPlus1);
+      final int downTo = Math.max(1, prefixLenPlus1);
+      for(int idx=lastInput.length; idx >= downTo; idx--) {
 
-      final UnCompiledNode<T> node = frontier[idx];
-      final UnCompiledNode<T> parent = frontier[idx-1];
+        boolean doPrune = false;
+        boolean doCompile = false;
 
-      if (node.inputCount < minSuffixCount1) {
-        doPrune = true;
-        doCompile = true;
-      } else if (idx > prefixLenPlus1) {
-        // prune if parent's inputCount is less than suffixMinCount2
-        if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) {
-          // my parent, about to be compiled, doesn't make the cut, so
-          // I'm definitely pruned 
+        final UnCompiledNode<T> node = frontier[idx];
+        final UnCompiledNode<T> parent = frontier[idx-1];
 
-          // if pruneCount2 is 1, we keep only up
-          // until the 'distinguished edge', ie we keep only the
-          // 'divergent' part of the FST. if my parent, about to be
-          // compiled, has inputCount 1 then we are already past the
-          // distinguished edge.  NOTE: this only works if
-          // the FST outputs are not "compressible" (simple
-          // ords ARE compressible).
+        if (node.inputCount < minSuffixCount1) {
           doPrune = true;
+          doCompile = true;
+        } else if (idx > prefixLenPlus1) {
+          // prune if parent's inputCount is less than suffixMinCount2
+          if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
+            // my parent, about to be compiled, doesn't make the cut, so
+            // I'm definitely pruned 
+
+            // if minSuffixCount2 is 1, we keep only up
+            // until the 'distinguished edge', ie we keep only the
+            // 'divergent' part of the FST. if my parent, about to be
+            // compiled, has inputCount 1 then we are already past the
+            // distinguished edge.  NOTE: this only works if
+            // the FST outputs are not "compressible" (simple
+            // ords ARE compressible).
+            doPrune = true;
+          } else {
+            // my parent, about to be compiled, does make the cut, so
+            // I'm definitely not pruned 
+            doPrune = false;
+          }
+          doCompile = true;
         } else {
-          // my parent, about to be compiled, does make the cut, so
-          // I'm definitely not pruned 
-          doPrune = false;
+          // if pruning is disabled (count is 0) we can always
+          // compile current node
+          doCompile = minSuffixCount2 == 0;
         }
-        doCompile = true;
-      } else {
-        // if pruning is disabled (count is 0) we can always
-        // compile current node
-        doCompile = minSuffixCount2 == 0;
-      }
 
-      //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
+        //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
 
-      if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) {
-        // drop all arcs
-        for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-          @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
-          target.clear();
+        if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
+          // drop all arcs
+          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+            @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
+            target.clear();
+          }
+          node.numArcs = 0;
         }
-        node.numArcs = 0;
-      }
 
-      if (doPrune) {
-        // this node doesn't make it -- deref it
-        node.clear();
-        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
-      } else {
+        if (doPrune) {
+          // this node doesn't make it -- deref it
+          node.clear();
+          parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
+        } else {
 
-        if (minSuffixCount2 != 0) {
-          compileAllTargets(node, lastInput.length-idx);
-        }
-        final T nextFinalOutput = node.output;
+          if (minSuffixCount2 != 0) {
+            compileAllTargets(node, lastInput.length-idx);
+          }
+          final T nextFinalOutput = node.output;
 
-        // We "fake" the node as being final if it has no
-        // outgoing arcs; in theory we could leave it
-        // as non-final (the FST can represent this), but
-        // FSTEnum, Util, etc., have trouble w/ non-final
-        // dead-end states:
-        final boolean isFinal = node.isFinal || node.numArcs == 0;
+          // We "fake" the node as being final if it has no
+          // outgoing arcs; in theory we could leave it
+          // as non-final (the FST can represent this), but
+          // FSTEnum, Util, etc., have trouble w/ non-final
+          // dead-end states:
+          final boolean isFinal = node.isFinal || node.numArcs == 0;
 
-        if (doCompile) {
-          // this node makes it and we now compile it.  first,
-          // compile any targets that were previously
-          // undecided:
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             compileNode(node, 1+lastInput.length-idx),
-                             nextFinalOutput,
-                             isFinal);
-        } else {
-          // replaceLast just to install
-          // nextFinalOutput/isFinal onto the arc
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             node,
-                             nextFinalOutput,
-                             isFinal);
-          // this node will stay in play for now, since we are
-          // undecided on whether to prune it.  later, it
-          // will be either compiled or pruned, so we must
-          // allocate a new node:
-          frontier[idx] = new UnCompiledNode<T>(this, idx);
+          if (doCompile) {
+            // this node makes it and we now compile it.  first,
+            // compile any targets that were previously
+            // undecided:
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               compileNode(node, 1+lastInput.length-idx),
+                               nextFinalOutput,
+                               isFinal);
+          } else {
+            // replaceLast just to install
+            // nextFinalOutput/isFinal onto the arc
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               node,
+                               nextFinalOutput,
+                               isFinal);
+            // this node will stay in play for now, since we are
+            // undecided on whether to prune it.  later, it
+            // will be either compiled or pruned, so we must
+            // allocate a new node:
+            frontier[idx] = new UnCompiledNode<T>(this, idx);
+          }
         }
       }
     }
@@ -320,11 +338,36 @@
     add(scratchIntsRef, output);
   }
 
+  // for debugging
+  /*
+  private String toString(BytesRef b) {
+    try {
+      return b.utf8ToString() + " " + b;
+    } catch (Throwable t) {
+      return b.toString();
+    }
+  }
+  */
+
   /** It's OK to add the same input twice in a row with
    *  different outputs, as long as outputs impls the merge
    *  method. */
   public void add(IntsRef input, T output) throws IOException {
-    //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
+    /*
+    if (DEBUG) {
+      BytesRef b = new BytesRef(input.length);
+      for(int x=0;x<input.length;x++) {
+        b.bytes[x] = (byte) input.ints[x];
+      }
+      b.length = input.length;
+      if (output == NO_OUTPUT) {
+        System.out.println("\nFST ADD: input=" + toString(b) + " " + b);
+      } else {
+        System.out.println("\nFST ADD: input=" + toString(b) + " " + b + " output=" + fst.outputs.outputToString(output));
+      }
+    }
+    */
+
     assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
     assert validOutput(output);
 
@@ -346,8 +389,8 @@
     int pos2 = input.offset;
     final int pos1Stop = Math.min(lastInput.length, input.length);
     while(true) {
-      //System.out.println("  incr " + pos1);
       frontier[pos1].inputCount++;
+      //System.out.println("  incr " + pos1 + " ct=" + frontier[pos1].inputCount + " n=" + frontier[pos1]);
       if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
         break;
       }
@@ -368,13 +411,12 @@
 
     // minimize/compile states from previous input's
     // orphan'd suffix
-    compilePrevTail(prefixLenPlus1);
+    freezeTail(prefixLenPlus1);
 
     // init tail states for current input
     for(int idx=prefixLenPlus1;idx<=input.length;idx++) {
       frontier[idx-1].addArc(input.ints[input.offset + idx - 1],
                              frontier[idx]);
-      //System.out.println("  incr tail " + idx);
       frontier[idx].inputCount++;
     }
 
@@ -433,34 +475,25 @@
    *  nothing is accepted by the FST. */
   public FST<T> finish() throws IOException {
 
+    final UnCompiledNode<T> root = frontier[0];
+
     // minimize nodes in the last word's suffix
-    compilePrevTail(1);
-    //System.out.println("finish: inputCount=" + frontier[0].inputCount);
-    if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
+    freezeTail(0);
+    if (root.inputCount < minSuffixCount1 || root.inputCount < minSuffixCount2 || root.numArcs == 0) {
       if (fst.emptyOutput == null) {
         return null;
       } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
         // empty string got pruned
         return null;
-      } else {
-        fst.finish(compileNode(frontier[0], lastInput.length).address);
-        //System.out.println("compile addr = " + fst.getStartNode());
-        return fst;
       }
     } else {
       if (minSuffixCount2 != 0) {
-        compileAllTargets(frontier[0], lastInput.length);
+        compileAllTargets(root, lastInput.length);
       }
-      //System.out.println("NOW: " + frontier[0].numArcs);
-      fst.finish(compileNode(frontier[0], lastInput.length).address);
     }
+    //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
+    fst.finish(compileNode(root, lastInput.length).address);
 
-    /*
-    if (dedupHash != null) {
-      System.out.println("NH: " + dedupHash.count()); 
-    }
-    */
-    
     return fst;
   }
 
@@ -479,7 +512,7 @@
     }
   }
 
-  static class Arc<T> {
+  public static class Arc<T> {
     public int label;                             // really an "unsigned" byte
     public Node target;
     public boolean isFinal;
@@ -502,16 +535,20 @@
     }
   }
 
-  static final class UnCompiledNode<T> implements Node {
+  public static final class UnCompiledNode<T> implements Node {
     final Builder<T> owner;
-    int numArcs;
-    Arc<T>[] arcs;
-    T output;
-    boolean isFinal;
-    long inputCount;
+    public int numArcs;
+    public Arc<T>[] arcs;
+    // TODO: instead of recording isFinal/output on the
+    // node, maybe we should use -1 arc to mean "end" (like
+    // we do when reading the FST).  Would simplify much
+    // code here...
+    public T output;
+    public boolean isFinal;
+    public long inputCount;
 
     /** This node's depth, starting from the automaton root. */
-    final int depth;
+    public final int depth;
 
     /**
      * @param depth
Index: lucene/src/java/org/apache/lucene/util/fst/FST.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/fst/FST.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/fst/FST.java	(working copy)
@@ -123,7 +123,7 @@
     public int label;
     public T output;
 
-    int target;
+    public int target;
 
     byte flags;
     public T nextFinalOutput;
@@ -274,6 +274,10 @@
     }
   }
 
+  public T getEmptyOutput() {
+    return emptyOutput;
+  }
+
   void setEmptyOutput(T v) throws IOException {
     if (emptyOutput != null) {
       emptyOutput = outputs.merge(emptyOutput, v);
@@ -597,9 +601,9 @@
       arc.label = END_LABEL;
       arc.output = follow.nextFinalOutput;
       if (follow.target <= 0) {
-        arc.flags = BIT_LAST_ARC;
+        arc.flags = BIT_LAST_ARC | BIT_FINAL_ARC;
       } else {
-        arc.flags = 0;
+        arc.flags = BIT_FINAL_ARC;
         arc.nextArc = follow.target;
       }
       //System.out.println("    insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
@@ -609,8 +613,7 @@
     }
   }
 
-  // Not private because NodeHash needs access:
-  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+  public Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
 
     final BytesReader in = getBytesReader(address);
 
@@ -693,7 +696,9 @@
     return readLabel(in);
   }
 
-  Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+  /** Never returns null, but you should never call this if
+   *  arc.isLast() is true. */
+  public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
     // this is a continuing arc in a fixed array
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
@@ -925,7 +930,7 @@
     }
   }
 
-  final BytesReader getBytesReader(int pos) {
+  public final BytesReader getBytesReader(int pos) {
     // TODO: maybe re-use via ThreadLocal?
     return new BytesReader(pos);
   }
Index: lucene/src/java/org/apache/lucene/util/BytesRef.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/BytesRef.java	(revision 1159650)
+++ lucene/src/java/org/apache/lucene/util/BytesRef.java	(working copy)
@@ -65,6 +65,18 @@
     this.bytes = new byte[capacity];
   }
 
+  /** Incoming IntsRef values must be Byte.MIN_VALUE -
+   *  Byte.MAX_VALUE. */
+  public BytesRef(IntsRef intsRef) {
+    bytes = new byte[intsRef.length];
+    for(int idx=0;idx<intsRef.length;idx++) {
+      final int v = intsRef.ints[intsRef.offset + idx];
+      assert v >= Byte.MIN_VALUE && v <= Byte.MAX_VALUE;
+      bytes[idx] = (byte) v;
+    }
+    length = intsRef.length;
+  }
+
   /**
    * @param text Initialize the byte[] from the UTF8 bytes
    * for the provided Sring.  This must be well-formed
Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java	(working copy)
@@ -182,9 +182,9 @@
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
-                                                                      state.segmentInfo,
-                                                                      state.context,
-                                                                      new MockIntFactory(baseBlockSize), state.codecId);
+                                                                           state.segmentInfo,
+                                                                           state.context,
+                                                                           new MockIntFactory(baseBlockSize), state.codecId);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java	(working copy)
@@ -159,7 +159,7 @@
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
-                                                                      state.segmentInfo,
+                                                                           state.segmentInfo,
                                                                       state.context,
                                                                       new MockIntFactory(blockSize), state.codecId);
 
Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java	(working copy)
@@ -66,6 +66,11 @@
     out.close();
   }
 
+  @Override
+  public String toString() {
+    return "MockSingleIntIndexOutput fp=" + out.getFilePointer();
+  }
+
   private class Index extends IntIndexOutput.Index {
     long fp;
     long lastFP;
@@ -74,8 +79,11 @@
       fp = out.getFilePointer();
     }
     @Override
-    public void set(IntIndexOutput.Index other) {
-      lastFP = fp = ((Index) other).fp;
+    public void copyFrom(IntIndexOutput.Index other, boolean copyLast) {
+      fp = ((Index) other).fp;
+      if (copyLast) {
+        lastFP = ((Index) other).fp;
+      }
     }
     @Override
     public void write(IndexOutput indexOut, boolean absolute)
Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java	(working copy)
@@ -30,6 +30,8 @@
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.Codec;
@@ -137,7 +139,7 @@
     final long seed = seedRandom.nextLong();
 
     if (LuceneTestCase.VERBOSE) {
-      System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " seed=" + seed);
+      System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " codecID=" + state.codecId + " seed=" + seed);
     }
 
     final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
@@ -151,9 +153,8 @@
     final Random random = new Random(seed);
     
     random.nextInt(); // consume a random for buffersize
-    
+
     PostingsWriterBase postingsWriter;
-
     if (random.nextBoolean()) {
       postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval);
     } else {
@@ -166,76 +167,107 @@
     if (random.nextBoolean()) {
       final int totTFCutoff = _TestUtil.nextInt(random, 1, 20);
       if (LuceneTestCase.VERBOSE) {
-        System.out.println("MockRandomCodec: pulsing postings with totTFCutoff=" + totTFCutoff);
+        System.out.println("MockRandomCodec: writing pulsing postings with totTFCutoff=" + totTFCutoff);
       }
       postingsWriter = new PulsingPostingsWriterImpl(totTFCutoff, postingsWriter);
     }
 
-    final TermsIndexWriterBase indexWriter;
-    boolean success = false;
+    final FieldsConsumer fields;
 
-    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 + ")");
+    if (random.nextBoolean()) {
+      // Use BlockTree terms dict
+
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing BlockTree terms dict");
+      }
+
+      // TODO: would be nice to allow 1 but this is very
+      // slow to write
+      final int minTermsInBlock = _TestUtil.nextInt(random, 2, 100);
+      final int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random.nextInt(100));
+
+      boolean success = false;
+      try {
+        fields = new BlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
         }
-        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);
+      }
+    } else {
+
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing Block terms dict");
+      }
+
+      boolean success = false;
+
+      final TermsIndexWriterBase indexWriter;
+      try {
+        if (random.nextBoolean()) {
+          state.termIndexInterval = _TestUtil.nextInt(random, 1, 100);
           if (LuceneTestCase.VERBOSE) {
-            System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")");
+            System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")");
           }
-        } 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);
+          indexWriter = new FixedGapTermsIndexWriter(state);
         } 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);
+          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, TermStats stats) {
-                return rand.nextInt(gap) == gap/2;
-              }
+                @Override
+                public boolean isIndexTerm(BytesRef term, TermStats stats) {
+                  return rand.nextInt(gap) == gap/2;
+                }
 
-              @Override
-              public void newField(FieldInfo fieldInfo) {
-              }
-            };
+                @Override
+                  public void newField(FieldInfo fieldInfo) {
+                }
+              };
+          }
+          indexWriter = new VariableGapTermsIndexWriter(state, selector);
         }
-        indexWriter = new VariableGapTermsIndexWriter(state, selector);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
+        }
       }
-      success = true;
-    } finally {
-      if (!success) {
-        postingsWriter.close();
-      }
-    }
 
-    success = false;
-    try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        try {
-          postingsWriter.close();
-        } finally {
-          indexWriter.close();
+      success = false;
+      try {
+        fields = new BlockTermsWriter(indexWriter, state, postingsWriter);
+        success = true;
+      } finally {
+        if (!success) {
+          try {
+            postingsWriter.close();
+          } finally {
+            indexWriter.close();
+          }
         }
       }
     }
+
+    return fields;
   }
 
   @Override
@@ -245,7 +277,7 @@
     final IndexInput in = state.dir.openInput(seedFileName, state.context);
     final long seed = in.readLong();
     if (LuceneTestCase.VERBOSE) {
-      System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " seed=" + seed);
+      System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " codecID=" + state.codecId + " seed=" + seed);
     }
     in.close();
 
@@ -259,6 +291,9 @@
     PostingsReaderBase postingsReader;
 
     if (random.nextBoolean()) {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading Sep postings");
+      }
       postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
                                                  state.context, new MockIntStreamFactory(random), state.codecId);
     } else {
@@ -276,74 +311,106 @@
       postingsReader = new PulsingPostingsReaderImpl(postingsReader);
     }
 
-    final TermsIndexReaderBase indexReader;
-    boolean success = false;
+    final FieldsProducer fields;
 
-    try {
-      if (random.nextBoolean()) {
-        // if termsIndexDivisor is set to -1, we should not touch it. It means a
-        // test explicitly instructed not to load the terms index.
+    if (random.nextBoolean()) {
+      // Use BlockTree terms dict
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading BlockTree terms dict");
+      }
+
+      boolean success = false;
+      try {
+        fields = new BlockTreeTermsReader(state.dir,
+                                          state.fieldInfos,
+                                          state.segmentInfo.name,
+                                          postingsReader,
+                                          state.context,
+                                          state.codecId,
+                                          state.termsIndexDivisor);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
+        }
+      }
+    } else {
+
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading Block terms dict");
+      }
+      final TermsIndexReaderBase indexReader;
+      boolean success = false;
+      try {
+        final boolean doFixedGap = random.nextBoolean();
+
+        // randomness diverges from writer, here:
         if (state.termsIndexDivisor != -1) {
           state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
         }
-        if (LuceneTestCase.VERBOSE) {
-          System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
+
+        if (doFixedGap) {
+          // if termsIndexDivisor is set to -1, we should not touch it. It means a
+          // test explicitly instructed not to load the terms index.
+          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, state.context);
+        } 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 + ")");
+          }
+          indexReader = new VariableGapTermsIndexReader(state.dir,
+                                                        state.fieldInfos,
+                                                        state.segmentInfo.name,
+                                                        state.termsIndexDivisor,
+                                                        state.codecId, state.context);
+
         }
-        indexReader = new FixedGapTermsIndexReader(state.dir,
-                                                   state.fieldInfos,
-                                                   state.segmentInfo.name,
-                                                   state.termsIndexDivisor,
-                                                   BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                   state.codecId, state.context);
-      } else {
-        final int n2 = random.nextInt(3);
-        if (n2 == 1) {
-          random.nextInt();
-        } else if (n2 == 2) {
-          random.nextLong();
+
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
         }
-        if (LuceneTestCase.VERBOSE) {
-          System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
-        }
-        if (state.termsIndexDivisor != -1) {
-          state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
-        }
-        indexReader = new VariableGapTermsIndexReader(state.dir,
-                                                      state.fieldInfos,
-                                                      state.segmentInfo.name,
-                                                      state.termsIndexDivisor,
-                                                      state.codecId, state.context);
       }
-      success = true;
-    } finally {
-      if (!success) {
-        postingsReader.close();
-      }
-    }
 
-    final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024);
+      final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024);
 
-    success = false;
-    try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
-                                                state.fieldInfos,
-                                                state.segmentInfo.name,
-                                                postingsReader,
-                                                state.context,
-                                                termsCacheSize,
-                                                state.codecId);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        try {
-          postingsReader.close();
-        } finally {
-          indexReader.close();
+      success = false;
+      try {
+        fields = new BlockTermsReader(indexReader,
+                                      state.dir,
+                                      state.fieldInfos,
+                                      state.segmentInfo.name,
+                                      postingsReader,
+                                      state.context,
+                                      termsCacheSize,
+                                      state.codecId);
+        success = true;
+      } finally {
+        if (!success) {
+          try {
+            postingsReader.close();
+          } finally {
+            indexReader.close();
+          }
         }
       }
     }
+
+    return fields;
   }
 
   @Override
@@ -353,6 +420,7 @@
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     StandardPostingsReader.files(dir, segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@@ -371,6 +439,7 @@
   public void getExtensions(Set<String> extensions) {
     SepPostingsWriterImpl.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
+    BlockTreeTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
     DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
Index: lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java	(working copy)
@@ -60,7 +60,7 @@
 
     private final Random r;
 
-    public MockIndexWriter(Random r,Directory dir, IndexWriterConfig conf) throws IOException {
+    public MockIndexWriter(Random r, Directory dir, IndexWriterConfig conf) throws IOException {
       super(dir, conf);
       // must make a private random since our methods are
       // called from different threads; else test failures may
Index: lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java	(revision 0)
+++ lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java	(revision 0)
@@ -0,0 +1,361 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.*;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
+
+/**
+ * Builds a minimal deterministic automaton that accepts a set of strings. The
+ * algorithm requires sorted input data, but is very fast (nearly linear with
+ * the input size).
+ */
+public final class DaciukMihovAutomatonBuilder {
+  /**
+   * DFSA state with <code>char</code> labels on transitions.
+   */
+  final static class State {
+    
+    /** An empty set of labels. */
+    private final static int[] NO_LABELS = new int[0];
+    
+    /** An empty set of states. */
+    private final static State[] NO_STATES = new State[0];
+    
+    /**
+     * Labels of outgoing transitions. Indexed identically to {@link #states}.
+     * Labels must be sorted lexicographically.
+     */
+    int[] labels = NO_LABELS;
+    
+    /**
+     * States reachable from outgoing transitions. Indexed identically to
+     * {@link #labels}.
+     */
+    State[] states = NO_STATES;
+    
+    /**
+     * <code>true</code> if this state corresponds to the end of at least one
+     * input sequence.
+     */
+    boolean is_final;
+    
+    /**
+     * Returns the target state of a transition leaving this state and labeled
+     * with <code>label</code>. If no such transition exists, returns
+     * <code>null</code>.
+     */
+    public State getState(int label) {
+      final int index = Arrays.binarySearch(labels, label);
+      return index >= 0 ? states[index] : null;
+    }
+    
+    /**
+     * Returns an array of outgoing transition labels. The array is sorted in
+     * lexicographic order and indexes correspond to states returned from
+     * {@link #getStates()}.
+     */
+    public int[] getTransitionLabels() {
+      return this.labels;
+    }
+    
+    /**
+     * Returns an array of outgoing transitions from this state. The returned
+     * array must not be changed.
+     */
+    public State[] getStates() {
+      return this.states;
+    }
+    
+    /**
+     * Two states are equal if:
+     * <ul>
+     * <li>they have an identical number of outgoing transitions, labeled with
+     * the same labels</li>
+     * <li>corresponding outgoing transitions lead to the same states (to states
+     * with an identical right-language).
+     * </ul>
+     */
+    @Override
+    public boolean equals(Object obj) {
+      final State other = (State) obj;
+      return is_final == other.is_final
+          && Arrays.equals(this.labels, other.labels)
+          && referenceEquals(this.states, other.states);
+    }
+    
+    /**
+     * Return <code>true</code> if this state has any children (outgoing
+     * transitions).
+     */
+    public boolean hasChildren() {
+      return labels.length > 0;
+    }
+    
+    /**
+     * Is this state a final state in the automaton?
+     */
+    public boolean isFinal() {
+      return is_final;
+    }
+    
+    /**
+     * Compute the hash code of the <i>current</i> status of this state.
+     */
+    @Override
+    public int hashCode() {
+      int hash = is_final ? 1 : 0;
+      
+      hash ^= hash * 31 + this.labels.length;
+      for (int c : this.labels)
+        hash ^= hash * 31 + c;
+      
+      /*
+       * Compare the right-language of this state using reference-identity of
+       * outgoing states. This is possible because states are interned (stored
+       * in registry) and traversed in post-order, so any outgoing transitions
+       * are already interned.
+       */
+      for (State s : this.states) {
+        hash ^= System.identityHashCode(s);
+      }
+      
+      return hash;
+    }
+    
+    /**
+     * Create a new outgoing transition labeled <code>label</code> and return
+     * the newly created target state for this transition.
+     */
+    State newState(int label) {
+      assert Arrays.binarySearch(labels, label) < 0 : "State already has transition labeled: "
+          + label;
+      
+      labels = copyOf(labels, labels.length + 1);
+      states = copyOf(states, states.length + 1);
+      
+      labels[labels.length - 1] = label;
+      return states[states.length - 1] = new State();
+    }
+    
+    /**
+     * Return the most recent transitions's target state.
+     */
+    State lastChild() {
+      assert hasChildren() : "No outgoing transitions.";
+      return states[states.length - 1];
+    }
+    
+    /**
+     * Return the associated state if the most recent transition is labeled with
+     * <code>label</code>.
+     */
+    State lastChild(int label) {
+      final int index = labels.length - 1;
+      State s = null;
+      if (index >= 0 && labels[index] == label) {
+        s = states[index];
+      }
+      assert s == getState(label);
+      return s;
+    }
+    
+    /**
+     * Replace the last added outgoing transition's target state with the given
+     * state.
+     */
+    void replaceLastChild(State state) {
+      assert hasChildren() : "No outgoing transitions.";
+      states[states.length - 1] = state;
+    }
+    
+    /**
+     * JDK1.5-replacement of {@link Arrays#copyOf(int[], int)}
+     */
+    private static int[] copyOf(int[] original, int newLength) {
+      int[] copy = new int[newLength];
+      System.arraycopy(original, 0, copy, 0,
+          Math.min(original.length, newLength));
+      return copy;
+    }
+    
+    /**
+     * JDK1.5-replacement of {@link Arrays#copyOf(char[], int)}
+     */
+    public static State[] copyOf(State[] original, int newLength) {
+      State[] copy = new State[newLength];
+      System.arraycopy(original, 0, copy, 0,
+          Math.min(original.length, newLength));
+      return copy;
+    }
+    
+    /**
+     * Compare two lists of objects for reference-equality.
+     */
+    private static boolean referenceEquals(Object[] a1, Object[] a2) {
+      if (a1.length != a2.length) return false;
+      
+      for (int i = 0; i < a1.length; i++)
+        if (a1[i] != a2[i]) return false;
+      
+      return true;
+    }
+  }
+  
+  /**
+   * "register" for state interning.
+   */
+  private HashMap<State,State> register = new HashMap<State,State>();
+  
+  /**
+   * Root automaton state.
+   */
+  private State root = new State();
+  
+  /**
+   * Previous sequence added to the automaton in {@link #add(CharSequence)}.
+   */
+  private CharsRef previous;
+  
+  private static final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
+
+  /**
+   * Add another character sequence to this automaton. The sequence must be
+   * lexicographically larger or equal compared to any previous sequences added
+   * to this automaton (the input must be sorted).
+   */
+  public void add(CharsRef current) {
+    assert register != null : "Automaton already built.";
+    assert previous == null
+        || comparator.compare(previous, current) <= 0 : "Input must be sorted: "
+        + previous + " >= " + current;
+    assert setPrevious(current);
+    
+    // Descend in the automaton (find matching prefix).
+    int pos = 0, max = current.length();
+    State next, state = root;
+    while (pos < max && (next = state.lastChild(Character.codePointAt(current, pos))) != null) {
+      state = next;
+      // todo, optimize me
+      pos += Character.charCount(Character.codePointAt(current, pos));
+    }
+    
+    if (state.hasChildren()) replaceOrRegister(state);
+    
+    addSuffix(state, current, pos);
+  }
+  
+  /**
+   * Finalize the automaton and return the root state. No more strings can be
+   * added to the builder after this call.
+   * 
+   * @return Root automaton state.
+   */
+  public State complete() {
+    if (this.register == null) throw new IllegalStateException();
+    
+    if (root.hasChildren()) replaceOrRegister(root);
+    
+    register = null;
+    return root;
+  }
+  
+  /**
+   * Internal recursive traversal for conversion.
+   */
+  private static org.apache.lucene.util.automaton.State convert(State s,
+      IdentityHashMap<State,org.apache.lucene.util.automaton.State> visited) {
+    org.apache.lucene.util.automaton.State converted = visited.get(s);
+    if (converted != null) return converted;
+    
+    converted = new org.apache.lucene.util.automaton.State();
+    converted.setAccept(s.is_final);
+    
+    visited.put(s, converted);
+    int i = 0;
+    int[] labels = s.labels;
+    for (DaciukMihovAutomatonBuilder.State target : s.states) {
+      converted.addTransition(new Transition(labels[i++], convert(target,
+          visited)));
+    }
+    
+    return converted;
+  }
+  
+  /**
+   * Build a minimal, deterministic automaton from a sorted list of strings.
+   */
+  public static Automaton build(Collection<BytesRef> input) {
+    final DaciukMihovAutomatonBuilder builder = new DaciukMihovAutomatonBuilder();
+    
+    CharsRef scratch = new CharsRef();
+    for (BytesRef b : input) {
+      UnicodeUtil.UTF8toUTF16(b, scratch);
+      builder.add(scratch);
+    }
+    
+    Automaton a = new Automaton();
+    a.initial = convert(builder.complete(), new IdentityHashMap<State,org.apache.lucene.util.automaton.State>());
+    a.deterministic = true;
+    return a;
+  }
+
+  /**
+   * Copy <code>current</code> into an internal buffer.
+   */
+  private boolean setPrevious(CharsRef current) {
+    // don't need to copy, once we fix https://issues.apache.org/jira/browse/LUCENE-3277
+    // still, called only from assert
+    previous = new CharsRef(current);
+    return true;
+  }
+  
+  /**
+   * Replace last child of <code>state</code> with an already registered state
+   * or register the last child state.
+   */
+  private void replaceOrRegister(State state) {
+    final State child = state.lastChild();
+    
+    if (child.hasChildren()) replaceOrRegister(child);
+    
+    final State registered = register.get(child);
+    if (registered != null) {
+      state.replaceLastChild(registered);
+    } else {
+      register.put(child, child);
+    }
+  }
+  
+  /**
+   * Add a suffix of <code>current</code> starting at <code>fromIndex</code>
+   * (inclusive) to state <code>state</code>.
+   */
+  private void addSuffix(State state, CharSequence current, int fromIndex) {
+    final int len = current.length();
+    while (fromIndex < len) {
+      int cp = Character.codePointAt(current, fromIndex);
+      state = state.newState(cp);
+      fromIndex += Character.charCount(cp);
+    }
+    state.is_final = true;
+  }
+}

Property changes on: lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java	(working copy)
@@ -280,7 +280,11 @@
     }
 
     swapCodec(new MockSepCodec(), cp);
-    swapCodec(new PulsingCodec(codecHasParam && "Pulsing".equals(codec) ? codecParam : 1 + random.nextInt(20)), cp);
+    // TODO: make it possible to specify min/max iterms per
+    // block via CL:
+    int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
+    int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
+    swapCodec(new PulsingCodec(codecHasParam && "Pulsing".equals(codec) ? codecParam : 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock), cp);
     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);
@@ -307,7 +311,7 @@
     cp.unregister(cp.lookup("MockFixedIntBlock"));
     cp.unregister(cp.lookup("MockVariableIntBlock"));
     cp.unregister(cp.lookup("MockRandom"));
-    swapCodec(new PulsingCodec(1), cp);
+    swapCodec(new PulsingCodec(), cp);
     cp.setDefaultFieldCodec(savedDefaultCodec);
   }
 
@@ -485,7 +489,7 @@
       System.err.println("NOTE: test params are: codec=" + codecDescription +
         ", locale=" + locale +
         ", timezone=" + (timeZone == null ? "(null)" : timeZone.getID()));
-    if (testsFailed) {
+    if (VERBOSE || testsFailed) {
       System.err.println("NOTE: all tests run in this JVM:");
       System.err.println(Arrays.toString(testClassesRun.toArray()));
       System.err.println("NOTE: " + System.getProperty("os.name") + " "
@@ -1561,9 +1565,17 @@
 
     RandomCodecProvider(Random random) {
       this.perFieldSeed = random.nextInt();
-      register(randomizCodec(random, new StandardCodec()));
+      // TODO: make it possible to specify min/max iterms per
+      // block via CL:
+      int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
+      int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
+      register(randomizCodec(random, new StandardCodec(minItemsPerBlock, maxItemsPerBlock)));
       register(randomizCodec(random, new PreFlexCodec()));
-      register(randomizCodec(random, new PulsingCodec( 1 + random.nextInt(20))));
+      // TODO: make it possible to specify min/max iterms per
+      // block via CL:
+      minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
+      maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
+      register(randomizCodec(random, new PulsingCodec( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock)));
       register(randomizCodec(random, new SimpleTextCodec()));
       register(randomizCodec(random, new MemoryCodec()));
       Collections.shuffle(knownCodecs, random);
Index: lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java	(revision 1159650)
+++ lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java	(working copy)
@@ -154,7 +154,7 @@
   public static CheckIndex.Status checkIndex(Directory dir, CodecProvider codecs) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
-    checker.setInfoStream(new PrintStream(bos));
+    checker.setInfoStream(new PrintStream(bos), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null, codecs);
     if (indexStatus == null || indexStatus.clean == false) {
       System.out.println("CheckIndex failed");
