You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/30 18:56:43 UTC

svn commit: r1237850 [2/4] - in /lucene/dev/branches/lucene3661: ./ lucene/ lucene/contrib/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/ lucene/contrib/misc/ lucene/contrib/misc/src/java/ lucene/contrib/misc/src/java/or...

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FST.java Mon Jan 30 17:56:40 2012
@@ -25,6 +25,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -33,12 +35,23 @@ import org.apache.lucene.store.OutputStr
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.Builder.UnCompiledNode;
 
+// TODO: break this into WritableFST and ReadOnlyFST.. then
+// we can have subclasses of ReadOnlyFST to handle the
+// different byte[] level encodings (packed or
+// not)... and things like nodeCount, arcCount are read only
+
 // TODO: if FST is pure prefix trie we can do a more compact
 // job, ie, once we are at a 'suffix only', just store the
 // completion labels as a string not as a series of arcs.
 
+// TODO: maybe make an explicit thread state that holds
+// reusable stuff eg BytesReader, a scratch arc
+
 // NOTE: while the FST is able to represent a non-final
 // dead-end state (NON_FINAL_END_NODE=0), the layers above
 // (FSTEnum, Util) have problems with this!!
@@ -52,13 +65,15 @@ import org.apache.lucene.util.fst.Builde
  *
  * @lucene.experimental
  */
