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 2013/01/14 19:54:24 UTC

svn commit: r1433035 [2/4] - in /lucene/dev/branches/lucene4547: ./ dev-tools/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/ lucene/analysis/kuromoji/src/resources/org/apache/lucene/an...

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Mon Jan 14 18:54:22 2013
@@ -27,8 +27,14 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.Map;
+/*
+import java.io.Writer;
+import java.io.OutputStreamWriter;
+import java.io.FileOutputStream;
+*/
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.InputStreamDataInput;
@@ -51,9 +57,6 @@ import org.apache.lucene.util.packed.Pac
 // 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!!
@@ -65,8 +68,6 @@ import org.apache.lucene.util.packed.Pac
  *  
  *  <p> See the {@link org.apache.lucene.util.fst package
  *      documentation} for some simple examples.
- *  <p><b>NOTE</b>: the FST cannot be larger than ~2.1 GB
- *  because it uses int to address the byte[].
  *
  * @lucene.experimental
  */
@@ -93,6 +94,8 @@ public final class FST<T> {
   // position:
   private final static int BIT_TARGET_DELTA = 1 << 6;
 
+  // We use this as a marker (because this one flag is
+  // illegal by itself ...):
   private final static byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
 
   /**
@@ -125,24 +128,27 @@ public final class FST<T> {
   /** Added optional packed format. */
   private final static int VERSION_PACKED = 3;
 
-  private final static int VERSION_CURRENT = VERSION_PACKED;
+  /** Changed from int to vInt for encoding arc targets. 
+   *  Also changed maxBytesPerArc from int to vInt in the array case. */
+  private final static int VERSION_VINT_TARGET = 4;
+
+  private final static int VERSION_CURRENT = VERSION_VINT_TARGET;
 
   // Never serialized; just used to represent the virtual
   // final node w/ no arcs:
-  private final static int FINAL_END_NODE = -1;
+  private final static long FINAL_END_NODE = -1;
 
   // Never serialized; just used to represent the virtual
   // non-final node w/ no arcs:
-  private final static int NON_FINAL_END_NODE = 0;
+  private final static long NON_FINAL_END_NODE = 0;
 
   // if non-null, this FST accepts the empty string and
   // produces this output
   T emptyOutput;
 
-  // Not private to avoid synthetic access$NNN methods:
-  byte[] bytes;
+  final BytesStore bytes;
 
-  private int startNode = -1;
+  private long startNode = -1;
 
   public final Outputs<T> outputs;
 
@@ -150,13 +156,13 @@ public final class FST<T> {
   // instead of storing the address of the target node for
   // a given arc, we mark a single bit noting that the next
   // node in the byte[] is the target node):
-  private int lastFrozenNode;
+  private long lastFrozenNode;
 
   private final T NO_OUTPUT;
 
-  public int nodeCount;
-  public int arcCount;
-  public int arcWithOutputCount;
+  public long nodeCount;
+  public long arcCount;
+  public long arcWithOutputCount;
 
   private final boolean packed;
   private PackedInts.Reader nodeRefToAddress;
@@ -175,19 +181,19 @@ public final class FST<T> {
 
     // From node (ord or address); currently only used when
     // building an FST w/ willPackFST=true:
-    int node;
+    long node;
 
     /** To node (ord or address) */
-    public int target;
+    public long target;
 
     byte flags;
     public T nextFinalOutput;
 
     // address (into the byte[]), or ord/address if label == END_LABEL
-    int nextArc;
+    long nextArc;
 
     // This is non-zero if current arcs are fixed array:
-    int posArcsStart;
+    long posArcsStart;
     int bytesPerArc;
     int arcIdx;
     int numArcs;
@@ -254,8 +260,6 @@ public final class FST<T> {
     return (flags & bit) != 0;
   }
 
-  private final BytesWriter writer;
-
   private GrowableWriter nodeAddress;
 
   // TODO: we could be smarter here, and prune periodically
@@ -263,23 +267,28 @@ public final class FST<T> {
   // clear early on:
   private GrowableWriter inCounts;
 
+  private final int version;
+
   // make a new empty FST, for building; Builder invokes
   // this ctor
-  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
+  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs, int bytesPageBits) {
     this.inputType = inputType;
     this.outputs = outputs;
     this.allowArrayArcs = allowArrayArcs;
-    bytes = new byte[128];
+    version = VERSION_CURRENT;
+    // 32 KB blocks:
+    bytes = new BytesStore(bytesPageBits);
+    // pad: ensure no node gets address 0 which is reserved to mean
+    // the stop state w/ no arcs
+    bytes.writeByte((byte) 0);
     NO_OUTPUT = outputs.getNoOutput();
     if (willPackFST) {
-      nodeAddress = new GrowableWriter(PackedInts.bitsRequired(bytes.length - 1), 8, acceptableOverheadRatio);
+      nodeAddress = new GrowableWriter(15, 8, acceptableOverheadRatio);
       inCounts = new GrowableWriter(1, 8, acceptableOverheadRatio);
     } else {
       nodeAddress = null;
       inCounts = null;
     }
-    
-    writer = new DefaultBytesWriter();
 
     emptyOutput = null;
     packed = false;
@@ -289,23 +298,29 @@ public final class FST<T> {
   /** 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_PACKED, VERSION_PACKED);
+    version = CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_VINT_TARGET);
     packed = in.readByte() == 1;
     if (in.readByte() == 1) {
       // accepts empty string
+      // 1 KB blocks:
+      BytesStore emptyBytes = new BytesStore(10);
       int numBytes = in.readVInt();
-      bytes = new byte[numBytes];
-      in.readBytes(bytes, 0, numBytes);
-      
+      emptyBytes.copyBytes(in, numBytes);
+
       // De-serialize empty-string output:
       BytesReader reader;
       if (packed) {
-        reader = new ForwardBytesReader(bytes, 0);
+        reader = emptyBytes.getForwardReader();
       } else {
-        reader = new ReverseBytesReader(bytes, bytes.length-1);
+        reader = emptyBytes.getReverseReader();
+        // NoOutputs uses 0 bytes when writing its output,
+        // so we have to check here else BytesStore gets
+        // angry:
+        if (numBytes > 0) {
+          reader.setPosition(numBytes-1);
+        }
       }
       emptyOutput = outputs.readFinalOutput(reader);
     } else {
@@ -331,12 +346,13 @@ public final class FST<T> {
       nodeRefToAddress = null;
     }
     startNode = in.readVInt();
-    nodeCount = in.readVInt();
-    arcCount = in.readVInt();
-    arcWithOutputCount = in.readVInt();
+    nodeCount = in.readVLong();
+    arcCount = in.readVLong();
+    arcWithOutputCount = in.readVLong();
 
-    bytes = new byte[in.readVInt()];
-    in.readBytes(bytes, 0, bytes.length);
+    int numBytes = in.readVInt();
+    bytes = new BytesStore(in, numBytes, Integer.MAX_VALUE);
+    
     NO_OUTPUT = outputs.getNoOutput();
 
     cacheRootArcs();
@@ -345,6 +361,15 @@ public final class FST<T> {
     // building; we need to break out mutable FST from
     // immutable
     allowArrayArcs = false;
+
+    /*
+    if (bytes.length == 665) {
+      Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+      Util.toDot(this, w, false, false);
+      w.close();
+      System.out.println("Wrote FST to out.dot");
+    }
+    */
   }
 
   public INPUT_TYPE getInputType() {
@@ -352,8 +377,8 @@ public final class FST<T> {
   }
 
   /** Returns bytes used to represent the FST */
-  public int sizeInBytes() {
-    int size = bytes.length;
+  public long sizeInBytes() {
+    long size = bytes.getPosition();
     if (packed) {
       size += nodeRefToAddress.ramBytesUsed();
     } else if (nodeAddress != null) {
@@ -363,25 +388,23 @@ public final class FST<T> {
     return size;
   }
 
-  void finish(int startNode) throws IOException {
-    if (startNode == FINAL_END_NODE && emptyOutput != null) {
-      startNode = 0;
-    }
+  void finish(long startNode) throws IOException {
     if (this.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
-    byte[] finalBytes = new byte[writer.getPosition()];
-    System.arraycopy(bytes, 0, finalBytes, 0, writer.getPosition());
-    bytes = finalBytes;
+    if (startNode == FINAL_END_NODE && emptyOutput != null) {
+      startNode = 0;
+    }
     this.startNode = startNode;
+    bytes.finish();
 
     cacheRootArcs();
   }
 
-  private int getNodeAddress(int node) {
+  private long getNodeAddress(long node) {
     if (nodeAddress != null) {
       // Deref
-      return (int) nodeAddress.get(node);
+      return nodeAddress.get((int) node);
     } else {
       // Straight
       return node;
@@ -481,12 +504,13 @@ public final class FST<T> {
     if (packed) {
       ((PackedInts.Mutable) nodeRefToAddress).save(out);
     }
-    out.writeVInt(startNode);
-    out.writeVInt(nodeCount);
-    out.writeVInt(arcCount);
-    out.writeVInt(arcWithOutputCount);
-    out.writeVInt(bytes.length);
-    out.writeBytes(bytes, 0, bytes.length);
+    out.writeVLong(startNode);
+    out.writeVLong(nodeCount);
+    out.writeVLong(arcCount);
+    out.writeVLong(arcWithOutputCount);
+    long numBytes = bytes.getPosition();
+    out.writeVLong(numBytes);
+    bytes.writeTo(out);
   }
   
   /**
@@ -526,17 +550,16 @@ public final class FST<T> {
     }
   }
 
-  private void writeLabel(int v) throws IOException {
+  private void writeLabel(DataOutput out, int v) throws IOException {
     assert v >= 0: "v=" + v;
     if (inputType == INPUT_TYPE.BYTE1) {
       assert v <= 255: "v=" + v;
-      writer.writeByte((byte) v);
+      out.writeByte((byte) v);
     } else if (inputType == INPUT_TYPE.BYTE2) {
       assert v <= 65535: "v=" + v;
-      writer.writeShort((short) v);
+      out.writeShort((short) v);
     } else {
-      //writeInt(v);
-      writer.writeVInt(v);
+      out.writeVInt(v);
     }
   }
 
@@ -562,8 +585,9 @@ public final class FST<T> {
 
   // serializes new node by appending its bytes to the end
   // of the current byte[]
-  int addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
-    //System.out.println("FST.addNode pos=" + writer.posWrite + " numArcs=" + nodeIn.numArcs);
+  long addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+
+    //System.out.println("FST.addNode pos=" + bytes.getPosition() + " numArcs=" + nodeIn.numArcs);
     if (nodeIn.numArcs == 0) {
       if (nodeIn.isFinal) {
         return FINAL_END_NODE;
@@ -572,38 +596,28 @@ public final class FST<T> {
       }
     }
 
-    int startAddress = writer.getPosition();
+    final long startAddress = bytes.getPosition();
     //System.out.println("  startAddr=" + startAddress);
 
     final boolean doFixedArray = shouldExpand(nodeIn);
-    final int fixedArrayStart;
     if (doFixedArray) {
+      //System.out.println("  fixedArray");
       if (bytesPerArc.length < nodeIn.numArcs) {
         bytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, 1)];
       }
-      // write a "false" first arc:
-      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
-      writer.writeInt(0);
-      fixedArrayStart = writer.getPosition();
-      //System.out.println("  do fixed arcs array arcsStart=" + fixedArrayStart);
-    } else {
-      fixedArrayStart = 0;
     }
 
     arcCount += nodeIn.numArcs;
     
     final int lastArc = nodeIn.numArcs-1;
 
-    int lastArcStart = writer.getPosition();
+    long lastArcStart = bytes.getPosition();
     int maxBytesPerArc = 0;
     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;
+      //System.out.println("  arc " + arcIdx + " label=" + arc.label + " -> target=" + target.node);
 
       if (arcIdx == lastArc) {
         flags += BIT_LAST_ARC;
@@ -630,111 +644,135 @@ public final class FST<T> {
       if (!targetHasArcs) {
         flags += BIT_STOP_NODE;
       } else if (inCounts != null) {
-        inCounts.set(target.node, inCounts.get(target.node) + 1);
+        inCounts.set((int) target.node, inCounts.get((int) target.node) + 1);
       }
 
       if (arc.output != NO_OUTPUT) {
         flags += BIT_ARC_HAS_OUTPUT;
       }
 
-      writer.writeByte((byte) flags);
-      writeLabel(arc.label);
+      bytes.writeByte((byte) flags);
+      writeLabel(bytes, arc.label);
 
-      // System.out.println("  write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + writer.posWrite + " output=" + outputs.outputToString(arc.output));
+      // System.out.println("  write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + bytes.getPosition() + " output=" + outputs.outputToString(arc.output));
 
       if (arc.output != NO_OUTPUT) {
-        outputs.write(arc.output, writer);
+        outputs.write(arc.output, bytes);
         //System.out.println("    write output");
         arcWithOutputCount++;
       }
 
       if (arc.nextFinalOutput != NO_OUTPUT) {
         //System.out.println("    write final output");
-        outputs.writeFinalOutput(arc.nextFinalOutput, writer);
+        outputs.writeFinalOutput(arc.nextFinalOutput, bytes);
       }
 
       if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) {
         assert target.node > 0;
         //System.out.println("    write target");
-        writer.writeInt(target.node);
+        bytes.writeVLong(target.node);
       }
 
       // just write the arcs "like normal" on first pass,
       // but record how many bytes each one took, and max
       // byte size:
       if (doFixedArray) {
-        bytesPerArc[arcIdx] = writer.getPosition() - lastArcStart;
-        lastArcStart = writer.getPosition();
+        bytesPerArc[arcIdx] = (int) (bytes.getPosition() - lastArcStart);
+        lastArcStart = bytes.getPosition();
         maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]);
         //System.out.println("    bytes=" + bytesPerArc[arcIdx]);
       }
     }
-
-    // TODO: if arc'd arrays will be "too wasteful" by some
-    // measure, eg if arcs have vastly different sized
-    // outputs, then we should selectively disable array for
-    // such cases
+    
+    // TODO: try to avoid wasteful cases: disable doFixedArray in that case
+    /* 
+     * 
+     * LUCENE-4682: what is a fair heuristic here?
+     * It could involve some of these:
+     * 1. how "busy" the node is: nodeIn.inputCount relative to frontier[0].inputCount?
+     * 2. how much binSearch saves over scan: nodeIn.numArcs
+     * 3. waste: numBytes vs numBytesExpanded
+     * 
+     * the one below just looks at #3
+    if (doFixedArray) {
+      // rough heuristic: make this 1.25 "waste factor" a parameter to the phd ctor????
+      int numBytes = lastArcStart - startAddress;
+      int numBytesExpanded = maxBytesPerArc * nodeIn.numArcs;
+      if (numBytesExpanded > numBytes*1.25) {
+        doFixedArray = false;
+      }
+    }
+    */
 
     if (doFixedArray) {
-      //System.out.println("  doFixedArray");
+      final int MAX_HEADER_SIZE = 11; // header(byte) + numArcs(vint) + numBytes(vint)
       assert maxBytesPerArc > 0;
       // 2nd pass just "expands" all arcs to take up a fixed
       // byte size
-      final int sizeNeeded = fixedArrayStart + nodeIn.numArcs * maxBytesPerArc;
-      assert ((long) fixedArrayStart) + ((long) nodeIn.numArcs) * maxBytesPerArc < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
 
-      bytes = ArrayUtil.grow(bytes, sizeNeeded);
-      // TODO: we could make this a vInt instead
-      bytes[fixedArrayStart-4] = (byte) (maxBytesPerArc >> 24);
-      bytes[fixedArrayStart-3] = (byte) (maxBytesPerArc >> 16);
-      bytes[fixedArrayStart-2] = (byte) (maxBytesPerArc >> 8);
-      bytes[fixedArrayStart-1] = (byte) maxBytesPerArc;
+      //System.out.println("write int @pos=" + (fixedArrayStart-4) + " numArcs=" + nodeIn.numArcs);
+      // create the header
+      // TODO: clean this up: or just rewind+reuse and deal with it
+      byte header[] = new byte[MAX_HEADER_SIZE]; 
+      ByteArrayDataOutput bad = new ByteArrayDataOutput(header);
+      // write a "false" first arc:
+      bad.writeByte(ARCS_AS_FIXED_ARRAY);
+      bad.writeVInt(nodeIn.numArcs);
+      bad.writeVInt(maxBytesPerArc);
+      int headerLen = bad.getPosition();
+      
+      final long fixedArrayStart = startAddress + headerLen;
 
       // expand the arcs in place, backwards
-      int srcPos = writer.getPosition();
-      int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
-      writer.setPosition(destPos);
-      for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
-        //System.out.println("  repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
-        destPos -= maxBytesPerArc;
-        srcPos -= bytesPerArc[arcIdx];
-        if (srcPos != destPos) {
-          assert destPos > srcPos: "destPos=" + destPos + " srcPos=" + srcPos + " arcIdx=" + arcIdx + " maxBytesPerArc=" + maxBytesPerArc + " bytesPerArc[arcIdx]=" + bytesPerArc[arcIdx] + " nodeIn.numArcs=" + nodeIn.numArcs;
-          System.arraycopy(bytes, srcPos, bytes, destPos, bytesPerArc[arcIdx]);
+      long srcPos = bytes.getPosition();
+      long destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
+      assert destPos >= srcPos;
+      if (destPos > srcPos) {
+        bytes.skipBytes((int) (destPos - srcPos));
+        for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
+          destPos -= maxBytesPerArc;
+          srcPos -= bytesPerArc[arcIdx];
+          //System.out.println("  repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
+          if (srcPos != destPos) {
+            //System.out.println("  copy len=" + bytesPerArc[arcIdx]);
+            assert destPos > srcPos: "destPos=" + destPos + " srcPos=" + srcPos + " arcIdx=" + arcIdx + " maxBytesPerArc=" + maxBytesPerArc + " bytesPerArc[arcIdx]=" + bytesPerArc[arcIdx] + " nodeIn.numArcs=" + nodeIn.numArcs;
+            bytes.copyBytes(srcPos, destPos, bytesPerArc[arcIdx]);
+          }
         }
       }
+      
+      // now write the header
+      bytes.writeBytes(startAddress, header, 0, headerLen);
     }
 
-    // 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 = writer.getPosition() - 1;
-
-    int left = startAddress;
-    int right = endAddress;
-    while (left < right) {
-      final byte b = bytes[left];
-      bytes[left++] = bytes[right];
-      bytes[right--] = b;
+    final long thisNodeAddress = bytes.getPosition()-1;
+
+    bytes.reverse(startAddress, thisNodeAddress);
+
+    // PackedInts uses int as the index, so we cannot handle
+    // > 2.1B nodes when packing:
+    if (nodeAddress != null && nodeCount == Integer.MAX_VALUE) {
+      throw new IllegalStateException("cannot create a packed FST with more than 2.1 billion nodes");
     }
-    //System.out.println("  endAddress=" + endAddress);
 
     nodeCount++;
-    final int node;
+    final long node;
     if (nodeAddress != null) {
+
       // Nodes are addressed by 1+ord:
-      if (nodeCount == nodeAddress.size()) {
+      if ((int) nodeCount == nodeAddress.size()) {
         nodeAddress = nodeAddress.resize(ArrayUtil.oversize(nodeAddress.size() + 1, nodeAddress.getBitsPerValue()));
         inCounts = inCounts.resize(ArrayUtil.oversize(inCounts.size() + 1, inCounts.getBitsPerValue()));
       }
-      nodeAddress.set(nodeCount, endAddress);
+      nodeAddress.set((int) nodeCount, thisNodeAddress);
       // System.out.println("  write nodeAddress[" + nodeCount + "] = " + endAddress);
       node = nodeCount;
     } else {
-      node = endAddress;
+      node = thisNodeAddress;
     }
     lastFrozenNode = node;
 
+    //System.out.println("  ret node=" + node + " address=" + thisNodeAddress + " nodeAddress=" + nodeAddress);
     return node;
   }
 
@@ -763,7 +801,7 @@ public final class FST<T> {
    * 
    * @return Returns the second argument
    * (<code>arc</code>). */
-  public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, FST.BytesReader in) throws IOException {
+  public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
     //System.out.println("readLast");
     if (!targetHasArcs(follow)) {
       //System.out.println("  end node");
@@ -774,19 +812,19 @@ public final class FST<T> {
       arc.flags = BIT_LAST_ARC;
       return arc;
     } else {
-      in.pos = getNodeAddress(follow.target);
+      in.setPosition(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();
-        if (packed) {
+        if (packed || version >= VERSION_VINT_TARGET) {
           arc.bytesPerArc = in.readVInt();
         } else {
           arc.bytesPerArc = in.readInt();
         }
         //System.out.println("  array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
-        arc.posArcsStart = in.pos;
+        arc.posArcsStart = in.getPosition();
         arc.arcIdx = arc.numArcs - 2;
       } else {
         arc.flags = b;
@@ -804,18 +842,16 @@ public final class FST<T> {
           }
           if (arc.flag(BIT_STOP_NODE)) {
           } else if (arc.flag(BIT_TARGET_NEXT)) {
+          } else if (packed) {
+            in.readVLong();
           } else {
-            if (packed) {
-              in.readVInt();
-            } else {
-              in.skip(4);
-            }
+            readUnpackedNodeTarget(in);
           }
           arc.flags = in.readByte();
         }
-        // Undo the byte flags we read: 
-        in.skip(-1);
-        arc.nextArc = in.pos;
+        // Undo the byte flags we read:
+        in.skipBytes(-1);
+        arc.nextArc = in.getPosition();
       }
       readNextRealArc(arc, in);
       assert arc.isLast();
@@ -823,6 +859,16 @@ public final class FST<T> {
     }
   }
 
+  private long readUnpackedNodeTarget(BytesReader in) throws IOException {
+    long target;
+    if (version < VERSION_VINT_TARGET) {
+      target = in.readInt();
+    } else {
+      target = in.readVLong();
+    }
+    return target;
+  }
+
   /**
    * Follow the <code>follow</code> arc and read the first arc of its target;
    * this changes the provided <code>arc</code> (2nd arg) in-place and returns
@@ -853,10 +899,9 @@ public final class FST<T> {
     }
   }
 
-  public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
-    assert in.bytes == bytes;
-    final int address = getNodeAddress(node);
-    in.pos = address;
+  public Arc<T> readFirstRealTargetArc(long node, Arc<T> arc, final BytesReader in) throws IOException {
+    final long address = getNodeAddress(node);
+    in.setPosition(address);
     //System.out.println("  readFirstRealTargtArc address="
     //+ address);
     //System.out.println("   flags=" + arc.flags);
@@ -866,13 +911,13 @@ public final class FST<T> {
       //System.out.println("  fixedArray");
       // this is first arc in a fixed-array
       arc.numArcs = in.readVInt();
-      if (packed) {
+      if (packed || version >= VERSION_VINT_TARGET) {
         arc.bytesPerArc = in.readVInt();
       } else {
         arc.bytesPerArc = in.readInt();
       }
       arc.arcIdx = -1;
-      arc.nextArc = arc.posArcsStart = in.pos;
+      arc.nextArc = arc.posArcsStart = in.getPosition();
       //System.out.println("  bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
     } else {
       //arc.flags = b;
@@ -889,11 +934,11 @@ public final class FST<T> {
    * @return Returns <code>true</code> if <code>arc</code> points to a state in an
    * expanded array format.
    */
-  boolean isExpandedTarget(Arc<T> follow, FST.BytesReader in) throws IOException {
+  boolean isExpandedTarget(Arc<T> follow, BytesReader in) throws IOException {
     if (!targetHasArcs(follow)) {
       return false;
     } else {
-      in.pos = getNodeAddress(follow.target);
+      in.setPosition(getNodeAddress(follow.target));
       return in.readByte() == ARCS_AS_FIXED_ARRAY;
     }
   }
@@ -917,30 +962,36 @@ public final class FST<T> {
     assert !arc.isLast();
 
     if (arc.label == END_LABEL) {
-      //System.out.println("    nextArc fake " + arc.nextArc);
-      int pos = in.pos = getNodeAddress(arc.nextArc);
+      //System.out.println("    nextArc fake " +
+      //arc.nextArc);
+      
+      long pos = getNodeAddress(arc.nextArc);
+      in.setPosition(pos);
+
       final byte b = in.readByte();
       if (b == ARCS_AS_FIXED_ARRAY) {
-        //System.out.println("    nextArc fake array");
+        //System.out.println("    nextArc fixed array");
         in.readVInt();
-        if (packed) {
+
+        // Skip bytesPerArc:
+        if (packed || version >= VERSION_VINT_TARGET) {
           in.readVInt();
         } else {
           in.readInt();
         }
       } else {
-        in.pos = pos;
+        in.setPosition(pos);
       }
     } else {
       if (arc.bytesPerArc != 0) {
         //System.out.println("    nextArc real array");
         // arcs are at fixed entries
-        in.pos = arc.posArcsStart;
-        in.skip((1+arc.arcIdx)*arc.bytesPerArc);
+        in.setPosition(arc.posArcsStart);
+        in.skipBytes((1+arc.arcIdx)*arc.bytesPerArc);
       } else {
         // arcs are packed
         //System.out.println("    nextArc real packed");
-        in.pos = arc.nextArc;
+        in.setPosition(arc.nextArc);
       }
     }
     // skip flags
@@ -951,7 +1002,6 @@ public final 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 {
-    assert in.bytes == bytes;
 
     // TODO: can't assert this because we call from readFirstArc
     // assert !flag(arc.flags, BIT_LAST_ARC);
@@ -961,10 +1011,11 @@ public final class FST<T> {
       // arcs are at fixed entries
       arc.arcIdx++;
       assert arc.arcIdx < arc.numArcs;
-      in.skip(arc.posArcsStart, arc.arcIdx*arc.bytesPerArc);
+      in.setPosition(arc.posArcsStart);
+      in.skipBytes(arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed
-      in.pos = arc.nextArc;
+      in.setPosition(arc.nextArc);
     }
     arc.flags = in.readByte();
     arc.label = readLabel(in);
@@ -987,9 +1038,9 @@ public final class FST<T> {
       } else {
         arc.target = NON_FINAL_END_NODE;
       }
-      arc.nextArc = in.pos;
+      arc.nextArc = in.getPosition();
     } else if (arc.flag(BIT_TARGET_NEXT)) {
-      arc.nextArc = in.pos;
+      arc.nextArc = in.getPosition();
       // TODO: would be nice to make this lazy -- maybe
       // caller doesn't need the target and is scanning arcs...
       if (nodeAddress == null) {
@@ -998,35 +1049,36 @@ public final class FST<T> {
             // must scan
             seekToNextNode(in);
           } else {
-            in.skip(arc.posArcsStart, arc.bytesPerArc * arc.numArcs);
+            in.setPosition(arc.posArcsStart);
+            in.skipBytes(arc.bytesPerArc * arc.numArcs);
           }
         }
-        arc.target = in.pos;
+        arc.target = in.getPosition();
       } else {
         arc.target = arc.node - 1;
         assert arc.target > 0;
       }
     } else {
       if (packed) {
-        final int pos = in.pos;
-        final int code = in.readVInt();
+        final long pos = in.getPosition();
+        final long code = in.readVLong();
         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.size()) {
           // Deref
-          arc.target = (int) nodeRefToAddress.get(code);
+          arc.target = nodeRefToAddress.get((int) code);
           //System.out.println("    deref code=" + code + " target=" + arc.target);
         } else {
           // Absolute
           arc.target = code;
-          //System.out.println("    abs code=" + code + " derefLen=" + nodeRefToAddress.length);
+          //System.out.println("    abs code=" + code);
         }
       } else {
-        arc.target = in.readInt();
+        arc.target = readUnpackedNodeTarget(in);
       }
-      arc.nextArc = in.pos;
+      arc.nextArc = in.getPosition();
     }
     return arc;
   }
@@ -1035,7 +1087,6 @@ public final class FST<T> {
    *  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, BytesReader in) throws IOException {
     assert cachedRootArcs != null;
-    assert in.bytes == bytes;
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1070,7 +1121,7 @@ public final class FST<T> {
       return null;
     }
 
-    in.pos = getNodeAddress(follow.target);
+    in.setPosition(getNodeAddress(follow.target));
 
     arc.node = follow.target;
 
@@ -1079,18 +1130,19 @@ public final class FST<T> {
     if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
       // Arcs are full array; do binary search:
       arc.numArcs = in.readVInt();
-      if (packed) {
+      if (packed || version >= VERSION_VINT_TARGET) {
         arc.bytesPerArc = in.readVInt();
       } else {
         arc.bytesPerArc = in.readInt();
       }
-      arc.posArcsStart = in.pos;
+      arc.posArcsStart = in.getPosition();
       int low = 0;
       int high = arc.numArcs-1;
       while (low <= high) {
         //System.out.println("    cycle");
         int mid = (low + high) >>> 1;
-        in.skip(arc.posArcsStart, arc.bytesPerArc*mid + 1);
+        in.setPosition(arc.posArcsStart);
+        in.skipBytes(arc.bytesPerArc*mid + 1);
         int midLabel = readLabel(in);
         final int cmp = midLabel - labelToMatch;
         if (cmp < 0) {
@@ -1145,9 +1197,9 @@ public final class FST<T> {
 
       if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
         if (packed) {
-          in.readVInt();
+          in.readVLong();
         } else {
-          in.readInt();
+          readUnpackedNodeTarget(in);
         }
       }
 
@@ -1157,16 +1209,16 @@ public final class FST<T> {
     }
   }
 
-  public int getNodeCount() {
+  public long getNodeCount() {
     // 1+ in order to count the -1 implicit final node
     return 1+nodeCount;
   }
   
-  public int getArcCount() {
+  public long getArcCount() {
     return arcCount;
   }
 
-  public int getArcWithOutputCount() {
+  public long getArcWithOutputCount() {
     return arcWithOutputCount;
   }
 
@@ -1191,56 +1243,6 @@ public final class FST<T> {
        node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
   }
   
-  static abstract class BytesWriter extends DataOutput {
-    public abstract void setPosition(int posWrite);
-    public abstract int getPosition();
-  }
-
-  // Non-static: writes to FST's byte[]
-  class DefaultBytesWriter extends BytesWriter {
-    int posWrite;
-
-    public DefaultBytesWriter() {
-      // pad: ensure no node gets address 0 which is reserved to mean
-      // the stop state w/ no arcs
-      posWrite = 1;
-    }
-
-    @Override
-    public void writeByte(byte b) {
-      assert posWrite <= bytes.length;
-      if (bytes.length == posWrite) {
-        assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
-        bytes = ArrayUtil.grow(bytes);
-      }
-      assert posWrite < bytes.length: "posWrite=" + posWrite + " bytes.length=" + bytes.length;
-      bytes[posWrite++] = b;
-    }
-
-    @Override
-    public int getPosition() {
-      return posWrite;
-    }
-
-    @Override
-    public void setPosition(int posWrite) {
-      this.posWrite = posWrite;
-      if (bytes.length < posWrite) {
-        assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
-        bytes = ArrayUtil.grow(bytes, posWrite);
-      }
-    }
-
-    @Override
-    public void writeBytes(byte[] b, int offset, int length) {
-      final int size = posWrite + length;
-      assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
-      bytes = ArrayUtil.grow(bytes, size);
-      System.arraycopy(b, offset, bytes, posWrite, length);
-      posWrite += length;
-    }
-  }
-
   /** Returns a {@link BytesReader} for this FST, positioned at
    *  position 0. */
   public BytesReader getBytesReader() {
@@ -1249,87 +1251,34 @@ public final class FST<T> {
 
   /** Returns a {@link BytesReader} for this FST, positioned at
    *  the provided position. */
-  public BytesReader getBytesReader(int pos) {
+  public BytesReader getBytesReader(long pos) {
     // TODO: maybe re-use via ThreadLocal?
+    BytesReader in;
     if (packed) {
-      return new ForwardBytesReader(bytes, pos);
+      in = bytes.getForwardReader();
     } else {
-      return new ReverseBytesReader(bytes, pos);
-    }
-  }
-
-  /** Reads the bytes from this FST.  Use {@link
-   *  #getBytesReader(int)} to obtain an instance for this
-   *  FST; re-use across calls (but only within a single
-   *  thread) for better performance. */
-  public static abstract class BytesReader extends DataInput {
-    protected int pos;
-    protected final byte[] bytes;
-    protected BytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
-    }
-    abstract void skip(int byteCount);
-    abstract void skip(int base, int byteCount);
-  }
-
-  final static class ReverseBytesReader extends BytesReader {
-
-    public ReverseBytesReader(byte[] bytes, int pos) {
-      super(bytes, pos);
-    }
-
-    @Override
-    public byte readByte() {
-      return bytes[pos--];
-    }
-
-    @Override
-    public void readBytes(byte[] b, int offset, int len) {
-      for(int i=0;i<len;i++) {
-        b[offset+i] = bytes[pos--];
-      }
-    }
-
-    @Override
-    public void skip(int count) {
-      pos -= count;
+      in = bytes.getReverseReader();
     }
-
-    @Override
-    public void skip(int base, int count) {
-      pos = base - count;
+    if (pos != 0) {
+      in.setPosition(pos);
     }
+    return in;
   }
 
-  // TODO: can we use just ByteArrayDataInput...?  need to
-  // add a .skipBytes to DataInput.. hmm and .setPosition
-  final static class ForwardBytesReader extends BytesReader {
-
-    public ForwardBytesReader(byte[] bytes, int pos) {
-      super(bytes, pos);
-    }
-
-    @Override
-    public byte readByte() {
-      return bytes[pos++];
-    }
+  /** Reads bytes stored in an FST. */
+  public static abstract class BytesReader extends DataInput {
+    /** Get current read position. */
+    public abstract long getPosition();
 
-    @Override
-    public void readBytes(byte[] b, int offset, int len) {
-      System.arraycopy(bytes, pos, b, offset, len);
-      pos += len;
-    }
+    /** Set current read position. */
+    public abstract void setPosition(long pos);
 
-    @Override
-    public void skip(int count) {
-      pos += count;
-    }
+    /** Returns true if this reader uses reversed bytes
+     *  under-the-hood. */
+    public abstract boolean reversed();
 
-    @Override
-    public void skip(int base, int count) {
-      pos = base + count;
-    }
+    /** Skips bytes. */
+    public abstract void skipBytes(int count);
   }
 
   private static class ArcAndState<T> {
@@ -1451,14 +1400,13 @@ public final class FST<T> {
  */
 
   // Creates a packed FST
-  private FST(INPUT_TYPE inputType, PackedInts.Reader nodeRefToAddress, Outputs<T> outputs) {
+  private FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
+    version = VERSION_CURRENT;
     packed = true;
     this.inputType = inputType;
-    bytes = new byte[128];
-    this.nodeRefToAddress = nodeRefToAddress;
+    bytes = new BytesStore(bytesPageBits);
     this.outputs = outputs;
     NO_OUTPUT = outputs.getNoOutput();
-    writer = new DefaultBytesWriter();
     
     // NOTE: bogus because this is only used during
     // building; we need to break out mutable FST from
@@ -1480,6 +1428,9 @@ public final class FST<T> {
    */
   FST<T> pack(int minInCountDeref, int maxDerefNodes, float acceptableOverheadRatio) throws IOException {
 
+    // NOTE: maxDerefNodes is intentionally int: we cannot
+    // support > 2.1B deref nodes
+
     // TODO: other things to try
     //   - renumber the nodes to get more next / better locality?
     //   - allow multiple input labels on an arc, so
@@ -1529,17 +1480,13 @@ public final class FST<T> {
       //System.out.println("map node=" + n.node + " inCount=" + n.count + " to newID=" + downTo);
     }
 
-    final FST<T> fst = new FST<T>(inputType, null, outputs);
-
-    final BytesWriter writer = fst.writer;
-
     // +1 because node ords start at 1 (0 is reserved as stop node):
     final GrowableWriter newNodeAddress = new GrowableWriter(
-        PackedInts.bitsRequired(bytes.length), 1 + nodeCount, acceptableOverheadRatio);
+                       PackedInts.bitsRequired(this.bytes.getPosition()), (int) (1 + nodeCount), acceptableOverheadRatio);
 
     // Fill initial coarse guess:
     for(int node=1;node<=nodeCount;node++) {
-      newNodeAddress.set(node, 1 + bytes.length - nodeAddress.get(node));
+      newNodeAddress.set(node, 1 + this.bytes.getPosition() - nodeAddress.get(node));
     }
 
     int absCount;
@@ -1547,6 +1494,8 @@ public final class FST<T> {
     int topCount;
     int nextCount;
 
+    FST<T> fst;
+
     // Iterate until we converge:
     while(true) {
 
@@ -1556,7 +1505,10 @@ public final class FST<T> {
       // for assert:
       boolean negDelta = false;
 
-      writer.setPosition(0);
+      fst = new FST<T>(inputType, outputs, bytes.getBlockBits());
+      
+      final BytesStore writer = fst.bytes;
+
       // Skip 0 byte since 0 is reserved target:
       writer.writeByte((byte) 0);
 
@@ -1568,19 +1520,20 @@ public final class FST<T> {
 
       int changedCount = 0;
 
-      int addressError = 0;
+      long 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--) {
+      for(int node=(int)nodeCount;node>=1;node--) {
         fst.nodeCount++;
-        final int address = writer.getPosition();
+        final long address = writer.getPosition();
+
         //System.out.println("  node: " + node + " address=" + address);
         if (address != newNodeAddress.get(node)) {
-          addressError = address - (int) newNodeAddress.get(node);
+          addressError = address - newNodeAddress.get(node);
           //System.out.println("    change: " + (address - newNodeAddress[node]));
           changed = true;
           newNodeAddress.set(node, address);
@@ -1600,6 +1553,7 @@ public final class FST<T> {
         writeNode:
         while(true) { // retry writing this node
 
+          //System.out.println("  cycle: retry");
           readFirstRealTargetArc(node, arc, r);
 
           final boolean useArcArray = arc.bytesPerArc != 0;
@@ -1617,9 +1571,9 @@ public final class FST<T> {
           int maxBytesPerArc = 0;
           //int wasted = 0;
           while(true) {  // iterate over all arcs for this node
+            //System.out.println("    cycle next arc");
 
-            //System.out.println("    arc label=" + arc.label + " target=" + arc.target + " pos=" + writer.posWrite);
-            final int arcStartPos = writer.getPosition();
+            final long arcStartPos = writer.getPosition();
             nodeArcCount++;
 
             byte flags = 0;
@@ -1654,19 +1608,18 @@ public final class FST<T> {
               flags += BIT_ARC_HAS_OUTPUT;
             }
 
-            final Integer ptr;
-            final int absPtr;
+            final long absPtr;
             final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
             if (doWriteTarget) {
 
-              ptr = topNodeMap.get(arc.target);
+              final Integer ptr = topNodeMap.get(arc.target);
               if (ptr != null) {
                 absPtr = ptr;
               } else {
-                absPtr = topNodeMap.size() + (int) newNodeAddress.get(arc.target) + addressError;
+                absPtr = topNodeMap.size() + newNodeAddress.get((int) arc.target) + addressError;
               }
 
-              int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition() - 2;
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition() - 2;
               if (delta < 0) {
                 //System.out.println("neg: " + delta);
                 anyNegDelta = true;
@@ -1677,12 +1630,13 @@ public final class FST<T> {
                 flags |= BIT_TARGET_DELTA;
               }
             } else {
-              ptr = null;
               absPtr = 0;
             }
 
+            assert flags != ARCS_AS_FIXED_ARRAY;
             writer.writeByte(flags);
-            fst.writeLabel(arc.label);
+
+            fst.writeLabel(writer, arc.label);
 
             if (arc.output != NO_OUTPUT) {
               outputs.write(arc.output, writer);
@@ -1696,7 +1650,7 @@ public final class FST<T> {
 
             if (doWriteTarget) {
 
-              int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition();
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition();
               if (delta < 0) {
                 anyNegDelta = true;
                 //System.out.println("neg: " + delta);
@@ -1705,7 +1659,7 @@ public final class FST<T> {
 
               if (flag(flags, BIT_TARGET_DELTA)) {
                 //System.out.println("        delta");
-                writer.writeVInt(delta);
+                writer.writeVLong(delta);
                 if (!retry) {
                   deltaCount++;
                 }
@@ -1717,7 +1671,7 @@ public final class FST<T> {
                   System.out.println("        abs");
                 }
                 */
-                writer.writeVInt(absPtr);
+                writer.writeVLong(absPtr);
                 if (!retry) {
                   if (absPtr >= topNodeMap.size()) {
                     absCount++;
@@ -1729,7 +1683,7 @@ public final class FST<T> {
             }
 
             if (useArcArray) {
-              final int arcBytes = writer.getPosition() - arcStartPos;
+              final int arcBytes = (int) (writer.getPosition() - arcStartPos);
               //System.out.println("  " + arcBytes + " bytes");
               maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
               // NOTE: this may in fact go "backwards", if
@@ -1739,7 +1693,7 @@ public final class FST<T> {
               // will retry (below) so it's OK to ovewrite
               // bytes:
               //wasted += bytesPerArc - arcBytes;
-              writer.setPosition(arcStartPos + bytesPerArc);
+              writer.skipBytes((int) (arcStartPos + bytesPerArc - writer.getPosition()));
             }
 
             if (arc.isLast()) {
@@ -1764,11 +1718,12 @@ public final class FST<T> {
 
           // Retry:
           bytesPerArc = maxBytesPerArc;
-          writer.setPosition(address);
+          writer.truncate(address);
           nodeArcCount = 0;
           retry = true;
           anyNegDelta = false;
         }
+
         negDelta |= anyNegDelta;
 
         fst.arcCount += nodeArcCount;
@@ -1788,8 +1743,8 @@ public final class FST<T> {
     }
 
     long maxAddress = 0;
-    for (int key : topNodeMap.keySet()) {
-      maxAddress = Math.max(maxAddress, newNodeAddress.get(key));
+    for (long key : topNodeMap.keySet()) {
+      maxAddress = Math.max(maxAddress, newNodeAddress.get((int) key));
     }
 
     PackedInts.Mutable nodeRefToAddressIn = PackedInts.getMutable(topNodeMap.size(),
@@ -1799,8 +1754,7 @@ public final class FST<T> {
     }
     fst.nodeRefToAddress = nodeRefToAddressIn;
     
-
-    fst.startNode = (int) newNodeAddress.get(startNode);
+    fst.startNode = newNodeAddress.get((int) startNode);
     //System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
 
     if (emptyOutput != null) {
@@ -1810,11 +1764,8 @@ public final class FST<T> {
     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.getPosition()];
-    //System.out.println("resize " + fst.bytes.length + " down to " + writer.posWrite);
-    System.arraycopy(fst.bytes, 0, finalBytes, 0, writer.getPosition());
-    fst.bytes = finalBytes;
+
+    fst.bytes.finish();
     fst.cacheRootArcs();
 
     //final int size = fst.sizeInBytes();

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java Mon Jan 14 18:54:22 2013
@@ -17,11 +17,11 @@ package org.apache.lucene.util.fst;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 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
@@ -153,8 +153,8 @@ abstract class FSTEnum<T> {
         boolean found = false;
         while (low <= high) {
           mid = (low + high) >>> 1;
-          in.pos = arc.posArcsStart;
-          in.skip(arc.bytesPerArc*mid+1);
+          in.setPosition(arc.posArcsStart);
+          in.skipBytes(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);
@@ -292,8 +292,8 @@ abstract class FSTEnum<T> {
         boolean found = false;
         while (low <= high) {
           mid = (low + high) >>> 1;
-          in.pos = arc.posArcsStart;
-          in.skip(arc.bytesPerArc*mid+1);
+          in.setPosition(arc.posArcsStart);
+          in.skipBytes(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);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Mon Jan 14 18:54:22 2013
@@ -19,22 +19,27 @@ package org.apache.lucene.util.fst;
 
 import java.io.IOException;
 
+import org.apache.lucene.util.packed.GrowableWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
 // Used to dedup states (lookup already-frozen states)
 final class NodeHash<T> {
 
-  private int[] table;
+  private GrowableWriter table;
   private int count;
   private int mask;
   private final FST<T> fst;
   private final FST.Arc<T> scratchArc = new FST.Arc<T>();
+  private final FST.BytesReader in;
 
-  public NodeHash(FST<T> fst) {
-    table = new int[16];
+  public NodeHash(FST<T> fst, FST.BytesReader in) {
+    table = new GrowableWriter(8, 16, PackedInts.COMPACT);
     mask = 15;
     this.fst = fst;
+    this.in = in;
   }
 
-  private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address, FST.BytesReader in) throws IOException {
+  private boolean nodesEqual(Builder.UnCompiledNode<T> node, long address) throws IOException {
     fst.readFirstRealTargetArc(address, scratchArc, in);
     if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
       return false;
@@ -73,7 +78,8 @@ final class NodeHash<T> {
       final Builder.Arc<T> arc = node.arcs[arcIdx];
       //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).node;
+      long n = ((Builder.CompiledNode) arc.target).node;
+      h = PRIME * h + (int) (n^(n>>32));
       h = PRIME * h + arc.output.hashCode();
       h = PRIME * h + arc.nextFinalOutput.hashCode();
       if (arc.isFinal) {
@@ -85,16 +91,15 @@ final class NodeHash<T> {
   }
 
   // hash code for a frozen node
-  private int hash(int node) throws IOException {
+  private int hash(long node) throws IOException {
     final int PRIME = 31;
-    final FST.BytesReader in = fst.getBytesReader(0);
     //System.out.println("hash frozen node=" + node);
     int h = 0;
     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());
+      //System.out.println("  label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal() + " pos=" + in.getPosition());
       h = PRIME * h + scratchArc.label;
-      h = PRIME * h + scratchArc.target;
+      h = PRIME * h + (int) (scratchArc.target^(scratchArc.target>>32));
       h = PRIME * h + scratchArc.output.hashCode();
       h = PRIME * h + scratchArc.nextFinalOutput.hashCode();
       if (scratchArc.isFinal()) {
@@ -109,26 +114,25 @@ final class NodeHash<T> {
     return h & Integer.MAX_VALUE;
   }
 
-  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);
+  public long add(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+    // System.out.println("hash: add count=" + count + " vs " + table.size());
     final int h = hash(nodeIn);
     int pos = h & mask;
     int c = 0;
     while(true) {
-      final int v = table[pos];
+      final long v = table.get(pos);
       if (v == 0) {
         // freeze & add
-        final int node = fst.addNode(nodeIn);
+        final long node = fst.addNode(nodeIn);
         //System.out.println("  now freeze node=" + node);
         assert hash(node) == h : "frozenHash=" + hash(node) + " vs h=" + h;
         count++;
-        table[pos] = node;
-        if (table.length < 2*count) {
+        table.set(pos, node);
+        if (table.size() < 2*count) {
           rehash();
         }
         return node;
-      } else if (nodesEqual(nodeIn, v, in)) {
+      } else if (nodesEqual(nodeIn, v)) {
         // same node is already here
         return v;
       }
@@ -139,12 +143,12 @@ final class NodeHash<T> {
   }
 
   // called only by rehash
-  private void addNew(int address) throws IOException {
+  private void addNew(long address) throws IOException {
     int pos = hash(address) & mask;
     int c = 0;
     while(true) {
-      if (table[pos] == 0) {
-        table[pos] = address;
+      if (table.get(pos) == 0) {
+        table.set(pos, address);
         break;
       }
 
@@ -154,16 +158,16 @@ final class NodeHash<T> {
   }
 
   private void rehash() throws IOException {
-    final int[] oldTable = table;
+    final GrowableWriter oldTable = table;
 
-    if (oldTable.length >= Integer.MAX_VALUE/2) {
+    if (oldTable.size() >= Integer.MAX_VALUE/2) {
       throw new IllegalStateException("FST too large (> 2.1 GB)");
     }
 
-    table = new int[2*table.length];
-    mask = table.length-1;
-    for(int idx=0;idx<oldTable.length;idx++) {
-      final int address = oldTable[idx];
+    table = new GrowableWriter(oldTable.getBitsPerValue(), 2*oldTable.size(), PackedInts.COMPACT);
+    mask = table.size()-1;
+    for(int idx=0;idx<oldTable.size();idx++) {
+      final long address = oldTable.get(idx);
       if (address != 0) {
         addNew(address);
       }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Mon Jan 14 18:54:22 2013
@@ -39,7 +39,7 @@ 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);
+    final BytesReader fstReader = fst.getBytesReader(0);
 
     // Accumulate output as we go
     T output = fst.outputs.getNoOutput();
@@ -64,7 +64,7 @@ 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);
+    final 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>());
@@ -101,7 +101,7 @@ public final class Util {
    *  fit this. */
   public static IntsRef getByOutput(FST<Long> fst, long targetOutput) throws IOException {
 
-    final FST.BytesReader in = fst.getBytesReader(0);
+    final BytesReader in = fst.getBytesReader(0);
 
     // TODO: would be nice not to alloc this on every lookup
     FST.Arc<Long> arc = fst.getFirstArc(new FST.Arc<Long>());
@@ -147,8 +147,8 @@ public final class Util {
           boolean exact = false;
           while (low <= high) {
             mid = (low + high) >>> 1;
-            in.pos = arc.posArcsStart;
-            in.skip(arc.bytesPerArc*mid);
+            in.setPosition(arc.posArcsStart);
+            in.skipBytes(arc.bytesPerArc*mid);
             final byte flags = in.readByte();
             fst.readLabel(in);
             final long minArcOutput;
@@ -273,7 +273,7 @@ public final class Util {
   public static class TopNSearcher<T> {
 
     private final FST<T> fst;
-    private final FST.BytesReader bytesReader;
+    private final BytesReader bytesReader;
     private final int topN;
     private final int maxQueueDepth;
 
@@ -374,7 +374,7 @@ public final class Util {
 
       //System.out.println("search topN=" + topN);
 
-      final FST.BytesReader fstReader = fst.getBytesReader(0);
+      final BytesReader fstReader = fst.getBytesReader(0);
       final T NO_OUTPUT = fst.outputs.getNoOutput();
 
       // TODO: we could enable FST to sorting arcs by weight
@@ -544,7 +544,9 @@ public final class Util {
    * </pre>
    * 
    * <p>
-   * Note: larger FSTs (a few thousand nodes) won't even render, don't bother.
+   * Note: larger FSTs (a few thousand nodes) won't even
+   * render, don't bother.  If the FST is > 2.1 GB in size
+   * then this method will throw strange exceptions.
    * 
    * @param sameRank
    *          If <code>true</code>, the resulting <code>dot</code> file will try
@@ -578,7 +580,7 @@ public final class Util {
 
     // A bitset of already seen states (target offset).
     final BitSet seen = new BitSet();
-    seen.set(startArc.target);
+    seen.set((int) startArc.target);
 
     // Shape for states.
     final String stateShape = "circle";
@@ -595,7 +597,7 @@ public final class Util {
     emitDotState(out, "initial", "point", "white", "");
 
     final T NO_OUTPUT = fst.outputs.getNoOutput();
-    final FST.BytesReader r = fst.getBytesReader(0);
+    final BytesReader r = fst.getBytesReader(0);
 
     // final FST.Arc<T> scratchArc = new FST.Arc<T>();
 
@@ -617,7 +619,7 @@ public final class Util {
         finalOutput = null;
       }
       
-      emitDotState(out, Integer.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
+      emitDotState(out, Long.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
     }
 
     out.write("  initial -> " + startArc.target + "\n");
@@ -638,7 +640,8 @@ public final class Util {
         if (FST.targetHasArcs(arc)) {
           // scan all target arcs
           //System.out.println("  readFirstTarget...");
-          final int node = arc.target;
+
+          final long node = arc.target;
 
           fst.readFirstRealTargetArc(arc.target, arc, r);
 
@@ -648,7 +651,7 @@ public final class Util {
 
             //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)) {
+            if (arc.target >= 0 && !seen.get((int) arc.target)) {
 
               /*
               boolean isFinal = false;
@@ -675,12 +678,12 @@ public final class Util {
                 finalOutput = "";
               }
 
-              emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, finalOutput);
+              emitDotState(out, Long.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);
+              seen.set((int) arc.target);
               nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
-              sameLevelStates.add(arc.target);
+              sameLevelStates.add((int) arc.target);
             }
 
             String outs;
@@ -893,8 +896,8 @@ public final class Util {
       // " targetLabel=" + targetLabel);
       while (low <= high) {
         mid = (low + high) >>> 1;
-        in.pos = arc.posArcsStart;
-        in.skip(arc.bytesPerArc * mid + 1);
+        in.setPosition(arc.posArcsStart);
+        in.skipBytes(arc.bytesPerArc * mid + 1);
         final int midLabel = fst.readLabel(in);
         final int cmp = midLabel - label;
         // System.out.println("  cycle low=" + low + " high=" + high + " mid=" +

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Jan 14 18:54:22 2013
@@ -99,7 +99,7 @@ public class TestBackwardsCompatibility 
     createIndex("index.nocfs", false, false);
   }
   */
-  
+
 /*
   // These are only needed for the special upgrade test to verify
   // that also single-segment indexes are correctly upgraded by IndexUpgrader.
@@ -115,8 +115,40 @@ public class TestBackwardsCompatibility 
   }
 
 */  
+
+  /*
+  public void testCreateMoreTermsIndex() throws Exception {
+    // we use a real directory name that is not cleaned up,
+    // because this method is only used to create backwards
+    // indexes:
+    File indexDir = new File("moreterms");
+    _TestUtil.rmDir(indexDir);
+    Directory dir = newFSDirectory(indexDir);
+
+    LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
+    mp.setUseCompoundFile(false);
+    mp.setNoCFSRatio(1.0);
+    mp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
+    // TODO: remove randomness
+    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
+      .setMergePolicy(mp);
+    conf.setCodec(Codec.forName("Lucene40"));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    LineFileDocs docs = new LineFileDocs(null, true);
+    for(int i=0;i<50;i++) {
+      writer.addDocument(docs.nextDoc());
+    }
+    writer.close();
+    dir.close();
+
+    // Gives you time to copy the index out!: (there is also
+    // a test option to not remove temp dir...):
+    Thread.sleep(100000);
+  }
+  */
+  
   final static String[] oldNames = {"40.cfs",
-                             "40.nocfs",
+                                    "40.nocfs",
   };
   
   final String[] unsupportedNames = {"19.cfs",
@@ -144,7 +176,7 @@ public class TestBackwardsCompatibility 
   };
   
   final static String[] oldSingleSegmentNames = {"40.optimized.cfs",
-                                          "40.optimized.nocfs",
+                                                 "40.optimized.nocfs",
   };
   
   static Map<String,Directory> oldIndexDirs;
@@ -916,4 +948,15 @@ public class TestBackwardsCompatibility 
       dir.close();
     }
   }
+
+  public static final String moreTermsIndex = "moreterms.40.zip";
+
+  public void testMoreTerms() throws Exception {
+    File oldIndexDir = _TestUtil.getTempDir("moreterms");
+    _TestUtil.unzip(getDataFile(moreTermsIndex), oldIndexDir);
+    Directory dir = newFSDirectory(oldIndexDir);
+    // TODO: more tests
+    _TestUtil.checkIndex(dir);
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java Mon Jan 14 18:54:22 2013
@@ -22,7 +22,6 @@ import java.util.Random;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
@@ -31,12 +30,14 @@ import org.apache.lucene.search.TermStat
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.similarities.Similarity.ExactSimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimWeight;
+import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 
 /**
  * 
@@ -86,39 +87,6 @@ public class TestCustomNorms extends Luc
     dir.close();
     docs.close();
   }
-  
-  public void testPackedNorms() throws IOException {
-    Directory dir = newDirectory();
-    IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    config.setSimilarity(new PackedNormSimilarity());
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
-    int num = _TestUtil.nextInt(random(), 1, 1000);
-    for (int i = 0; i < num; i++) {
-      Document doc = new Document();
-      doc.add(new StringField("len", Integer.toString(i), Field.Store.YES));
-      StringBuilder sb = new StringBuilder();
-      for (int j = 0; j < i; j++) {
-        sb.append(" token");
-      }
-      doc.add(new TextField("content", sb.toString(), Field.Store.NO));
-      writer.addDocument(doc);
-    }
-    
-    DirectoryReader ir = writer.getReader();
-    writer.close();
-    for (AtomicReaderContext context : ir.leaves()) {
-      AtomicReader reader = context.reader();
-      DocValues norms = reader.normValues("content");
-      assertNotNull(norms);
-      Source source = norms.getSource();
-      assertEquals(Type.VAR_INTS, source.getType());
-      for (int i = 0; i < reader.maxDoc(); i++) {
-        assertEquals(source.getInt(i), Long.parseLong(reader.document(i).get("len")));
-      }
-    }
-    ir.close();
-    dir.close();
-  }
 
   public void testExceptionOnRandomType() throws IOException {
     Directory dir = newDirectory();
@@ -334,28 +302,5 @@ public class TestCustomNorms extends Luc
       throw new UnsupportedOperationException();
     }
   }
-  
-  class PackedNormSimilarity extends Similarity {
-
-    @Override
-    public void computeNorm(FieldInvertState state, Norm norm) {
-      norm.setPackedLong(state.getLength());
-    }
-
-    @Override
-    public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-  }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Mon Jan 14 18:54:22 2013
@@ -310,7 +310,7 @@ public class TestFSTs extends LuceneTest
 
     final boolean doRewrite = random().nextBoolean();
 
-    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, true);
+    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, PackedInts.DEFAULT, true, 15);
 
     boolean storeOrd = random().nextBoolean();
     if (VERBOSE) {
@@ -453,7 +453,7 @@ public class TestFSTs extends LuceneTest
       this.outputs = outputs;
       this.doPack = doPack;
 
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, !noArcArrays);
+      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -484,8 +484,13 @@ public class TestFSTs extends LuceneTest
           }
         }
 
+        long tMid = System.currentTimeMillis();
+        System.out.println(((tMid-tStart) / 1000.0) + " sec to add all terms");
+
         assert builder.getTermCount() == ord;
         FST<T> fst = builder.finish();
+        long tEnd = System.currentTimeMillis();
+        System.out.println(((tEnd-tMid) / 1000.0) + " sec to finish/pack");
         if (fst == null) {
           System.out.println("FST was fully pruned!");
           System.exit(0);
@@ -513,6 +518,12 @@ public class TestFSTs extends LuceneTest
           return;
         }
 
+        /*
+        IndexInput in = dir.openInput("fst.bin", IOContext.DEFAULT);
+        fst = new FST<T>(in, outputs);
+        in.close();
+        */
+
         System.out.println("\nNow verify...");
 
         while(true) {
@@ -576,7 +587,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
+  // java -cp ../build/codecs/classes/java:../test-framework/lib/randomizedtesting-runner-2.0.8.jar:../build/core/classes/test:../build/core/classes/test-framework:../build/core/classes/java:../build/test-framework/classes/java:../test-framework/lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /xold/tmp/allTerms3.txt out
   public static void main(String[] args) throws IOException {
     int prune = 0;
     int limit = Integer.MAX_VALUE;
@@ -1022,7 +1033,7 @@ public class TestFSTs extends LuceneTest
         throws IOException {
         if (FST.targetHasArcs(arc)) {
           int childCount = 0;
-          FST.BytesReader fstReader = fst.getBytesReader(0);
+          BytesReader fstReader = fst.getBytesReader(0);
           for (arc = fst.readFirstTargetArc(arc, arc, fstReader);; 
                arc = fst.readNextArc(arc, fstReader), childCount++)
           {
@@ -1062,7 +1073,7 @@ public class TestFSTs extends LuceneTest
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
 
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
     builder.add(Util.toUTF32("station", new IntsRef()), 10L);
     final FST<Long> fst = builder.finish();
@@ -1077,7 +1088,7 @@ public class TestFSTs extends LuceneTest
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
     final boolean willRewrite = random().nextBoolean();
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();
@@ -1100,7 +1111,7 @@ public class TestFSTs extends LuceneTest
     final Long nothing = outputs.getNoOutput();
     final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
-    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true);
+    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true, 15);
 
     final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);
 

Modified: lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java (original)
+++ lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java Mon Jan 14 18:54:22 2013
@@ -46,7 +46,7 @@ public class SearchFiles {
   /** Simple command-line based search demo. */
   public static void main(String[] args) throws Exception {
     String usage =
-      "Usage:\tjava org.apache.lucene.demo.SearchFiles [-index dir] [-field f] [-repeat n] [-queries file] [-query string] [-raw] [-paging hitsPerPage]\n\nSee http://lucene.apache.org/java/4_0/demo.html for details.";
+      "Usage:\tjava org.apache.lucene.demo.SearchFiles [-index dir] [-field f] [-repeat n] [-queries file] [-query string] [-raw] [-paging hitsPerPage]\n\nSee http://lucene.apache.org/core/4_1_0/demo/ for details.";
     if (args.length > 0 && ("-h".equals(args[0]) || "-help".equals(args[0]))) {
       System.out.println(usage);
       System.exit(0);

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -3,7 +3,7 @@ package org.apache.lucene.facet.associat
 import java.io.IOException;
 
 import org.apache.lucene.facet.search.PayloadIterator;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.BytesRef;
@@ -46,13 +46,22 @@ public abstract class AssociationsPayloa
    * It is assumed that all association values can be deserialized with the
    * given {@link CategoryAssociation}.
    */
-  public AssociationsPayloadIterator(IndexReader reader, String field, T association) throws IOException {
-    pi = new PayloadIterator(reader, new Term(field, association.getCategoryListID()));
-    hasAssociations = pi.init();
+  public AssociationsPayloadIterator(String field, T association) throws IOException {
+    pi = new PayloadIterator(new Term(field, association.getCategoryListID()));
     this.association = association;
   }
 
   /**
+   * Sets the {@link AtomicReaderContext} for which {@link #setNextDoc(int)}
+   * calls will be made. Returns true iff this reader has associations for any
+   * of the documents belonging to the association given to the constructor.
+   */
+  public final boolean setNextReader(AtomicReaderContext context) throws IOException {
+    hasAssociations = pi.setNextReader(context);
+    return hasAssociations;
+  }
+  
+  /**
    * Skip to the requested document. Returns true iff the document has category
    * association values and they were read successfully. Associations are
    * handled through {@link #handleAssociation(int, CategoryAssociation)} by

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.collections.IntToFloatMap;
 
 /*
@@ -31,9 +30,8 @@ public class FloatAssociationsPayloadIte
 
   private final IntToFloatMap ordinalAssociations = new IntToFloatMap();
 
-  public FloatAssociationsPayloadIterator(IndexReader reader, String field, CategoryFloatAssociation association) 
-      throws IOException {
-    super(reader, field, association);
+  public FloatAssociationsPayloadIterator(String field, CategoryFloatAssociation association) throws IOException {
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.collections.IntToIntMap;
 
 /*
@@ -31,9 +30,8 @@ public class IntAssociationsPayloadItera
 
   private final IntToIntMap ordinalAssociations = new IntToIntMap();
 
-  public IntAssociationsPayloadIterator(IndexReader reader, String field, CategoryIntAssociation association) 
-      throws IOException {
-    super(reader, field, association);
+  public IntAssociationsPayloadIterator(String field, CategoryIntAssociation association) throws IOException {
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java Mon Jan 14 18:54:22 2013
@@ -3,13 +3,10 @@ package org.apache.lucene.facet.index.pa
 import java.io.IOException;
 import java.io.Serializable;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-
 import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
-import org.apache.lucene.facet.search.TotalFacetCounts;
 import org.apache.lucene.facet.util.PartitionsUtils;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.util.encoding.DGapIntEncoder;
 import org.apache.lucene.util.encoding.IntDecoder;
 import org.apache.lucene.util.encoding.IntEncoder;
@@ -98,11 +95,6 @@ public class CategoryListParams implemen
     return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
   }
 
-  /**
-   * Equality is defined by the 'term' that defines this category list.  
-   * Sub-classes should override this method if a more complex calculation
-   * is needed to ensure equality. 
-   */
   @Override
   public boolean equals(Object o) {
     if (o == this) {
@@ -121,29 +113,16 @@ public class CategoryListParams implemen
     return this.term.equals(other.term);
   }
 
-  /**
-   * Hashcode is similar to {@link #equals(Object)}, in that it uses
-   * the term that defines this category list to derive the hashcode.
-   * Subclasses need to ensure that equality/hashcode is correctly defined,
-   * or there could be side-effects in the {@link TotalFacetCounts} caching 
-   * mechanism (as the filename for a Total Facet Counts array cache 
-   * is dependent on the hashCode, so it should consistently return the same
-   * hash for identity).
-   */
   @Override
   public int hashCode() {
     return this.hashCode;
   }
 
-  /**
-   * Create the category list iterator for the specified partition.
-   */
-  public CategoryListIterator createCategoryListIterator(IndexReader reader,
-      int partition) throws IOException {
+  /** Create the {@link CategoryListIterator} for the specified partition. */
+  public CategoryListIterator createCategoryListIterator(int partition) throws IOException {
     String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
     Term payloadTerm = new Term(term.field(), categoryListTermStr);
-    return new PayloadCategoryListIteraor(reader, payloadTerm,
-        createEncoder().createMatchingDecoder());
+    return new PayloadCategoryListIteraor(payloadTerm, createEncoder().createMatchingDecoder());
   }
   
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java Mon Jan 14 18:54:22 2013
@@ -50,7 +50,7 @@ public final class AdaptiveFacetsAccumul
    * Create an {@link AdaptiveFacetsAccumulator} 
    * @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader)
    */
-  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
+  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, 
       TaxonomyReader taxonomyReader) {
     super(searchParams, indexReader, taxonomyReader);
   }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java Mon Jan 14 18:54:22 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -23,6 +24,8 @@ import org.apache.lucene.util.IntsRef;
 
 /**
  * An interface for obtaining the category ordinals of documents.
+ * {@link #getOrdinals(int, IntsRef)} calls are done with document IDs that are
+ * local to the reader given to {@link #setNextReader(AtomicReaderContext)}.
  * <p>
  * <b>NOTE:</b> this class operates as a key to a map, and therefore you should
  * implement {@code equals()} and {@code hashCode()} for proper behavior.
@@ -32,19 +35,20 @@ import org.apache.lucene.util.IntsRef;
 public interface CategoryListIterator {
 
   /**
-   * Initializes the iterator. This method must be called before any calls to
-   * {@link #getOrdinals(int, IntsRef)}, and its return value indicates whether there are
-   * any relevant documents for this iterator.
+   * Sets the {@link AtomicReaderContext} for which
+   * {@link #getOrdinals(int, IntsRef)} calls will be made. Returns true iff any
+   * of the documents in this reader have category ordinals. This method must be
+   * called before any calls to {@link #getOrdinals(int, IntsRef)}.
    */
-  public boolean init() throws IOException;
-
+  public boolean setNextReader(AtomicReaderContext context) throws IOException;
+  
   /**
    * Stores the category ordinals of the given document ID in the given
    * {@link IntsRef}, starting at position 0 upto {@link IntsRef#length}. Grows
    * the {@link IntsRef} if it is not large enough.
    * 
    * <p>
-   * <b>NOTE:</b> if the requested document does not category ordinals
+   * <b>NOTE:</b> if the requested document does not have category ordinals
    * associated with it, {@link IntsRef#length} is set to zero.
    */
   public void getOrdinals(int docID, IntsRef ints) throws IOException;

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -34,17 +34,15 @@ import org.apache.lucene.util.encoding.I
 public class PayloadCategoryListIteraor implements CategoryListIterator {
 
   private final IntDecoder decoder;
-  private final IndexReader indexReader;
   private final Term term;
   private final PayloadIterator pi;
   private final int hashCode;
   
-  public PayloadCategoryListIteraor(IndexReader indexReader, Term term, IntDecoder decoder) throws IOException {
-    pi = new PayloadIterator(indexReader, term);
+  public PayloadCategoryListIteraor(Term term, IntDecoder decoder) throws IOException {
+    pi = new PayloadIterator(term);
     this.decoder = decoder;
-    hashCode = indexReader.hashCode() ^ term.hashCode();
+    hashCode = term.hashCode();
     this.term = term;
-    this.indexReader = indexReader;
   }
 
   @Override
@@ -58,7 +56,7 @@ public class PayloadCategoryListIteraor 
     }
     
     // Hash codes are the same, check equals() to avoid cases of hash-collisions.
-    return indexReader.equals(that.indexReader) && term.equals(that.term);
+    return term.equals(that.term);
   }
 
   @Override
@@ -67,8 +65,8 @@ public class PayloadCategoryListIteraor 
   }
 
   @Override
-  public boolean init() throws IOException {
-    return pi.init();
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return pi.setNextReader(context);
   }
   
   @Override