You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/01/12 16:55:51 UTC

svn commit: r1432459 - in /lucene/dev/trunk/lucene: ./ analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/ core/src/java/org/apache/lucene/util/fst/ core/src/test/org/apache/lucene/util/fst/

Author: mikemccand
Date: Sat Jan 12 15:55:50 2013
New Revision: 1432459

URL: http://svn.apache.org/viewvc?rev=1432459&view=rev
Log:
LUCENE-4678: use paged byte[] under the hood for FST

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/TokenInfoFST.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sat Jan 12 15:55:50 2013
@@ -19,6 +19,10 @@ Changes in backwards compatibility polic
   (Nikola Tanković, Uwe Schindler, Chris Male, Mike McCandless,
   Robert Muir)
 
+* LUCENE-4678: FST now uses a paged byte[] structure instead of a
+  single byte[] internally, to avoid large memory spikes during
+  building (James Dyer, Mike McCandless)
+
 ======================= Lucene 4.1.0 =======================
 
 Changes in backwards compatibility policy

Modified: lucene/dev/trunk/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/TokenInfoFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/TokenInfoFST.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/TokenInfoFST.java (original)
+++ lucene/dev/trunk/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/TokenInfoFST.java Sat Jan 12 15:55:50 2013
@@ -19,8 +19,8 @@ package org.apache.lucene.analysis.ja.di
 
 import java.io.IOException;
 
