lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1159905 [5/6] - in /lucene/dev/trunk: lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/ lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/index/codecs/ lucene/src/java/org/apache/lucene/index/codec...
Date Sat, 20 Aug 2011 19:20:48 GMT
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Builder.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Builder.java Sat Aug 20 19:20:45 2011
@@ -53,6 +53,8 @@ public class Builder<T> {
   private final FST<T> fst;
   private final T NO_OUTPUT;
 
+  // private static final boolean DEBUG = false;
+
   // simplistic pruning: we prune node (and all following
   // nodes) if less than this number of terms go through it:
   private final int minSuffixCount1;
@@ -73,13 +75,21 @@ public class Builder<T> {
   // current "frontier"
   private UnCompiledNode<T>[] frontier;
 
+  // Expert: you pass an instance of this if you want to do
+  // something "custom" as suffixes are "frozen":
+  public static abstract class FreezeTail<T> {
+    public abstract void freeze(final UnCompiledNode<T>[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException;
+  }
+
+  private final FreezeTail<T> freezeTail;
+
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs)} with 
    * 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 @@ public class Builder<T> {
    *    singleton output object.
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
-                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs) {
+                 boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
+                 FreezeTail<T> freezeTail) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
+    this.freezeTail = freezeTail;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
     fst = new FST<T>(inputType, outputs);
@@ -179,94 +191,100 @@ public class Builder<T> {
     return fn;
   }
 
-  private void compilePrevTail(int prefixLenPlus1) throws IOException {
-    assert prefixLenPlus1 >= 1;
-    //System.out.println("  compileTail " + prefixLenPlus1);
-    for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
-      boolean doPrune = false;
-      boolean doCompile = false;
+  private void freezeTail(int prefixLenPlus1) throws IOException {
+    if (freezeTail != null) {
+      // Custom plugin:
+      freezeTail.freeze(frontier, prefixLenPlus1, lastInput);
+    } else {
+      //System.out.println("  compileTail " + prefixLenPlus1);
+      final int downTo = Math.max(1, prefixLenPlus1);
+      for(int idx=lastInput.length; idx >= downTo; idx--) {
 
-      final UnCompiledNode<T> node = frontier[idx];
-      final UnCompiledNode<T> parent = frontier[idx-1];
+        boolean doPrune = false;
+        boolean doCompile = false;
 
-      if (node.inputCount < minSuffixCount1) {
-        doPrune = true;
-        doCompile = true;
-      } else if (idx > prefixLenPlus1) {
-        // prune if parent's inputCount is less than suffixMinCount2
-        if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) {
-          // my parent, about to be compiled, doesn't make the cut, so
-          // I'm definitely pruned 
-
-          // if pruneCount2 is 1, we keep only up
-          // until the 'distinguished edge', ie we keep only the
-          // 'divergent' part of the FST. if my parent, about to be
-          // compiled, has inputCount 1 then we are already past the
-          // distinguished edge.  NOTE: this only works if
-          // the FST outputs are not "compressible" (simple
-          // ords ARE compressible).
+        final UnCompiledNode<T> node = frontier[idx];
+        final UnCompiledNode<T> parent = frontier[idx-1];
+
+        if (node.inputCount < minSuffixCount1) {
           doPrune = true;
+          doCompile = true;
+        } else if (idx > prefixLenPlus1) {
+          // prune if parent's inputCount is less than suffixMinCount2
+          if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
+            // my parent, about to be compiled, doesn't make the cut, so
+            // I'm definitely pruned 
+
+            // if minSuffixCount2 is 1, we keep only up
+            // until the 'distinguished edge', ie we keep only the
+            // 'divergent' part of the FST. if my parent, about to be
+            // compiled, has inputCount 1 then we are already past the
+            // distinguished edge.  NOTE: this only works if
+            // the FST outputs are not "compressible" (simple
+            // ords ARE compressible).
+            doPrune = true;
+          } else {
+            // my parent, about to be compiled, does make the cut, so
+            // I'm definitely not pruned 
+            doPrune = false;
+          }
+          doCompile = true;
         } else {
-          // my parent, about to be compiled, does make the cut, so
-          // I'm definitely not pruned 
-          doPrune = false;
+          // if pruning is disabled (count is 0) we can always
+          // compile current node
+          doCompile = minSuffixCount2 == 0;
         }
-        doCompile = true;
-      } else {
-        // if pruning is disabled (count is 0) we can always
-        // compile current node
-        doCompile = minSuffixCount2 == 0;
-      }
 
-      //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
-
-      if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) {
-        // drop all arcs
-        for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-          @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
-          target.clear();
-        }
-        node.numArcs = 0;
-      }
+        //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
 
-      if (doPrune) {
-        // this node doesn't make it -- deref it
-        node.clear();
-        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
-      } else {
-
-        if (minSuffixCount2 != 0) {
-          compileAllTargets(node, lastInput.length-idx);
+        if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
+          // drop all arcs
+          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+            @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
+            target.clear();
+          }
+          node.numArcs = 0;
         }
-        final T nextFinalOutput = node.output;
 
-        // We "fake" the node as being final if it has no
-        // outgoing arcs; in theory we could leave it
-        // as non-final (the FST can represent this), but
-        // FSTEnum, Util, etc., have trouble w/ non-final
-        // dead-end states:
-        final boolean isFinal = node.isFinal || node.numArcs == 0;
-
-        if (doCompile) {
-          // this node makes it and we now compile it.  first,
-          // compile any targets that were previously
-          // undecided:
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             compileNode(node, 1+lastInput.length-idx),
-                             nextFinalOutput,
-                             isFinal);
+        if (doPrune) {
+          // this node doesn't make it -- deref it
+          node.clear();
+          parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
         } else {
-          // replaceLast just to install
-          // nextFinalOutput/isFinal onto the arc
-          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                             node,
-                             nextFinalOutput,
-                             isFinal);
-          // this node will stay in play for now, since we are
-          // undecided on whether to prune it.  later, it
-          // will be either compiled or pruned, so we must
-          // allocate a new node:
-          frontier[idx] = new UnCompiledNode<T>(this, idx);
+
+          if (minSuffixCount2 != 0) {
+            compileAllTargets(node, lastInput.length-idx);
+          }
+          final T nextFinalOutput = node.output;
+
+          // We "fake" the node as being final if it has no
+          // outgoing arcs; in theory we could leave it
+          // as non-final (the FST can represent this), but
+          // FSTEnum, Util, etc., have trouble w/ non-final
+          // dead-end states:
+          final boolean isFinal = node.isFinal || node.numArcs == 0;
+
+          if (doCompile) {
+            // this node makes it and we now compile it.  first,
+            // compile any targets that were previously
+            // undecided:
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               compileNode(node, 1+lastInput.length-idx),
+                               nextFinalOutput,
+                               isFinal);
+          } else {
+            // replaceLast just to install
+            // nextFinalOutput/isFinal onto the arc
+            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                               node,
+                               nextFinalOutput,
+                               isFinal);
+            // this node will stay in play for now, since we are
+            // undecided on whether to prune it.  later, it
+            // will be either compiled or pruned, so we must
+            // allocate a new node:
+            frontier[idx] = new UnCompiledNode<T>(this, idx);
+          }
         }
       }
     }