-public class FST<T> {
+public final class FST<T> {
   public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
   public final INPUT_TYPE inputType;
 
   private final static int BIT_FINAL_ARC = 1 << 0;
   private final static int BIT_LAST_ARC = 1 << 1;
-  private final static int BIT_TARGET_NEXT = 1 << 2;
+  final static int BIT_TARGET_NEXT = 1 << 2;
+
+  // TODO: we can free up a bit if we can nuke this:
   private final static int BIT_STOP_NODE = 1 << 3;
   private final static int BIT_ARC_HAS_OUTPUT = 1 << 4;
   private final static int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
@@ -66,7 +81,12 @@ public class FST<T> {
   // Arcs are stored as fixed-size (per entry) array, so
   // that we can find an arc using binary search.  We do
   // this when number of arcs is > NUM_ARCS_ARRAY:
-  private final static int BIT_ARCS_AS_FIXED_ARRAY = 1 << 6;
+
+  // If set, the target node is delta coded vs current
+  // position:
+  private final static int BIT_TARGET_DELTA = 1 << 6;
+
+  private final static byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
 
   /**
    * @see #shouldExpand(UnCompiledNode)
@@ -95,7 +115,10 @@ public class FST<T> {
   /** Write BYTE2 labels as 2-byte short, not vInt. */
   private final static int VERSION_SHORT_BYTE2_LABELS = 2;
 
-  private final static int VERSION_CURRENT = VERSION_SHORT_BYTE2_LABELS;
+  /** Added optional packed format. */
+  private final static int VERSION_PACKED = 3;
+
+  private final static int VERSION_CURRENT = VERSION_PACKED;
 
   // Never serialized; just used to represent the virtual
   // final node w/ no arcs:
@@ -126,6 +149,9 @@ public class FST<T> {
   public int arcCount;
   public int arcWithOutputCount;
 
+  private final boolean packed;
+  private final int[] nodeRefToAddress;
+
   // If arc has this label then that arc is final/accepted
   public static final int END_LABEL = -1;
 
@@ -137,10 +163,17 @@ public class FST<T> {
     public int label;
     public T output;
 
+    // From node (ord or address); currently only used when
+    // building an FST w/ willPackFST=true:
+    int node;
+
+    // To node (ord or address):
     public int target;
 
     byte flags;
     public T nextFinalOutput;
+
+    // address (into the byte[]), or ord/address if label == END_LABEL
     int nextArc;
 
     // This is non-zero if current arcs are fixed array:
@@ -151,19 +184,18 @@ public class FST<T> {
 
     /** Returns this */
     public Arc<T> copyFrom(Arc<T> other) {
+      node = other.node;
       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;
+      bytesPerArc = other.bytesPerArc;
+      if (bytesPerArc != 0) {
         posArcsStart = other.posArcsStart;
         arcIdx = other.arcIdx;
         numArcs = other.numArcs;
-      } else {
-        bytesPerArc = 0;
       }
       return this;
     }
@@ -179,40 +211,91 @@ public class FST<T> {
     public boolean isFinal() {
       return flag(BIT_FINAL_ARC);
     }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("node=" + node);
+      b.append(" target=" + target);
+      b.append(" label=" + label);
+      if (flag(BIT_LAST_ARC)) {
+        b.append(" last");
+      }
+      if (flag(BIT_FINAL_ARC)) {
+        b.append(" final");
+      }
+      if (flag(BIT_TARGET_NEXT)) {
+        b.append(" targetNext");
+      }
+      if (flag(BIT_ARC_HAS_OUTPUT)) {
+        b.append(" hasOutput");
+      }
+      if (flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+        b.append(" hasFinalOutput");
+      }
+      if (bytesPerArc != 0) {
+        b.append(" arcArray(idx=" + arcIdx + " of " + numArcs + ")");
+      }
+      return b.toString();
+    }
   };
 
-  static boolean flag(int flags, int bit) {
+  private final static boolean flag(int flags, int bit) {
     return (flags & bit) != 0;
   }
 
   private final BytesWriter writer;
 
-  // make a new empty FST, for building
-  public FST(INPUT_TYPE inputType, Outputs<T> outputs) {
+  // TODO: we can save RAM here by using growable packed
+  // ints...:
+  private int[] nodeAddress;
+
+  // TODO: we could be smarter here, and prune periodically
+  // as we go; high in-count nodes will "usually" become
+  // clear early on:
+  private int[] inCounts;
+
+  // make a new empty FST, for building; Builder invokes
+  // this ctor
+  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST) {
     this.inputType = inputType;
     this.outputs = outputs;
     bytes = new byte[128];
     NO_OUTPUT = outputs.getNoOutput();
+    if (willPackFST) {
+      nodeAddress = new int[8];
+      inCounts = new int[8];
+    } else {
+      nodeAddress = null;
+      inCounts = null;
+    }
     
     writer = new BytesWriter();
 
     emptyOutput = null;
+    packed = false;
+    nodeRefToAddress = null;
   }
 
-  // create an existing FST
+  /** Load a previously saved FST. */
   public FST(DataInput in, Outputs<T> outputs) throws IOException {
     this.outputs = outputs;
     writer = null;
     // NOTE: only reads most recent format; we don't have
     // back-compat promise for FSTs (they are experimental):
-    CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_SHORT_BYTE2_LABELS, VERSION_SHORT_BYTE2_LABELS);
+    CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_PACKED);
+    packed = in.readByte() == 1;
     if (in.readByte() == 1) {
       // accepts empty string
       int numBytes = in.readVInt();
       // messy
       bytes = new byte[numBytes];
       in.readBytes(bytes, 0, numBytes);
-      emptyOutput = outputs.read(getBytesReader(numBytes-1));
+      if (packed) {
+        emptyOutput = outputs.read(getBytesReader(0));
+      } else {
+        emptyOutput = outputs.read(getBytesReader(numBytes-1));
+      }
     } else {
       emptyOutput = null;
     }
@@ -230,6 +313,15 @@ public class FST<T> {
     default:
       throw new IllegalStateException("invalid input type " + t);
     }
+    if (packed) {
+      final int nodeRefCount = in.readVInt();
+      nodeRefToAddress = new int[nodeRefCount];
+      for(int idx=0;idx<nodeRefCount;idx++) {
+        nodeRefToAddress[idx] = in.readVInt();
+      }
+    } else {
+      nodeRefToAddress = null;
+    }
     startNode = in.readVInt();
     nodeCount = in.readVInt();
     arcCount = in.readVInt();
@@ -248,7 +340,14 @@ public class FST<T> {
 
   /** Returns bytes used to represent the FST */
   public int sizeInBytes() {
-    return bytes.length;
+    int size = bytes.length;
+    if (packed) {
+      size += nodeRefToAddress.length * RamUsageEstimator.NUM_BYTES_INT;
+    } else if (nodeAddress != null) {
+      size += nodeAddress.length * RamUsageEstimator.NUM_BYTES_INT;
+      size += inCounts.length * RamUsageEstimator.NUM_BYTES_INT;
+    }
+    return size;
   }
 
   void finish(int startNode) throws IOException {
@@ -266,15 +365,25 @@ public class FST<T> {
     cacheRootArcs();
   }
 
+  private int getNodeAddress(int node) {
+    if (nodeAddress != null) {
+      // Deref
+      return nodeAddress[node];
+    } else {
+      // Straight
+      return node;
+    }
+  }
+
   // Caches first 128 labels
   @SuppressWarnings("unchecked")
   private void cacheRootArcs() throws IOException {
-    cachedRootArcs = (FST.Arc<T>[]) new FST.Arc[0x80];
-    final FST.Arc<T> arc = new FST.Arc<T>();
+    cachedRootArcs = (Arc<T>[]) new Arc[0x80];
+    final Arc<T> arc = new Arc<T>();
     getFirstArc(arc);
     final BytesReader in = getBytesReader(0);
     if (targetHasArcs(arc)) {
-      readFirstRealArc(arc.target, arc, in);
+      readFirstRealTargetArc(arc.target, arc, in);
       while(true) {
         assert arc.label != END_LABEL;
         if (arc.label < cachedRootArcs.length) {
@@ -307,14 +416,16 @@ public class FST<T> {
     outputs.write(emptyOutput, writer);
     emptyOutputBytes = new byte[writer.posWrite-posSave];
 
-    // reverse
-    final int stopAt = (writer.posWrite - posSave)/2;
-    int upto = 0;
-    while(upto < stopAt) {
-      final byte b = bytes[posSave + upto];
-      bytes[posSave+upto] = bytes[writer.posWrite-upto-1];
-      bytes[writer.posWrite-upto-1] = b;
-      upto++;
+    if (!packed) {
+      // reverse
+      final int stopAt = (writer.posWrite - posSave)/2;
+      int upto = 0;
+      while(upto < stopAt) {
+        final byte b = bytes[posSave + upto];
+        bytes[posSave+upto] = bytes[writer.posWrite-upto-1];
+        bytes[writer.posWrite-upto-1] = b;
+        upto++;
+      }
     }
     System.arraycopy(bytes, posSave, emptyOutputBytes, 0, writer.posWrite-posSave);
     writer.posWrite = posSave;
@@ -324,7 +435,15 @@ public class FST<T> {
     if (startNode == -1) {
       throw new IllegalStateException("call finish first");
     }
+    if (nodeAddress != null) {
+      throw new IllegalStateException("cannot save an FST pre-packed FST; it must first be packed");
+    }
     CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT);
+    if (packed) {
+      out.writeByte((byte) 1);
+    } else {
+      out.writeByte((byte) 0);
+    }
     // TODO: really we should encode this as an arc, arriving
     // to the root node, instead of special casing here:
     if (emptyOutput != null) {
@@ -343,6 +462,13 @@ public class FST<T> {
       t = 2;
     }
     out.writeByte(t);
+    if (packed) {
+      assert nodeRefToAddress != null;
+      out.writeVInt(nodeRefToAddress.length);
+      for(int idx=0;idx<nodeRefToAddress.length;idx++) {
+        out.writeVInt(nodeRefToAddress[idx]);
+      }
+    }
     out.writeVInt(startNode);
     out.writeVInt(nodeCount);
     out.writeVInt(arcCount);
@@ -418,16 +544,16 @@ public class FST<T> {
 
   // returns true if the node at this address has any
   // outgoing arcs
-  public boolean targetHasArcs(Arc<T> arc) {
+  public static<T> boolean targetHasArcs(Arc<T> arc) {
     return arc.target > 0;
   }
 
   // serializes new node by appending its bytes to the end
   // of the current byte[]
-  int addNode(Builder.UnCompiledNode<T> node) throws IOException {
-    //System.out.println("FST.addNode pos=" + posWrite + " numArcs=" + node.numArcs);
-    if (node.numArcs == 0) {
-      if (node.isFinal) {
+  int addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+    //System.out.println("FST.addNode pos=" + writer.posWrite + " numArcs=" + nodeIn.numArcs);
+    if (nodeIn.numArcs == 0) {
+      if (nodeIn.isFinal) {
         return FINAL_END_NODE;
       } else {
         return NON_FINAL_END_NODE;
@@ -437,15 +563,15 @@ public class FST<T> {
     int startAddress = writer.posWrite;
     //System.out.println("  startAddr=" + startAddress);
 
-    final boolean doFixedArray = shouldExpand(node);
+    final boolean doFixedArray = shouldExpand(nodeIn);
     final int fixedArrayStart;
     if (doFixedArray) {
-      if (bytesPerArc.length < node.numArcs) {
-        bytesPerArc = new int[ArrayUtil.oversize(node.numArcs, 1)];
+      if (bytesPerArc.length < nodeIn.numArcs) {
+        bytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, 1)];
       }
       // write a "false" first arc:
-      writer.writeByte((byte) BIT_ARCS_AS_FIXED_ARRAY);
-      writer.writeVInt(node.numArcs);
+      writer.writeByte(ARCS_AS_FIXED_ARRAY);
+      writer.writeVInt(nodeIn.numArcs);
       // placeholder -- we'll come back and write the number
       // of bytes per arc (int) here:
       // TODO: we could make this a vInt instead
@@ -456,15 +582,14 @@ public class FST<T> {
       fixedArrayStart = 0;
     }
 
-    nodeCount++;
-    arcCount += node.numArcs;
+    arcCount += nodeIn.numArcs;
     
-    final int lastArc = node.numArcs-1;
+    final int lastArc = nodeIn.numArcs-1;
 
     int lastArcStart = writer.posWrite;
     int maxBytesPerArc = 0;
-    for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-      final Builder.Arc<T> arc = node.arcs[arcIdx];
+    for(int arcIdx=0;arcIdx<nodeIn.numArcs;arcIdx++) {
+      final Builder.Arc<T> arc = nodeIn.arcs[arcIdx];
       final Builder.CompiledNode target = (Builder.CompiledNode) arc.target;
       int flags = 0;
 
@@ -472,7 +597,10 @@ public class FST<T> {
         flags += BIT_LAST_ARC;
       }
 
-      if (lastFrozenNode == target.address && !doFixedArray) {
+      if (lastFrozenNode == target.node && !doFixedArray) {
+        // TODO: for better perf (but more RAM used) we
+        // could avoid this except when arc is "near" the
+        // last arc:
         flags += BIT_TARGET_NEXT;
       }
 
@@ -485,10 +613,12 @@ public class FST<T> {
         assert arc.nextFinalOutput == NO_OUTPUT;
       }
 
-      boolean targetHasArcs = target.address > 0;
+      boolean targetHasArcs = target.node > 0;
 
       if (!targetHasArcs) {
         flags += BIT_STOP_NODE;
+      } else if (inCounts != null) {
+        inCounts[target.node]++;
       }
 
       if (arc.output != NO_OUTPUT) {
@@ -498,19 +628,23 @@ public class FST<T> {
       writer.writeByte((byte) flags);
       writeLabel(arc.label);
 
-      //System.out.println("  write arc: label=" + arc.label + " flags=" + flags);
+      // System.out.println("  write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + writer.posWrite + " output=" + outputs.outputToString(arc.output));
 
       if (arc.output != NO_OUTPUT) {
         outputs.write(arc.output, writer);
+        //System.out.println("    write output");
         arcWithOutputCount++;
       }
+
       if (arc.nextFinalOutput != NO_OUTPUT) {
+        //System.out.println("    write final output");
         outputs.write(arc.nextFinalOutput, writer);
       }
 
-      if (targetHasArcs && (doFixedArray || lastFrozenNode != target.address)) {
-        assert target.address > 0;
-        writer.writeInt(target.address);
+      if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) {
+        assert target.node > 0;
+        //System.out.println("    write target");
+        writer.writeInt(target.node);
       }
 
       // just write the arcs "like normal" on first pass,
@@ -530,10 +664,11 @@ public class FST<T> {
     // such cases
 
     if (doFixedArray) {
+      //System.out.println("  doFixedArray");
       assert maxBytesPerArc > 0;
       // 2nd pass just "expands" all arcs to take up a fixed
       // byte size
-      final int sizeNeeded = fixedArrayStart + node.numArcs * maxBytesPerArc;
+      final int sizeNeeded = fixedArrayStart + nodeIn.numArcs * maxBytesPerArc;
       bytes = ArrayUtil.grow(bytes, sizeNeeded);
       // TODO: we could make this a vInt instead
       bytes[fixedArrayStart-4] = (byte) (maxBytesPerArc >> 24);
@@ -543,9 +678,9 @@ public class FST<T> {
 
       // expand the arcs in place, backwards
       int srcPos = writer.posWrite;
-      int destPos = fixedArrayStart + node.numArcs*maxBytesPerArc;
+      int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
       writer.posWrite = destPos;
-      for(int arcIdx=node.numArcs-1;arcIdx>=0;arcIdx--) {
+      for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
         //System.out.println("  repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
         destPos -= maxBytesPerArc;
         srcPos -= bytesPerArc[arcIdx];
@@ -559,7 +694,7 @@ public class FST<T> {
     // reverse bytes in-place; we do this so that the
     // "BIT_TARGET_NEXT" opto can work, ie, it reads the
     // node just before the current one
-    final int endAddress = lastFrozenNode = writer.posWrite - 1;
+    final int endAddress = writer.posWrite - 1;
 
     int left = startAddress;
     int right = endAddress;
@@ -568,13 +703,31 @@ public class FST<T> {
       bytes[left++] = bytes[right];
       bytes[right--] = b;
     }
+    //System.out.println("  endAddress=" + endAddress);
 
-    return endAddress;
+    nodeCount++;
+    final int node;
+    if (nodeAddress != null) {
+      // Nodes are addressed by 1+ord:
+      if (nodeCount == nodeAddress.length) {
+        nodeAddress = ArrayUtil.grow(nodeAddress);
+        inCounts = ArrayUtil.grow(inCounts);
+      }
+      nodeAddress[nodeCount] = endAddress;
+      // System.out.println("  write nodeAddress[" + nodeCount + "] = " + endAddress);
+      node = nodeCount;
+    } else {
+      node = endAddress;
+    }
+    lastFrozenNode = node;
+
+    return node;
   }
 
   /** 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;
@@ -585,7 +738,7 @@ public class FST<T> {
     arc.output = NO_OUTPUT;
 
     // If there are no nodes, ie, the FST only accepts the
-    // empty string, then startNode is 0, and then readFirstTargetArc
+    // empty string, then startNode is 0
     arc.target = startNode;
     return arc;
   }
@@ -602,20 +755,27 @@ public class FST<T> {
       //System.out.println("  end node");
       assert follow.isFinal();
       arc.label = END_LABEL;
+      arc.target = FINAL_END_NODE;
       arc.output = follow.nextFinalOutput;
       arc.flags = BIT_LAST_ARC;
       return arc;
     } else {
-      final BytesReader in = getBytesReader(follow.target);
-      arc.flags = in.readByte();
-      if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+      final BytesReader in = getBytesReader(getNodeAddress(follow.target));
+      arc.node = follow.target;
+      final byte b = in.readByte();
+      if (b == ARCS_AS_FIXED_ARRAY) {
         // array: jump straight to end
         arc.numArcs = in.readVInt();
-        arc.bytesPerArc = in.readInt();
+        if (packed) {
+          arc.bytesPerArc = in.readVInt();
+        } else {
+          arc.bytesPerArc = in.readInt();
+        }
         //System.out.println("  array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
         arc.posArcsStart = in.pos;
         arc.arcIdx = arc.numArcs - 2;
       } else {
+        arc.flags = b;
         // non-array: linear scan
         arc.bytesPerArc = 0;
         //System.out.println("  scan");
@@ -631,11 +791,17 @@ public class FST<T> {
           if (arc.flag(BIT_STOP_NODE)) {
           } else if (arc.flag(BIT_TARGET_NEXT)) {
           } else {
-            in.pos -= 4;
+            if (packed) {
+              in.readVInt();
+            } else {
+              in.skip(4);
+            }
           }
           arc.flags = in.readByte();
         }
-        arc.nextArc = in.pos+1;
+        // Undo the byte flags we read: 
+        in.skip(-1);
+        arc.nextArc = in.pos;
       }
       readNextRealArc(arc, in);
       assert arc.isLast();
@@ -657,35 +823,48 @@ public class FST<T> {
       // Insert "fake" final first arc:
       arc.label = END_LABEL;
       arc.output = follow.nextFinalOutput;
+      arc.flags = BIT_FINAL_ARC;
       if (follow.target <= 0) {
-        arc.flags = BIT_LAST_ARC | BIT_FINAL_ARC;
+        arc.flags |= BIT_LAST_ARC;
       } else {
-        arc.flags = BIT_FINAL_ARC;
+        arc.node = follow.target;
+        // NOTE: nextArc is a node (not an address!) in this case:
         arc.nextArc = follow.target;
       }
+      arc.target = FINAL_END_NODE;
       //System.out.println("    insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
       return arc;
     } else {
-      return readFirstRealArc(follow.target, arc, getBytesReader(0));
+      return readFirstRealTargetArc(follow.target, arc, getBytesReader(0));
     }
   }
 
-  public Arc<T> readFirstRealArc(int address, Arc<T> arc, final BytesReader in) throws IOException {
+  public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
+    final int address = getNodeAddress(node);
     in.pos = address;
-    arc.flags = in.readByte();
+    //System.out.println("  readFirstRealTargtArc address="
+    //+ address);
+    //System.out.println("   flags=" + arc.flags);
+    arc.node = node;
 
-    if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+    if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
       //System.out.println("  fixedArray");
       // this is first arc in a fixed-array
       arc.numArcs = in.readVInt();
-      arc.bytesPerArc = in.readInt();
+      if (packed) {
+        arc.bytesPerArc = in.readVInt();
+      } else {
+        arc.bytesPerArc = in.readInt();
+      }
       arc.arcIdx = -1;
       arc.nextArc = arc.posArcsStart = in.pos;
       //System.out.println("  bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
     } else {
+      //arc.flags = b;
       arc.nextArc = address;
       arc.bytesPerArc = 0;
     }
+
     return readNextRealArc(arc, in);
   }
 
@@ -699,9 +878,8 @@ public class FST<T> {
     if (!targetHasArcs(follow)) {
       return false;
     } else {
-      final BytesReader in = getBytesReader(follow.target);
-      final byte b = in.readByte();
-      return (b & BIT_ARCS_AS_FIXED_ARRAY) != 0;
+      final BytesReader in = getBytesReader(getNodeAddress(follow.target));
+      return in.readByte() == ARCS_AS_FIXED_ARRAY;
     }
   }
 
@@ -710,10 +888,9 @@ public class FST<T> {
     if (arc.label == END_LABEL) {
       // 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;
+        throw new IllegalArgumentException("cannot readNextArc when arc.isLast()=true");
       }
-      return readFirstRealArc(arc.nextArc, arc, getBytesReader(0));
+      return readFirstRealTargetArc(arc.nextArc, arc, getBytesReader(0));
     } else {
       return readNextRealArc(arc, getBytesReader(0));
     }
@@ -727,19 +904,24 @@ public class FST<T> {
     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)) {
+      in = getBytesReader(getNodeAddress(arc.nextArc));
+      final byte b = bytes[in.pos];
+      if (b == ARCS_AS_FIXED_ARRAY) {
         //System.out.println("    nextArc fake array");
-        in.pos--;
+        in.skip(1);
         in.readVInt();
-        in.readInt();
+        if (packed) {
+          in.readVInt();
+        } else {
+          in.readInt();
+        }
       }
     } 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);
+        in = getBytesReader(arc.posArcsStart);
+        in.skip((1+arc.arcIdx)*arc.bytesPerArc);
       } else {
         // arcs are packed
         //System.out.println("    nextArc real packed");
@@ -754,12 +936,16 @@ public class FST<T> {
   /** 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 {
+
+    // TODO: can't assert this because we call from readFirstArc
+    // assert !flag(arc.flags, BIT_LAST_ARC);
+
     // this is a continuing arc in a fixed array
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
       arc.arcIdx++;
       assert arc.arcIdx < arc.numArcs;
-      in.pos = arc.posArcsStart - arc.arcIdx*arc.bytesPerArc;
+      in.skip(arc.posArcsStart, arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed
       in.pos = arc.nextArc;
@@ -788,45 +974,61 @@ public class FST<T> {
       arc.nextArc = in.pos;
     } else if (arc.flag(BIT_TARGET_NEXT)) {
       arc.nextArc = in.pos;
-      if (!arc.flag(BIT_LAST_ARC)) {
-        if (arc.bytesPerArc == 0) {
-          // must scan
-          seekToNextNode(in);
-        } else {
-          in.pos = arc.posArcsStart - arc.bytesPerArc * arc.numArcs;
+      // TODO: would be nice to make this lazy -- maybe
+      // caller doesn't need the target and is scanning arcs...
+      if (nodeAddress == null) {
+        if (!arc.flag(BIT_LAST_ARC)) {
+          if (arc.bytesPerArc == 0) {
+            // must scan
+            seekToNextNode(in);
+          } else {
+            in.skip(arc.posArcsStart, arc.bytesPerArc * arc.numArcs);
+          }
         }
+        arc.target = in.pos;
+      } else {
+        arc.target = arc.node - 1;
+        assert arc.target > 0;
       }
-      arc.target = in.pos;
     } else {
-      arc.target = in.readInt();
+      if (packed) {
+        final int pos = in.pos;
+        final int code = in.readVInt();
+        if (arc.flag(BIT_TARGET_DELTA)) {
+          // Address is delta-coded from current address:
+          arc.target = pos + code;
+          //System.out.println("    delta pos=" + pos + " delta=" + code + " target=" + arc.target);
+        } else if (code < nodeRefToAddress.length) {
+          // Deref
+          arc.target = nodeRefToAddress[code];
+          //System.out.println("    deref code=" + code + " target=" + arc.target);
+        } else {
+          // Absolute
+          arc.target = code;
+          //System.out.println("    abs code=" + code + " derefLen=" + nodeRefToAddress.length);
+        }
+      } else {
+        arc.target = in.readInt();
+      }
       arc.nextArc = in.pos;
     }
-
     return arc;
   }
 
   /** 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 {
+  public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
     assert cachedRootArcs != null;
-    // Short-circuit if this arc is in the root arc cache:
-    if (follow.target == startNode && labelToMatch != END_LABEL && labelToMatch < cachedRootArcs.length) {
-      final Arc<T> result = cachedRootArcs[labelToMatch];
-      if (result == null) {
-        return result;
-      } else {
-        arc.copyFrom(result);
-        return arc;
-      }
-    }
- 
+
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
         if (follow.target <= 0) {
           arc.flags = BIT_LAST_ARC;
         } else {
           arc.flags = 0;
+          // NOTE: nextArc is a node (not an address!) in this case:
           arc.nextArc = follow.target;
+          arc.node = follow.target;
         }
         arc.output = follow.nextFinalOutput;
         arc.label = END_LABEL;
@@ -836,35 +1038,49 @@ public class FST<T> {
       }
     }
 
+    // Short-circuit if this arc is in the root arc cache:
+    if (follow.target == startNode && labelToMatch < cachedRootArcs.length) {
+      final Arc<T> result = cachedRootArcs[labelToMatch];
+      if (result == null) {
+        return result;
+      } else {
+        arc.copyFrom(result);
+        return arc;
+      }
+    }
+
     if (!targetHasArcs(follow)) {
       return null;
     }
 
-    // TODO: maybe make an explicit thread state that holds
-    // reusable stuff eg BytesReader:
-    final BytesReader in = getBytesReader(follow.target);
+    in.pos = getNodeAddress(follow.target);
+
+    arc.node = follow.target;
 
     // System.out.println("fta label=" + (char) labelToMatch);
 
-    if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) {
+    if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
       // Arcs are full array; do binary search:
       arc.numArcs = in.readVInt();
-      //System.out.println("  bs " + arc.numArcs);
-      arc.bytesPerArc = in.readInt();
+      if (packed) {
+        arc.bytesPerArc = in.readVInt();
+      } else {
+        arc.bytesPerArc = in.readInt();
+      }
       arc.posArcsStart = in.pos;
       int low = 0;
       int high = arc.numArcs-1;
       while (low <= high) {
         //System.out.println("    cycle");
         int mid = (low + high) >>> 1;
-        in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+        in.skip(arc.posArcsStart, arc.bytesPerArc*mid + 1);
         int midLabel = readLabel(in);
         final int cmp = midLabel - labelToMatch;
-        if (cmp < 0)
+        if (cmp < 0) {
           low = mid + 1;
-        else if (cmp > 0)
+        } else if (cmp > 0) {
           high = mid - 1;
-        else {
+        } else {
           arc.arcIdx = mid-1;
           //System.out.println("    found!");
           return readNextRealArc(arc, in);
@@ -875,7 +1091,8 @@ public class FST<T> {
     }
 
     // Linear scan
-    readFirstTargetArc(follow, arc);
+    readFirstRealTargetArc(follow.target, arc, in);
+
     while(true) {
       //System.out.println("  non-bs cycle");
       // TODO: we should fix this code to not have to create
@@ -889,7 +1106,7 @@ public class FST<T> {
       } else if (arc.isLast()) {
         return null;
       } else {
-        readNextArc(arc);
+        readNextRealArc(arc, in);
       }
     }
   }
@@ -910,7 +1127,11 @@ public class FST<T> {
       }
 
       if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
-        in.readInt();
+        if (packed) {
+          in.readVInt();
+        } else {
+          in.readInt();
+        }
       }
 
       if (flag(flags, BIT_LAST_ARC)) {
@@ -969,6 +1190,7 @@ public class FST<T> {
 
     @Override
     public void writeByte(byte b) {
+      assert posWrite <= bytes.length;
       if (bytes.length == posWrite) {
         bytes = ArrayUtil.grow(bytes);
       }
@@ -976,6 +1198,13 @@ public class FST<T> {
       bytes[posWrite++] = b;
     }
 
+    public void setPosWrite(int posWrite) {
+      this.posWrite = posWrite;
+      if (bytes.length < posWrite) {
+        bytes = ArrayUtil.grow(bytes, posWrite);
+      }
+    }
+
     @Override
     public void writeBytes(byte[] b, int offset, int length) {
       final int size = posWrite + length;
@@ -987,15 +1216,24 @@ public class FST<T> {
 
   public final BytesReader getBytesReader(int pos) {
     // TODO: maybe re-use via ThreadLocal?
-    return new BytesReader(bytes, pos);
+    if (packed) {
+      return new ForwardBytesReader(bytes, pos);
+    } else {
+      return new ReverseBytesReader(bytes, pos);
+    }
   }
 
   /** Expert */
-  public final static class BytesReader extends DataInput {
-    final byte[] bytes;
+  public static abstract class BytesReader extends DataInput {
     int pos;
+    abstract void skip(int byteCount);
+    abstract void skip(int base, int byteCount);
+  }
+
+  final static class ReverseBytesReader extends BytesReader {
+    final byte[] bytes;
 
-    public BytesReader(byte[] bytes, int pos) {
+    public ReverseBytesReader(byte[] bytes, int pos) {
       this.bytes = bytes;
       this.pos = pos;
     }
@@ -1011,5 +1249,549 @@ public class FST<T> {
         b[offset+i] = bytes[pos--];
       }
     }
+
+    public void skip(int count) {
+      pos -= count;
+    }
+
+    public void skip(int base, int count) {
+      pos = base - count;
+    }
+  }
+
+  // TODO: can we use just ByteArrayDataInput...?  need to
+  // add a .skipBytes to DataInput.. hmm and .setPosition
+  final static class ForwardBytesReader extends BytesReader {
+    final byte[] bytes;
+
+    public ForwardBytesReader(byte[] bytes, int pos) {
+      this.bytes = bytes;
+      this.pos = pos;
+    }
+
+    @Override
+    public byte readByte() {
+      return bytes[pos++];
+    }
+
+    @Override
+    public void readBytes(byte[] b, int offset, int len) {
+      System.arraycopy(bytes, pos, b, offset, len);
+      pos += len;
+    }
+
+    public void skip(int count) {
+      pos += count;
+    }
+
+    public void skip(int base, int count) {
+      pos = base + count;
+    }
+  }
+
+  private static class ArcAndState<T> {
+    final Arc<T> arc;
+    final IntsRef chain;
+
+    public ArcAndState(Arc<T> arc, IntsRef chain) {
+      this.arc = arc;
+      this.chain = chain;
+    }
+  }
+
+  /*
+  public void countSingleChains() throws IOException {
+    // TODO: must assert this FST was built with
+    // "willRewrite"
+
+    final List<ArcAndState<T>> queue = new ArrayList<ArcAndState<T>>();
+
+    // TODO: use bitset to not revisit nodes already
+    // visited
+
+    FixedBitSet seen = new FixedBitSet(1+nodeCount);
+    int saved = 0;
+
+    queue.add(new ArcAndState<T>(getFirstArc(new Arc<T>()), new IntsRef()));
+    Arc<T> scratchArc = new Arc<T>();
+    while(queue.size() > 0) {
+      //System.out.println("cycle size=" + queue.size());
+      //for(ArcAndState<T> ent : queue) {
+      //  System.out.println("  " + Util.toBytesRef(ent.chain, new BytesRef()));
+      //  }
+      final ArcAndState<T> arcAndState = queue.get(queue.size()-1);
+      seen.set(arcAndState.arc.node);
+      final BytesRef br = Util.toBytesRef(arcAndState.chain, new BytesRef());
+      if (br.length > 0 && br.bytes[br.length-1] == -1) {
+        br.length--;
+      }
+      //System.out.println("  top node=" + arcAndState.arc.target + " chain=" + br.utf8ToString());
+      if (targetHasArcs(arcAndState.arc) && !seen.get(arcAndState.arc.target)) {
+        // push
+        readFirstTargetArc(arcAndState.arc, scratchArc);
+        //System.out.println("  push label=" + (char) scratchArc.label);
+        //System.out.println("    tonode=" + scratchArc.target + " last?=" + scratchArc.isLast());
+        
+        final IntsRef chain = IntsRef.deepCopyOf(arcAndState.chain);
+        chain.grow(1+chain.length);
+        // TODO
+        //assert scratchArc.label != END_LABEL;
+        chain.ints[chain.length] = scratchArc.label;
+        chain.length++;
+
+        if (scratchArc.isLast()) {
+          if (scratchArc.target != -1 && inCounts[scratchArc.target] == 1) {
+            //System.out.println("    append");
+          } else {
+            if (arcAndState.chain.length > 1) {
+              saved += chain.length-2;
+              try {
+                System.out.println("chain: " + Util.toBytesRef(chain, new BytesRef()).utf8ToString());
+              } catch (AssertionError ae) {
+                System.out.println("chain: " + Util.toBytesRef(chain, new BytesRef()));
+              }
+            }
+            chain.length = 0;
+          }
+        } else {
+          //System.out.println("    reset");
+          if (arcAndState.chain.length > 1) {
+            saved += arcAndState.chain.length-2;
+            try {
+              System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()).utf8ToString());
+            } catch (AssertionError ae) {
+              System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()));
+            }
+          }
+          if (scratchArc.target != -1 && inCounts[scratchArc.target] != 1) {
+            chain.length = 0;
+          } else {
+            chain.ints[0] = scratchArc.label;
+            chain.length = 1;
+          }
+        }
+        // TODO: instead of new Arc() we can re-use from
+        // a by-depth array
+        queue.add(new ArcAndState<T>(new Arc<T>().copyFrom(scratchArc), chain));
+      } else if (!arcAndState.arc.isLast()) {
+        // next
+        readNextArc(arcAndState.arc);
+        //System.out.println("  next label=" + (char) arcAndState.arc.label + " len=" + arcAndState.chain.length);
+        if (arcAndState.chain.length != 0) {
+          arcAndState.chain.ints[arcAndState.chain.length-1] = arcAndState.arc.label;
+        }
+      } else {
+        if (arcAndState.chain.length > 1) {
+          saved += arcAndState.chain.length-2;
+          System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()).utf8ToString());
+        }
+        // pop
+        //System.out.println("  pop");
+        queue.remove(queue.size()-1);
+        while(queue.size() > 0 && queue.get(queue.size()-1).arc.isLast()) {
+          queue.remove(queue.size()-1);
+        }
+        if (queue.size() > 0) {
+          final ArcAndState<T> arcAndState2 = queue.get(queue.size()-1);
+          readNextArc(arcAndState2.arc);
+          //System.out.println("  read next=" + (char) arcAndState2.arc.label + " queue=" + queue.size());
+          assert arcAndState2.arc.label != END_LABEL;
+          if (arcAndState2.chain.length != 0) {
+            arcAndState2.chain.ints[arcAndState2.chain.length-1] = arcAndState2.arc.label;
+          }
+        }
+      }
+    }
+
+    System.out.println("TOT saved " + saved);
+  }
+ */
+
+  // Creates a packed FST
+  private FST(INPUT_TYPE inputType, int[] nodeRefToAddress, Outputs<T> outputs) {
+    packed = true;
+    this.inputType = inputType;
+    bytes = new byte[128];
+    this.nodeRefToAddress = nodeRefToAddress;
+    this.outputs = outputs;
+    NO_OUTPUT = outputs.getNoOutput();
+    writer = new BytesWriter();
+  }
+
+  /** Expert: creates an FST by packing this one.  This
+   *  process requires substantial additional RAM (currently
+   *  ~8 bytes per node), but then should produce a smaller FST. */
+  public FST<T> pack(int minInCountDeref, int maxDerefNodes) throws IOException {
+
+    // TODO: other things to try
+    //   - renumber the nodes to get more next / better locality?
+    //   - allow multiple input labels on an arc, so
+    //     singular chain of inputs can take one arc (on
+    //     wikipedia terms this could save another ~6%)
+    //   - in the ord case, the output '1' is presumably
+    //     very common (after NO_OUTPUT)... maybe use a bit
+    //     for it..?
+    //   - use spare bits in flags.... for top few labels /
+    //     outputs / targets
+
+    if (nodeAddress == null) {
+      throw new IllegalArgumentException("this FST was not built with willPackFST=true");
+    }
+
+    Arc<T> arc = new Arc<T>();
+
+    final BytesReader r = getBytesReader(0);
+
+    final int topN = Math.min(maxDerefNodes, inCounts.length);
+
+    // Find top nodes with highest number of incoming arcs:
+    NodeQueue q = new NodeQueue(topN);
+
+    // TODO: we could use more RAM efficient selection algo here...
+    NodeAndInCount bottom = null;
+    for(int node=0;node<inCounts.length;node++) {
+      if (inCounts[node] >= minInCountDeref) {
+        if (bottom == null) {
+          q.add(new NodeAndInCount(node, inCounts[node]));
+          if (q.size() == topN) {
+            bottom = q.top();
+          }
+        } else if (inCounts[node] > bottom.count) {
+          q.insertWithOverflow(new NodeAndInCount(node, inCounts[node]));
+        }
+      }
+    }
+
+    // Free up RAM:
+    inCounts = null;
+
+    final Map<Integer,Integer> topNodeMap = new HashMap<Integer,Integer>();
+    for(int downTo=q.size()-1;downTo>=0;downTo--) {
+      NodeAndInCount n = q.pop();
+      topNodeMap.put(n.node, downTo);
+      //System.out.println("map node=" + n.node + " inCount=" + n.count + " to newID=" + downTo);
+    }
+
+    // TODO: we can use packed ints:
+    // +1 because node ords start at 1 (0 is reserved as
+    // stop node):
+    final int[] nodeRefToAddressIn = new int[topNodeMap.size()];
+
+    final FST<T> fst = new FST<T>(inputType, nodeRefToAddressIn, outputs);
+
+    final BytesWriter writer = fst.writer;
+    
+    final int[] newNodeAddress = new int[1+nodeCount];
+
+    // Fill initial coarse guess:
+    for(int node=1;node<=nodeCount;node++) {
+      newNodeAddress[node] = 1 + bytes.length - nodeAddress[node];
+    }
+
+    int absCount;
+    int deltaCount;
+    int topCount;
+    int nextCount;
+
+    // Iterate until we converge:
+    while(true) {
+
+      //System.out.println("\nITER");
+      boolean changed = false;
+
+      // for assert:
+      boolean negDelta = false;
+
+      writer.posWrite = 0;
+      // Skip 0 byte since 0 is reserved target:
+      writer.writeByte((byte) 0);
+
+      fst.arcWithOutputCount = 0;
+      fst.nodeCount = 0;
+      fst.arcCount = 0;
+
+      absCount = deltaCount = topCount = nextCount = 0;
+
+      int changedCount = 0;
+
+      int addressError = 0;
+
+      //int totWasted = 0;
+
+      // Since we re-reverse the bytes, we now write the
+      // nodes backwards, so that BIT_TARGET_NEXT is
+      // unchanged:
+      for(int node=nodeCount;node>=1;node--) {
+        fst.nodeCount++;
+        final int address = writer.posWrite;
+        //System.out.println("  node: " + node + " address=" + address);
+        if (address != newNodeAddress[node]) {
+          addressError = address - newNodeAddress[node];
+          //System.out.println("    change: " + (address - newNodeAddress[node]));
+          changed = true;
+          newNodeAddress[node] = address;
+          changedCount++;
+        }
+
+        int nodeArcCount = 0;
+        int bytesPerArc = 0;
+
+        boolean retry = false;
+
+        // for assert:
+        boolean anyNegDelta = false;
+
+        // Retry loop: possibly iterate more than once, if
+        // this is an array'd node and bytesPerArc changes:
+        writeNode:
+        while(true) { // retry writing this node
+
+          readFirstRealTargetArc(node, arc, r);
+
+          final boolean useArcArray = arc.bytesPerArc != 0;
+          if (useArcArray) {
+            // Write false first arc:
+            if (bytesPerArc == 0) {
+              bytesPerArc = arc.bytesPerArc;
+            }
+            writer.writeByte(ARCS_AS_FIXED_ARRAY);
+            writer.writeVInt(arc.numArcs);
+            writer.writeVInt(bytesPerArc);
+            //System.out.println("node " + node + ": " + arc.numArcs + " arcs");
+          }
+
+          int maxBytesPerArc = 0;
+          //int wasted = 0;
+          while(true) {  // iterate over all arcs for this node
+
+            //System.out.println("    arc label=" + arc.label + " target=" + arc.target + " pos=" + writer.posWrite);
+            final int arcStartPos = writer.posWrite;
+            nodeArcCount++;
+
+            byte flags = 0;
+
+            if (arc.isLast()) {
+              flags += BIT_LAST_ARC;
+            }
+            /*
+            if (!useArcArray && nodeUpto < nodes.length-1 && arc.target == nodes[nodeUpto+1]) {
+              flags += BIT_TARGET_NEXT;
+            }
+            */
+            if (!useArcArray && node != 1 && arc.target == node-1) {
+              flags += BIT_TARGET_NEXT;
+              if (!retry) {
+                nextCount++;
+              }
+            }
+            if (arc.isFinal()) {
+              flags += BIT_FINAL_ARC;
+              if (arc.nextFinalOutput != NO_OUTPUT) {
+                flags += BIT_ARC_HAS_FINAL_OUTPUT;
+              }
+            } else {
+              assert arc.nextFinalOutput == NO_OUTPUT;
+            }
+            if (!targetHasArcs(arc)) {
+              flags += BIT_STOP_NODE;
+            }
+
+            if (arc.output != NO_OUTPUT) {
+              flags += BIT_ARC_HAS_OUTPUT;
+            }
+
+            final Integer ptr;
+            final int absPtr;
+            final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
+            if (doWriteTarget) {
+
+              ptr = topNodeMap.get(arc.target);
+              if (ptr != null) {
+                absPtr = ptr;
+              } else {
+                absPtr = topNodeMap.size() + newNodeAddress[arc.target] + addressError;
+              }
+
+              int delta = newNodeAddress[arc.target] + addressError - writer.posWrite - 2;
+              if (delta < 0) {
+                //System.out.println("neg: " + delta);
+                anyNegDelta = true;
+                delta = 0;
+              }
+
+              if (delta < absPtr) {
+                flags |= BIT_TARGET_DELTA;
+              }
+            } else {
+              ptr = null;
+              absPtr = 0;
+            }
+
+            writer.writeByte(flags);
+            fst.writeLabel(arc.label);
+
+            if (arc.output != NO_OUTPUT) {
+              outputs.write(arc.output, writer);
+              if (!retry) {
+                fst.arcWithOutputCount++;
+              }
+            }
+            if (arc.nextFinalOutput != NO_OUTPUT) {
+              outputs.write(arc.nextFinalOutput, writer);
+            }
+
+            if (doWriteTarget) {
+
+              int delta = newNodeAddress[arc.target] + addressError - writer.posWrite;
+              if (delta < 0) {
+                anyNegDelta = true;
+                //System.out.println("neg: " + delta);
+                delta = 0;
+              }
+
+              if (flag(flags, BIT_TARGET_DELTA)) {
+                //System.out.println("        delta");
+                writer.writeVInt(delta);
+                if (!retry) {
+                  deltaCount++;
+                }
+              } else {
+                /*
+                if (ptr != null) {
+                  System.out.println("        deref");
+                } else {
+                  System.out.println("        abs");
+                }
+                */
+                writer.writeVInt(absPtr);
+                if (!retry) {
+                  if (absPtr >= topNodeMap.size()) {
+                    absCount++;
+                  } else {
+                    topCount++;
+                  }
+                }
+              }
+            }
+
+            if (useArcArray) {
+              final int arcBytes = writer.posWrite - arcStartPos;
+              //System.out.println("  " + arcBytes + " bytes");
+              maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
+              // NOTE: this may in fact go "backwards", if
+              // somehow (rarely, possibly never) we use
+              // more bytesPerArc in this rewrite than the
+              // incoming FST did... but in this case we
+              // will retry (below) so it's OK to ovewrite
+              // bytes:
+              //wasted += bytesPerArc - arcBytes;
+              writer.setPosWrite(arcStartPos + bytesPerArc);
+            }
+
+            if (arc.isLast()) {
+              break;
+            }
+
+            readNextRealArc(arc, r);
+          }
+
+          if (useArcArray) {
+            if (maxBytesPerArc == bytesPerArc || (retry && maxBytesPerArc <= bytesPerArc)) {
+              // converged
+              //System.out.println("  bba=" + bytesPerArc + " wasted=" + wasted);
+              //totWasted += wasted;
+              break;
+            }
+          } else {
+            break;
+          }
+
+          //System.out.println("  retry this node maxBytesPerArc=" + maxBytesPerArc + " vs " + bytesPerArc);
+
+          // Retry:
+          bytesPerArc = maxBytesPerArc;
+          writer.posWrite = address;
+          nodeArcCount = 0;
+          retry = true;
+          anyNegDelta = false;
+        }
+        negDelta |= anyNegDelta;
+
+        fst.arcCount += nodeArcCount;
+      }
+
+      if (!changed) {
+        // We don't renumber the nodes (just reverse their
+        // order) so nodes should only point forward to
+        // other nodes because we only produce acyclic FSTs
+        // w/ nodes only pointing "forwards":
+        assert !negDelta;
+        //System.out.println("TOT wasted=" + totWasted);
+        // Converged!
+        break;
+      }
+      //System.out.println("  " + changedCount + " of " + fst.nodeCount + " changed; retry");
+    }
+
+    for(Map.Entry<Integer,Integer> ent : topNodeMap.entrySet()) {
+      nodeRefToAddressIn[ent.getValue()] = newNodeAddress[ent.getKey()];
+    }
+
+    fst.startNode = newNodeAddress[startNode];
+    //System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
+
+    if (emptyOutput != null) {
+      fst.setEmptyOutput(emptyOutput);
+    }
+
+    assert fst.nodeCount == nodeCount: "fst.nodeCount=" + fst.nodeCount + " nodeCount=" + nodeCount;
+    assert fst.arcCount == arcCount;
+    assert fst.arcWithOutputCount == arcWithOutputCount: "fst.arcWithOutputCount=" + fst.arcWithOutputCount + " arcWithOutputCount=" + arcWithOutputCount;
+    
+    final byte[] finalBytes = new byte[writer.posWrite];
+    //System.out.println("resize " + fst.bytes.length + " down to " + writer.posWrite);
+    System.arraycopy(fst.bytes, 0, finalBytes, 0, writer.posWrite);
+    fst.bytes = finalBytes;
+    fst.cacheRootArcs();
+
+    //final int size = fst.sizeInBytes();
+    //System.out.println("nextCount=" + nextCount + " topCount=" + topCount + " deltaCount=" + deltaCount + " absCount=" + absCount);
+
+    return fst;
+  }
+
+  private static class NodeAndInCount implements Comparable<NodeAndInCount> {
+    final int node;
+    final int count;
+
+    public NodeAndInCount(int node, int count) {
+      this.node = node;
+      this.count = count;
+    }
+    
+    @Override
+    public int compareTo(NodeAndInCount other) {
+      if (count > other.count) {
+        return 1;
+      } else if (count < other.count) {
+        return -1;
+      } else {
+        // Tie-break: smaller node compares as greater than
+        return other.node - node;
+      }
+    }
+  }
+
+  private static class NodeQueue extends PriorityQueue<NodeAndInCount> {
+    public NodeQueue(int topN) {
+      super(topN, false);
+    }
+
+    @Override
+    public boolean lessThan(NodeAndInCount a, NodeAndInCount b) {
+      final int cmp = a.compareTo(b);
+      assert cmp != 0;
+      return cmp < 0;
+    }
   }
 }

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java Mon Jan 30 17:56:40 2012
@@ -151,7 +151,8 @@ abstract class FSTEnum<T> {
         boolean found = false;
         while (low <= high) {
           mid = (low + high) >>> 1;
-          in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+          in.pos = arc.posArcsStart;
+          in.skip(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);
@@ -275,7 +276,7 @@ abstract class FSTEnum<T> {
 
     // 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());
+      //System.out.println("  cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast() + " bba=" + arc.bytesPerArc);
 
       if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
         // Arcs are fixed array -- use binary search to find
@@ -289,15 +290,16 @@ abstract class FSTEnum<T> {
         boolean found = false;
         while (low <= high) {
           mid = (low + high) >>> 1;
-          in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+          in.pos = arc.posArcsStart;
+          in.skip(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)
+          if (cmp < 0) {
             low = mid + 1;
-          else if (cmp > 0)
+          } else if (cmp > 0) {
             high = mid - 1;
-          else {
+          } else {
             found = true;
             break;
           }
@@ -430,9 +432,11 @@ abstract class FSTEnum<T> {
     FST.Arc<T> arc = getArc(upto-1);
     int targetLabel = getTargetLabel();
 
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
+
     while(true) {
       //System.out.println("  cycle target=" + (targetLabel == -1 ? "-1" : (char) targetLabel));
-      final FST.Arc<T> nextArc = fst.findTargetArc(targetLabel, arc, getArc(upto));
+      final FST.Arc<T> nextArc = fst.findTargetArc(targetLabel, arc, getArc(upto), fstReader);
       if (nextArc == null) {
         // short circuit
         //upto--;

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java Mon Jan 30 17:56:40 2012
@@ -35,7 +35,7 @@ final class NodeHash<T> {
   }
 
   private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address, FST.BytesReader in) throws IOException {
-    fst.readFirstRealArc(address, scratchArc, in);
+    fst.readFirstRealTargetArc(address, scratchArc, in);
     if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
       return false;
     }
@@ -43,7 +43,7 @@ final class NodeHash<T> {
       final Builder.Arc<T> arc = node.arcs[arcUpto];
       if (arc.label != scratchArc.label ||
           !arc.output.equals(scratchArc.output) ||
-          ((Builder.CompiledNode) arc.target).address != scratchArc.target ||
+          ((Builder.CompiledNode) arc.target).node != scratchArc.target ||
           !arc.nextFinalOutput.equals(scratchArc.nextFinalOutput) ||
           arc.isFinal != scratchArc.isFinal()) {
         return false;
@@ -71,9 +71,9 @@ final class NodeHash<T> {
     // TODO: maybe if number of arcs is high we can safely subsample?
     for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
       final Builder.Arc<T> arc = node.arcs[arcIdx];
-      //System.out.println("  label=" + arc.label + " target=" + ((Builder.CompiledNode) arc.target).address + " h=" + h + " output=" + fst.outputs.outputToString(arc.output) + " isFinal?=" + arc.isFinal);
+      //System.out.println("  label=" + arc.label + " target=" + ((Builder.CompiledNode) arc.target).node + " h=" + h + " output=" + fst.outputs.outputToString(arc.output) + " isFinal?=" + arc.isFinal);
       h = PRIME * h + arc.label;
-      h = PRIME * h + ((Builder.CompiledNode) arc.target).address;
+      h = PRIME * h + ((Builder.CompiledNode) arc.target).node;
       h = PRIME * h + arc.output.hashCode();
       h = PRIME * h + arc.nextFinalOutput.hashCode();
       if (arc.isFinal) {
@@ -88,9 +88,9 @@ final class NodeHash<T> {
   private int hash(int node) throws IOException {
     final int PRIME = 31;
     final FST.BytesReader in = fst.getBytesReader(0);
-    //System.out.println("hash frozen");
+    //System.out.println("hash frozen node=" + node);
     int h = 0;
-    fst.readFirstRealArc(node, scratchArc, in);
+    fst.readFirstRealTargetArc(node, scratchArc, in);
     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;
@@ -109,26 +109,26 @@ final class NodeHash<T> {
     return h & Integer.MAX_VALUE;
   }
 
-  public int add(Builder.UnCompiledNode<T> node) throws IOException {
+  public int add(Builder.UnCompiledNode<T> nodeIn) throws IOException {
     // System.out.println("hash: add count=" + count + " vs " + table.length);
     final FST.BytesReader in = fst.getBytesReader(0);
-    final int h = hash(node);
+    final int h = hash(nodeIn);
     int pos = h & mask;
     int c = 0;
     while(true) {
       final int v = table[pos];
       if (v == 0) {
         // freeze & add
-        final int address = fst.addNode(node);
-        //System.out.println("  now freeze addr=" + address);
-        assert hash(address) == h : "frozenHash=" + hash(address) + " vs h=" + h;
+        final int node = fst.addNode(nodeIn);
+        //System.out.println("  now freeze node=" + node);
+        assert hash(node) == h : "frozenHash=" + hash(node) + " vs h=" + h;
         count++;
-        table[pos] = address;
+        table[pos] = node;
         if (table.length < 2*count) {
           rehash();
         }
-        return address;
-      } else if (nodesEqual(node, v, in)) {
+        return node;
+      } else if (nodesEqual(nodeIn, v, in)) {
         // same node is already here
         return v;
       }

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Outputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Outputs.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Outputs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Outputs.java Mon Jan 30 17:56:40 2012
@@ -26,6 +26,10 @@ import org.apache.lucene.store.DataOutpu
  * Represents the outputs for an FST, providing the basic
  * algebra needed for the FST.
  *
+ * <p>Note that any operation that returns NO_OUTPUT must
+ * return the same singleton object from {@link
+ * #getNoOutput}.</p>
+ *
  * @lucene.experimental
  */
 
@@ -56,6 +60,8 @@ public abstract class Outputs<T> {
 
   public abstract String outputToString(T output);
 
+  // TODO: maybe make valid(T output) public...?  for asserts
+
   public T merge(T first, T second) {
     throw new UnsupportedOperationException();
   }

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java Mon Jan 30 17:56:40 2012
@@ -38,7 +38,8 @@ public class PairOutputs<A,B> extends Ou
     public final A output1;
     public final B output2;
 
-    public Pair(A output1, B output2) {
+    // use newPair
+    private Pair(A output1, B output2) {
       this.output1 = output1;
       this.output2 = output2;
     }
@@ -66,35 +67,79 @@ public class PairOutputs<A,B> extends Ou
     this.outputs2 = outputs2;
     NO_OUTPUT = new Pair<A,B>(outputs1.getNoOutput(), outputs2.getNoOutput());
   }
-  
-  public Pair<A,B> get(A output1, B output2) {
-    if (output1 == outputs1.getNoOutput() && output2 == outputs2.getNoOutput()) {
+
+  /** Create a new Pair */
+  public Pair<A,B> newPair(A a, B b) {
+    if (a.equals(outputs1.getNoOutput())) {
+      a = outputs1.getNoOutput();
+    }
+    if (b.equals(outputs2.getNoOutput())) {
+      b = outputs2.getNoOutput();
+    }
+
+    if (a == outputs1.getNoOutput() && b == outputs2.getNoOutput()) {
       return NO_OUTPUT;
     } else {
-      return new Pair<A,B>(output1, output2);
+      final Pair<A,B> p = new Pair<A,B>(a, b);
+      assert valid(p);
+      return p;
     }
   }
- 
+
+  // for assert
+  private boolean valid(Pair<A,B> pair) {
+    final boolean noOutput1 = pair.output1.equals(outputs1.getNoOutput());
+    final boolean noOutput2 = pair.output2.equals(outputs2.getNoOutput());
+
+    if (noOutput1 && pair.output1 != outputs1.getNoOutput()) {
+      System.out.println("invalid0");
+      return false;
+    }
+
+    if (noOutput2 && pair.output2 != outputs2.getNoOutput()) {
+      System.out.println("invalid1");
+      return false;
+    }
+
+    if (noOutput1 && noOutput2) {
+      if (pair != NO_OUTPUT) {
+        System.out.println("invalid2");
+        return false;
+      } else {
+        return true;
+      }
+    } else {
+      return true;
+    }
+  }
+  
   @Override
   public Pair<A,B> common(Pair<A,B> pair1, Pair<A,B> pair2) {
-    return get(outputs1.common(pair1.output1, pair2.output1),
-               outputs2.common(pair1.output2, pair2.output2));
+    assert valid(pair1);
+    assert valid(pair2);
+    return newPair(outputs1.common(pair1.output1, pair2.output1),
+                   outputs2.common(pair1.output2, pair2.output2));
   }
 
   @Override
   public Pair<A,B> subtract(Pair<A,B> output, Pair<A,B> inc) {
-    return get(outputs1.subtract(output.output1, inc.output1),
-               outputs2.subtract(output.output2, inc.output2));
+    assert valid(output);
+    assert valid(inc);
+    return newPair(outputs1.subtract(output.output1, inc.output1),
+                    outputs2.subtract(output.output2, inc.output2));
   }
 
   @Override
   public Pair<A,B> add(Pair<A,B> prefix, Pair<A,B> output) {
-    return get(outputs1.add(prefix.output1, output.output1),
-               outputs2.add(prefix.output2, output.output2));
+    assert valid(prefix);
+    assert valid(output);
+    return newPair(outputs1.add(prefix.output1, output.output1),
+                    outputs2.add(prefix.output2, output.output2));
   }
 
   @Override
   public void write(Pair<A,B> output, DataOutput writer) throws IOException {
+    assert valid(output);
     outputs1.write(output.output1, writer);
     outputs2.write(output.output2, writer);
   }
@@ -103,7 +148,7 @@ public class PairOutputs<A,B> extends Ou
   public Pair<A,B> read(DataInput in) throws IOException {
     A output1 = outputs1.read(in);
     B output2 = outputs2.read(in);
-    return get(output1, output2);
+    return newPair(output1, output2);
   }
 
   @Override
@@ -113,6 +158,12 @@ public class PairOutputs<A,B> extends Ou
 
   @Override
   public String outputToString(Pair<A,B> output) {
+    assert valid(output);
     return "<pair:" + outputs1.outputToString(output.output1) + "," + outputs2.outputToString(output.output2) + ">";
   }
+
+  @Override
+  public String toString() {
+    return "PairOutputs<" + outputs1 + "," + outputs2 + ">";
+  }
 }

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java Mon Jan 30 17:56:40 2012
@@ -25,10 +25,7 @@ import org.apache.lucene.store.DataOutpu
 /**
  * Output is a long, for each input term.  NOTE: the
  * resulting FST is not guaranteed to be minimal!  See
- * {@link Builder}.  You must use {@link #get} to obtain the
- * output for a given long value -- do not use autoboxing
- * nor create your own Long instance (the value 0
- * must map to the {@link #getNoOutput} singleton).
+ * {@link Builder}.
  *
  * @lucene.experimental
  */
@@ -50,14 +47,6 @@ public final class PositiveIntOutputs ex
     return doShare ? singletonShare : singletonNoShare;
   }
 
-  public Long get(long v) {
-    if (v == 0) {
-      return NO_OUTPUT;
-    } else {
-      return Long.valueOf(v);
-    }
-  }
-
   @Override
   public Long common(Long output1, Long output2) {
     assert valid(output1);

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Util.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/Util.java Mon Jan 30 17:56:40 2012
@@ -37,23 +37,21 @@ public final class Util {
     // TODO: would be nice not to alloc this on every lookup
     final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
 
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
+
     // Accumulate output as we go
-    final T NO_OUTPUT = fst.outputs.getNoOutput();
-    T output = NO_OUTPUT;
+    T output = fst.outputs.getNoOutput();
     for(int i=0;i<input.length;i++) {
-      if (fst.findTargetArc(input.ints[input.offset + i], arc, arc) == null) {
+      if (fst.findTargetArc(input.ints[input.offset + i], arc, arc, fstReader) == null) {
         return null;
-      } else if (arc.output != NO_OUTPUT) {
-        output = fst.outputs.add(output, arc.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);
+    if (arc.isFinal()) {
+      return fst.outputs.add(output, arc.nextFinalOutput);
     } else {
-      return output;
+      return null;
     }
   }
 
@@ -64,26 +62,24 @@ public final class Util {
   public static<T> T get(FST<T> fst, BytesRef input) throws IOException {
     assert fst.inputType == FST.INPUT_TYPE.BYTE1;
 
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
+
     // 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;
+    T output = fst.outputs.getNoOutput();
     for(int i=0;i<input.length;i++) {
-      if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc) == null) {
+      if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc, fstReader) == null) {
         return null;
-      } else if (arc.output != NO_OUTPUT) {
-        output = fst.outputs.add(output, arc.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);
+    if (arc.isFinal()) {
+      return fst.outputs.add(output, arc.nextFinalOutput);
     } else {
-      return output;
+      return null;
     }
   }
 
@@ -142,7 +138,7 @@ public final class Util {
           result.grow(1+upto);
         }
         
-        fst.readFirstRealArc(arc.target, arc, in);
+        fst.readFirstRealTargetArc(arc.target, arc, in);
 
         FST.Arc<Long> prevArc = null;
 
@@ -238,6 +234,7 @@ public final class Util {
     // A queue of transitions to consider when processing the next level.
     final List<FST.Arc<T>> nextLevelQueue = new ArrayList<FST.Arc<T>>();
     nextLevelQueue.add(startArc);
+    //System.out.println("toDot: startArc: " + startArc);
     
     // A list of states on the same level (for ranking).
     final List<Integer> sameLevelStates = new ArrayList<Integer>();
@@ -289,8 +286,11 @@ public final class Util {
 
     int level = 0;
 
+    final FST.BytesReader r = fst.getBytesReader(0);
+
     while (!nextLevelQueue.isEmpty()) {
       // we could double buffer here, but it doesn't matter probably.
+      //System.out.println("next level=" + level);
       thisLevelQueue.addAll(nextLevelQueue);
       nextLevelQueue.clear();
 
@@ -298,19 +298,19 @@ 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);
+        //System.out.println("  pop: " + arc);
         if (fst.targetHasArcs(arc)) {
-          // scan all arcs
+          // scan all target arcs
+          //System.out.println("  readFirstTarget...");
           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);
-          }
+          fst.readFirstRealTargetArc(arc.target, arc, r);
+
+          //System.out.println("    firstTarget: " + arc);
 
           while (true) {
 
+            //System.out.println("  cycle arc=" + arc);
             // Emit the unseen state and add it to the queue for the next level.
             if (arc.target >= 0 && !seen.get(arc.target)) {
 
@@ -329,7 +329,7 @@ public final class Util {
               if (fst.isExpandedTarget(arc)) {
                 stateColor = expandedNodeColor;
               } else {
-               stateColor = null;
+                stateColor = null;
               }
 
               final String finalOutput;
@@ -339,7 +339,9 @@ public final class Util {
                 finalOutput = "";
               }
 
-              emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput);
+              emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, finalOutput);
+              // To see the node address, use this instead:
+              //emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, String.valueOf(arc.target));
               seen.set(arc.target);
               nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
               sameLevelStates.add(arc.target);
@@ -362,14 +364,22 @@ public final class Util {
               outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
             }
 
+            final String arcColor;
+            if (arc.flag(FST.BIT_TARGET_NEXT)) {
+              arcColor = "red";
+            } else {
+              arcColor = "black";
+            }
+
             assert arc.label != FST.END_LABEL;
-            out.write("  " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n");
+            out.write("  " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"" + (arc.isFinal() ? " style=\"bold\"" : "" ) + " color=\"" + arcColor + "\"]\n");
                    
             // Break the loop if we're on the last arc of this state.
             if (arc.isLast()) {
+              //System.out.println("    break");
               break;
             }
-            fst.readNextArc(arc);
+            fst.readNextRealArc(arc, r);
           }
         }
       }

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java Mon Jan 30 17:56:40 2012
@@ -703,12 +703,6 @@ public class MockDirectoryWrapper extend
   }
 
   @Override
-  public synchronized long fileModified(String name) throws IOException {
-    maybeYield();
-    return delegate.fileModified(name);
-  }
-
-  @Override
   public synchronized long fileLength(String name) throws IOException {
     maybeYield();
     return delegate.fileLength(name);

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Mon Jan 30 17:56:40 2012
@@ -228,14 +228,6 @@ public class TestCrashCausesCorruptIndex
      * {@inheritDoc}
      */
     @Override
-    public long fileModified(String name) throws IOException {
-      return realDirectory.fileModified(name);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public String[] listAll() throws IOException {
       return realDirectory.listAll();
     }

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java Mon Jan 30 17:56:40 2012
@@ -18,10 +18,12 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
-import java.util.Collection;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -32,7 +34,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -47,20 +48,12 @@ public class TestDeletionPolicy extends 
     final IndexCommit firstCommit =  commits.get(0);
     long last = SegmentInfos.generationFromSegmentsFileName(firstCommit.getSegmentsFileName());
     assertEquals(last, firstCommit.getGeneration());
-    long lastVersion = firstCommit.getVersion();
-    long lastTimestamp = firstCommit.getTimestamp();
     for(int i=1;i<commits.size();i++) {
       final IndexCommit commit =  commits.get(i);
       long now = SegmentInfos.generationFromSegmentsFileName(commit.getSegmentsFileName());
-      long nowVersion = commit.getVersion();
-      long nowTimestamp = commit.getTimestamp();
       assertTrue("SegmentInfos commits are out-of-order", now > last);
-      assertTrue("SegmentInfos versions are out-of-order", nowVersion > lastVersion);
-      assertTrue("SegmentInfos timestamps are out-of-order: now=" + nowTimestamp + " vs last=" + lastTimestamp, nowTimestamp >= lastTimestamp);
       assertEquals(now, commit.getGeneration());
       last = now;
-      lastVersion = nowVersion;
-      lastTimestamp = nowTimestamp;
     }
   }
 
@@ -158,6 +151,10 @@ public class TestDeletionPolicy extends 
     }
   }
 
+  static long getCommitTime(IndexCommit commit) throws IOException {
+    return Long.parseLong(commit.getUserData().get("commitTime"));
+  }
+
   /*
    * Delete a commit only when it has been obsoleted by N
    * seconds.
@@ -184,10 +181,10 @@ public class TestDeletionPolicy extends 
       IndexCommit lastCommit = commits.get(commits.size()-1);
 
       // Any commit older than expireTime should be deleted:
-      double expireTime = dir.fileModified(lastCommit.getSegmentsFileName())/1000.0 - expirationTimeSeconds;
+      double expireTime = getCommitTime(lastCommit)/1000.0 - expirationTimeSeconds;
 
       for (final IndexCommit commit : commits) {
-        double modTime = dir.fileModified(commit.getSegmentsFileName())/1000.0;
+        double modTime = getCommitTime(commit)/1000.0;
         if (commit != lastCommit && modTime < expireTime) {
           commit.delete();
           numDelete += 1;
@@ -213,6 +210,9 @@ public class TestDeletionPolicy extends 
       ((LogMergePolicy) mp).setUseCompoundFile(true);
     }
     IndexWriter writer = new IndexWriter(dir, conf);
+    Map<String,String> commitData = new HashMap<String,String>();
+    commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+    writer.commit(commitData);
     writer.close();
 
     final int ITER = 9;
@@ -233,6 +233,9 @@ public class TestDeletionPolicy extends 
       for(int j=0;j<17;j++) {
         addDoc(writer);
       }
+      commitData = new HashMap<String,String>();
+      commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+      writer.commit(commitData);
       writer.close();
 
       if (i < ITER-1) {
@@ -269,7 +272,9 @@ public class TestDeletionPolicy extends 
         // if we are on a filesystem that seems to have only
         // 1 second resolution, allow +1 second in commit
         // age tolerance:
-        long modTime = dir.fileModified(fileName);
+        SegmentInfos sis = new SegmentInfos();
+        sis.read(dir, fileName);
+        long modTime = Long.parseLong(sis.getUserData().get("commitTime"));
         oneSecondResolution &= (modTime % 1000) == 0;
         final long leeway = (long) ((SECONDS + (oneSecondResolution ? 1.0:0.0))*1000);
 

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java Mon Jan 30 17:56:40 2012
@@ -126,10 +126,6 @@ public class TestFieldsReader extends Lu
       return fsDir.fileExists(name);
     }
     @Override
-    public long fileModified(String name) throws IOException {
-      return fsDir.fileModified(name);
-    }
-    @Override
     public void deleteFile(String name) throws IOException {
       fsDir.deleteFile(name);
     }

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java Mon Jan 30 17:56:40 2012
@@ -34,12 +34,10 @@ public class TestIndexCommit extends Luc
     
     IndexCommit ic1 = new IndexCommit() {
       @Override public String getSegmentsFileName() { return "a"; }
-      @Override public long getVersion() { return 12; }
       @Override public Directory getDirectory() { return dir; }
       @Override public Collection<String> getFileNames() throws IOException { return null; }
       @Override public void delete() {}
       @Override public long getGeneration() { return 0; }
-      @Override public long getTimestamp() throws IOException { return 1;}
       @Override public Map<String, String> getUserData() throws IOException { return null; }
       @Override public boolean isDeleted() { return false; }
       @Override public int getSegmentCount() { return 2; }
@@ -47,12 +45,10 @@ public class TestIndexCommit extends Luc
     
     IndexCommit ic2 = new IndexCommit() {
       @Override public String getSegmentsFileName() { return "b"; }
-      @Override public long getVersion() { return 12; }
       @Override public Directory getDirectory() { return dir; }
       @Override public Collection<String> getFileNames() throws IOException { return null; }
       @Override public void delete() {}
       @Override public long getGeneration() { return 0; }
-      @Override public long getTimestamp() throws IOException { return 1;}
       @Override public Map<String, String> getUserData() throws IOException { return null; }
       @Override public boolean isDeleted() { return false; }
       @Override public int getSegmentCount() { return 2; }

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexReader.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestIndexReader.java Mon Jan 30 17:56:40 2012
@@ -381,60 +381,6 @@ public class TestIndexReader extends Luc
         _TestUtil.rmDir(dirFile);
     }
 
-    public void testLastModified() throws Exception {
-      for(int i=0;i<2;i++) {
-        final Directory dir = newDirectory();
-        assertFalse(IndexReader.indexExists(dir));
-        IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
-        addDocumentWithFields(writer);
-        assertTrue(IndexWriter.isLocked(dir));		// writer open, so dir is locked
-        writer.close();
-        assertTrue(IndexReader.indexExists(dir));
-        IndexReader reader = IndexReader.open(dir);
-        assertFalse(IndexWriter.isLocked(dir));		// reader only, no lock
-        long version = IndexReader.lastModified(dir);
-        if (i == 1) {
-          long version2 = IndexReader.lastModified(dir);
-          assertEquals(version, version2);
-        }
-        reader.close();
-        // modify index and check version has been
-        // incremented:
-        Thread.sleep(1000);
-
-        writer  = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
-        addDocumentWithFields(writer);
-        writer.close();
-        reader = IndexReader.open(dir);
-        assertTrue("old lastModified is " + version + "; new lastModified is " + IndexReader.lastModified(dir), version <= IndexReader.lastModified(dir));
-        reader.close();
-        dir.close();
-      }
-    }
-
-    public void testVersion() throws IOException {
-      Directory dir = newDirectory();
-      assertFalse(IndexReader.indexExists(dir));
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
-      addDocumentWithFields(writer);
-      assertTrue(IndexWriter.isLocked(dir));		// writer open, so dir is locked
-      writer.close();
-      assertTrue(IndexReader.indexExists(dir));
-      IndexReader reader = IndexReader.open(dir);
-      assertFalse(IndexWriter.isLocked(dir));		// reader only, no lock
-      long version = IndexReader.getCurrentVersion(dir);
-      reader.close();
-      // modify index and check version has been
-      // incremented:
-      writer  = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
-      addDocumentWithFields(writer);
-      writer.close();
-      reader = IndexReader.open(dir);
-      assertTrue("old version is " + version + "; new version is " + IndexReader.getCurrentVersion(dir), version < IndexReader.getCurrentVersion(dir));
-      reader.close();
-      dir.close();
-    }
-
     public void testOpenReaderAfterDelete() throws IOException {
       File dirFile = _TestUtil.getTempDir("deletetest");
       Directory dir = newFSDirectory(dirFile);

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1237850&r1=1237849&r2=1237850&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Mon Jan 30 17:56:40 2012
@@ -348,12 +348,6 @@ public class TestBufferedIndexInput exte
         dir.deleteFile(name);
       }
       @Override
-      public long fileModified(String name)
-        throws IOException
-      {
-        return dir.fileModified(name);
-      }
-      @Override
       public boolean fileExists(String name)
         throws IOException
       {