You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/01/05 11:52:06 UTC

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

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java Wed Jan  5 10:52:04 2011
@@ -17,173 +17,31 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.RamUsageEstimator;
-
 import java.io.IOException;
 
+import org.apache.lucene.util.BytesRef;
+
 /** Can next() and advance() through the terms in an FST
   * @lucene.experimental
 */
 
-public class BytesRefFSTEnum<T> {
-  private final FST<T> fst;
-
-  private BytesRef current = new BytesRef(10);
-  @SuppressWarnings("unchecked") private FST.Arc<T>[] arcs = new FST.Arc[10];
-  // outputs are cumulative
-  @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10];
-
-  private boolean lastFinal;
-  private boolean didEmpty;
-  private final T NO_OUTPUT;
+public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
+  private final BytesRef current = new BytesRef(10);
   private final InputOutput<T> result = new InputOutput<T>();
+  private BytesRef target;
 
   public static class InputOutput<T> {
     public BytesRef input;
     public T output;
   }
-  
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
   public BytesRefFSTEnum(FST<T> fst) {
-    this.fst = fst;
+    super(fst);
     result.input = current;
-    NO_OUTPUT = fst.outputs.getNoOutput();
-  }
-
-  public void reset() {
-    lastFinal = false;
-    didEmpty = false;
-    current.length = 0;
-    result.output = NO_OUTPUT;
-  }
-
-  /** NOTE: target must be >= where we are already
-   *  positioned */
-  public InputOutput<T> advance(BytesRef target) throws IOException {
-
-    assert target.compareTo(current) >= 0;
-
-    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
-
-    // special case empty string
-    if (current.length == 0) {
-      if (target.length == 0) {
-        final T output = fst.getEmptyOutput();      
-        if (output != null) {
-          if (!didEmpty) {
-            current.length = 0;
-            lastFinal = true;
-            result.output = output;
-            didEmpty = true;
-          }
-          return result;
-        } else {
-          return next();
-        }
-      }
-      
-      if (fst.noNodes()) {
-        return null;
-      }
-    }
-
-    // TODO: possibly caller could/should provide common
-    // prefix length?  ie this work may be redundant if
-    // caller is in fact intersecting against its own
-    // automaton
-
-    // what prefix does target share w/ current
-    int idx = 0;
-    while (idx < current.length && idx < target.length) {
-      if (current.bytes[idx] != target.bytes[target.offset + idx]) {
-        break;
-      }
-      idx++;
-    }
-
-    //System.out.println("  shared " + idx);
-
-    FST.Arc<T> arc;
-    if (current.length == 0) {
-      // new enum (no seek/next yet)
-      arc = fst.readFirstArc(fst.getStartNode(), getArc(0));
-      //System.out.println("  new enum");
-    } else if (idx < current.length) {
-      // roll back to shared point
-      lastFinal = false;
-      current.length = idx;
-      arc = arcs[idx];
-      if (arc.isLast()) {
-        if (idx == 0) {
-          return null;
-        } else {
-          return next();
-        }
-      }
-      arc = fst.readNextArc(arc);
-    } else if (idx == target.length) {
-      // degenerate case -- seek to term we are already on
-      assert target.equals(current);
-      return result;
-    } else {
-      // current is a full prefix of target
-      if (lastFinal) {
-        arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      } else {
-        return next();
-      }
-    }
-
-    lastFinal = false;
-
-    assert arc == arcs[current.length];
-    int targetLabel = target.bytes[target.offset+current.length] & 0xFF;
-
-    while(true) {
-      //System.out.println("    cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label));
-      if (arc.label == targetLabel) {
-        grow();
-        current.bytes[current.length] = (byte) arc.label;
-        appendOutput(arc.output);
-        current.length++;
-        grow();
-        if (current.length == target.length) {
-          result.output = output[current.length-1];
-          if (arc.isFinal()) {
-            // target is exact match
-            if (fst.hasArcs(arc.target)) {
-              // target is also a proper prefix of other terms
-              lastFinal = true;
-              appendFinalOutput(arc.nextFinalOutput);
-            }
-          } else {
-            // target is not a match but is a prefix of
-            // other terms
-            current.length--;
-            push();
-          }
-          return result;
-        } else if (!fst.hasArcs(arc.target)) {
-          // we only match a prefix of the target
-          return next();
-        } else {
-          targetLabel = target.bytes[target.offset+current.length] & 0xFF;
-          arc = fst.readFirstArc(arc.target, getArc(current.length));
-        }
-      } else if (arc.label > targetLabel) {
-        // we are now past the target
-        push();
-        return result;
-      } else if (arc.isLast()) {
-        if (current.length == 0) {
-          return null;
-        }
-        return next();
-      } else {
-        arc = fst.readNextArc(getArc(current.length));
-      }
-    }
+    current.offset = 1;
   }
 
   public InputOutput<T> current() {
@@ -192,124 +50,58 @@ public class BytesRefFSTEnum<T> {
 
   public InputOutput<T> next() throws IOException {
     //System.out.println("  enum.next");
-
-    if (current.length == 0) {
-      final T output = fst.getEmptyOutput();
-      if (output != null) {
-        if (!didEmpty) {
-          current.length = 0;
-          lastFinal = true;
-          result.output = output;
-          didEmpty = true;
-          return result;
-        } else {
-          lastFinal = false;
-        }
-      }
-      if (fst.noNodes()) {
-        return null;
-      }
-      fst.readFirstArc(fst.getStartNode(), getArc(0));
-      push();
-    } else if (lastFinal) {
-      lastFinal = false;
-      assert current.length > 0;
-      // resume pushing
-      fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      push();
-    } else {
-      //System.out.println("    pop/push");
-      pop();
-      if (current.length == 0) {
-        // enum done
-        return null;
-      } else {
-        current.length--;
-        fst.readNextArc(arcs[current.length]);
-        push();
-      }
-    }
-
-    return result;
-  }
-
-  private void grow() {
-    final int l = current.length + 1;
-    current.grow(l);
-    if (arcs.length < l) {
-      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
-        new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
-      arcs = newArcs;
-    }
-    if (output.length < l) {
-      @SuppressWarnings("unchecked") final T[] newOutput =
-        (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(output, 0, newOutput, 0, output.length);
-      output = newOutput;
-    }
+    doNext();
+    return setResult();
   }
 
-  private void appendOutput(T addedOutput) {
-    T newOutput;
-    if (current.length == 0) {
-      newOutput = addedOutput;
-    } else if (addedOutput == NO_OUTPUT) {
-      output[current.length] = output[current.length-1];
-      return;
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
     } else {
-      newOutput = fst.outputs.add(output[current.length-1], addedOutput);
+      return target.bytes[target.offset + upto - 1] & 0xFF;
     }
-    output[current.length] = newOutput;
   }
 
-  private void appendFinalOutput(T addedOutput) {
-    if (current.length == 0) {
-      result.output = addedOutput;
-    } else {
-      result.output = fst.outputs.add(output[current.length-1], addedOutput);
-    }
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.bytes[upto] & 0xFF;
   }
 
-  private void push() throws IOException {
-
-    FST.Arc<T> arc = arcs[current.length];
-    assert arc != null;
-
-    while(true) {
-      grow();
-      
-      current.bytes[current.length] = (byte) arc.label;
-      appendOutput(arc.output);
-      //System.out.println("    push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output));
-      current.length++;
-      grow();
-
-      if (!fst.hasArcs(arc.target)) {
-        break;
-      }
-
-      if (arc.isFinal()) {
-        appendFinalOutput(arc.nextFinalOutput);
-        lastFinal = true;
-        return;
-      }
-
-      arc = fst.readFirstArc(arc.target, getArc(current.length));
-    }
-    result.output = output[current.length-1];
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.bytes[upto] = (byte) label;
   }
 
-  private void pop() {
-    while (current.length > 0 && arcs[current.length-1].isLast()) {
-      current.length--;
-    }
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
   }
 
-  private FST.Arc<T> getArc(int idx) {
-    if (arcs[idx] == null) {
-      arcs[idx] = new FST.Arc<T>();
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
     }
-    return arcs[idx];
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Wed Jan  5 10:52:04 2011
@@ -18,20 +18,13 @@ package org.apache.lucene.util.automaton
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
-import org.apache.lucene.util.IntsRef;
 
 /** Represents an FST using a compact byte[] format.
  *  <p> The format is similar to what's used by Morfologik
@@ -40,7 +33,7 @@ import org.apache.lucene.util.IntsRef;
  */
 public class FST<T> {
   public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
-  private final INPUT_TYPE inputType;
+  public final INPUT_TYPE inputType;
 
   private final static int BIT_FINAL_ARC = 1 << 0;
   private final static int BIT_LAST_ARC = 1 << 1;
@@ -76,7 +69,7 @@ public class FST<T> {
 
   // if non-null, this FST accepts the empty string and
   // produces this output
-  private T emptyOutput;
+  T emptyOutput;
   private byte[] emptyOutputBytes;
 
   private byte[] bytes;
@@ -94,11 +87,16 @@ public class FST<T> {
   public int arcCount;
   public int arcWithOutputCount;
 
+  // If arc has this label then that arc is final/accepted
+  public static int END_LABEL = -1;
+
   public final static class Arc<T> {
-    int label;  // really a "unsigned" byte
+    public int label;
+    public T output;
+
     int target;
+
     byte flags;
-    T output;
     T nextFinalOutput;
     int nextArc;
 
@@ -108,13 +106,26 @@ public class FST<T> {
     int arcIdx;
     int numArcs;
 
-    // Must call this before re-using an Arc instance on a
-    // new node
-    public void reset() {
-      bytesPerArc = 0;
+    /** Returns this */
+    public Arc<T> copyFrom(Arc<T> other) {
+      label = other.label;
+      target = other.target;
+      flags = other.flags;
+      output = other.output;
+      nextFinalOutput = other.nextFinalOutput;
+      nextArc = other.nextArc;
+      if (other.bytesPerArc != 0) {
+        bytesPerArc = other.bytesPerArc;
+        posArcsStart = other.posArcsStart;
+        arcIdx = other.arcIdx;
+        numArcs = other.numArcs;
+      } else {
+        bytesPerArc = 0;
+      }
+      return this;
     }
 
-    public boolean flag(int flag) {
+    boolean flag(int flag) {
       return FST.flag(flags, flag);
     }
 
@@ -122,7 +133,7 @@ public class FST<T> {
       return flag(BIT_LAST_ARC);
     }
 
-    public boolean isFinal() {
+    boolean isFinal() {
       return flag(BIT_FINAL_ARC);
     }
   };
@@ -156,7 +167,7 @@ public class FST<T> {
       // messy
       bytes = new byte[numBytes];
       in.readBytes(bytes, 0, numBytes);
-      emptyOutput = outputs.read(new BytesReader(numBytes-1));
+      emptyOutput = outputs.read(getBytesReader(numBytes-1));
     } else {
       emptyOutput = null;
     }
@@ -203,9 +214,9 @@ public class FST<T> {
     this.startNode = startNode;
   }
 
-  public void setEmptyOutput(T v) throws IOException {
-    if (emptyOutput != null) {
-      throw new IllegalStateException("empty output is already set");
+  void setEmptyOutput(T v) throws IOException {
+    if (emptyOutput != null && !emptyOutput.equals(v)) {
+      throw new IllegalStateException("empty output is already set: " + outputs.outputToString(emptyOutput) + " vs " + outputs.outputToString(v));
     }
     emptyOutput = v;
 
@@ -271,7 +282,7 @@ public class FST<T> {
     }
   }
 
-  private int readLabel(DataInput in) throws IOException {
+  int readLabel(DataInput in) throws IOException {
     final int v;
     if (inputType == INPUT_TYPE.BYTE1) {
       v = in.readByte()&0xFF;
@@ -285,21 +296,8 @@ public class FST<T> {
 
   // returns true if the node at this address has any
   // outgoing arcs
-  public boolean hasArcs(int address) {
-    return address != FINAL_END_NODE && address != NON_FINAL_END_NODE;
-  }
-
-  public int getStartNode() {
-    if (startNode == -1) {
-      throw new IllegalStateException("call finish first");
-    }
-    return startNode;
-  }
-
-  // returns null if this FST does not accept the empty
-  // string, else, the output for the empty string
-  public T getEmptyOutput() {
-    return emptyOutput;
+  public boolean targetHasArcs(Arc<T> arc) {
+    return arc.target > 0;
   }
 
   // serializes new node by appending its bytes to the end
@@ -364,7 +362,7 @@ public class FST<T> {
         assert arc.nextFinalOutput == NO_OUTPUT;
       }
 
-      boolean targetHasArcs = hasArcs(target.address);
+      boolean targetHasArcs = target.address > 0;
 
       if (!targetHasArcs) {
         flags += BIT_STOP_NODE;
@@ -453,10 +451,49 @@ public class FST<T> {
     return endAddress-1;
   }
 
-  public Arc<T> readFirstArc(int address, Arc<T> arc) throws IOException {
-    //System.out.println("readFirstArc addr=" + address);
+  /** Fills virtual 'start' arc, ie, an empty incoming arc to
+   *  the FST's start node */
+  public Arc<T> getFirstArc(Arc<T> arc) {
+    if (emptyOutput != null) {
+      arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC;
+      arc.nextFinalOutput = emptyOutput;
+    } else {
+      arc.flags = BIT_LAST_ARC;
+    }
+
+    // If there are no nodes, ie, the FST only accepts the
+    // empty string, then startNode is 0, and then readFirstTargetArc
+    arc.target = startNode;
+    return arc;
+  }
+
+  /** Follow the follow arc and read the first arc of its
+   *  target; this changes the provide arc (2nd arg) in-place
+   *  and returns it. */
+  public Arc<T> readFirstTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
     //int pos = address;
-    final BytesReader in = new BytesReader(address);
+    //System.out.println("    readFirstTarget follow.target=" + follow.target + " isFinal=" + follow.isFinal());
+    if (follow.isFinal()) {
+      // Insert "fake" final first arc:
+      arc.label = -1;
+      arc.output = follow.nextFinalOutput;
+      if (follow.target <= 0) {
+        arc.flags = BIT_LAST_ARC;
+      } else {
+        arc.flags = 0;
+        arc.nextArc = follow.target;
+      }
+      //System.out.println("    insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
+      return arc;
+    } else {
+      return readFirstRealArc(follow.target, arc);
+    }
+  }
+
+  // Not private beacaus NodeHash needs access:
+  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+
+    final BytesReader in = getBytesReader(address);
 
     arc.flags = in.readByte();
 
@@ -473,19 +510,66 @@ public class FST<T> {
       arc.bytesPerArc = 0;
     }
     arc.nextArc = in.pos;
+    arc.label = 0;
     return readNextArc(arc);
   }
 
+  /** In-place read; returns the arc. */
   public Arc<T> readNextArc(Arc<T> arc) throws IOException {
+    if (arc.label == -1) {
+      // This was a fake inserted "final" arc
+      if (arc.nextArc <= 0) {
+        // This arc went to virtual final node, ie has no outgoing arcs
+        return null;
+      }
+      return readFirstRealArc(arc.nextArc, arc);
+    } else {
+      return readNextRealArc(arc);
+    }
+  }
+
+  /** Peeks at next arc's label; does not alter arc.  Do
+   *  not call this if arc.isLast()! */
+  public int readNextArcLabel(Arc<T> arc) throws IOException {
+    assert !arc.isLast();
+
+    final BytesReader in;
+    if (arc.label == END_LABEL) {
+      //System.out.println("    nextArc fake " + arc.nextArc);
+      in = getBytesReader(arc.nextArc);
+      byte flags = bytes[in.pos];
+      if (flag(flags, BIT_ARCS_AS_FIXED_ARRAY)) {
+        //System.out.println("    nextArc fake array");
+        in.pos--;
+        in.readVInt();
+        in.readByte();
+      }
+    } else {
+      if (arc.bytesPerArc != 0) {
+        //System.out.println("    nextArc real array");
+        // arcs are at fixed entries
+        in = getBytesReader(arc.posArcsStart - (1+arc.arcIdx)*arc.bytesPerArc);
+      } else {
+        // arcs are packed
+        //System.out.println("    nextArc real packed");
+        in = getBytesReader(arc.nextArc);
+      }
+    }
+    // skip flags
+    in.readByte();
+    return readLabel(in);
+  }
+
+  Arc<T> readNextRealArc(Arc<T> arc) throws IOException {
     // this is a continuing arc in a fixed array
     final BytesReader in;
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
       arc.arcIdx++;
-      in = new BytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
+      in = getBytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed
-      in = new BytesReader(arc.nextArc);
+      in = getBytesReader(arc.nextArc);
     }
     arc.flags = in.readByte();
     arc.label = readLabel(in);
@@ -504,6 +588,7 @@ public class FST<T> {
 
     if (arc.flag(BIT_STOP_NODE)) {
       arc.target = FINAL_END_NODE;
+      arc.flags |= BIT_FINAL_ARC;
       arc.nextArc = in.pos;
     } else if (arc.flag(BIT_TARGET_NEXT)) {
       arc.nextArc = in.pos;
@@ -524,14 +609,30 @@ public class FST<T> {
     return arc;
   }
 
-  public Arc<T> findArc(int address, int labelToMatch, Arc<T> arc) throws IOException {
+  /** Finds an arc leaving the incoming arc, replacing the arc in place.
+   *  This returns null if the arc was not found, else the incoming arc. */
+  public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc) throws IOException {
+
+    if (labelToMatch == END_LABEL) {
+      if (follow.isFinal()) {
+        arc.output = follow.nextFinalOutput;
+        arc.label = END_LABEL;
+        return arc;
+      } else {
+        return null;
+      }
+    }
+
+    if (!targetHasArcs(follow)) {
+      return null;
+    }
+
     // TODO: maybe make an explicit thread state that holds
     // reusable stuff eg BytesReader:
-    final BytesReader in = new BytesReader(address);
+    final BytesReader in = getBytesReader(follow.target);
 
     if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) {
       // Arcs are full array; do binary search:
-      //System.out.println("findArc: array label=" + labelToMatch);
       arc.numArcs = in.readVInt();
       arc.bytesPerArc = in.readByte() & 0xFF;
       arc.posArcsStart = in.pos;
@@ -548,19 +649,20 @@ public class FST<T> {
           high = mid - 1;
         else {
           arc.arcIdx = mid-1;
-          return readNextArc(arc);
+          return readNextRealArc(arc);
         }
       }
 
       return null;
     }
-    //System.out.println("findArc: scan");
-
-    readFirstArc(address, arc);
 
+    // Linear scan
+    readFirstTargetArc(follow, arc);
     while(true) {
       if (arc.label == labelToMatch) {
         return arc;
+      } else if (arc.label > labelToMatch) {
+        return null;
       } else if (arc.isLast()) {
         return null;
       } else {
@@ -569,191 +671,6 @@ public class FST<T> {
     }
   }
 
-  /** Looks up the output for this input, or null if the
-   *  input is not accepted. FST must be
-   *  INPUT_TYPE.BYTE4. */
-  public T get(IntsRef input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    if (input.length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    T output = NO_OUTPUT;
-    for(int i=0;i<input.length;i++) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      if (findArc(node, input.ints[input.offset + i], arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Logically casts input to UTF32 ints then looks up the output
-   *  or null if the input is not accepted.  FST must be
-   *  INPUT_TYPE.BYTE4.  */
-  public T get(char[] input, int offset, int length) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    if (length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    int charIdx = offset;
-    final int charLimit = offset + length;
-    T output = NO_OUTPUT;
-    while(charIdx < charLimit) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      final int utf32 = Character.codePointAt(input, charIdx);
-      charIdx += Character.charCount(utf32);
-
-      if (findArc(node, utf32, arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-
-  /** Logically casts input to UTF32 ints then looks up the output
-   *  or null if the input is not accepted.  FST must be
-   *  INPUT_TYPE.BYTE4.  */
-  public T get(CharSequence input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    final int len = input.length();
-    if (len == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    int charIdx = 0;
-    final int charLimit = input.length();
-    T output = NO_OUTPUT;
-    while(charIdx < charLimit) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      final int utf32 = Character.codePointAt(input, charIdx);
-      charIdx += Character.charCount(utf32);
-
-      if (findArc(node, utf32, arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Looks up the output for this input, or null if the
-   *  input is not accepted */
-  public T get(BytesRef input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE1;
-
-    if (input.length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    T output = NO_OUTPUT;
-    for(int i=0;i<input.length;i++) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      if (findArc(node, input.bytes[i+input.offset], arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Returns true if this FST has no nodes */
-  public boolean noNodes() {
-    //System.out.println("isempty startNode=" + startNode);
-    return startNode == 0;
-  }
-
   private void seekToNextNode(BytesReader in) throws IOException {
 
     while(true) {
@@ -779,85 +696,6 @@ public class FST<T> {
     }
   }
 
-  // NOTE: this consumes alot of RAM!
-  // final arcs have a flat end (not arrow)
-  // arcs w/ NEXT opto are in blue
-  /*
-    eg:
-      PrintStream ps = new PrintStream("out.dot");
-      fst.toDot(ps);
-      ps.close();
-      System.out.println("SAVED out.dot");
-      
-    then dot -Tpng out.dot > /x/tmp/out.png
-  */
-  public void toDot(PrintStream out) throws IOException {
-
-    final List<Integer> queue = new ArrayList<Integer>();
-    queue.add(startNode);
-
-    final Set<Integer> seen = new HashSet<Integer>();
-    seen.add(startNode);
-    
-    out.println("digraph FST {");
-    out.println("  rankdir = LR;");
-    //out.println("  " + startNode + " [shape=circle label=" + startNode + "];");
-    out.println("  " + startNode + " [label=\"\" shape=circle];");
-    out.println("  initial [shape=point color=white label=\"\"];");
-    if (emptyOutput != null) {
-      out.println("  initial -> " + startNode + " [arrowhead=tee label=\"(" + outputs.outputToString(emptyOutput) + ")\"];");
-    } else {
-      out.println("  initial -> " + startNode);
-    }
-
-    final Arc<T> arc = new Arc<T>();
-
-    while(queue.size() != 0) {
-      Integer node = queue.get(queue.size()-1);
-      queue.remove(queue.size()-1);
-
-      if (node == FINAL_END_NODE || node == NON_FINAL_END_NODE) {
-        continue;
-      }
-
-      // scan all arcs
-      readFirstArc(node, arc);
-      while(true) {
-
-        if (!seen.contains(arc.target)) {
-          //out.println("  " + arc.target + " [label=" + arc.target + "];");
-          out.println("  " + arc.target + " [label=\"\" shape=circle];");
-          seen.add(arc.target);
-          queue.add(arc.target);
-        }
-        String outs;
-        if (arc.output != NO_OUTPUT) {
-          outs = "/" + outputs.outputToString(arc.output);
-        } else {
-          outs = "";
-        }
-        if (arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
-          outs += " (" + outputs.outputToString(arc.nextFinalOutput) + ")";
-        }
-        out.print("  " + node + " -> " + arc.target + " [label=\"" + arc.label + outs + "\"");
-        if (arc.isFinal()) {
-          out.print(" arrowhead=tee");
-        }
-        if (arc.flag(BIT_TARGET_NEXT)) {
-          out.print(" color=blue");
-        }
-        out.println("];");
-        
-        if (arc.isLast()) {
-          break;
-        } else {
-          readNextArc(arc);
-        }
-      }
-    }
-    out.println("}");
-  }
-  
   public int getNodeCount() {
     // 1+ in order to count the -1 implicit final node
     return 1+nodeCount;
@@ -872,7 +710,7 @@ public class FST<T> {
   }
 
   // Non-static: writes to FST's byte[]
-  private class BytesWriter extends DataOutput {
+  class BytesWriter extends DataOutput {
     int posWrite;
 
     public BytesWriter() {
@@ -899,8 +737,13 @@ public class FST<T> {
     }
   }
 
+  final BytesReader getBytesReader(int pos) {
+    // TODO: maybe re-use via ThreadLocal?
+    return new BytesReader(pos);
+  }
+
   // Non-static: reads byte[] from FST
-  private class BytesReader extends DataInput {
+  class BytesReader extends DataInput {
     int pos;
 
     public BytesReader(int pos) {

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java?rev=1055405&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java Wed Jan  5 10:52:04 2011
@@ -0,0 +1,483 @@
+package org.apache.lucene.util.automaton.fst;
+
+/**
+ * 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 org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+
+/** Can next() and advance() through the terms in an FST
+  * @lucene.experimental
+*/
+
+abstract class FSTEnum<T> {
+  protected final FST<T> fst;
+
+  @SuppressWarnings("unchecked") protected FST.Arc<T>[] arcs = new FST.Arc[10];
+  // outputs are cumulative
+  @SuppressWarnings("unchecked") protected T[] output = (T[]) new Object[10];
+
+  protected final T NO_OUTPUT;
+  protected final FST.Arc<T> scratchArc = new FST.Arc<T>();
+
+  protected int upto;
+  protected int targetLength;
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
+  protected FSTEnum(FST<T> fst) {
+    this.fst = fst;
+    NO_OUTPUT = fst.outputs.getNoOutput();
+    fst.getFirstArc(getArc(0));
+    output[0] = NO_OUTPUT;
+  }
+
+  protected abstract int getTargetLabel();
+  protected abstract int getCurrentLabel();
+
+  protected abstract void setCurrentLabel(int label);
+  protected abstract void grow();
+
+  /** Rewinds enum state to match the shared prefix between
+   *  current term and target term */
+  protected final void rewindPrefix() throws IOException {
+    if (upto == 0) {
+      //System.out.println("  init");
+      upto = 1;
+      fst.readFirstTargetArc(getArc(0), getArc(1));
+      return;
+    }
+    //System.out.println("  rewind upto=" + upto + " vs targetLength=" + targetLength);
+
+    final int currentLimit = upto;
+    upto = 1;
+    while (upto < currentLimit && upto <= targetLength+1) {
+      final int cmp = getCurrentLabel() - getTargetLabel();
+      if (cmp < 0) {
+        // seek forward
+        break;
+      } else if (cmp > 0) {
+        // seek backwards -- reset this arc to the first arc
+        final FST.Arc<T> arc = getArc(upto);
+        fst.readFirstTargetArc(getArc(upto-1), arc);
+        //System.out.println("    seek first arc");
+        break;
+      }
+      upto++;
+    }
+  }
+
+  protected void doNext() throws IOException {
+    //System.out.println("FE: next upto=" + upto);
+    if (upto == 0) {
+      //System.out.println("  init");
+      upto = 1;
+      fst.readFirstTargetArc(getArc(0), getArc(1));
+    } else {
+      // pop
+      //System.out.println("  check pop curArc target=" + arcs[upto].target + " label=" + arcs[upto].label + " isLast?=" + arcs[upto].isLast());
+      while (arcs[upto].isLast()) {
+        upto--;
+        if (upto == 0) {
+          //System.out.println("  eof");
+          return;
+        }
+      }
+      fst.readNextArc(arcs[upto]);
+    }
+
+    pushFirst();
+  }
+
+  // TODO: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND /
+  // SEEK_END)?  saves the eq check above?
+
+  /** Seeks to smallest term that's >= target. */
+  protected void doSeekCeil() throws IOException {
+
+    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
+
+    // TODO: possibly caller could/should provide common
+    // prefix length?  ie this work may be redundant if
+    // caller is in fact intersecting against its own
+    // automaton
+
+    //System.out.println("FE.seekCeil upto=" + upto);
+
+    // Save time by starting at the end of the shared prefix
+    // b/w our current term & the target:
+    rewindPrefix();
+    //System.out.println("  after rewind upto=" + upto);
+
+    FST.Arc<T> arc = getArc(upto);
+    int targetLabel = getTargetLabel();
+    //System.out.println("  init targetLabel=" + targetLabel);
+
+    // Now scan forward, matching the new suffix of the target
+    while(true) {
+
+      //System.out.println("  cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") vs targetLabel=" + targetLabel);
+
+      if (arc.bytesPerArc != 0 && arc.label != -1) {
+
+        // Arcs are fixed array -- use binary search to find
+        // the target.
+
+        final FST.BytesReader in = fst.getBytesReader(0);
+        int low = arc.arcIdx;
+        int high = arc.numArcs-1;
+        int mid = 0;
+        //System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
+        boolean found = false;
+        while (low <= high) {
+          mid = (low + high) >>> 1;
+          in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+          final int midLabel = fst.readLabel(in);
+          final int cmp = midLabel - targetLabel;
+          //System.out.println("  cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
+          if (cmp < 0)
+            low = mid + 1;
+          else if (cmp > 0)
+            high = mid - 1;
+          else {
+            found = true;
+            break;
+          }
+        }
+
+        // NOTE: this code is dup'd w/ the code below (in
+        // the outer else clause):
+        if (found) {
+          // Match
+          arc.arcIdx = mid-1;
+          fst.readNextRealArc(arc);
+          assert arc.arcIdx == mid;
+          assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+          continue;
+        } else if (low == arc.numArcs) {
+          // Dead end
+          arc.arcIdx = arc.numArcs-2;
+          fst.readNextRealArc(arc);
+          assert arc.isLast();
+          // Dead end (target is after the last arc);
+          // rollback to last fork then push
+          upto--;
+          while(true) {
+            if (upto == 0) {
+              return;
+            }
+            final FST.Arc<T> prevArc = getArc(upto);
+            //System.out.println("  rollback upto=" + upto + " arc.label=" + prevArc.label + " isLast?=" + prevArc.isLast());
+            if (!prevArc.isLast()) {
+              fst.readNextArc(prevArc);
+              pushFirst();
+              return;
+            }
+            upto--;
+          }
+        } else {
+          arc.arcIdx = (low > high ? low : high)-1;
+          fst.readNextRealArc(arc);
+          assert arc.label > targetLabel;
+          pushFirst();
+          return;
+        }
+      } else {
+        // Arcs are not array'd -- must do linear scan:
+        if (arc.label == targetLabel) {
+          // recurse
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+        } else if (arc.label > targetLabel) {
+          pushFirst();
+          return;
+        } else if (arc.isLast()) {
+          // Dead end (target is after the last arc);
+          // rollback to last fork then push
+          upto--;
+          while(true) {
+            if (upto == 0) {
+              return;
+            }
+            final FST.Arc<T> prevArc = getArc(upto);
+            //System.out.println("  rollback upto=" + upto + " arc.label=" + prevArc.label + " isLast?=" + prevArc.isLast());
+            if (!prevArc.isLast()) {
+              fst.readNextArc(prevArc);
+              pushFirst();
+              return;
+            }
+            upto--;
+          }
+        } else {
+          // keep scanning
+          //System.out.println("    next scan");
+          fst.readNextArc(arc);
+        }
+      }
+    }
+  }
+
+  // TODO: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND /
+  // SEEK_END)?  saves the eq check above?
+  /** Seeks to largest term that's <= target. */
+  protected void doSeekFloor() throws IOException {
+
+    // TODO: possibly caller could/should provide common
+    // prefix length?  ie this work may be redundant if
+    // caller is in fact intersecting against its own
+    // automaton
+    //System.out.println("FE: seek floor upto=" + upto);
+
+    // Save CPU by starting at the end of the shared prefix
+    // b/w our current term & the target:
+    rewindPrefix();
+
+    //System.out.println("FE: after rewind upto=" + upto);
+
+    FST.Arc<T> arc = getArc(upto);
+    int targetLabel = getTargetLabel();
+
+    //System.out.println("FE: init targetLabel=" + targetLabel);
+
+    // Now scan forward, matching the new suffix of the target
+    while(true) {
+      //System.out.println("  cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast());
+
+      if (arc.bytesPerArc != 0 && arc.label != -1) {
+        // Arcs are fixed array -- use binary search to find
+        // the target.
+
+        final FST.BytesReader in = fst.getBytesReader(0);
+        int low = arc.arcIdx;
+        int high = arc.numArcs-1;
+        int mid = 0;
+        //System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
+        boolean found = false;
+        while (low <= high) {
+          mid = (low + high) >>> 1;
+          in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+          final int midLabel = fst.readLabel(in);
+          final int cmp = midLabel - targetLabel;
+          //System.out.println("  cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
+          if (cmp < 0)
+            low = mid + 1;
+          else if (cmp > 0)
+            high = mid - 1;
+          else {
+            found = true;
+            break;
+          }
+        }
+
+        // NOTE: this code is dup'd w/ the code below (in
+        // the outer else clause):
+        if (found) {
+          // Match -- recurse
+          //System.out.println("  match!  arcIdx=" + mid);
+          arc.arcIdx = mid-1;
+          fst.readNextRealArc(arc);
+          assert arc.arcIdx == mid;
+          assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+          continue;
+        } else if (high == -1) {
+          //System.out.println("  before first");
+          // Very first arc is after our target
+          // TODO: if each arc could somehow read the arc just
+          // before, we can save this re-scan.  The ceil case
+          // doesn't need this because it reads the next arc
+          // instead:
+          while(true) {
+            // First, walk backwards until we find a first arc
+            // that's before our target label:
+            fst.readFirstTargetArc(getArc(upto-1), arc);
+            if (arc.label < targetLabel) {
+              // Then, scan forwards to the arc just before
+              // the targetLabel:
+              while(!arc.isLast() && fst.readNextArcLabel(arc) < targetLabel) {
+                fst.readNextArc(arc);
+              }
+              pushLast();
+              return;
+            }
+            upto--;
+            if (upto == 0) {
+              return;
+            }
+            targetLabel = getTargetLabel();
+            arc = getArc(upto);
+          }
+        } else {
+          // There is a floor arc:
+          arc.arcIdx = (low > high ? high : low)-1;
+          //System.out.println(" hasFloor arcIdx=" + (arc.arcIdx+1));
+          fst.readNextRealArc(arc);
+          assert arc.isLast() || fst.readNextArcLabel(arc) > targetLabel;
+          assert arc.label < targetLabel;
+          pushLast();
+          return;
+        }        
+      } else {
+
+        if (arc.label == targetLabel) {
+          // Match -- recurse
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+        } else if (arc.label > targetLabel) {
+          // TODO: if each arc could somehow read the arc just
+          // before, we can save this re-scan.  The ceil case
+          // doesn't need this because it reads the next arc
+          // instead:
+          while(true) {
+            // First, walk backwards until we find a first arc
+            // that's before our target label:
+            fst.readFirstTargetArc(getArc(upto-1), arc);
+            if (arc.label < targetLabel) {
+              // Then, scan forwards to the arc just before
+              // the targetLabel:
+              while(!arc.isLast() && fst.readNextArcLabel(arc) < targetLabel) {
+                fst.readNextArc(arc);
+              }
+              pushLast();
+              return;
+            }
+            upto--;
+            if (upto == 0) {
+              return;
+            }
+            targetLabel = getTargetLabel();
+            arc = getArc(upto);
+          }
+        } else if (!arc.isLast()) {
+          //System.out.println("  check next label=" + fst.readNextArcLabel(arc) + " (" + (char) fst.readNextArcLabel(arc) + ")");
+          if (fst.readNextArcLabel(arc) > targetLabel) {
+            pushLast();
+            return;
+          } else {
+            // keep scanning
+            fst.readNextArc(arc);
+          }
+        } else {
+          pushLast();
+          return;
+        }
+      }
+    }
+  }
+
+  private void incr() {
+    upto++;
+    grow();
+    if (arcs.length <= upto) {
+      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
+        new FST.Arc[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
+      arcs = newArcs;
+    }
+    if (output.length <= upto) {
+      @SuppressWarnings("unchecked") final T[] newOutput =
+        (T[]) new Object[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(output, 0, newOutput, 0, output.length);
+      output = newOutput;
+    }
+  }
+
+  // Appends current arc, and then recurses from its target,
+  // appending first arc all the way to the final node
+  private void pushFirst() throws IOException {
+
+    FST.Arc<T> arc = arcs[upto];
+    assert arc != null;
+
+    while (true) {
+      output[upto] = fst.outputs.add(output[upto-1], arc.output);
+      if (arc.label == FST.END_LABEL) {
+        // Final node
+        break;
+      }
+      //System.out.println("  pushFirst label=" + (char) arc.label + " upto=" + upto + " output=" + fst.outputs.outputToString(output[upto]));
+      setCurrentLabel(arc.label);
+      incr();
+      
+      final FST.Arc<T> nextArc = getArc(upto);
+      fst.readFirstTargetArc(arc, nextArc);
+      arc = nextArc;
+    }
+  }
+
+  // Recurses from current arc, appending last arc all the
+  // way to the first final node
+  private void pushLast() throws IOException {
+
+    FST.Arc<T> arc = arcs[upto];
+    assert arc != null;
+
+    while (true) {
+      setCurrentLabel(arc.label);
+      output[upto] = fst.outputs.add(output[upto-1], arc.output);
+      if (arc.label == FST.END_LABEL) {
+        // Final node
+        break;
+      }
+      incr();
+
+      final FST.Arc<T> nextArc = getArc(upto);
+      fst.readFirstTargetArc(arc, nextArc);
+      arc = nextArc;
+      while(!arc.isLast()) {
+        fst.readNextArc(arc);
+      }
+    }
+  }
+
+  private FST.Arc<T> getArc(int idx) {
+    if (arcs[idx] == null) {
+      arcs[idx] = new FST.Arc<T>();
+    }
+    return arcs[idx];
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java Wed Jan  5 10:52:04 2011
@@ -17,9 +17,7 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
 
@@ -27,163 +25,23 @@ import java.io.IOException;
   * @lucene.experimental
 */
 
-public class IntsRefFSTEnum<T> {
-  private final FST<T> fst;
-
-  private IntsRef current = new IntsRef(10);
-  @SuppressWarnings("unchecked") private FST.Arc<T>[] arcs = new FST.Arc[10];
-  // outputs are cumulative
-  @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10];
-
-  private boolean lastFinal;
-  private boolean didEmpty;
-  private final T NO_OUTPUT;
+public final class IntsRefFSTEnum<T> extends FSTEnum<T> {
+  private final IntsRef current = new IntsRef(10);
   private final InputOutput<T> result = new InputOutput<T>();
+  private IntsRef target;
 
   public static class InputOutput<T> {
     public IntsRef input;
     public T output;
   }
-  
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
   public IntsRefFSTEnum(FST<T> fst) {
-    this.fst = fst;
+    super(fst);
     result.input = current;
-    NO_OUTPUT = fst.outputs.getNoOutput();
-  }
-
-  public void reset() {
-    lastFinal = false;
-    didEmpty = false;
-    current.length = 0;
-    result.output = NO_OUTPUT;
-  }
-
-  /** NOTE: target must be >= where we are already
-   *  positioned */
-  public InputOutput<T> advance(IntsRef target) throws IOException {
-
-    assert target.compareTo(current) >= 0;
-
-    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
-
-    // special case empty string
-    if (current.length == 0) {
-      if (target.length == 0) {
-        final T output = fst.getEmptyOutput();      
-        if (output != null) {
-          if (!didEmpty) {
-            current.length = 0;
-            lastFinal = true;
-            result.output = output;
-            didEmpty = true;
-          }
-          return result;
-        } else {
-          return next();
-        }
-      }
-      
-      if (fst.noNodes()) {
-        return null;
-      }
-    }
-
-    // TODO: possibly caller could/should provide common
-    // prefix length?  ie this work may be redundant if
-    // caller is in fact intersecting against its own
-    // automaton
-
-    // what prefix does target share w/ current
-    int idx = 0;
-    while (idx < current.length && idx < target.length) {
-      if (current.ints[idx] != target.ints[target.offset + idx]) {
-        break;
-      }
-      idx++;
-    }
-
-    //System.out.println("  shared " + idx);
-
-    FST.Arc<T> arc;
-    if (current.length == 0) {
-      // new enum (no seek/next yet)
-      arc = fst.readFirstArc(fst.getStartNode(), getArc(0));
-      //System.out.println("  new enum");
-    } else if (idx < current.length) {
-      // roll back to shared point
-      lastFinal = false;
-      current.length = idx;
-      arc = arcs[idx];
-      if (arc.isLast()) {
-        if (idx == 0) {
-          return null;
-        } else {
-          return next();
-        }
-      }
-      arc = fst.readNextArc(arc);
-    } else if (idx == target.length) {
-      // degenerate case -- seek to term we are already on
-      assert target.equals(current);
-      return result;
-    } else {
-      // current is a full prefix of target
-      if (lastFinal) {
-        arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      } else {
-        return next();
-      }
-    }
-
-    lastFinal = false;
-
-    assert arc == arcs[current.length];
-    int targetLabel = target.ints[target.offset+current.length];
-
-    while(true) {
-      //System.out.println("    cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label));
-      if (arc.label == targetLabel) {
-        grow();
-        current.ints[current.length] = arc.label;
-        appendOutput(arc.output);
-        current.length++;
-        grow();
-        if (current.length == target.length) {
-          result.output = output[current.length-1];
-          if (arc.isFinal()) {
-            // target is exact match
-            if (fst.hasArcs(arc.target)) {
-              // target is also a proper prefix of other terms
-              lastFinal = true;
-              appendFinalOutput(arc.nextFinalOutput);
-            }
-          } else {
-            // target is not a match but is a prefix of
-            // other terms
-            current.length--;
-            push();
-          }
-          return result;
-        } else if (!fst.hasArcs(arc.target)) {
-          // we only match a prefix of the target
-          return next();
-        } else {
-          targetLabel = target.ints[target.offset+current.length];
-          arc = fst.readFirstArc(arc.target, getArc(current.length));
-        }
-      } else if (arc.label > targetLabel) {
-        // we are now past the target
-        push();
-        return result;
-      } else if (arc.isLast()) {
-        if (current.length == 0) {
-          return null;
-        }
-        return next();
-      } else {
-        arc = fst.readNextArc(getArc(current.length));
-      }
-    }
+    current.offset = 1;
   }
 
   public InputOutput<T> current() {
@@ -192,124 +50,58 @@ public class IntsRefFSTEnum<T> {
 
   public InputOutput<T> next() throws IOException {
     //System.out.println("  enum.next");
-
-    if (current.length == 0) {
-      final T output = fst.getEmptyOutput();
-      if (output != null) {
-        if (!didEmpty) {
-          current.length = 0;
-          lastFinal = true;
-          result.output = output;
-          didEmpty = true;
-          return result;
-        } else {
-          lastFinal = false;
-        }
-      }
-      if (fst.noNodes()) {
-        return null;
-      }
-      fst.readFirstArc(fst.getStartNode(), getArc(0));
-      push();
-    } else if (lastFinal) {
-      lastFinal = false;
-      assert current.length > 0;
-      // resume pushing
-      fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      push();
-    } else {
-      //System.out.println("    pop/push");
-      pop();
-      if (current.length == 0) {
-        // enum done
-        return null;
-      } else {
-        current.length--;
-        fst.readNextArc(arcs[current.length]);
-        push();
-      }
-    }
-
-    return result;
+    doNext();
+    return setResult();
   }
 
-  private void grow() {
-    final int l = current.length + 1;
-    current.grow(l);
-    if (arcs.length < l) {
-      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
-        new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
-      arcs = newArcs;
-    }
-    if (output.length < l) {
-      @SuppressWarnings("unchecked") final T[] newOutput =
-        (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(output, 0, newOutput, 0, output.length);
-      output = newOutput;
-    }
-  }
-
-  private void appendOutput(T addedOutput) {
-    T newOutput;
-    if (current.length == 0) {
-      newOutput = addedOutput;
-    } else if (addedOutput == NO_OUTPUT) {
-      output[current.length] = output[current.length-1];
-      return;
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
     } else {
-      newOutput = fst.outputs.add(output[current.length-1], addedOutput);
+      return target.ints[target.offset + upto - 1];
     }
-    output[current.length] = newOutput;
   }
 
-  private void appendFinalOutput(T addedOutput) {
-    if (current.length == 0) {
-      result.output = addedOutput;
-    } else {
-      result.output = fst.outputs.add(output[current.length-1], addedOutput);
-    }
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.ints[upto];
   }
 
-  private void push() throws IOException {
-
-    FST.Arc<T> arc = arcs[current.length];
-    assert arc != null;
-
-    while(true) {
-      grow();
-      
-      current.ints[current.length] = arc.label;
-      appendOutput(arc.output);
-      //System.out.println("    push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output));
-      current.length++;
-      grow();
-
-      if (!fst.hasArcs(arc.target)) {
-        break;
-      }
-
-      if (arc.isFinal()) {
-        appendFinalOutput(arc.nextFinalOutput);
-        lastFinal = true;
-        return;
-      }
-
-      arc = fst.readFirstArc(arc.target, getArc(current.length));
-    }
-    result.output = output[current.length-1];
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.ints[upto] = label;
   }
 
-  private void pop() {
-    while (current.length > 0 && arcs[current.length-1].isLast()) {
-      current.length--;
-    }
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
   }
 
-  private FST.Arc<T> getArc(int idx) {
-    if (arcs[idx] == null) {
-      arcs[idx] = new FST.Arc<T>();
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
     }
-    return arcs[idx];
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java Wed Jan  5 10:52:04 2011
@@ -35,7 +35,7 @@ final class NodeHash<T> {
   }
 
   private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address) throws IOException {
-    fst.readFirstArc(address, scratchArc);
+    fst.readFirstRealArc(address, scratchArc);
     if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
       return false;
     }
@@ -56,7 +56,7 @@ final class NodeHash<T> {
           return false;
         }
       }
-      fst.readNextArc(scratchArc);
+      fst.readNextRealArc(scratchArc);
     }
 
     return false;
@@ -89,7 +89,7 @@ final class NodeHash<T> {
     final int PRIME = 31;
     //System.out.println("hash frozen");
     int h = 0;
-    fst.readFirstArc(node, scratchArc);
+    fst.readFirstRealArc(node, scratchArc);
     while(true) {
       //System.out.println("  label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal());
       h = PRIME * h + scratchArc.label;
@@ -102,7 +102,7 @@ final class NodeHash<T> {
       if (scratchArc.isLast()) {
         break;
       }
-      fst.readNextArc(scratchArc);
+      fst.readNextRealArc(scratchArc);
     }
     //System.out.println("  ret " + (h&Integer.MAX_VALUE));
     return h & Integer.MAX_VALUE;

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java?rev=1055405&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java Wed Jan  5 10:52:04 2011
@@ -0,0 +1,249 @@
+package org.apache.lucene.util.automaton.fst;
+
+/**
+ * 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.io.PrintStream;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.HashSet;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+/** Static helper methods */
+public final class Util {
+  private Util() {
+  }
+
+  /** Looks up the output for this input, or null if the
+   *  input is not accepted. FST must be
+   *  INPUT_TYPE.BYTE4. */
+  public static<T> T get(FST<T> fst, IntsRef input) throws IOException {
+    assert fst.inputType == FST.INPUT_TYPE.BYTE4;
+
+    // TODO: would be nice not to alloc this on every lookup
+    final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+
+    // Accumulate output as we go
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+    T output = NO_OUTPUT;
+    for(int i=0;i<input.length;i++) {
+      if (fst.findTargetArc(input.ints[input.offset + i], arc, arc) == null) {
+        return null;
+      } else if (arc.output != NO_OUTPUT) {
+        output = fst.outputs.add(output, arc.output);
+      }
+    }
+
+    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
+      return null;
+    } else if (arc.output != NO_OUTPUT) {
+      return fst.outputs.add(output, arc.output);
+    } else {
+      return output;
+    }
+  }
+
+  /** Logically casts input to UTF32 ints then looks up the output
+   *  or null if the input is not accepted.  FST must be
+   *  INPUT_TYPE.BYTE4.  */
+  public static<T> T get(FST<T> fst, char[] input, int offset, int length) throws IOException {
+    assert fst.inputType == FST.INPUT_TYPE.BYTE4;
+
+    // TODO: would be nice not to alloc this on every lookup
+    final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+
+    int charIdx = offset;
+    final int charLimit = offset + length;
+
+    // Accumulate output as we go
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+    T output = NO_OUTPUT;
+    while(charIdx < charLimit) {
+      final int utf32 = Character.codePointAt(input, charIdx);
+      charIdx += Character.charCount(utf32);
+
+      if (fst.findTargetArc(utf32, arc, arc) == null) {
+        return null;
+      } else if (arc.output != NO_OUTPUT) {
+        output = fst.outputs.add(output, arc.output);
+      }
+    }
+
+    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
+      return null;
+    } else if (arc.output != NO_OUTPUT) {
+      return fst.outputs.add(output, arc.output);
+    } else {
+      return output;
+    }
+  }
+
+
+  /** Logically casts input to UTF32 ints then looks up the output
+   *  or null if the input is not accepted.  FST must be
+   *  INPUT_TYPE.BYTE4.  */
+  public static<T> T get(FST<T> fst, CharSequence input) throws IOException {
+    assert fst.inputType == FST.INPUT_TYPE.BYTE4;
+    
+    // TODO: would be nice not to alloc this on every lookup
+    final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+
+    int charIdx = 0;
+    final int charLimit = input.length();
+
+    // Accumulate output as we go
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+    T output = NO_OUTPUT;
+
+    while(charIdx < charLimit) {
+      final int utf32 = Character.codePointAt(input, charIdx);
+      charIdx += Character.charCount(utf32);
+
+      if (fst.findTargetArc(utf32, arc, arc) == null) {
+        return null;
+      } else if (arc.output != NO_OUTPUT) {
+        output = fst.outputs.add(output, arc.output);
+      }
+    }
+
+    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
+      return null;
+    } else if (arc.output != NO_OUTPUT) {
+      return fst.outputs.add(output, arc.output);
+    } else {
+      return output;
+    }
+  }
+
+  /** Looks up the output for this input, or null if the
+   *  input is not accepted */
+  public static<T> T get(FST<T> fst, BytesRef input) throws IOException {
+    assert fst.inputType == FST.INPUT_TYPE.BYTE1;
+
+    // TODO: would be nice not to alloc this on every lookup
+    final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+
+    // Accumulate output as we go
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+    T output = NO_OUTPUT;
+    for(int i=0;i<input.length;i++) {
+      if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc) == null) {
+        return null;
+      } else if (arc.output != NO_OUTPUT) {
+        output = fst.outputs.add(output, arc.output);
+      }
+    }
+
+    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
+      return null;
+    } else if (arc.output != NO_OUTPUT) {
+      return fst.outputs.add(output, arc.output);
+    } else {
+      return output;
+    }
+  }
+
+
+  // NOTE: this consumes alot of RAM!
+  // arcs w/ NEXT opto are in blue
+  /*
+    eg:
+      PrintStream ps = new PrintStream("out.dot");
+      fst.toDot(ps);
+      ps.close();
+      System.out.println("SAVED out.dot");
+      
+    then dot -Tpng out.dot > /x/tmp/out.png
+  */
+
+  public static<T> void toDot(FST<T> fst, PrintStream out) throws IOException {
+    
+    final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
+
+    final List<FST.Arc<T>> queue = new ArrayList<FST.Arc<T>>();
+    queue.add(startArc);
+
+    final Set<Integer> seen = new HashSet<Integer>();
+    seen.add(startArc.target);
+    
+    out.println("digraph FST {");
+    out.println("  rankdir = LR;");
+    //out.println("  " + startNode + " [shape=circle label=" + startNode + "];");
+    out.println("  " + startArc.target + " [label=\"\" shape=circle];");
+    out.println("  initial [shape=point color=white label=\"\"];");
+    out.println("  initial -> " + startArc.target);
+
+    final T NO_OUTPUT = fst.outputs.getNoOutput();
+
+    while(queue.size() != 0) {
+      FST.Arc<T> arc = queue.get(queue.size()-1);
+      queue.remove(queue.size()-1);
+      //System.out.println("dot cycle target=" + arc.target);
+
+      if (fst.targetHasArcs(arc)) {
+
+        // scan all arcs
+        final int node = arc.target;
+        fst.readFirstTargetArc(arc, arc);
+        while(true) {
+
+          //System.out.println("  cycle label=" + arc.label + " (" + (char) arc.label + ") target=" + arc.target);
+          if (!seen.contains(arc.target)) {
+            final String shape;
+            if (arc.target == -1) {
+              shape = "doublecircle";
+            } else {
+              shape = "circle";
+            }
+            out.println("  " + arc.target + " [shape=" + shape + "];");
+            seen.add(arc.target);
+            queue.add(new FST.Arc<T>().copyFrom(arc));
+            //System.out.println("    new!");
+          }
+          String outs;
+          if (arc.output != NO_OUTPUT) {
+            outs = "/" + fst.outputs.outputToString(arc.output);
+          } else {
+            outs = "";
+          }
+          final char cl;
+          if (arc.label == FST.END_LABEL) {
+            cl = '~';
+          } else {
+            cl = (char) arc.label;
+          }
+          out.println("  " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]");
+          //if (arc.flag(FST.BIT_TARGET_NEXT)) {
+          //out.print(" color=blue");
+          //}
+          //out.println("];");
+        
+          if (arc.isLast()) {
+            break;
+          } else {
+            fst.readNextArc(arc);
+          }
+        }
+      }
+    }
+    out.println("}");
+  }
+}

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java Wed Jan  5 10:52:04 2011
@@ -87,6 +87,7 @@ public class RandomIndexWriter implement
     if (LuceneTestCase.VERBOSE) {
       System.out.println("RIW config=" + w.getConfig());
       System.out.println("codec default=" + w.getConfig().getCodecProvider().getDefaultFieldCodec());
+      w.setInfoStream(System.out);
     }
   } 
 

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=1055405&r1=1055404&r2=1055405&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 Wed Jan  5 10:52:04 2011
@@ -23,14 +23,15 @@ import java.util.HashSet;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PostingsConsumer;
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
+import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
@@ -40,6 +41,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util._TestUtil;
 
 // TODO: test multiple codecs here?
 
@@ -68,43 +70,6 @@ public class TestCodecs extends LuceneTe
   private final static int DOC_FREQ_RAND = 500; // must be > 16 to test skipping
   private final static int TERM_DOC_FREQ_RAND = 20;
 
-  // start is inclusive and end is exclusive
-  public int nextInt(final int start, final int end) {
-    return start + random.nextInt(end-start);
-  }
-
-  private int nextInt(final int lim) {
-    return random.nextInt(lim);
-  }
-
-  char[] getRandomText() {
-
-    final int len = 1+this.nextInt(10);
-    final char[] buffer = new char[len+1];
-    for(int i=0;i<len;i++) {
-      buffer[i] = (char) this.nextInt(97, 123);
-      /*
-      final int t = nextInt(5);
-      if (0 == t && i < len-1) {
-        // Make a surrogate pair
-        // High surrogate
-        buffer[i++] = (char) nextInt(0xd800, 0xdc00);
-        // Low surrogate
-        buffer[i] = (char) nextInt(0xdc00, 0xe000);
-      } else if (t <= 1)
-        buffer[i] = (char) nextInt(0x80);
-      else if (2 == t)
-        buffer[i] = (char) nextInt(0x80, 0x800);
-      else if (3 == t)
-        buffer[i] = (char) nextInt(0x800, 0xd800);
-      else
-        buffer[i] = (char) nextInt(0xe000, 0xffff);
-    */
-    }
-    buffer[len] = 0xffff;
-    return buffer;
-  }
-
   class FieldData implements Comparable {
     final FieldInfo fieldInfo;
     final TermData[] terms;
@@ -163,7 +128,7 @@ public class TestCodecs extends LuceneTe
     }
 
     public int compareTo(final Object o) {
-      return text2.compareTo(((TermData) o).text2);
+      return text.compareTo(((TermData) o).text);
     }
 
     public void write(final TermsConsumer termsConsumer) throws Throwable {
@@ -191,7 +156,7 @@ public class TestCodecs extends LuceneTe
   final private static String SEGMENT = "0";
 
   TermData[] makeRandomTerms(final boolean omitTF, final boolean storePayloads) {
-    final int numTerms = 1+this.nextInt(NUM_TERMS_RAND);
+    final int numTerms = 1+random.nextInt(NUM_TERMS_RAND);
     //final int numTerms = 2;
     final TermData[] terms = new TermData[numTerms];
 
@@ -200,18 +165,16 @@ public class TestCodecs extends LuceneTe
     for(int i=0;i<numTerms;i++) {
 
       // Make term text
-      char[] text;
       String text2;
       while(true) {
-        text = this.getRandomText();
-        text2 = new String(text, 0, text.length-1);
-        if (!termsSeen.contains(text2)) {
+        text2 = _TestUtil.randomUnicodeString(random);
+        if (!termsSeen.contains(text2) && !text2.endsWith(".")) {
           termsSeen.add(text2);
           break;
         }
       }
 
-      final int docFreq = 1+this.nextInt(DOC_FREQ_RAND);
+      final int docFreq = 1+random.nextInt(DOC_FREQ_RAND);
       final int[] docs = new int[docFreq];
       PositionData[][] positions;
 
@@ -222,21 +185,21 @@ public class TestCodecs extends LuceneTe
 
       int docID = 0;
       for(int j=0;j<docFreq;j++) {
-        docID += this.nextInt(1, 10);
+        docID += _TestUtil.nextInt(random, 1, 10);
         docs[j] = docID;
 
         if (!omitTF) {
-          final int termFreq = 1+this.nextInt(TERM_DOC_FREQ_RAND);
+          final int termFreq = 1+random.nextInt(TERM_DOC_FREQ_RAND);
           positions[j] = new PositionData[termFreq];
           int position = 0;
           for(int k=0;k<termFreq;k++) {
-            position += this.nextInt(1, 10);
+            position += _TestUtil.nextInt(random, 1, 10);
 
             final BytesRef payload;
-            if (storePayloads && this.nextInt(4) == 0) {
-              final byte[] bytes = new byte[1+this.nextInt(5)];
+            if (storePayloads && random.nextInt(4) == 0) {
+              final byte[] bytes = new byte[1+random.nextInt(5)];
               for(int l=0;l<bytes.length;l++) {
-                bytes[l] = (byte) this.nextInt(255);
+                bytes[l] = (byte) random.nextInt(255);
               }
               payload = new BytesRef(bytes);
             } else {
@@ -269,7 +232,7 @@ public class TestCodecs extends LuceneTe
     final FieldData[] fields = new FieldData[] {field};
 
     final Directory dir = newDirectory();
-    this.write(fieldInfos, dir, fields);
+    this.write(fieldInfos, dir, fields, true);
     final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
     si.setHasProx(false);
 
@@ -317,19 +280,25 @@ public class TestCodecs extends LuceneTe
 
     final Directory dir = newDirectory();
 
-    this.write(fieldInfos, dir, fields);
+    if (VERBOSE) {
+      System.out.println("TEST: now write postings");
+    }
+    this.write(fieldInfos, dir, fields, false);
     final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
 
+    if (VERBOSE) {
+      System.out.println("TEST: now read postings");
+    }
     final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
     final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
     for(int i=0;i<NUM_TEST_THREADS-1;i++) {
-      threads[i] = new Verify(fields, terms);
+      threads[i] = new Verify(si, fields, terms);
       threads[i].setDaemon(true);
       threads[i].start();
     }
 
-    new Verify(fields, terms).run();
+    new Verify(si, fields, terms).run();
 
     for(int i=0;i<NUM_TEST_THREADS-1;i++) {
       threads[i].join();
@@ -409,11 +378,13 @@ public class TestCodecs extends LuceneTe
   private class Verify extends Thread {
     final Fields termsDict;
     final FieldData[] fields;
+    final SegmentInfo si;
     volatile boolean failed;
 
-    Verify(final FieldData[] fields, final Fields termsDict) {
+    Verify(final SegmentInfo si, final FieldData[] fields, final Fields termsDict) {
       this.fields = fields;
       this.termsDict = termsDict;
+      this.si = si;
     }
 
     @Override
@@ -446,7 +417,7 @@ public class TestCodecs extends LuceneTe
         assertEquals(positions[i].pos, pos);
         if (positions[i].payload != null) {
           assertTrue(posEnum.hasPayload());
-          if (TestCodecs.this.nextInt(3) < 2) {
+          if (TestCodecs.random.nextInt(3) < 2) {
             // Verify the payload bytes
             final BytesRef otherPayload = posEnum.getPayload();
             assertTrue("expected=" + positions[i].payload.toString() + " got=" + otherPayload.toString(), positions[i].payload.equals(otherPayload));
@@ -460,22 +431,28 @@ public class TestCodecs extends LuceneTe
     public void _run() throws Throwable {
 
       for(int iter=0;iter<NUM_TEST_ITER;iter++) {
-        final FieldData field = fields[TestCodecs.this.nextInt(fields.length)];
+        final FieldData field = fields[TestCodecs.random.nextInt(fields.length)];
         final TermsEnum termsEnum = termsDict.terms(field.fieldInfo.name).iterator();
 
-        // Test straight enum of the terms:
+        if (si.getSegmentCodecs().codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
+          // code below expects unicode sort order
+          continue;
+        }
+
         int upto = 0;
+        // Test straight enum of the terms:
         while(true) {
           final BytesRef term = termsEnum.next();
           if (term == null) {
             break;
           }
-          assertTrue(new BytesRef(field.terms[upto++].text2).bytesEquals(term));
+          final BytesRef expected = new BytesRef(field.terms[upto++].text2);
+          assertTrue("expected=" + expected + " vs actual " + term, expected.bytesEquals(term));
         }
         assertEquals(upto, field.terms.length);
 
         // Test random seek:
-        TermData term = field.terms[TestCodecs.this.nextInt(field.terms.length)];
+        TermData term = field.terms[TestCodecs.random.nextInt(field.terms.length)];
         TermsEnum.SeekStatus status = termsEnum.seek(new BytesRef(term.text2));
         assertEquals(status, TermsEnum.SeekStatus.FOUND);
         assertEquals(term.docs.length, termsEnum.docFreq());
@@ -486,7 +463,7 @@ public class TestCodecs extends LuceneTe
         }
 
         // Test random seek by ord:
-        final int idx = TestCodecs.this.nextInt(field.terms.length);
+        final int idx = TestCodecs.random.nextInt(field.terms.length);
         term = field.terms[idx];
         try {
           status = termsEnum.seek(idx);
@@ -507,8 +484,7 @@ public class TestCodecs extends LuceneTe
 
         // Test seek to non-existent terms:
         for(int i=0;i<100;i++) {
-          final char[] text = TestCodecs.this.getRandomText();
-          final String text2 = new String(text, 0, text.length-1) + ".";
+          final String text2 = _TestUtil.randomUnicodeString(random) + ".";
           status = termsEnum.seek(new BytesRef(text2));
           assertTrue(status == TermsEnum.SeekStatus.NOT_FOUND ||
                      status == TermsEnum.SeekStatus.END);
@@ -533,7 +509,7 @@ public class TestCodecs extends LuceneTe
         // Seek to non-existent empty-string term
         status = termsEnum.seek(new BytesRef(""));
         assertNotNull(status);
-        assertEquals(status, TermsEnum.SeekStatus.NOT_FOUND);
+        //assertEquals(TermsEnum.SeekStatus.NOT_FOUND, status);
 
         // Make sure we're now pointing to first term
         assertTrue(termsEnum.term().bytesEquals(new BytesRef(field.terms[0].text2)));
@@ -543,7 +519,7 @@ public class TestCodecs extends LuceneTe
         upto = 0;
         do {
           term = field.terms[upto];
-          if (TestCodecs.this.nextInt(3) == 1) {
+          if (TestCodecs.random.nextInt(3) == 1) {
             final DocsEnum docs = termsEnum.docs(null, null);
             final DocsAndPositionsEnum postings = termsEnum.docsAndPositions(null, null);
 
@@ -558,10 +534,10 @@ public class TestCodecs extends LuceneTe
               // Maybe skip:
               final int left = term.docs.length-upto2;
               int doc;
-              if (TestCodecs.this.nextInt(3) == 1 && left >= 1) {
-                final int inc = 1+TestCodecs.this.nextInt(left-1);
+              if (TestCodecs.random.nextInt(3) == 1 && left >= 1) {
+                final int inc = 1+TestCodecs.random.nextInt(left-1);
                 upto2 += inc;
-                if (TestCodecs.this.nextInt(2) == 1) {
+                if (TestCodecs.random.nextInt(2) == 1) {
                   doc = docsEnum.advance(term.docs[upto2]);
                   assertEquals(term.docs[upto2], doc);
                 } else {
@@ -586,7 +562,7 @@ public class TestCodecs extends LuceneTe
               assertEquals(term.docs[upto2], doc);
               if (!field.omitTF) {
                 assertEquals(term.positions[upto2].length, docsEnum.freq());
-                if (TestCodecs.this.nextInt(2) == 1) {
+                if (TestCodecs.random.nextInt(2) == 1) {
                   this.verifyPositions(term.positions[upto2], postings);
                 }
               }
@@ -603,15 +579,19 @@ public class TestCodecs extends LuceneTe
     }
   }
 
-  private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
+  private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields, boolean allowPreFlex) throws Throwable {
 
-    final int termIndexInterval = this.nextInt(13, 27);
+    final int termIndexInterval = _TestUtil.nextInt(random, 13, 27);
     final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, CodecProvider.getDefault());
     final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo);
 
     final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
     Arrays.sort(fields);
     for (final FieldData field : fields) {
+      if (!allowPreFlex && codecInfo.codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
+        // code below expects unicode sort order
+        continue;
+      }
       field.write(consumer);
     }
     consumer.close();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFlex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFlex.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFlex.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFlex.java Wed Jan  5 10:52:04 2011
@@ -71,7 +71,11 @@ public class TestFlex extends LuceneTest
     IndexReader r = w.getReader();
     TermsEnum terms = r.getSequentialSubReaders()[0].fields().terms("f").iterator();
     assertTrue(terms.next() != null);
-    assertEquals(0, terms.ord());
+    try {
+      assertEquals(0, terms.ord());
+    } catch (UnsupportedOperationException uoe) {
+      // ok -- codec is not required to support this op
+    }
     r.close();
     w.close();
     d.close();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java Wed Jan  5 10:52:04 2011
@@ -56,6 +56,7 @@ public class TestPerFieldCodecSupport ex
     conf.setMergePolicy(logByteSizeMergePolicy);
 
     final IndexWriter writer = new IndexWriter(dir, conf);
+    writer.setInfoStream(VERBOSE ? System.out : null);
     return writer;
   }
 
@@ -110,12 +111,15 @@ public class TestPerFieldCodecSupport ex
   }
 
   /*
-   * Test is hetrogenous index segements are merge sucessfully
+   * Test that heterogeneous index segments are merged sucessfully
    */
   @Test
   public void testChangeCodecAndMerge() throws IOException {
     Directory dir = newDirectory();
     CodecProvider provider = new MockCodecProvider();
+    if (VERBOSE) {
+      System.out.println("TEST: make new index");
+    }
     IndexWriterConfig iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT,
              new MockAnalyzer()).setOpenMode(OpenMode.CREATE).setCodecProvider(provider);
     iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
@@ -125,6 +129,9 @@ public class TestPerFieldCodecSupport ex
     addDocs(writer, 10);
     writer.commit();
     assertQuery(new Term("content", "aaa"), dir, 10, provider);
+    if (VERBOSE) {
+      System.out.println("TEST: addDocs3");
+    }
     addDocs3(writer, 10);
     writer.commit();
     writer.close();
@@ -144,6 +151,9 @@ public class TestPerFieldCodecSupport ex
     iwconf.setCodecProvider(provider);
     writer = newWriter(dir, iwconf);
     // swap in new codec for currently written segments
+    if (VERBOSE) {
+      System.out.println("TEST: add docs w/ Standard codec for content field");
+    }
     addDocs2(writer, 10);
     writer.commit();
     Codec origContentCodec = provider.lookup("MockSep");
@@ -152,9 +162,12 @@ public class TestPerFieldCodecSupport ex
         origContentCodec, origContentCodec, newContentCodec);
     assertEquals(30, writer.maxDoc());
     assertQuery(new Term("content", "bbb"), dir, 10, provider);
-    assertQuery(new Term("content", "ccc"), dir, 10, provider);
+    assertQuery(new Term("content", "ccc"), dir, 10, provider);   ////
     assertQuery(new Term("content", "aaa"), dir, 10, provider);
 
+    if (VERBOSE) {
+      System.out.println("TEST: add more docs w/ new codec");
+    }
     addDocs2(writer, 10);
     writer.commit();
     assertQuery(new Term("content", "ccc"), dir, 10, provider);
@@ -162,6 +175,9 @@ public class TestPerFieldCodecSupport ex
     assertQuery(new Term("content", "aaa"), dir, 10, provider);
     assertEquals(40, writer.maxDoc());
 
+    if (VERBOSE) {
+      System.out.println("TEST: now optimize");
+    }
     writer.optimize();
     assertEquals(40, writer.maxDoc());
     writer.close();
@@ -206,6 +222,9 @@ public class TestPerFieldCodecSupport ex
 
   public void assertQuery(Term t, Directory dir, int num, CodecProvider codecs)
       throws CorruptIndexException, IOException {
+    if (VERBOSE) {
+      System.out.println("\nTEST: assertQuery " + t);
+    }
     IndexReader reader = IndexReader.open(dir, null, true,
         IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, codecs);
     IndexSearcher searcher = new IndexSearcher(reader);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentTermEnum.java?rev=1055405&r1=1055404&r2=1055405&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentTermEnum.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentTermEnum.java Wed Jan  5 10:52:04 2011
@@ -74,7 +74,7 @@ public class TestSegmentTermEnum extends
 
   public void testPrevTermAtEnd() throws IOException
   {
-    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setCodecProvider(_TestUtil.alwaysCodec("Standard")));
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setCodecProvider(_TestUtil.alwaysCodec("Standard")));
     addDoc(writer, "aaa bbb");
     writer.close();
     SegmentReader reader = getOnlySegmentReader(IndexReader.open(dir, false));
@@ -82,7 +82,14 @@ public class TestSegmentTermEnum extends
     assertNotNull(terms.next());
     assertEquals("aaa", terms.term().utf8ToString());
     assertNotNull(terms.next());
-    long ordB = terms.ord();
+    long ordB;
+    try {
+      ordB = terms.ord();
+    } catch (UnsupportedOperationException uoe) {
+      // ok -- codec is not required to support ord
+      reader.close();
+      return;
+    }
     assertEquals("bbb", terms.term().utf8ToString());
     assertNull(terms.next());