@@ -320,11 +338,36 @@ public class Builder<T> {
     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 @@ public class Builder<T> {
     int pos2 = input.offset;
     final int pos1Stop = Math.min(lastInput.length, input.length);
     while(true) {
-      //System.out.println("  incr " + pos1);
       frontier[pos1].inputCount++;
+      //System.out.println("  incr " + pos1 + " ct=" + frontier[pos1].inputCount + " n=" + frontier[pos1]);
       if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
         break;
       }
@@ -368,13 +411,12 @@ public class Builder<T> {
 
     // minimize/compile states from previous input's
     // orphan'd suffix
-    compilePrevTail(prefixLenPlus1);
+    freezeTail(prefixLenPlus1);
 
     // init tail states for current input
     for(int idx=prefixLenPlus1;idx<=input.length;idx++) {
       frontier[idx-1].addArc(input.ints[input.offset + idx - 1],
                              frontier[idx]);
-      //System.out.println("  incr tail " + idx);
       frontier[idx].inputCount++;
     }
 
@@ -433,34 +475,25 @@ public class Builder<T> {
    *  nothing is accepted by the FST. */
   public FST<T> finish() throws IOException {
 
+    final UnCompiledNode<T> root = frontier[0];
+
     // minimize nodes in the last word's suffix
-    compilePrevTail(1);
-    //System.out.println("finish: inputCount=" + frontier[0].inputCount);
-    if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
+    freezeTail(0);
+    if (root.inputCount < minSuffixCount1 || root.inputCount < minSuffixCount2 || root.numArcs == 0) {
       if (fst.emptyOutput == null) {
         return null;
       } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
         // empty string got pruned
         return null;
-      } else {
-        fst.finish(compileNode(frontier[0], lastInput.length).address);
-        //System.out.println("compile addr = " + fst.getStartNode());
-        return fst;
       }
     } else {
       if (minSuffixCount2 != 0) {
-        compileAllTargets(frontier[0], lastInput.length);
+        compileAllTargets(root, lastInput.length);
       }
-      //System.out.println("NOW: " + frontier[0].numArcs);
-      fst.finish(compileNode(frontier[0], lastInput.length).address);
     }
+    //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
+    fst.finish(compileNode(root, lastInput.length).address);
 
-    /*
-    if (dedupHash != null) {
-      System.out.println("NH: " + dedupHash.count()); 
-    }
-    */
-    
     return fst;
   }
 
@@ -479,7 +512,7 @@ public class Builder<T> {
     }
   }
 
-  static class Arc<T> {
+  public static class Arc<T> {
     public int label;                             // really an "unsigned" byte
     public Node target;
     public boolean isFinal;
@@ -502,16 +535,20 @@ public class Builder<T> {
     }
   }
 
-  static final class UnCompiledNode<T> implements Node {
+  public static final class UnCompiledNode<T> implements Node {
     final Builder<T> owner;
-    int numArcs;
-    Arc<T>[] arcs;
-    T output;
-    boolean isFinal;
-    long inputCount;
+    public int numArcs;
+    public Arc<T>[] arcs;
+    // TODO: instead of recording isFinal/output on the
+    // node, maybe we should use -1 arc to mean "end" (like
+    // we do when reading the FST).  Would simplify much
+    // code here...
+    public T output;
+    public boolean isFinal;
+    public long inputCount;
 
     /** This node's depth, starting from the automaton root. */
-    final int depth;
+    public final int depth;
 
     /**
      * @param depth

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Sat Aug 20 19:20:45 2011
@@ -133,6 +133,6 @@ public final class ByteSequenceOutputs e
 
   @Override
   public String outputToString(BytesRef output) {
-    return output.utf8ToString();
+    return output.toString();
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/FST.java Sat Aug 20 19:20:45 2011
@@ -123,7 +123,7 @@ public class FST<T> {
     public int label;
     public T output;
 
-    int target;
+    public int target;
 
     byte flags;
     public T nextFinalOutput;
@@ -274,6 +274,10 @@ public class FST<T> {
     }
   }
 
+  public T getEmptyOutput() {
+    return emptyOutput;
+  }
+
   void setEmptyOutput(T v) throws IOException {
     if (emptyOutput != null) {
       emptyOutput = outputs.merge(emptyOutput, v);
@@ -597,9 +601,9 @@ public class FST<T> {
       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 @@ public class FST<T> {
     }
   }
 
-  // Not private because NodeHash needs access:
-  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+  public Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
 
     final BytesReader in = getBytesReader(address);
 
@@ -693,7 +696,9 @@ public class FST<T> {
     return readLabel(in);
   }
 
-  Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+  /** Never returns null, but you should never call this if
+   *  arc.isLast() is true. */
+  public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
     // this is a continuing arc in a fixed array
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
@@ -925,7 +930,7 @@ public class FST<T> {
     }
   }
 
-  final BytesReader getBytesReader(int pos) {
+  public final BytesReader getBytesReader(int pos) {
     // TODO: maybe re-use via ThreadLocal?
     return new BytesReader(pos);
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java Sat Aug 20 19:20:45 2011
@@ -39,8 +39,8 @@ import org.apache.lucene.store.DataOutpu
 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;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java Sat Aug 20 19:20:45 2011
@@ -213,6 +213,7 @@ public final class Util {
 
     // Shape for states.
     final String stateShape = "circle";
+    final String finalStateShape = "doublecircle";
 
     // Emit DOT prologue.
     out.write("digraph FST {\n");
@@ -223,12 +224,34 @@ public final class Util {
     }
 
     emitDotState(out, "initial", "point", "white", "");
-    emitDotState(out, Integer.toString(startArc.target), stateShape, 
-        fst.isExpandedTarget(startArc) ? expandedNodeColor : null, 
-        "");
-    out.write("  initial -> " + startArc.target + "\n");
 
     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");
+
     int level = 0;
 
     while (!nextLevelQueue.isEmpty()) {
@@ -240,19 +263,48 @@ public final class Util {
       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 @@ public final class Util {
               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 @@ public final class Util {
     }
 
     // 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");

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Sat Aug 20 19:20:45 2011
@@ -60,7 +60,7 @@ public class RandomIndexWriter implement
 
     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

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Sat Aug 20 19:20:45 2011
@@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsReader;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
@@ -127,7 +127,7 @@ public class MockFixedIntBlockCodec exte
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(blockSize));
+    PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(blockSize));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -158,10 +158,10 @@ public class MockFixedIntBlockCodec exte
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
-                                                                      state.segmentInfo,
-                                                                      state.context,
-                                                                      new MockIntFactory(blockSize), state.codecId);
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
+                                                              state.segmentInfo,
+                                                              state.context,
+                                                              new MockIntFactory(blockSize), state.codecId);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
@@ -204,7 +204,7 @@ public class MockFixedIntBlockCodec exte
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
-    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@@ -212,7 +212,7 @@ public class MockFixedIntBlockCodec exte
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Sat Aug 20 19:20:45 2011
@@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsReader;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
@@ -150,7 +150,7 @@ public class MockVariableIntBlockCodec e
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(baseBlockSize));
+    PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(baseBlockSize));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -181,10 +181,10 @@ public class MockVariableIntBlockCodec e
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
-                                                                      state.segmentInfo,
-                                                                      state.context,
-                                                                      new MockIntFactory(baseBlockSize), state.codecId);
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
+                                                              state.segmentInfo,
+                                                              state.context,
+                                                              new MockIntFactory(baseBlockSize), state.codecId);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
@@ -227,7 +227,7 @@ public class MockVariableIntBlockCodec e
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
-    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@@ -235,7 +235,7 @@ public class MockVariableIntBlockCodec e
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Sat Aug 20 19:20:45 2011
@@ -30,6 +30,8 @@ import org.apache.lucene.index.PerDocWri
 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;
@@ -51,13 +53,13 @@ import org.apache.lucene.index.codecs.Va
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
 import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
-import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReaderImpl;
-import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReader;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriter;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsReader;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
 import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
 import org.apache.lucene.store.Directory;
@@ -137,7 +139,7 @@ public class MockRandomCodec extends Cod
     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,11 +153,10 @@ public class MockRandomCodec extends Cod
     final Random random = new Random(seed);
     
     random.nextInt(); // consume a random for buffersize
-    
-    PostingsWriterBase postingsWriter;
 
+    PostingsWriterBase postingsWriter;
     if (random.nextBoolean()) {
-      postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval);
+      postingsWriter = new SepPostingsWriter(state, new MockIntStreamFactory(random), skipInterval);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: writing Standard postings");
@@ -166,76 +167,107 @@ public class MockRandomCodec extends Cod
     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);
+      postingsWriter = new PulsingPostingsWriter(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 + ")");
+          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 void newField(FieldInfo fieldInfo) {
+                }
+              };
           }
