You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/01/18 19:31:23 UTC

svn commit: r1435287 [10/41] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/analysis/icu/ dev-tools/maven/ dev-tools/maven/lucene/benchmark/ dev-tools/maven/solr/ dev-tools/...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Fri Jan 18 18:30:54 2013
@@ -27,12 +27,19 @@ 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;
 import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -50,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!!
@@ -64,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
  */
@@ -92,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;
 
   /**
@@ -124,36 +128,41 @@ 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;
-  private byte[] emptyOutputBytes;
 
-  // Not private to avoid synthetic access$NNN methods:
-  byte[] bytes;
-  int byteUpto = 0;
+  final BytesStore bytes;
 
-  private int startNode = -1;
+  private long startNode = -1;
 
   public final Outputs<T> outputs;
 
-  private int lastFrozenNode;
+  // Used for the BIT_TARGET_NEXT optimization (whereby
+  // 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 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;
@@ -161,7 +170,7 @@ public final class FST<T> {
   /** If arc has this label then that arc is final/accepted */
   public static final int END_LABEL = -1;
 
-  private boolean allowArrayArcs = true;
+  private final boolean allowArrayArcs;
 
   private Arc<T> cachedRootArcs[];
 
@@ -172,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;
@@ -251,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
@@ -260,22 +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) {
+  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs, int bytesPageBits) {
     this.inputType = inputType;
     this.outputs = outputs;
-    bytes = new byte[128];
+    this.allowArrayArcs = allowArrayArcs;
+    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 BytesWriter();
 
     emptyOutput = null;
     packed = false;
@@ -285,22 +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();
-      // messy
-      bytes = new byte[numBytes];
-      in.readBytes(bytes, 0, numBytes);
+      emptyBytes.copyBytes(in, numBytes);
+
+      // De-serialize empty-string output:
       BytesReader reader;
       if (packed) {
-        reader = getBytesReader(0);
+        reader = emptyBytes.getForwardReader();
       } else {
-        reader = getBytesReader(numBytes-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 {
@@ -326,15 +346,30 @@ 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();
+
+    // NOTE: bogus because this is only used during
+    // 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() {
@@ -342,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) {
@@ -353,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.posWrite];
-    System.arraycopy(bytes, 0, finalBytes, 0, writer.posWrite);
-    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;
@@ -384,7 +417,7 @@ public final class FST<T> {
     cachedRootArcs = (Arc<T>[]) new Arc[0x80];
     final Arc<T> arc = new Arc<T>();
     getFirstArc(arc);
-    final BytesReader in = getBytesReader(0);
+    final BytesReader in = getBytesReader();
     if (targetHasArcs(arc)) {
       readFirstRealTargetArc(arc.target, arc, in);
       while(true) {
@@ -412,26 +445,6 @@ public final class FST<T> {
     } else {
       emptyOutput = v;
     }
-
-    // TODO: this is messy -- replace with sillyBytesWriter; maybe make
-    // bytes private
-    final int posSave = writer.posWrite;
-    outputs.writeFinalOutput(emptyOutput, writer);
-    emptyOutputBytes = new byte[writer.posWrite-posSave];
-
-    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;
   }
 
   public void save(DataOutput out) throws IOException {
@@ -453,7 +466,27 @@ public final class FST<T> {
     // TODO: really we should encode this as an arc, arriving
     // to the root node, instead of special casing here:
     if (emptyOutput != null) {
+      // Accepts empty string
       out.writeByte((byte) 1);
+
+      // Serialize empty-string output:
+      RAMOutputStream ros = new RAMOutputStream();
+      outputs.writeFinalOutput(emptyOutput, ros);
+      
+      byte[] emptyOutputBytes = new byte[(int) ros.getFilePointer()];
+      ros.writeTo(emptyOutputBytes, 0);
+
+      if (!packed) {
+        // reverse
+        final int stopAt = emptyOutputBytes.length/2;
+        int upto = 0;
+        while(upto < stopAt) {
+          final byte b = emptyOutputBytes[upto];
+          emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
+          emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
+          upto++;
+        }
+      }
       out.writeVInt(emptyOutputBytes.length);
       out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
     } else {
@@ -471,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);
   }
   
   /**
@@ -516,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);
     }
   }
 
@@ -552,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;
@@ -562,38 +596,28 @@ public final class FST<T> {
       }
     }
 
-    int startAddress = writer.posWrite;
+    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.posWrite;
-      //System.out.println("  do fixed arcs array arcsStart=" + fixedArrayStart);
-    } else {
-      fixedArrayStart = 0;
     }
 
     arcCount += nodeIn.numArcs;
     
     final int lastArc = nodeIn.numArcs-1;
 
-    int lastArcStart = writer.posWrite;
+    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;
@@ -620,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.posWrite - lastArcStart;
-        lastArcStart = writer.posWrite;
+        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.posWrite;
-      int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
-      writer.posWrite = 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.posWrite - 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;
   }
 
@@ -753,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");
@@ -764,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;
@@ -794,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();
@@ -813,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
@@ -843,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);
@@ -856,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;
@@ -879,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;
     }
   }
@@ -907,29 +962,36 @@ public final class FST<T> {
     assert !arc.isLast();
 
     if (arc.label == END_LABEL) {
-      //System.out.println("    nextArc fake " + arc.nextArc);
-      in.pos = getNodeAddress(arc.nextArc);
-      final byte b = bytes[in.pos];
+      //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");
-        in.skip(1);
+        //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.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
@@ -940,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);
@@ -950,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);
@@ -976,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) {
@@ -987,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;
   }
@@ -1024,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()) {
@@ -1059,7 +1121,7 @@ public final class FST<T> {
       return null;
     }
 
-    in.pos = getNodeAddress(follow.target);
+    in.setPosition(getNodeAddress(follow.target));
 
     arc.node = follow.target;
 
@@ -1068,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) {
@@ -1134,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);
         }
       }
 
@@ -1146,23 +1209,19 @@ 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;
   }
 
-  public void setAllowArrayArcs(boolean v) {
-    allowArrayArcs = v;
-  }
-  
   /**
    * Nodes will be expanded if their depth (distance from the root node) is
    * &lt;= this value and their number of arcs is &gt;=
@@ -1183,131 +1242,33 @@ public final class FST<T> {
       ((node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) || 
        node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
   }
-
-  // Non-static: writes to FST's byte[]
-  class BytesWriter extends DataOutput {
-    int posWrite;
-
-    public BytesWriter() {
-      // 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;
-    }
-
-    public void setPosWrite(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() {
-    return getBytesReader(0);
-  }
-
-  /** Returns a {@link BytesReader} for this FST, positioned at
-   *  the provided position. */
-  public BytesReader getBytesReader(int 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);
+      in = bytes.getReverseReader();
     }
+    return in;
   }
 
-  /** 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. */
+  /** Reads bytes stored in an FST. */
   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);
-    }
+    /** Get current read position. */
+    public abstract long getPosition();
 
-    @Override
-    public byte readByte() {
-      return bytes[pos--];
-    }
+    /** Set current read position. */
+    public abstract void setPosition(long pos);
 
-    @Override
-    public void readBytes(byte[] b, int offset, int len) {
-      for(int i=0;i<len;i++) {
-        b[offset+i] = bytes[pos--];
-      }
-    }
-
-    public void skip(int count) {
-      pos -= count;
-    }
+    /** Returns true if this reader uses reversed bytes
+     *  under-the-hood. */
+    public abstract boolean reversed();
 
-    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 {
-
-    public ForwardBytesReader(byte[] bytes, int pos) {
-      super(bytes, 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;
-    }
+    /** Skips bytes. */
+    public abstract void skipBytes(int count);
   }
 
   private static class ArcAndState<T> {
@@ -1429,14 +1390,18 @@ 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 BytesWriter();
+    
+    // NOTE: bogus because this is only used during
+    // building; we need to break out mutable FST from
+    // immutable
+    allowArrayArcs = false;
   }
 
   /** Expert: creates an FST by packing this one.  This
@@ -1451,7 +1416,10 @@ public final class FST<T> {
    *  However, this is not a strict implementation of the
    *  algorithms described in this paper.
    */
-  public FST<T> pack(int minInCountDeref, int maxDerefNodes, float acceptableOverheadRatio) throws IOException {
+  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?
@@ -1470,7 +1438,7 @@ public final class FST<T> {
 
     Arc<T> arc = new Arc<T>();
 
-    final BytesReader r = getBytesReader(0);
+    final BytesReader r = getBytesReader();
 
     final int topN = Math.min(maxDerefNodes, inCounts.size());
 
@@ -1502,17 +1470,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;
@@ -1520,6 +1484,8 @@ public final class FST<T> {
     int topCount;
     int nextCount;
 
+    FST<T> fst;
+
     // Iterate until we converge:
     while(true) {
 
@@ -1529,7 +1495,10 @@ public final class FST<T> {
       // for assert:
       boolean negDelta = false;
 
-      writer.posWrite = 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);
 
@@ -1541,19 +1510,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.posWrite;
+        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);
@@ -1573,6 +1543,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;
@@ -1590,9 +1561,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.posWrite;
+            final long arcStartPos = writer.getPosition();
             nodeArcCount++;
 
             byte flags = 0;
@@ -1627,19 +1598,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.posWrite - 2;
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition() - 2;
               if (delta < 0) {
                 //System.out.println("neg: " + delta);
                 anyNegDelta = true;
@@ -1650,12 +1620,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);
@@ -1669,7 +1640,7 @@ public final class FST<T> {
 
             if (doWriteTarget) {
 
-              int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.posWrite;
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition();
               if (delta < 0) {
                 anyNegDelta = true;
                 //System.out.println("neg: " + delta);
@@ -1678,7 +1649,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++;
                 }
@@ -1690,7 +1661,7 @@ public final class FST<T> {
                   System.out.println("        abs");
                 }
                 */
-                writer.writeVInt(absPtr);
+                writer.writeVLong(absPtr);
                 if (!retry) {
                   if (absPtr >= topNodeMap.size()) {
                     absCount++;
@@ -1702,7 +1673,7 @@ public final class FST<T> {
             }
 
             if (useArcArray) {
-              final int arcBytes = writer.posWrite - 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
@@ -1712,7 +1683,7 @@ public final class FST<T> {
               // will retry (below) so it's OK to ovewrite
               // bytes:
               //wasted += bytesPerArc - arcBytes;
-              writer.setPosWrite(arcStartPos + bytesPerArc);
+              writer.skipBytes((int) (arcStartPos + bytesPerArc - writer.getPosition()));
             }
 
             if (arc.isLast()) {
@@ -1737,11 +1708,12 @@ public final class FST<T> {
 
           // Retry:
           bytesPerArc = maxBytesPerArc;
-          writer.posWrite = address;
+          writer.truncate(address);
           nodeArcCount = 0;
           retry = true;
           anyNegDelta = false;
         }
+
         negDelta |= anyNegDelta;
 
         fst.arcCount += nodeArcCount;
@@ -1761,8 +1733,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(),
@@ -1772,8 +1744,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) {
@@ -1783,11 +1754,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.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.bytes.finish();
     fst.cacheRootArcs();
 
     //final int size = fst.sizeInBytes();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java Fri Jan 18 18:30:54 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
@@ -46,7 +46,7 @@ abstract class FSTEnum<T> {
    *  term before target.  */
   protected FSTEnum(FST<T> fst) {
     this.fst = fst;
-    fstReader = fst.getBytesReader(0);
+    fstReader = fst.getBytesReader();
     NO_OUTPUT = fst.outputs.getNoOutput();
     fst.getFirstArc(getArc(0));
     output[0] = NO_OUTPUT;
@@ -145,7 +145,7 @@ abstract class FSTEnum<T> {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
-        final FST.BytesReader in = fst.getBytesReader(0);
+        final FST.BytesReader in = fst.getBytesReader();
         int low = arc.arcIdx;
         int high = arc.numArcs-1;
         int mid = 0;
@@ -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);
@@ -284,7 +284,7 @@ abstract class FSTEnum<T> {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
-        final FST.BytesReader in = fst.getBytesReader(0);
+        final FST.BytesReader in = fst.getBytesReader();
         int low = arc.arcIdx;
         int high = arc.numArcs-1;
         int mid = 0;
@@ -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);
@@ -434,7 +434,7 @@ abstract class FSTEnum<T> {
     FST.Arc<T> arc = getArc(upto-1);
     int targetLabel = getTargetLabel();
 
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
 
     while(true) {
       //System.out.println("  cycle target=" + (targetLabel == -1 ? "-1" : (char) targetLabel));

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Fri Jan 18 18:30:54 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/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Fri Jan 18 18:30:54 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();
 
     // 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();
 
     // 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();
 
     // 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;
 
@@ -285,7 +285,7 @@ public final class Util {
 
     public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator) {
       this.fst = fst;
-      this.bytesReader = fst.getBytesReader(0);
+      this.bytesReader = fst.getBytesReader();
       this.topN = topN;
       this.maxQueueDepth = maxQueueDepth;
       this.comparator = comparator;
@@ -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();
       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();
 
     // 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/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValue.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValue.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/mutable/MutableValue.java Fri Jan 18 18:30:54 2013
@@ -34,6 +34,7 @@ public abstract class MutableValue imple
     return exists;
   }
 
+  @Override
   public int compareTo(MutableValue other) {
     Class<? extends MutableValue> c1 = this.getClass();
     Class<? extends MutableValue> c2 = other.getClass();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java Fri Jan 18 18:30:54 2013
@@ -54,14 +54,17 @@ final class Direct16 extends PackedInts.
     return values[index] & 0xFFFFL;
   }
 
+  @Override
   public void set(final int index, final long value) {
     values[index] = (short) (value);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(values);
   }
 
+  @Override
   public void clear() {
     Arrays.fill(values, (short) 0L);
   }
@@ -89,6 +92,7 @@ final class Direct16 extends PackedInts.
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java Fri Jan 18 18:30:54 2013
@@ -54,14 +54,17 @@ final class Direct32 extends PackedInts.
     return values[index] & 0xFFFFFFFFL;
   }
 
+  @Override
   public void set(final int index, final long value) {
     values[index] = (int) (value);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(values);
   }
 
+  @Override
   public void clear() {
     Arrays.fill(values, (int) 0L);
   }
@@ -89,6 +92,7 @@ final class Direct32 extends PackedInts.
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java Fri Jan 18 18:30:54 2013
@@ -49,14 +49,17 @@ final class Direct64 extends PackedInts.
     return values[index];
   }
 
+  @Override
   public void set(final int index, final long value) {
     values[index] = (value);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(values);
   }
 
+  @Override
   public void clear() {
     Arrays.fill(values, 0L);
   }
@@ -82,6 +85,7 @@ final class Direct64 extends PackedInts.
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java Fri Jan 18 18:30:54 2013
@@ -52,14 +52,17 @@ final class Direct8 extends PackedInts.M
     return values[index] & 0xFFL;
   }
 
+  @Override
   public void set(final int index, final long value) {
     values[index] = (byte) (value);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(values);
   }
 
+  @Override
   public void clear() {
     Arrays.fill(values, (byte) 0L);
   }
@@ -87,6 +90,7 @@ final class Direct8 extends PackedInts.M
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java Fri Jan 18 18:30:54 2013
@@ -40,14 +40,17 @@ public class GrowableWriter implements P
     currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
   }
 
+  @Override
   public long get(int index) {
     return current.get(index);
   }
 
+  @Override
   public int size() {
     return current.size();
   }
 
+  @Override
   public int getBitsPerValue() {
     return current.getBitsPerValue();
   }
@@ -79,11 +82,13 @@ public class GrowableWriter implements P
     currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
   }
 
+  @Override
   public void set(int index, long value) {
     ensureCapacity(value);
     current.set(index, value);
   }
 
+  @Override
   public void clear() {
     current.clear();
   }
@@ -95,6 +100,7 @@ public class GrowableWriter implements P
     return next;
   }
 
+  @Override
   public int get(int index, long[] arr, int off, int len) {
     return current.get(index, arr, off, len);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java Fri Jan 18 18:30:54 2013
@@ -114,6 +114,7 @@ final class Packed16ThreeBlocks extends 
     Arrays.fill(blocks, (short) 0);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(blocks);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Fri Jan 18 18:30:54 2013
@@ -58,6 +58,7 @@ abstract class Packed64SingleBlock exten
     Arrays.fill(blocks, 0L);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(blocks);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java Fri Jan 18 18:30:54 2013
@@ -112,6 +112,7 @@ final class Packed8ThreeBlocks extends P
     Arrays.fill(blocks, (byte) 0);
   }
 
+  @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.sizeOf(blocks);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Fri Jan 18 18:30:54 2013
@@ -576,22 +576,27 @@ public class PackedInts {
       this.valueCount = valueCount;
     }
 
+    @Override
     public int getBitsPerValue() {
       return bitsPerValue;
     }
 
+    @Override
     public int size() {
       return valueCount;
     }
 
+    @Override
     public Object getArray() {
       return null;
     }
 
+    @Override
     public boolean hasArray() {
       return false;
     }
 
+    @Override
     public int get(int index, long[] arr, int off, int len) {
       assert len > 0 : "len must be > 0 (got " + len + ")";
       assert index >= 0 && index < valueCount;
@@ -612,6 +617,7 @@ public class PackedInts {
       super(valueCount, bitsPerValue);
     }
 
+    @Override
     public int set(int index, long[] arr, int off, int len) {
       assert len > 0 : "len must be > 0 (got " + len + ")";
       assert index >= 0 && index < valueCount;
@@ -624,6 +630,7 @@ public class PackedInts {
       return len;
     }
 
+    @Override
     public void fill(int fromIndex, int toIndex, long val) {
       assert val <= maxValue(bitsPerValue);
       assert fromIndex <= toIndex;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java Fri Jan 18 18:30:54 2013
@@ -224,8 +224,11 @@ public class TestCharTermAttributeImpl e
     // finally use a completely custom CharSequence that is not catched by instanceof checks
     final String longTestString = "012345678901234567890123456789";
     t.append(new CharSequence() {
+      @Override
       public char charAt(int i) { return longTestString.charAt(i); }
+      @Override
       public int length() { return longTestString.length(); }
+      @Override
       public CharSequence subSequence(int start, int end) { return longTestString.subSequence(start, end); }
       @Override
       public String toString() { return longTestString; }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java Fri Jan 18 18:30:54 2013
@@ -58,7 +58,7 @@ public class TestReuseDocsEnum extends L
       IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
       MatchNoBits bits = new Bits.MatchNoBits(indexReader.maxDoc());
       while ((iterator.next()) != null) {
-        DocsEnum docs = iterator.docs(random().nextBoolean() ? bits : new Bits.MatchNoBits(indexReader.maxDoc()), null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        DocsEnum docs = iterator.docs(random().nextBoolean() ? bits : new Bits.MatchNoBits(indexReader.maxDoc()), null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       
@@ -85,7 +85,7 @@ public class TestReuseDocsEnum extends L
       MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc());
       DocsEnum docs = null;
       while ((iterator.next()) != null) {
-        docs = iterator.docs(bits, docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(bits, docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       
@@ -94,7 +94,7 @@ public class TestReuseDocsEnum extends L
       iterator = terms.iterator(null);
       docs = null;
       while ((iterator.next()) != null) {
-        docs = iterator.docs(new Bits.MatchNoBits(open.maxDoc()), docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(new Bits.MatchNoBits(open.maxDoc()), docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       assertEquals(terms.size(), enums.size());
@@ -103,7 +103,7 @@ public class TestReuseDocsEnum extends L
       iterator = terms.iterator(null);
       docs = null;
       while ((iterator.next()) != null) {
-        docs = iterator.docs(null, docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(null, docs, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       assertEquals(1, enums.size());  
@@ -135,7 +135,7 @@ public class TestReuseDocsEnum extends L
       DocsEnum docs = null;
       BytesRef term = null;
       while ((term = iterator.next()) != null) {
-        docs = iterator.docs(null, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(null, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       assertEquals(terms.size(), enums.size());
@@ -144,7 +144,7 @@ public class TestReuseDocsEnum extends L
       enums.clear();
       docs = null;
       while ((term = iterator.next()) != null) {
-        docs = iterator.docs(bits, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(bits, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
         enums.put(docs, true);
       }
       assertEquals(terms.size(), enums.size());
@@ -163,7 +163,7 @@ public class TestReuseDocsEnum extends L
     }
     TermsEnum iterator = terms.iterator(null);
     if (iterator.seekExact(term, true)) {
-      return iterator.docs(bits, null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+      return iterator.docs(bits, null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
     }
     return null;
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java Fri Jan 18 18:30:54 2013
@@ -332,17 +332,17 @@ public class TestBlockPostingsFormat3 ex
                                 rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
         
         // with positions only
-        assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, 0),
-                                   rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, 0));
-        assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, 0),
-                                   rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, 0));
+        assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
+                                   rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
+        assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
+                                   rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
 
         assertPositionsSkipping(leftTermsEnum.docFreq(), 
-                                leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, 0),
-                                rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, 0));
+                                leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
+                                rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
         assertPositionsSkipping(leftTermsEnum.docFreq(), 
-                                leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, 0),
-                                rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, 0));
+                                leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
+                                rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
         
         // with freqs:
         assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs),
@@ -351,10 +351,10 @@ public class TestBlockPostingsFormat3 ex
             rightDocs = rightTermsEnum.docs(randomBits, rightDocs));
 
         // w/o freqs:
-        assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs, 0),
-            rightDocs = rightTermsEnum.docs(null, rightDocs, 0));
-        assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs, 0),
-            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, 0));
+        assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
+        assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
         
         // with freqs:
         assertDocsSkipping(leftTermsEnum.docFreq(), 
@@ -366,11 +366,11 @@ public class TestBlockPostingsFormat3 ex
 
         // w/o freqs:
         assertDocsSkipping(leftTermsEnum.docFreq(), 
-            leftDocs = leftTermsEnum.docs(null, leftDocs, 0),
-            rightDocs = rightTermsEnum.docs(null, rightDocs, 0));
+            leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
         assertDocsSkipping(leftTermsEnum.docFreq(), 
-            leftDocs = leftTermsEnum.docs(randomBits, leftDocs, 0),
-            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, 0));
+            leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
       }
     }
     assertNull(rightTermsEnum.next());

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Fri Jan 18 18:30:54 2013
@@ -20,13 +20,9 @@ package org.apache.lucene.document;
 import java.io.StringReader;
 import java.util.List;
 
-import org.apache.lucene.analysis.EmptyTokenizer;
-import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
@@ -318,7 +314,7 @@ public class TestDocument extends Lucene
   // LUCENE-3616
   public void testInvalidFields() {
     try {
-      new Field("foo", new EmptyTokenizer(new StringReader("")), StringField.TYPE_STORED);
+      new Field("foo", new MockTokenizer(new StringReader("")), StringField.TYPE_STORED);
       fail("did not hit expected exc");
     } catch (IllegalArgumentException iae) {
       // expected