-import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST;
 
 /**
  * Thin wrapper around an FST with root-arc caching for Japanese.
@@ -48,7 +48,7 @@ public final class TokenInfoFST {
     rootCache = cacheRootArcs();
   }
   
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({"rawtypes","unchecked"})
   private FST.Arc<Long>[] cacheRootArcs() throws IOException {
     FST.Arc<Long> rootCache[] = new FST.Arc[1+(cacheCeiling-0x3040)];
     FST.Arc<Long> firstArc = new FST.Arc<Long>();

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java?rev=1432459&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java Sat Jan 12 15:55:50 2013
@@ -0,0 +1,430 @@
+package org.apache.lucene.util.fst;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+// TODO: merge with PagedBytes, except PagedBytes doesn't
+// let you read while writing which FST needs
+
+class BytesStore extends DataOutput {
+
+  private final List<byte[]> blocks = new ArrayList<byte[]>();
+
+  private final int blockSize;
+  private final int blockBits;
+  private final int blockMask;
+
+  private byte[] current;
+  private int nextWrite;
+
+  public BytesStore(int blockBits) {
+    this.blockBits = blockBits;
+    blockSize = 1 << blockBits;
+    blockMask = blockSize-1;
+    nextWrite = blockSize;
+  }
+
+  /** Pulls bytes from the provided IndexInput.  */
+  public BytesStore(DataInput in, int numBytes, int maxBlockSize) throws IOException {
+    int blockSize = 2;
+    int blockBits = 1;
+    while(blockSize < numBytes && blockSize < maxBlockSize) {
+      blockSize *= 2;
+      blockBits++;
+    }
+    this.blockBits = blockBits;
+    this.blockSize = blockSize;
+    this.blockMask = blockSize-1;
+    int left = numBytes;
+    while(left > 0) {
+      final int chunk = Math.min(blockSize, left);
+      byte[] block = new byte[chunk];
+      in.readBytes(block, 0, block.length);
+      blocks.add(block);
+      left -= chunk;
+    }
+
+    // So .getPosition still works
+    nextWrite = blocks.get(blocks.size()-1).length;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    if (nextWrite == blockSize) {
+      current = new byte[blockSize];
+      blocks.add(current);
+      nextWrite = 0;
+    }
+    current[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    while (len > 0) {
+      int chunk = blockSize - nextWrite;
+      if (len <= chunk) {
+        System.arraycopy(b, offset, current, nextWrite, len);
+        nextWrite += len;
+        break;
+      } else {
+        if (chunk > 0) {
+          System.arraycopy(b, offset, current, nextWrite, chunk);
+          offset += chunk;
+          len -= chunk;
+        }
+        current = new byte[blockSize];
+        blocks.add(current);
+        nextWrite = 0;
+      }
+    }
+  }
+
+  /** Absolute writeBytes without changing the current
+   *  position.  Note: this cannot "grow" the bytes, so you
+   *  must only call it on already written parts. */
+  void writeBytes(int dest, byte[] b, int offset, int len) {
+    //System.out.println("  BS.writeBytes dest=" + dest + " offset=" + offset + " len=" + len);
+    assert dest + len <= getPosition(): "dest=" + dest + " pos=" + getPosition() + " len=" + len;
+
+    // Note: weird: must go "backwards" because copyBytes
+    // calls us with overlapping src/dest.  If we
+    // go forwards then we overwrite bytes before we can
+    // copy them:
+
+    /*
+    int blockIndex = dest >> blockBits;
+    int upto = dest & blockMask;
+    byte[] block = blocks.get(blockIndex);
+    while (len > 0) {
+      int chunk = blockSize - upto;
+      System.out.println("    cycle chunk=" + chunk + " len=" + len);
+      if (len <= chunk) {
+        System.arraycopy(b, offset, block, upto, len);
+        break;
+      } else {
+        System.arraycopy(b, offset, block, upto, chunk);
+        offset += chunk;
+        len -= chunk;
+        blockIndex++;
+        block = blocks.get(blockIndex);
+        upto = 0;
+      }
+    }
+    */
+
+    final int end = dest + len;
+    int blockIndex = end >> blockBits;
+    int downTo = end & blockMask;
+    if (downTo == 0) {
+      blockIndex--;
+      downTo = blockSize;
+    }
+    byte[] block = blocks.get(blockIndex);
+
+    while (len > 0) {
+      //System.out.println("    cycle downTo=" + downTo + " len=" + len);
+      if (len <= downTo) {
+        //System.out.println("      final: offset=" + offset + " len=" + len + " dest=" + (downTo-len));
+        System.arraycopy(b, offset, block, downTo-len, len);
+        break;
+      } else {
+        len -= downTo;
+        //System.out.println("      partial: offset=" + (offset + len) + " len=" + downTo + " dest=0");
+        System.arraycopy(b, offset + len, block, 0, downTo);
+        blockIndex--;
+        block = blocks.get(blockIndex);
+        downTo = blockSize;
+      }
+    }
+  }
+
+  /** Absolute copy bytes self to self, without changing the
+   *  position. Note: this cannot "grow" the bytes, so must
+   *  only call it on already written parts. */
+  public void copyBytes(int src, int dest, int len) {
+    //System.out.println("BS.copyBytes src=" + src + " dest=" + dest + " len=" + len);
+    assert src < dest;
+
+    // Note: weird: must go "backwards" because copyBytes
+    // calls us with overlapping src/dest.  If we
+    // go forwards then we overwrite bytes before we can
+    // copy them:
+
+    /*
+    int blockIndex = src >> blockBits;
+    int upto = src & blockMask;
+    byte[] block = blocks.get(blockIndex);
+    while (len > 0) {
+      int chunk = blockSize - upto;
+      System.out.println("  cycle: chunk=" + chunk + " len=" + len);
+      if (len <= chunk) {
+        writeBytes(dest, block, upto, len);
+        break;
+      } else {
+        writeBytes(dest, block, upto, chunk);
+        blockIndex++;
+        block = blocks.get(blockIndex);
+        upto = 0;
+        len -= chunk;
+        dest += chunk;
+      }
+    }
+    */
+
+    int end = src + len;
+
+    int blockIndex = end >> blockBits;
+    int downTo = end & blockMask;
+    if (downTo == 0) {
+      blockIndex--;
+      downTo = blockSize;
+    }
+    byte[] block = blocks.get(blockIndex);
+
+    while (len > 0) {
+      //System.out.println("  cycle downTo=" + downTo);
+      if (len <= downTo) {
+        //System.out.println("    finish");
+        writeBytes(dest, block, downTo-len, len);
+        break;
+      } else {
+        //System.out.println("    partial");
+        len -= downTo;
+        writeBytes(dest + len, block, 0, downTo);
+        blockIndex--;
+        block = blocks.get(blockIndex);
+        downTo = blockSize;
+      }
+    }
+  }
+
+  /** Writes an int at the absolute position without
+   *  changing the current pointer. */
+  public void writeInt(int pos, int value) {
+    int blockIndex = pos >> blockBits;
+    int upto = pos & blockMask;
+    byte[] block = blocks.get(blockIndex);
+    int shift = 24;
+    for(int i=0;i<4;i++) {
+      block[upto++] = (byte) (value >> shift);
+      shift -= 8;
+      if (upto == blockSize) {
+        upto = 0;
+        blockIndex++;
+        block = blocks.get(blockIndex);
+      }
+    }
+  }
+
+  /** Reverse the last numBytes. */
+  public void reverse(int srcPos, int destPos) {
+    assert srcPos < destPos;
+    //System.out.println("reverse src=" + srcPos + " dest=" + destPos);
+
+    int srcBlockIndex = srcPos >> blockBits;
+    int src = srcPos & blockMask;
+    byte[] srcBlock = blocks.get(srcBlockIndex);
+
+    int destBlockIndex = destPos >> blockBits;
+    int dest = destPos & blockMask;
+    byte[] destBlock = blocks.get(destBlockIndex);
+    //System.out.println("  srcBlock=" + srcBlockIndex + " destBlock=" + destBlockIndex);
+
+    int limit = (destPos - srcPos + 1)/2;
+    for(int i=0;i<limit;i++) {
+      //System.out.println("  cycle src=" + src + " dest=" + dest);
+      byte b = srcBlock[src];
+      srcBlock[src] = destBlock[dest];
+      destBlock[dest] = b;
+      src++;
+      if (src == blockSize) {
+        srcBlockIndex++;
+        srcBlock = blocks.get(srcBlockIndex);
+        //System.out.println("  set destBlock=" + destBlock + " srcBlock=" + srcBlock);
+        src = 0;
+      }
+
+      dest--;
+      if (dest == -1) {
+        destBlockIndex--;
+        destBlock = blocks.get(destBlockIndex);
+        //System.out.println("  set destBlock=" + destBlock + " srcBlock=" + srcBlock);
+        dest = blockSize-1;
+      }
+    }
+  }
+
+  public void skip(int len) {
+    while (len > 0) {
+      int chunk = blockSize - nextWrite;
+      if (len <= chunk) {
+        nextWrite += len;
+        break;
+      } else {
+        len -= chunk;
+        current = new byte[blockSize];
+        blocks.add(current);
+        nextWrite = 0;
+      }
+    }
+  }
+
+  public int getPosition() {
+    return (blocks.size()-1) * blockSize + nextWrite;
+  }
+
+  public void finish() {
+    if (current != null) {
+      byte[] lastBuffer = new byte[nextWrite];
+      System.arraycopy(current, 0, lastBuffer, 0, nextWrite);
+      blocks.set(blocks.size()-1, lastBuffer);
+      current = null;
+    }
+  }
+
+  public void writeTo(DataOutput out) throws IOException {
+    for(byte[] block : blocks) {
+      out.writeBytes(block, 0, block.length);
+    }
+  }
+
+  public FST.BytesReader getForwardReader() {
+    if (blocks.size() == 1) {
+      return new ForwardBytesReader(blocks.get(0));
+    }
+    return new FST.BytesReader() {
+      private byte[] current;
+      private int nextBuffer;
+      private int nextRead = blockSize;
+
+      @Override
+      public byte readByte() {
+        if (nextRead == blockSize) {
+          current = blocks.get(nextBuffer++);
+          nextRead = 0;
+        }
+        return current[nextRead++];
+      }
+
+      @Override
+      public void skipBytes(int count) {
+        setPosition(getPosition() + count);
+      }
+
+      @Override
+      public void readBytes(byte[] b, int offset, int len) {
+        while(len > 0) {
+          int chunkLeft = blockSize - nextRead;
+          if (len <= chunkLeft) {
+            System.arraycopy(current, nextRead, b, offset, len);
+            nextRead += len;
+            break;
+          } else {
+            if (chunkLeft > 0) {
+              System.arraycopy(current, nextRead, b, offset, chunkLeft);
+              offset += chunkLeft;
+              len -= chunkLeft;
+            }
+            current = blocks.get(nextBuffer++);
+            nextRead = 0;
+          }
+        }
+      }
+
+      @Override
+      public int getPosition() {
+        return (nextBuffer-1)*blockSize + nextRead;
+      }
+
+      @Override
+      public void setPosition(int pos) {
+        int bufferIndex = pos >> blockBits;
+        nextBuffer = bufferIndex+1;
+        current = blocks.get(bufferIndex);
+        nextRead = pos & blockMask;
+        assert getPosition() == pos;
+      }
+
+      @Override
+      public boolean reversed() {
+        return false;
+      }
+    };
+  }
+
+  public FST.BytesReader getReverseReader() {
+    if (blocks.size() == 1) {
+      return new ReverseBytesReader(blocks.get(0));
+    }
+    return new FST.BytesReader() {
+      private byte[] current = blocks.size() == 0 ? null : blocks.get(0);
+      private int nextBuffer = -1;
+      private int nextRead = 0;
+
+      @Override
+      public byte readByte() {
+        if (nextRead == -1) {
+          current = blocks.get(nextBuffer--);
+          nextRead = blockSize-1;
+        }
+        return current[nextRead--];
+      }
+
+      @Override
+      public void skipBytes(int count) {
+        setPosition(getPosition() - count);
+      }
+
+      @Override
+      public void readBytes(byte[] b, int offset, int len) {
+        for(int i=0;i<len;i++) {
+          b[offset+i] = readByte();
+        }
+      }
+
+      @Override
+      public int getPosition() {
+        return (nextBuffer+1)*blockSize + nextRead;
+      }
+
+      @Override
+      public void setPosition(int pos) {
+        // NOTE: a little weird because if you
+        // setPosition(0), the next byte you read is
+        // bytes[0] ... but I would expect bytes[-1] (ie,
+        // EOF)...?
+        int bufferIndex = pos >> blockBits;
+        nextBuffer = bufferIndex-1;
+        current = blocks.get(bufferIndex);
+        nextRead = pos & blockMask;
+        assert getPosition() == pos: "pos=" + pos + " getPos()=" + getPosition();
+      }
+
+      @Override
+      public boolean reversed() {
+        return true;
+      }
+    };
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sat Jan 12 15:55:50 2013
@@ -51,9 +51,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!!
@@ -93,6 +90,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;
 
   /**
@@ -139,8 +138,7 @@ public final class FST<T> {
   // produces this output
   T emptyOutput;
 
-  // Not private to avoid synthetic access$NNN methods:
-  byte[] bytes;
+  private final BytesStore bytes;
 
   private int startNode = -1;
 
@@ -254,8 +252,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
@@ -269,17 +265,19 @@ public final class FST<T> {
     this.inputType = inputType;
     this.outputs = outputs;
     this.allowArrayArcs = allowArrayArcs;
-    bytes = new byte[128];
+    // 32 KB blocks:
+    bytes = new BytesStore(15);
+    // 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 +287,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);
     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 {
@@ -335,8 +339,9 @@ public final class FST<T> {
     arcCount = in.readVInt();
     arcWithOutputCount = in.readVInt();
 
-    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();
@@ -353,7 +358,7 @@ public final class FST<T> {
 
   /** Returns bytes used to represent the FST */
   public int sizeInBytes() {
-    int size = bytes.length;
+    int size = bytes.getPosition();
     if (packed) {
       size += nodeRefToAddress.ramBytesUsed();
     } else if (nodeAddress != null) {
@@ -370,10 +375,8 @@ public final class FST<T> {
     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;
     this.startNode = startNode;
+    bytes.finish();
 
     cacheRootArcs();
   }
@@ -485,8 +488,9 @@ public final class FST<T> {
     out.writeVInt(nodeCount);
     out.writeVInt(arcCount);
     out.writeVInt(arcWithOutputCount);
-    out.writeVInt(bytes.length);
-    out.writeBytes(bytes, 0, bytes.length);
+    int numBytes = bytes.getPosition();
+    out.writeVInt(numBytes);
+    bytes.writeTo(out);
   }
   
   /**
@@ -526,17 +530,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);
     }
   }
 
@@ -563,7 +566,7 @@ 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);
+    //System.out.println("FST.addNode pos=" + bytes.getPosition() + " numArcs=" + nodeIn.numArcs);
     if (nodeIn.numArcs == 0) {
       if (nodeIn.isFinal) {
         return FINAL_END_NODE;
@@ -572,23 +575,24 @@ public final class FST<T> {
       }
     }
 
-    int startAddress = writer.getPosition();
+    int 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);
+      bytes.writeByte(ARCS_AS_FIXED_ARRAY);
+      bytes.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();
+      bytes.writeInt(0);
+      fixedArrayStart = bytes.getPosition();
       //System.out.println("  do fixed arcs array arcsStart=" + fixedArrayStart);
     } else {
       fixedArrayStart = 0;
@@ -598,12 +602,13 @@ public final class FST<T> {
     
     final int lastArc = nodeIn.numArcs-1;
 
-    int lastArcStart = writer.getPosition();
+    int 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;
@@ -637,34 +642,34 @@ public final class FST<T> {
         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.writeInt(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] = bytes.getPosition() - lastArcStart;
+        lastArcStart = bytes.getPosition();
         maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]);
         //System.out.println("    bytes=" + bytesPerArc[arcIdx]);
       }
@@ -676,48 +681,38 @@ public final class FST<T> {
     // such cases
 
     if (doFixedArray) {
-      //System.out.println("  doFixedArray");
       assert maxBytesPerArc > 0;
       // 2nd pass just "expands" all arcs to take up a fixed
       // byte size
       final int sizeNeeded = fixedArrayStart + nodeIn.numArcs * maxBytesPerArc;
       assert ((long) fixedArrayStart) + ((long) nodeIn.numArcs) * maxBytesPerArc < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
 
-      bytes = ArrayUtil.grow(bytes, sizeNeeded);
+      //System.out.println("write int @pos=" + (fixedArrayStart-4) + " numArcs=" + nodeIn.numArcs);
       // 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;
+      bytes.writeInt(fixedArrayStart-4, maxBytesPerArc);
 
       // expand the arcs in place, backwards
-      int srcPos = writer.getPosition();
+      int srcPos = bytes.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]);
+      assert destPos >= srcPos;
+      if (destPos > srcPos) {
+        bytes.skip(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]);
+          }
         }
       }
     }
 
-    // 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;
-    }
-    //System.out.println("  endAddress=" + endAddress);
+    final int thisNodeAddress = bytes.getPosition()-1;
+
+    bytes.reverse(startAddress, thisNodeAddress);
 
     nodeCount++;
     final int node;
@@ -727,14 +722,15 @@ public final class FST<T> {
         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(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 +759,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,7 +770,7 @@ 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) {
@@ -786,7 +782,7 @@ public final class FST<T> {
           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;
@@ -808,14 +804,14 @@ public final class FST<T> {
             if (packed) {
               in.readVInt();
             } else {
-              in.skip(4);
+              in.skipBytes(4);
             }
           }
           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();
@@ -854,9 +850,8 @@ 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;
+    in.setPosition(address);
     //System.out.println("  readFirstRealTargtArc address="
     //+ address);
     //System.out.println("   flags=" + arc.flags);
@@ -872,7 +867,7 @@ public final class FST<T> {
         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 +884,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,8 +912,12 @@ 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);
+      
+      int pos = getNodeAddress(arc.nextArc);
+      in.setPosition(pos);
+
       final byte b = in.readByte();
       if (b == ARCS_AS_FIXED_ARRAY) {
         //System.out.println("    nextArc fake array");
@@ -929,18 +928,18 @@ public final class FST<T> {
           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 +950,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 +959,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 +986,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,17 +997,18 @@ 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 pos = in.getPosition();
         final int code = in.readVInt();
         if (arc.flag(BIT_TARGET_DELTA)) {
           // Address is delta-coded from current address:
@@ -1021,12 +1021,12 @@ public final class FST<T> {
         } 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.nextArc = in.pos;
+      arc.nextArc = in.getPosition();
     }
     return arc;
   }
@@ -1035,7 +1035,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 +1069,7 @@ public final class FST<T> {
       return null;
     }
 
-    in.pos = getNodeAddress(follow.target);
+    in.setPosition(getNodeAddress(follow.target));
 
     arc.node = follow.target;
 
@@ -1084,13 +1083,14 @@ public final class FST<T> {
       } 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) {
@@ -1196,51 +1196,6 @@ public final class FST<T> {
     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() {
@@ -1251,85 +1206,32 @@ public final class FST<T> {
    *  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);
-    }
-  }
-
-  /** 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--];
-      }
+      in = bytes.getReverseReader();
     }
-
-    @Override
-    public void skip(int count) {
-      pos -= count;
-    }
-
-    @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);
-    }
+  /** Reads bytes stored in an FST. */
+  public static abstract class BytesReader extends DataInput {
+    /** Get current read position. */
+    public abstract int getPosition();
 
-    @Override
-    public byte readByte() {
-      return bytes[pos++];
-    }
+    /** Set current read position. */
+    public abstract void setPosition(int pos);
 
-    @Override
-    public void readBytes(byte[] b, int offset, int len) {
-      System.arraycopy(bytes, pos, b, offset, len);
-      pos += len;
-    }
+    /** Returns true if this reader uses reversed bytes
+     *  under-the-hood. */
+    public abstract boolean reversed();
 
-    @Override
-    public void skip(int count) {
-      pos += count;
-    }
-
-    @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 +1353,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) {
     packed = true;
     this.inputType = inputType;
-    bytes = new byte[128];
-    this.nodeRefToAddress = nodeRefToAddress;
+    // 32 KB blocks:
+    bytes = new BytesStore(15);
     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
@@ -1495,6 +1396,9 @@ public final class FST<T> {
       throw new IllegalArgumentException("this FST was not built with willPackFST=true");
     }
 
+    final RAMOutputStream buffer = new RAMOutputStream();
+    byte[] bufferBytes = new byte[64];
+
     Arc<T> arc = new Arc<T>();
 
     final BytesReader r = getBytesReader(0);
@@ -1529,17 +1433,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()), 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 +1447,8 @@ public final class FST<T> {
     int topCount;
     int nextCount;
 
+    FST<T> fst;
+
     // Iterate until we converge:
     while(true) {
 
@@ -1556,7 +1458,10 @@ public final class FST<T> {
       // for assert:
       boolean negDelta = false;
 
-      writer.setPosition(0);
+      fst = new FST<T>(inputType, outputs);
+      
+      final BytesStore writer = fst.bytes;
+
       // Skip 0 byte since 0 is reserved target:
       writer.writeByte((byte) 0);
 
@@ -1578,6 +1483,7 @@ public final class FST<T> {
       for(int node=nodeCount;node>=1;node--) {
         fst.nodeCount++;
         final int address = writer.getPosition();
+
         //System.out.println("  node: " + node + " address=" + address);
         if (address != newNodeAddress.get(node)) {
           addressError = address - (int) newNodeAddress.get(node);
@@ -1599,7 +1505,8 @@ public final class FST<T> {
         // this is an array'd node and bytesPerArc changes:
         writeNode:
         while(true) { // retry writing this node
-
+          assert buffer.getFilePointer() == 0;
+          //System.out.println("  cycle: retry");
           readFirstRealTargetArc(node, arc, r);
 
           final boolean useArcArray = arc.bytesPerArc != 0;
@@ -1608,18 +1515,18 @@ public final class FST<T> {
             if (bytesPerArc == 0) {
               bytesPerArc = arc.bytesPerArc;
             }
-            writer.writeByte(ARCS_AS_FIXED_ARRAY);
-            writer.writeVInt(arc.numArcs);
-            writer.writeVInt(bytesPerArc);
+            buffer.writeByte(ARCS_AS_FIXED_ARRAY);
+            buffer.writeVInt(arc.numArcs);
+            buffer.writeVInt(bytesPerArc);
             //System.out.println("node " + node + ": " + arc.numArcs + " arcs");
           }
 
           int maxBytesPerArc = 0;
           //int wasted = 0;
           while(true) {  // iterate over all arcs for this node
+            //System.out.println("    cycle next arc");
 
-            //System.out.println("    arc label=" + arc.label + " target=" + arc.target + " pos=" + writer.posWrite);
-            final int arcStartPos = writer.getPosition();
+            final int arcStartPos = (int) buffer.getFilePointer();
             nodeArcCount++;
 
             byte flags = 0;
@@ -1666,7 +1573,7 @@ public final class FST<T> {
                 absPtr = topNodeMap.size() + (int) newNodeAddress.get(arc.target) + addressError;
               }
 
-              int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition() - 2;
+              int delta = (int) (newNodeAddress.get(arc.target) + addressError - buffer.getFilePointer() - address - 2);
               if (delta < 0) {
                 //System.out.println("neg: " + delta);
                 anyNegDelta = true;
@@ -1681,22 +1588,22 @@ public final class FST<T> {
               absPtr = 0;
             }
 
-            writer.writeByte(flags);
-            fst.writeLabel(arc.label);
+            buffer.writeByte(flags);
+            fst.writeLabel(buffer, arc.label);
 
             if (arc.output != NO_OUTPUT) {
-              outputs.write(arc.output, writer);
+              outputs.write(arc.output, buffer);
               if (!retry) {
                 fst.arcWithOutputCount++;
               }
             }
             if (arc.nextFinalOutput != NO_OUTPUT) {
-              outputs.writeFinalOutput(arc.nextFinalOutput, writer);
+              outputs.writeFinalOutput(arc.nextFinalOutput, buffer);
             }
 
             if (doWriteTarget) {
 
-              int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition();
+              int delta = (int) (newNodeAddress.get(arc.target) + addressError - buffer.getFilePointer() - address);
               if (delta < 0) {
                 anyNegDelta = true;
                 //System.out.println("neg: " + delta);
@@ -1705,7 +1612,7 @@ public final class FST<T> {
 
               if (flag(flags, BIT_TARGET_DELTA)) {
                 //System.out.println("        delta");
-                writer.writeVInt(delta);
+                buffer.writeVInt(delta);
                 if (!retry) {
                   deltaCount++;
                 }
@@ -1717,7 +1624,7 @@ public final class FST<T> {
                   System.out.println("        abs");
                 }
                 */
-                writer.writeVInt(absPtr);
+                buffer.writeVInt(absPtr);
                 if (!retry) {
                   if (absPtr >= topNodeMap.size()) {
                     absCount++;
@@ -1729,7 +1636,7 @@ public final class FST<T> {
             }
 
             if (useArcArray) {
-              final int arcBytes = writer.getPosition() - arcStartPos;
+              final int arcBytes = (int) (buffer.getFilePointer() - arcStartPos);
               //System.out.println("  " + arcBytes + " bytes");
               maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
               // NOTE: this may in fact go "backwards", if
@@ -1739,7 +1646,11 @@ public final class FST<T> {
               // will retry (below) so it's OK to ovewrite
               // bytes:
               //wasted += bytesPerArc - arcBytes;
-              writer.setPosition(arcStartPos + bytesPerArc);
+              int skip = (int) (arcStartPos + bytesPerArc - buffer.getFilePointer());
+              while(skip > 0) {
+                buffer.writeByte((byte) 0);
+                skip--;
+              }
             }
 
             if (arc.isLast()) {
@@ -1764,11 +1675,19 @@ public final class FST<T> {
 
           // Retry:
           bytesPerArc = maxBytesPerArc;
-          writer.setPosition(address);
+          buffer.reset();
           nodeArcCount = 0;
           retry = true;
           anyNegDelta = false;
         }
+
+        if (bufferBytes.length < (int) buffer.getFilePointer()) {
+          bufferBytes = ArrayUtil.grow(bufferBytes, (int) buffer.getFilePointer());
+        }
+        buffer.writeTo(bufferBytes, 0);
+        writer.writeBytes(bufferBytes, 0, (int) buffer.getFilePointer());
+        buffer.reset();
+
         negDelta |= anyNegDelta;
 
         fst.arcCount += nodeArcCount;
@@ -1799,7 +1718,6 @@ public final class FST<T> {
     }
     fst.nodeRefToAddress = nodeRefToAddressIn;
     
-
     fst.startNode = (int) newNodeAddress.get(startNode);
     //System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
 
@@ -1810,11 +1728,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/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java Sat Jan 12 15:55:50 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);

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java?rev=1432459&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java Sat Jan 12 15:55:50 2013
@@ -0,0 +1,62 @@
+package org.apache.lucene.util.fst;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// TODO: can we use just ByteArrayDataInput...?  need to
+// add a .skipBytes to DataInput.. hmm and .setPosition
+
+/** Reads from a single byte[]. */
+final class ForwardBytesReader extends FST.BytesReader {
+  private final byte[] bytes;
+  private int pos;
+
+  public ForwardBytesReader(byte[] bytes) {
+    this.bytes = bytes;
+  }
+
+  @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;
+  }
+
+  @Override
+  public void skipBytes(int count) {
+    pos += count;
+  }
+
+  @Override
+  public int getPosition() {
+    return pos;
+  }
+
+  @Override
+  public void setPosition(int pos) {
+    this.pos = pos;
+  }
+
+  @Override
+  public boolean reversed() {
+    return false;
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Sat Jan 12 15:55:50 2013
@@ -92,7 +92,7 @@ final class NodeHash<T> {
     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 + scratchArc.output.hashCode();

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java?rev=1432459&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java Sat Jan 12 15:55:50 2013
@@ -0,0 +1,61 @@
+package org.apache.lucene.util.fst;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Reads in reverse from a single byte[]. */
+final class ReverseBytesReader extends FST.BytesReader {
+  private final byte[] bytes;
+  private int pos;
+
+  public ReverseBytesReader(byte[] bytes) {
+    this.bytes = bytes;
+  }
+
+  @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 skipBytes(int count) {
+    pos -= count;
+  }
+
+  @Override
+  public int getPosition() {
+    return pos;
+  }
+
+  @Override
+  public void setPosition(int pos) {
+    this.pos = pos;
+  }
+
+  @Override
+  public boolean reversed() {
+    return true;
+  }
+}
+

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Sat Jan 12 15:55:50 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
@@ -595,7 +595,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>();
 
@@ -893,8 +893,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=" +

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java?rev=1432459&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java Sat Jan 12 15:55:50 2013
@@ -0,0 +1,328 @@
+package org.apache.lucene.util.fst;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+public class TestBytesStore extends LuceneTestCase {
+
+  public void testRandom() throws Exception {
+
+    final int iters = atLeast(10);
+    for(int iter=0;iter<iters;iter++) {
+      final int numBytes = _TestUtil.nextInt(random(), 1, 200000);
+      final byte[] expected = new byte[numBytes];
+      final int blockBits = _TestUtil.nextInt(random(), 8, 15);
+      final BytesStore bytes = new BytesStore(blockBits);
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter + " numBytes=" + numBytes + " blockBits=" + blockBits);
+      }
+
+      int pos = 0;
+      while(pos < numBytes) {
+        int op = random().nextInt(7);
+        if (VERBOSE) {
+          System.out.println("  cycle pos=" + pos);
+        }
+        switch(op) {
+
+        case 0:
+          {
+            // write random byte
+            byte b = (byte) random().nextInt(256);
+            if (VERBOSE) {
+              System.out.println("    writeByte b=" + b);
+            }
+
+            expected[pos++] = b;
+            bytes.writeByte(b);
+          }
+          break;
+
+        case 1:
+          {
+            // write random byte[]
+            int len = random().nextInt(Math.min(numBytes - pos, 100));
+            byte[] temp = new byte[len];
+            random().nextBytes(temp);
+            if (VERBOSE) {
+              System.out.println("    writeBytes len=" + len + " bytes=" + Arrays.toString(temp));
+            }
+            System.arraycopy(temp, 0, expected, pos, temp.length);
+            bytes.writeBytes(temp, 0, temp.length);
+            pos += len;
+          }
+          break;
+
+        case 2:
+          {
+            // write int @ absolute pos
+            if (pos > 4) {
+              int x = random().nextInt();
+              int randomPos = random().nextInt(pos-4);
+              if (VERBOSE) {
+                System.out.println("    abs writeInt pos=" + randomPos + " x=" + x);
+              }
+              bytes.writeInt(randomPos, x);
+              expected[randomPos++] = (byte) (x >> 24);
+              expected[randomPos++] = (byte) (x >> 16);
+              expected[randomPos++] = (byte) (x >> 8);
+              expected[randomPos++] = (byte) x;
+            }
+          }
+          break;
+
+        case 3:
+          {
+            // reverse bytes
+            if (pos > 0) {
+              int len = _TestUtil.nextInt(random(), 1, Math.min(100, pos));
+              int start;
+              if (len == pos) {
+                start = 0;
+              } else {
+                start = random().nextInt(pos - len);
+              }
+              int end = start + len;
+              if (VERBOSE) {
+                System.out.println("    reverse start=" + start + " end=" + end + " len=" + len);
+              }
+              bytes.reverse(start, end);
+
+              while(start < end) {
+                byte b = expected[end];
+                expected[end] = expected[start];
+                expected[start] = b;
+                start++;
+                end--;
+              }
+            }
+          }
+          break;
+
+        case 4:
+          {
+            // abs write random byte[]
+            if (pos > 2) {
+              int randomPos = random().nextInt(pos-1);
+              int len = _TestUtil.nextInt(random(), 1, Math.min(pos - randomPos - 1, 100));
+              byte[] temp = new byte[len];
+              random().nextBytes(temp);
+              if (VERBOSE) {
+                System.out.println("    abs writeBytes pos=" + randomPos + " len=" + len + " bytes=" + Arrays.toString(temp));
+              }
+              System.arraycopy(temp, 0, expected, randomPos, temp.length);
+              bytes.writeBytes(randomPos, temp, 0, temp.length);
+            }
+          }
+          break;
+
+        case 5:
+          {
+            // copyBytes
+            if (pos > 1) {
+              int src = random().nextInt(pos-1);
+              int dest = _TestUtil.nextInt(random(), src+1, pos-1);
+              int len = _TestUtil.nextInt(random(), 1, Math.min(300, pos - dest));
+              if (VERBOSE) {
+                System.out.println("    copyBytes src=" + src + " dest=" + dest + " len=" + len);
+              }
+              System.arraycopy(expected, src, expected, dest, len);
+              bytes.copyBytes(src, dest, len);
+            }
+          }
+          break;
+
+        case 6:
+          {
+            // skip
+            int len = random().nextInt(Math.min(100, numBytes - pos));
+            pos += len;
+            bytes.skip(len);
+            if (VERBOSE) {
+              System.out.println("    skip len=" + len);
+            }
+          }
+          break;
+        }
+
+        assertEquals(pos, bytes.getPosition());
+
+        if ((pos > 0 && random().nextInt(200) == 17)) {
+          verify(bytes, expected, pos);
+        }
+      }
+
+      BytesStore bytesToVerify;
+
+      if (random().nextBoolean()) {
+        if (VERBOSE) {
+          System.out.println("TEST: save/load final bytes");
+        }
+        Directory dir = newDirectory();
+        IndexOutput out = dir.createOutput("bytes", IOContext.DEFAULT);
+        bytes.writeTo(out);
+        out.close();
+        IndexInput in = dir.openInput("bytes", IOContext.DEFAULT);
+        bytesToVerify = new BytesStore(in, numBytes, _TestUtil.nextInt(random(), 256, Integer.MAX_VALUE));
+        in.close();
+        dir.close();
+      } else {
+        bytesToVerify = bytes;
+      }
+
+      verify(bytesToVerify, expected, numBytes);
+    }
+  }
+
+  private void verify(BytesStore bytes, byte[] expected, int totalLength) throws Exception {
+    assertEquals(totalLength, bytes.getPosition());
+    if (totalLength == 0) {
+      return;
+    }
+    if (VERBOSE) {
+      System.out.println("  verify...");
+    }
+    
+    // First verify whole thing in one blast:
+    byte[] actual = new byte[totalLength];
+    if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("    bulk: reversed");
+      }
+      // reversed
+      FST.BytesReader r = bytes.getReverseReader();
+      assertTrue(r.reversed());
+      r.setPosition(totalLength-1);
+      r.readBytes(actual, 0, actual.length);
+      int start = 0;
+      int end = totalLength - 1;
+      while(start < end) {
+        byte b = actual[start];
+        actual[start] = actual[end];
+        actual[end] = b;
+        start++;
+        end--;
+      }
+    } else {
+      // forward
+      if (VERBOSE) {
+        System.out.println("    bulk: forward");
+      }
+      FST.BytesReader r = bytes.getForwardReader();
+      assertFalse(r.reversed());
+      r.readBytes(actual, 0, actual.length);
+    }
+
+    for(int i=0;i<totalLength;i++) {
+      assertEquals("byte @ index=" + i, expected[i], actual[i]);
+    }
+
+    FST.BytesReader r;
+
+    // Then verify ops:
+    boolean reversed = random().nextBoolean();
+    if (reversed) {
+      if (VERBOSE) {
+        System.out.println("    ops: reversed");
+      }
+      r = bytes.getReverseReader();
+    } else {
+      if (VERBOSE) {
+        System.out.println("    ops: forward");
+      }
+      r = bytes.getForwardReader();
+    }
+
+    if (totalLength > 1) {
+      int numOps = _TestUtil.nextInt(random(), 100, 200);
+      for(int op=0;op<numOps;op++) {
+
+        int numBytes = random().nextInt(Math.min(1000, totalLength-1));
+        int pos;
+        if (reversed) {
+          pos = _TestUtil.nextInt(random(), numBytes, totalLength-1);
+        } else {
+          pos = random().nextInt(totalLength-numBytes);
+        }
+        if (VERBOSE) {
+          System.out.println("    op iter=" + op + " reversed=" + reversed + " numBytes=" + numBytes + " pos=" + pos);
+        }
+        byte[] temp = new byte[numBytes];
+        r.setPosition(pos);
+        assertEquals(pos, r.getPosition());
+        r.readBytes(temp, 0, temp.length);
+        for(int i=0;i<numBytes;i++) {
+          byte expectedByte;
+          if (reversed) {
+            expectedByte = expected[pos - i];
+          } else {
+            expectedByte = expected[pos + i];
+          }
+          assertEquals("byte @ index=" + i, expectedByte, temp[i]);
+        }
+
+        int left;
+        int expectedPos;
+
+        if (reversed) {
+          expectedPos = pos-numBytes;
+          left = r.getPosition();
+        } else {
+          expectedPos = pos+numBytes;
+          left = totalLength - r.getPosition();
+        }
+        assertEquals(expectedPos, r.getPosition());
+
+        if (left > 4) {
+          int skipBytes = random().nextInt(left-4);
+
+          int expectedInt = 0;
+          if (reversed) {
+            expectedPos -= skipBytes;
+            expectedInt |= (expected[expectedPos--]&0xFF)<<24;
+            expectedInt |= (expected[expectedPos--]&0xFF)<<16;
+            expectedInt |= (expected[expectedPos--]&0xFF)<<8;
+            expectedInt |= (expected[expectedPos--]&0xFF);
+          } else {
+            expectedPos += skipBytes;
+            expectedInt |= (expected[expectedPos++]&0xFF)<<24;
+            expectedInt |= (expected[expectedPos++]&0xFF)<<16;
+            expectedInt |= (expected[expectedPos++]&0xFF)<<8;
+            expectedInt |= (expected[expectedPos++]&0xFF);
+          }
+
+          if (VERBOSE) {
+            System.out.println("    skip numBytes=" + skipBytes);
+            System.out.println("    readInt");
+          }
+
+          r.skipBytes(skipBytes);
+          assertEquals(expectedInt, r.readInt());
+        }
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1432459&r1=1432458&r2=1432459&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Sat Jan 12 15:55:50 2013
@@ -483,6 +483,8 @@ public class TestFSTs extends LuceneTest
             break;
           }
         }
+        long t = System.currentTimeMillis() - tStart;
+        System.out.println((t / 1000.0) + " sec to build");
 
         assert builder.getTermCount() == ord;
         FST<T> fst = builder.finish();
@@ -513,6 +515,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 +584,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 +1030,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++)
           {