-          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 void newField(FieldInfo fieldInfo) {
-              }
-            };
+          indexWriter = new VariableGapTermsIndexWriter(state, selector);
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
         }
-        indexWriter = new VariableGapTermsIndexWriter(state, selector);
-      }
-      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 @@ public class MockRandomCodec extends Cod
     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,8 +291,11 @@ public class MockRandomCodec extends Cod
     PostingsReaderBase postingsReader;
 
     if (random.nextBoolean()) {
-      postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
-                                                 state.context, new MockIntStreamFactory(random), state.codecId);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading Sep postings");
+      }
+      postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
+                                             state.context, new MockIntStreamFactory(random), state.codecId);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Standard postings");
@@ -273,86 +308,119 @@ public class MockRandomCodec extends Cod
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff);
       }
-      postingsReader = new PulsingPostingsReaderImpl(postingsReader);
+      postingsReader = new PulsingPostingsReader(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 + ")");
-        }
-        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 + ")");
+
+        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);
+
         }
-        if (state.termsIndexDivisor != -1) {
-          state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
+
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
         }
-        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
   public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
     final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);    
     files.add(seedFileName);
-    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    SepPostingsReader.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());
@@ -369,8 +437,9 @@ public class MockRandomCodec extends Cod
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
+    BlockTreeTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
     DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java Sat Aug 20 19:20:45 2011
@@ -40,8 +40,8 @@ import org.apache.lucene.index.codecs.Bl
 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.sep.SepPostingsWriterImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
+import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 
@@ -60,7 +60,7 @@ public class MockSepCodec extends Codec 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
 
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory());
+    PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockSingleIntFactory());
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -92,7 +92,7 @@ public class MockSepCodec extends Codec 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
         state.context, new MockSingleIntFactory(), state.codecId);
 
     TermsIndexReaderBase indexReader;
@@ -136,7 +136,7 @@ public class MockSepCodec extends Codec 
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
-    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@@ -149,7 +149,7 @@ public class MockSepCodec extends Codec 
   }
 
   public static void getSepExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
   }

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java Sat Aug 20 19:20:45 2011
@@ -66,6 +66,11 @@ public class MockSingleIntIndexOutput ex
     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 @@ public class MockSingleIntIndexOutput ex
       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)

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Sat Aug 20 19:20:45 2011
@@ -280,7 +280,11 @@ public abstract class LuceneTestCase ext
     }
 
     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 @@ public abstract class LuceneTestCase ext
     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 @@ public abstract class LuceneTestCase ext
       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 @@ public abstract class LuceneTestCase ext
 
     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);

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Sat Aug 20 19:20:45 2011
@@ -154,7 +154,7 @@ public class _TestUtil {
   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");

Added: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java?rev=1159905&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java (added)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java Sat Aug 20 19:20:45 2011
@@ -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;
+  }
+}

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java Sat Aug 20 19:20:45 2011
@@ -86,7 +86,7 @@ public class TestSearchForDuplicates ext
       }
       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);
       }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Sat Aug 20 19:20:45 2011
@@ -155,9 +155,9 @@ public class Test2BTerms extends LuceneT
     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 @@ public class Test2BTerms extends LuceneT
                                       .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:

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java Sat Aug 20 19:20:45 2011
@@ -40,8 +40,6 @@ import org.apache.lucene.search.PhraseQu
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
@@ -504,6 +502,9 @@ public class TestCodecs extends LuceneTe
         }
 
         // 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 +513,9 @@ public class TestCodecs extends LuceneTe
         }
 
         // 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());

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java Sat Aug 20 19:20:45 2011
@@ -462,7 +462,7 @@ public class TestDocTermOrds extends Luc
     */
 
     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;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocsAndPositions.java Sat Aug 20 19:20:45 2011
@@ -194,7 +194,8 @@ public class TestDocsAndPositions extend
   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 @@ public class TestDocsAndPositions extend
       writer.addDocument(doc);
     }
 
-    // now do seaches
+    // now do searches
     IndexReader reader = writer.getReader();
     writer.close();
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Sat Aug 20 19:20:45 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.search.DefaultSimilarity;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -169,7 +168,7 @@ public class TestIndexFileDeleter extend
     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);
     }
   }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Sat Aug 20 19:20:45 2011
@@ -74,8 +74,15 @@ public class TestIndexWriterDelete exten
     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);
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterMerging.java Sat Aug 20 19:20:45 2011
@@ -221,6 +221,8 @@ public class TestIndexWriterMerging exte
             setMergePolicy(newLogMergePolicy(50))
     );
 
+    writer.setInfoStream(VERBOSE ? System.out : null);
+
     Document document = new Document();
 
     document = new Document();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLongPostings.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLongPostings.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLongPostings.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLongPostings.java Sat Aug 20 19:20:45 2011
@@ -355,7 +355,7 @@ public class TestLongPostings extends Lu
       }
 
       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 @@ public class TestLongPostings extends Lu
           
           if (random.nextInt(6) == 3) {
             final int freq = postings.freq();
-            assertTrue(freq >=1 && freq <= 4);
+            assertTrue("got invalid freq=" + freq, freq >=1 && freq <= 4);
           }
         }
       }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiFields.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiFields.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiFields.java Sat Aug 20 19:20:45 2011
@@ -29,10 +29,15 @@ public class TestMultiFields extends Luc
 
     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 @@ public class TestMultiFields extends Luc
       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 @@ public class TestMultiFields extends Luc
           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 @@ public class TestMultiFields extends Luc
       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);



Mime
View raw message