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/14 19:31:18 UTC

svn commit: r1433026 - in /lucene/dev/trunk/lucene: ./ analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ codecs/src/java/org/apache/lucene/codecs/memory/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/util/fs...

Author: mikemccand
Date: Mon Jan 14 18:31:17 2013
New Revision: 1433026

URL: http://svn.apache.org/viewvc?rev=1433026&view=rev
Log:
LUCENE-3298: FSTs can now be larger than 2GB, have more than 2B nodes

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.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/ForwardBytesReader.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/ReverseBytesReader.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/TestBytesStore.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Mon Jan 14 18:31:17 2013
@@ -26,6 +26,9 @@ Changes in backwards compatibility polic
   single byte[] internally, to avoid large memory spikes during
   building (James Dyer, Mike McCandless)
 
+* LUCENE-3298: FST can now be larger than 2.1 GB / 2.1 B nodes.
+  (James Dyer, Mike McCandless)
+
 ======================= Lucene 4.1.0 =======================
 
 Changes in backwards compatibility policy

Modified: lucene/dev/trunk/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java (original)
+++ lucene/dev/trunk/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java Mon Jan 14 18:31:17 2013
@@ -132,7 +132,7 @@ public class TokenInfoDictionaryBuilder 
     System.out.println("  encode...");
 
     PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton(true);
-    Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true, true);
+    Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true, PackedInts.DEFAULT, true, 15);
     IntsRef scratch = new IntsRef();
     long ord = -1; // first ord will be 0
     String lastValue = null;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Mon Jan 14 18:31:17 2013
@@ -113,7 +113,7 @@ public final class MemoryPostingsFormat 
       this.field = field;
       this.doPackFST = doPackFST;
       this.acceptableOverheadRatio = acceptableOverheadRatio;
-      builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true);
+      builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true, 15);
     }
 
     private class PostingsWriter extends PostingsConsumer {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Mon Jan 14 18:31:17 2013
@@ -276,13 +276,13 @@ public class BlockTreeTermsReader extend
    */
   public static class Stats {
     /** How many nodes in the index FST. */
-    public int indexNodeCount;
+    public long indexNodeCount;
 
     /** How many arcs in the index FST. */
-    public int indexArcCount;
+    public long indexArcCount;
 
     /** Byte size of the index. */
-    public int indexNumBytes;
+    public long indexNumBytes;
 
     /** Total number of terms in the field. */
     public long totalTermCount;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Mon Jan 14 18:31:17 2013
@@ -23,7 +23,6 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
@@ -41,6 +40,7 @@ import org.apache.lucene.util.fst.BytesR
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.NoOutputs;
 import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.PackedInts;
 
 /*
   TODO:
@@ -187,7 +187,7 @@ public class BlockTreeTermsWriter extend
   public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
 
   //public final static boolean DEBUG = false;
-  private final static boolean SAVE_DOT_FILES = false;
+  //private final static boolean SAVE_DOT_FILES = false;
 
   static final int OUTPUT_FLAGS_NUM_BITS = 2;
   static final int OUTPUT_FLAGS_MASK = 0x3;
@@ -419,7 +419,8 @@ public class BlockTreeTermsWriter extend
       final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
       final Builder<BytesRef> indexBuilder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1,
                                                                    0, 0, true, false, Integer.MAX_VALUE,
-                                                                   outputs, null, false, true);
+                                                                   outputs, null, false,
+                                                                   PackedInts.COMPACT, true, 15);
       //if (DEBUG) {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}
@@ -962,7 +963,9 @@ public class BlockTreeTermsWriter extend
                                          0, 0, true,
                                          true, Integer.MAX_VALUE,
                                          noOutputs,
-                                         new FindBlocks(), false, true);
+                                         new FindBlocks(), false,
+                                         PackedInts.COMPACT,
+                                         true, 15);
 
       postingsWriter.setField(fieldInfo);
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Mon Jan 14 18:31:17 2013
@@ -36,9 +36,13 @@ import org.apache.lucene.util.packed.Pac
  * <p>NOTE: The algorithm is described at
  * http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.24.3698</p>
  *
- * The parameterized type T is the output type.  See the
+ * <p>The parameterized type T is the output type.  See the
  * subclasses of {@link Outputs}.
  *
+ * <p>FSTs larger than 2.1GB are now possible (as of Lucene
+ * 4.2).  FSTs containing more than 2.1B nodes are also now
+ * possible, however they cannot be packed.
+ *
  * @lucene.experimental
  */
 
@@ -84,22 +88,11 @@ public class Builder<T> {
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean,
-   * boolean, int, Outputs, FreezeTail, boolean, boolean)} with
-   * pruning options turned off.
+   * boolean, int, Outputs, FreezeTail, boolean, float,
+   * boolean, int)} with pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT, true);
-  }
-
-  /**
-   * Instantiates an FST/FSA builder with {@link PackedInts#DEFAULT}
-   * <code>acceptableOverheadRatio</code>.
-   */
-  public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
-      boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-      FreezeTail<T> freezeTail, boolean willPackFST, boolean allowArrayArcs) {
-    this(inputType, minSuffixCount1, minSuffixCount2, doShareSuffix, doShareNonSingletonNodes,
-         shareMaxTailLength, outputs, freezeTail, willPackFST, PackedInts.DEFAULT, allowArrayArcs);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT, true, 15);
   }
 
   /**
@@ -147,10 +140,16 @@ public class Builder<T> {
    * @param allowArrayArcs Pass false to disable the array arc optimization
    *    while building the FST; this will make the resulting
    *    FST smaller but slower to traverse.
+   *
+   * @param bytesPageBits How many bits wide to make each
+   *    byte[] block in the BytesStore; if you know the FST
+   *    will be large then make this larger.  For example 15
+   *    bits = 32768 byte pages.
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
                  boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-                 FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
+                 FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
+                 int bytesPageBits) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
     this.freezeTail = freezeTail;
@@ -158,7 +157,7 @@ public class Builder<T> {
     this.shareMaxTailLength = shareMaxTailLength;
     this.doPackFST = doPackFST;
     this.acceptableOverheadRatio = acceptableOverheadRatio;
-    fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs);
+    fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs, bytesPageBits);
     if (doShareSuffix) {
       dedupHash = new NodeHash<T>(fst, fst.bytes.getReverseReader(false));
     } else {
@@ -174,7 +173,7 @@ public class Builder<T> {
     }
   }
 
-  public int getTotStateCount() {
+  public long getTotStateCount() {
     return fst.nodeCount;
   }
 
@@ -182,12 +181,12 @@ public class Builder<T> {
     return frontier[0].inputCount;
   }
 
-  public int getMappedStateCount() {
+  public long getMappedStateCount() {
     return dedupHash == null ? 0 : fst.nodeCount;
   }
 
   private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
-    final int node;
+    final long node;
     if (dedupHash != null && (doShareNonSingletonNodes || nodeIn.numArcs <= 1) && tailLength <= shareMaxTailLength) {
       if (nodeIn.numArcs == 0) {
         node = fst.addNode(nodeIn);
@@ -475,7 +474,7 @@ public class Builder<T> {
     fst.finish(compileNode(root, lastInput.length).node);
 
     if (doPackFST) {
-      return fst.pack(3, Math.max(10, fst.getNodeCount()/4), acceptableOverheadRatio);
+      return fst.pack(3, Math.max(10, (int) (fst.getNodeCount()/4)), acceptableOverheadRatio);
     } else {
       return fst;
     }
@@ -513,8 +512,12 @@ public class Builder<T> {
     boolean isCompiled();
   }
 
+  public long fstSizeInBytes() {
+    return fst.sizeInBytes();
+  }
+
   static final class CompiledNode implements Node {
-    int node;
+    long node;
     @Override
     public boolean isCompiled() {
       return true;

Modified: 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=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java Mon Jan 14 18:31:17 2013
@@ -108,10 +108,14 @@ class BytesStore extends DataOutput {
     }
   }
 
+  int getBlockBits() {
+    return blockBits;
+  }
+
   /** 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) {
+  void writeBytes(long 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;
 
@@ -141,9 +145,9 @@ class BytesStore extends DataOutput {
     }
     */
 
-    final int end = dest + len;
-    int blockIndex = end >> blockBits;
-    int downTo = end & blockMask;
+    final long end = dest + len;
+    int blockIndex = (int) (end >> blockBits);
+    int downTo = (int) (end & blockMask);
     if (downTo == 0) {
       blockIndex--;
       downTo = blockSize;
@@ -170,7 +174,7 @@ class BytesStore extends DataOutput {
   /** 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) {
+  public void copyBytes(long src, long dest, int len) {
     //System.out.println("BS.copyBytes src=" + src + " dest=" + dest + " len=" + len);
     assert src < dest;
 
@@ -200,10 +204,10 @@ class BytesStore extends DataOutput {
     }
     */
 
-    int end = src + len;
+    long end = src + len;
 
-    int blockIndex = end >> blockBits;
-    int downTo = end & blockMask;
+    int blockIndex = (int) (end >> blockBits);
+    int downTo = (int) (end & blockMask);
     if (downTo == 0) {
       blockIndex--;
       downTo = blockSize;
@@ -229,9 +233,9 @@ class BytesStore extends DataOutput {
 
   /** 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;
+  public void writeInt(long pos, int value) {
+    int blockIndex = (int) (pos >> blockBits);
+    int upto = (int) (pos & blockMask);
     byte[] block = blocks.get(blockIndex);
     int shift = 24;
     for(int i=0;i<4;i++) {
@@ -246,21 +250,21 @@ class BytesStore extends DataOutput {
   }
 
   /** Reverse from srcPos, inclusive, to destPos, inclusive. */
-  public void reverse(int srcPos, int destPos) {
+  public void reverse(long srcPos, long destPos) {
     assert srcPos < destPos;
     assert destPos < getPosition();
     //System.out.println("reverse src=" + srcPos + " dest=" + destPos);
 
-    int srcBlockIndex = srcPos >> blockBits;
-    int src = srcPos & blockMask;
+    int srcBlockIndex = (int) (srcPos >> blockBits);
+    int src = (int) (srcPos & blockMask);
     byte[] srcBlock = blocks.get(srcBlockIndex);
 
-    int destBlockIndex = destPos >> blockBits;
-    int dest = destPos & blockMask;
+    int destBlockIndex = (int) (destPos >> blockBits);
+    int dest = (int) (destPos & blockMask);
     byte[] destBlock = blocks.get(destBlockIndex);
     //System.out.println("  srcBlock=" + srcBlockIndex + " destBlock=" + destBlockIndex);
 
-    int limit = (destPos - srcPos + 1)/2;
+    int limit = (int) (destPos - srcPos + 1)/2;
     for(int i=0;i<limit;i++) {
       //System.out.println("  cycle src=" + src + " dest=" + dest);
       byte b = srcBlock[src];
@@ -299,17 +303,17 @@ class BytesStore extends DataOutput {
     }
   }
 
-  public int getPosition() {
-    return (blocks.size()-1) * blockSize + nextWrite;
+  public long getPosition() {
+    return ((long) blocks.size()-1) * blockSize + nextWrite;
   }
 
   /** Pos must be less than the max position written so far!
    *  Ie, you cannot "grow" the file with this! */
-  public void truncate(int newLen) {
+  public void truncate(long newLen) {
     assert newLen <= getPosition();
     assert newLen >= 0;
-    int blockIndex = newLen >> blockBits;
-    nextWrite = newLen & blockMask;
+    int blockIndex = (int) (newLen >> blockBits);
+    nextWrite = (int) (newLen & blockMask);
     if (nextWrite == 0) {
       blockIndex--;
       nextWrite = blockSize;
@@ -332,6 +336,7 @@ class BytesStore extends DataOutput {
     }
   }
 
+  /** Writes all of our bytes to the target {@link DataOutput}. */
   public void writeTo(DataOutput out) throws IOException {
     for(byte[] block : blocks) {
       out.writeBytes(block, 0, block.length);
@@ -382,16 +387,16 @@ class BytesStore extends DataOutput {
       }
 
       @Override
-      public int getPosition() {
-        return (nextBuffer-1)*blockSize + nextRead;
+      public long getPosition() {
+        return ((long) nextBuffer-1)*blockSize + nextRead;
       }
 
       @Override
-      public void setPosition(int pos) {
-        int bufferIndex = pos >> blockBits;
+      public void setPosition(long pos) {
+        int bufferIndex = (int) (pos >> blockBits);
         nextBuffer = bufferIndex+1;
         current = blocks.get(bufferIndex);
-        nextRead = pos & blockMask;
+        nextRead = (int) (pos & blockMask);
         assert getPosition() == pos;
       }
 
@@ -437,20 +442,20 @@ class BytesStore extends DataOutput {
       }
 
       @Override
-      public int getPosition() {
-        return (nextBuffer+1)*blockSize + nextRead;
+      public long getPosition() {
+        return ((long) nextBuffer+1)*blockSize + nextRead;
       }
 
       @Override
-      public void setPosition(int pos) {
+      public void setPosition(long 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;
+        int bufferIndex = (int) (pos >> blockBits);
         nextBuffer = bufferIndex-1;
         current = blocks.get(bufferIndex);
-        nextRead = pos & blockMask;
+        nextRead = (int) (pos & blockMask);
         assert getPosition() == pos: "pos=" + pos + " getPos()=" + getPosition();
       }
 

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=1433026&r1=1433025&r2=1433026&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 Mon Jan 14 18:31:17 2013
@@ -68,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
  */
@@ -138,11 +136,11 @@ public final class FST<T> {
 
   // 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
@@ -150,7 +148,7 @@ public final class FST<T> {
 
   final BytesStore bytes;
 
-  private int startNode = -1;
+  private long startNode = -1;
 
   public final Outputs<T> outputs;
 
@@ -158,13 +156,13 @@ public final class FST<T> {
   // instead of storing the address of the target node for
   // a given arc, we mark a single bit noting that the next
   // node in the byte[] is the target node):
-  private int lastFrozenNode;
+  private long lastFrozenNode;
 
   private final T NO_OUTPUT;
 
-  public int nodeCount;
-  public int arcCount;
-  public int arcWithOutputCount;
+  public long nodeCount;
+  public long arcCount;
+  public long arcWithOutputCount;
 
   private final boolean packed;
   private PackedInts.Reader nodeRefToAddress;
@@ -183,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;
@@ -273,13 +271,13 @@ public final class FST<T> {
 
   // make a new empty FST, for building; Builder invokes
   // this ctor
-  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
+  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs, int bytesPageBits) {
     this.inputType = inputType;
     this.outputs = outputs;
     this.allowArrayArcs = allowArrayArcs;
     version = VERSION_CURRENT;
     // 32 KB blocks:
-    bytes = new BytesStore(15);
+    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);
@@ -348,9 +346,9 @@ 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();
 
     int numBytes = in.readVInt();
     bytes = new BytesStore(in, numBytes, Integer.MAX_VALUE);
@@ -379,8 +377,8 @@ public final class FST<T> {
   }
 
   /** Returns bytes used to represent the FST */
-  public int sizeInBytes() {
-    int size = bytes.getPosition();
+  public long sizeInBytes() {
+    long size = bytes.getPosition();
     if (packed) {
       size += nodeRefToAddress.ramBytesUsed();
     } else if (nodeAddress != null) {
@@ -390,23 +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");
     }
+    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;
@@ -506,12 +504,12 @@ public final class FST<T> {
     if (packed) {
       ((PackedInts.Mutable) nodeRefToAddress).save(out);
     }
-    out.writeVInt(startNode);
-    out.writeVInt(nodeCount);
-    out.writeVInt(arcCount);
-    out.writeVInt(arcWithOutputCount);
-    int numBytes = bytes.getPosition();
-    out.writeVInt(numBytes);
+    out.writeVLong(startNode);
+    out.writeVLong(nodeCount);
+    out.writeVLong(arcCount);
+    out.writeVLong(arcWithOutputCount);
+    long numBytes = bytes.getPosition();
+    out.writeVLong(numBytes);
     bytes.writeTo(out);
   }
   
@@ -587,7 +585,8 @@ 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 {
+  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) {
@@ -597,10 +596,10 @@ public final class FST<T> {
       }
     }
 
-    final int startAddress = bytes.getPosition();
+    final long startAddress = bytes.getPosition();
     //System.out.println("  startAddr=" + startAddress);
 
-    boolean doFixedArray = shouldExpand(nodeIn);
+    final boolean doFixedArray = shouldExpand(nodeIn);
     if (doFixedArray) {
       //System.out.println("  fixedArray");
       if (bytesPerArc.length < nodeIn.numArcs) {
@@ -612,7 +611,7 @@ public final class FST<T> {
     
     final int lastArc = nodeIn.numArcs-1;
 
-    int lastArcStart = bytes.getPosition();
+    long lastArcStart = bytes.getPosition();
     int maxBytesPerArc = 0;
     for(int arcIdx=0;arcIdx<nodeIn.numArcs;arcIdx++) {
       final Builder.Arc<T> arc = nodeIn.arcs[arcIdx];
@@ -645,7 +644,7 @@ 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) {
@@ -671,14 +670,14 @@ public final class FST<T> {
       if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) {
         assert target.node > 0;
         //System.out.println("    write target");
-        bytes.writeVInt(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] = bytes.getPosition() - lastArcStart;
+        bytesPerArc[arcIdx] = (int) (bytes.getPosition() - lastArcStart);
         lastArcStart = bytes.getPosition();
         maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]);
         //System.out.println("    bytes=" + bytesPerArc[arcIdx]);
@@ -710,7 +709,6 @@ public final class FST<T> {
       assert maxBytesPerArc > 0;
       // 2nd pass just "expands" all arcs to take up a fixed
       // byte size
-      assert ((long) startAddress+MAX_HEADER_SIZE) + ((long) nodeIn.numArcs) * maxBytesPerArc < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
 
       //System.out.println("write int @pos=" + (fixedArrayStart-4) + " numArcs=" + nodeIn.numArcs);
       // create the header
@@ -723,14 +721,14 @@ public final class FST<T> {
       bad.writeVInt(maxBytesPerArc);
       int headerLen = bad.getPosition();
       
-      final int fixedArrayStart = startAddress + headerLen;
+      final long fixedArrayStart = startAddress + headerLen;
 
       // expand the arcs in place, backwards
-      int srcPos = bytes.getPosition();
-      int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
+      long srcPos = bytes.getPosition();
+      long destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
       assert destPos >= srcPos;
       if (destPos > srcPos) {
-        bytes.skipBytes(destPos - srcPos);
+        bytes.skipBytes((int) (destPos - srcPos));
         for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
           destPos -= maxBytesPerArc;
           srcPos -= bytesPerArc[arcIdx];
@@ -747,19 +745,26 @@ public final class FST<T> {
       bytes.writeBytes(startAddress, header, 0, headerLen);
     }
 
-    final int thisNodeAddress = bytes.getPosition()-1;
+    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");
+    }
+
     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, thisNodeAddress);
+      nodeAddress.set((int) nodeCount, thisNodeAddress);
       // System.out.println("  write nodeAddress[" + nodeCount + "] = " + endAddress);
       node = nodeCount;
     } else {
@@ -838,7 +843,7 @@ public final class FST<T> {
           if (arc.flag(BIT_STOP_NODE)) {
           } else if (arc.flag(BIT_TARGET_NEXT)) {
           } else if (packed) {
-            in.readVInt();
+            in.readVLong();
           } else {
             readUnpackedNodeTarget(in);
           }
@@ -854,12 +859,12 @@ public final class FST<T> {
     }
   }
 
-  private int readUnpackedNodeTarget(BytesReader in) throws IOException {
-    int target;
+  private long readUnpackedNodeTarget(BytesReader in) throws IOException {
+    long target;
     if (version < VERSION_VINT_TARGET) {
       target = in.readInt();
     } else {
-      target = in.readVInt();
+      target = in.readVLong();
     }
     return target;
   }
@@ -894,8 +899,8 @@ public final class FST<T> {
     }
   }
 
-  public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
-    final int address = getNodeAddress(node);
+  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);
@@ -960,7 +965,7 @@ public final class FST<T> {
       //System.out.println("    nextArc fake " +
       //arc.nextArc);
       
-      int pos = getNodeAddress(arc.nextArc);
+      long pos = getNodeAddress(arc.nextArc);
       in.setPosition(pos);
 
       final byte b = in.readByte();
@@ -1055,15 +1060,15 @@ public final class FST<T> {
       }
     } else {
       if (packed) {
-        final int pos = in.getPosition();
-        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
@@ -1192,7 +1197,7 @@ public final class FST<T> {
 
       if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
         if (packed) {
-          in.readVInt();
+          in.readVLong();
         } else {
           readUnpackedNodeTarget(in);
         }
@@ -1204,16 +1209,16 @@ public final class FST<T> {
     }
   }
 
-  public int getNodeCount() {
+  public long getNodeCount() {
     // 1+ in order to count the -1 implicit final node
     return 1+nodeCount;
   }
   
-  public int getArcCount() {
+  public long getArcCount() {
     return arcCount;
   }
 
-  public int getArcWithOutputCount() {
+  public long getArcWithOutputCount() {
     return arcWithOutputCount;
   }
 
@@ -1238,11 +1243,6 @@ public final class FST<T> {
        node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
   }
   
-  static abstract class BytesWriter extends DataOutput {
-    public abstract void setPosition(int posWrite);
-    public abstract int getPosition();
-  }
-
   /** Returns a {@link BytesReader} for this FST, positioned at
    *  position 0. */
   public BytesReader getBytesReader() {
@@ -1251,7 +1251,7 @@ public final class FST<T> {
 
   /** Returns a {@link BytesReader} for this FST, positioned at
    *  the provided position. */
-  public BytesReader getBytesReader(int pos) {
+  public BytesReader getBytesReader(long pos) {
     // TODO: maybe re-use via ThreadLocal?
     BytesReader in;
     if (packed) {
@@ -1268,10 +1268,10 @@ public final class FST<T> {
   /** Reads bytes stored in an FST. */
   public static abstract class BytesReader extends DataInput {
     /** Get current read position. */
-    public abstract int getPosition();
+    public abstract long getPosition();
 
     /** Set current read position. */
-    public abstract void setPosition(int pos);
+    public abstract void setPosition(long pos);
 
     /** Returns true if this reader uses reversed bytes
      *  under-the-hood. */
@@ -1400,12 +1400,11 @@ public final class FST<T> {
  */
 
   // Creates a packed FST
-  private FST(INPUT_TYPE inputType, Outputs<T> outputs) {
+  private FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
     version = VERSION_CURRENT;
     packed = true;
     this.inputType = inputType;
-    // 32 KB blocks:
-    bytes = new BytesStore(15);
+    bytes = new BytesStore(bytesPageBits);
     this.outputs = outputs;
     NO_OUTPUT = outputs.getNoOutput();
     
@@ -1429,6 +1428,9 @@ public final class FST<T> {
    */
   FST<T> pack(int minInCountDeref, int maxDerefNodes, float acceptableOverheadRatio) throws IOException {
 
+    // NOTE: maxDerefNodes is intentionally int: we cannot
+    // support > 2.1B deref nodes
+
     // TODO: other things to try
     //   - renumber the nodes to get more next / better locality?
     //   - allow multiple input labels on an arc, so
@@ -1480,7 +1482,7 @@ public final class FST<T> {
 
     // +1 because node ords start at 1 (0 is reserved as stop node):
     final GrowableWriter newNodeAddress = new GrowableWriter(
-                    PackedInts.bitsRequired(this.bytes.getPosition()), 1 + nodeCount, acceptableOverheadRatio);
+                       PackedInts.bitsRequired(this.bytes.getPosition()), (int) (1 + nodeCount), acceptableOverheadRatio);
 
     // Fill initial coarse guess:
     for(int node=1;node<=nodeCount;node++) {
@@ -1503,7 +1505,7 @@ public final class FST<T> {
       // for assert:
       boolean negDelta = false;
 
-      fst = new FST<T>(inputType, outputs);
+      fst = new FST<T>(inputType, outputs, bytes.getBlockBits());
       
       final BytesStore writer = fst.bytes;
 
@@ -1518,20 +1520,20 @@ public final class FST<T> {
 
       int changedCount = 0;
 
-      int addressError = 0;
+      long addressError = 0;
 
       //int totWasted = 0;
 
       // Since we re-reverse the bytes, we now write the
       // nodes backwards, so that BIT_TARGET_NEXT is
       // unchanged:
-      for(int node=nodeCount;node>=1;node--) {
+      for(int node=(int)nodeCount;node>=1;node--) {
         fst.nodeCount++;
-        final int address = writer.getPosition();
+        final long address = writer.getPosition();
 
         //System.out.println("  node: " + node + " address=" + address);
         if (address != newNodeAddress.get(node)) {
-          addressError = address - (int) newNodeAddress.get(node);
+          addressError = address - newNodeAddress.get(node);
           //System.out.println("    change: " + (address - newNodeAddress[node]));
           changed = true;
           newNodeAddress.set(node, address);
@@ -1571,7 +1573,7 @@ public final class FST<T> {
           while(true) {  // iterate over all arcs for this node
             //System.out.println("    cycle next arc");
 
-            final int arcStartPos = writer.getPosition();
+            final long arcStartPos = writer.getPosition();
             nodeArcCount++;
 
             byte flags = 0;
@@ -1606,19 +1608,18 @@ public final class FST<T> {
               flags += BIT_ARC_HAS_OUTPUT;
             }
 
-            final Integer ptr;
-            final int absPtr;
+            final long absPtr;
             final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
             if (doWriteTarget) {
 
-              ptr = topNodeMap.get(arc.target);
+              final Integer ptr = topNodeMap.get(arc.target);
               if (ptr != null) {
                 absPtr = ptr;
               } else {
-                absPtr = topNodeMap.size() + (int) newNodeAddress.get(arc.target) + addressError;
+                absPtr = topNodeMap.size() + newNodeAddress.get((int) arc.target) + addressError;
               }
 
-              int delta = (int) (newNodeAddress.get(arc.target) + addressError - writer.getPosition() - 2);
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition() - 2;
               if (delta < 0) {
                 //System.out.println("neg: " + delta);
                 anyNegDelta = true;
@@ -1629,7 +1630,6 @@ public final class FST<T> {
                 flags |= BIT_TARGET_DELTA;
               }
             } else {
-              ptr = null;
               absPtr = 0;
             }
 
@@ -1650,7 +1650,7 @@ public final class FST<T> {
 
             if (doWriteTarget) {
 
-              int delta = (int) (newNodeAddress.get(arc.target) + addressError - writer.getPosition());
+              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition();
               if (delta < 0) {
                 anyNegDelta = true;
                 //System.out.println("neg: " + delta);
@@ -1659,7 +1659,7 @@ public final class FST<T> {
 
               if (flag(flags, BIT_TARGET_DELTA)) {
                 //System.out.println("        delta");
-                writer.writeVInt(delta);
+                writer.writeVLong(delta);
                 if (!retry) {
                   deltaCount++;
                 }
@@ -1671,7 +1671,7 @@ public final class FST<T> {
                   System.out.println("        abs");
                 }
                 */
-                writer.writeVInt(absPtr);
+                writer.writeVLong(absPtr);
                 if (!retry) {
                   if (absPtr >= topNodeMap.size()) {
                     absCount++;
@@ -1683,7 +1683,7 @@ public final class FST<T> {
             }
 
             if (useArcArray) {
-              final int arcBytes = writer.getPosition() - arcStartPos;
+              final int arcBytes = (int) (writer.getPosition() - arcStartPos);
               //System.out.println("  " + arcBytes + " bytes");
               maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
               // NOTE: this may in fact go "backwards", if
@@ -1693,7 +1693,7 @@ public final class FST<T> {
               // will retry (below) so it's OK to ovewrite
               // bytes:
               //wasted += bytesPerArc - arcBytes;
-              writer.skipBytes(arcStartPos + bytesPerArc - writer.getPosition());
+              writer.skipBytes((int) (arcStartPos + bytesPerArc - writer.getPosition()));
             }
 
             if (arc.isLast()) {
@@ -1743,8 +1743,8 @@ public final class FST<T> {
     }
 
     long maxAddress = 0;
-    for (int key : topNodeMap.keySet()) {
-      maxAddress = Math.max(maxAddress, newNodeAddress.get(key));
+    for (long key : topNodeMap.keySet()) {
+      maxAddress = Math.max(maxAddress, newNodeAddress.get((int) key));
     }
 
     PackedInts.Mutable nodeRefToAddressIn = PackedInts.getMutable(topNodeMap.size(),
@@ -1754,7 +1754,7 @@ public final class FST<T> {
     }
     fst.nodeRefToAddress = nodeRefToAddressIn;
     
-    fst.startNode = (int) newNodeAddress.get(startNode);
+    fst.startNode = newNodeAddress.get((int) startNode);
     //System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
 
     if (emptyOutput != null) {

Modified: 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=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java Mon Jan 14 18:31:17 2013
@@ -46,13 +46,13 @@ final class ForwardBytesReader extends F
   }
 
   @Override
-  public int getPosition() {
+  public long getPosition() {
     return pos;
   }
 
   @Override
-  public void setPosition(int pos) {
-    this.pos = pos;
+  public void setPosition(long pos) {
+    this.pos = (int) pos;
   }
 
   @Override

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=1433026&r1=1433025&r2=1433026&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 Mon Jan 14 18:31:17 2013
@@ -19,10 +19,13 @@ 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;
@@ -30,13 +33,13 @@ final class NodeHash<T> {
   private final FST.BytesReader in;
 
   public NodeHash(FST<T> fst, FST.BytesReader in) {
-    table = new int[16];
+    table = new GrowableWriter(8, 16, PackedInts.COMPACT);
     mask = 15;
     this.fst = fst;
     this.in = in;
   }
 
-  private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address) 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;
@@ -75,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) {
@@ -87,7 +91,7 @@ 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;
     //System.out.println("hash frozen node=" + node);
     int h = 0;
@@ -95,7 +99,7 @@ final class NodeHash<T> {
     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() + " 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()) {
@@ -110,21 +114,21 @@ 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);
+  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;
@@ -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/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=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java Mon Jan 14 18:31:17 2013
@@ -44,13 +44,13 @@ final class ReverseBytesReader extends F
   }
 
   @Override
-  public int getPosition() {
+  public long getPosition() {
     return pos;
   }
 
   @Override
-  public void setPosition(int pos) {
-    this.pos = pos;
+  public void setPosition(long pos) {
+    this.pos = (int) pos;
   }
 
   @Override

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=1433026&r1=1433025&r2=1433026&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 Mon Jan 14 18:31:17 2013
@@ -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";
@@ -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;

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java?rev=1433026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java Mon Jan 14 18:31:17 2013
@@ -0,0 +1,261 @@
+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 java.util.Random;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TimeUnits;
+import org.apache.lucene.util.packed.PackedInts;
+import org.junit.Ignore;
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
+
+@Ignore("Requires tons of heap to run (10G works)")
+@TimeoutSuite(millis = 100 * TimeUnits.HOUR)
+public class Test2BFST extends LuceneTestCase {
+
+  private static long LIMIT = 3L*1024*1024*1024;
+
+  public void test() throws Exception {
+    int[] ints = new int[7];
+    IntsRef input = new IntsRef(ints, 0, ints.length);
+    long seed = random().nextLong();
+
+    for(int doPackIter=0;doPackIter<2;doPackIter++) {
+      boolean doPack = doPackIter == 1;
+
+      // Build FST w/ NoOutputs and stop when nodeCount > 3B
+      if (!doPack) {
+        System.out.println("\nTEST: 3B nodes; doPack=false output=NO_OUTPUTS");
+        Outputs<Object> outputs = NoOutputs.getSingleton();
+        Object NO_OUTPUT = outputs.getNoOutput();
+        final Builder<Object> b = new Builder<Object>(FST.INPUT_TYPE.BYTE1, 0, 0, false, false, Integer.MAX_VALUE, outputs,
+                                                      null, doPack, PackedInts.COMPACT, true, 15);
+
+        int count = 0;
+        Random r = new Random(seed);
+        int[] ints2 = new int[200];
+        IntsRef input2 = new IntsRef(ints2, 0, ints2.length);
+        while(true) {
+          //System.out.println("add: " + input + " -> " + output);
+          for(int i=10;i<ints2.length;i++) {
+            ints2[i] = r.nextInt(256);
+          }
+          b.add(input2, NO_OUTPUT);
+          count++;
+          if (count % 100000 == 0) {
+            System.out.println(count + ": " + b.fstSizeInBytes() + " bytes; " + b.getTotStateCount() + " nodes");
+          }
+          if (b.getTotStateCount() > LIMIT) {
+            break;
+          }
+          nextInput(r, ints2);
+        }
+
+        FST<Object> fst = b.finish();
+
+        System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
+
+        Arrays.fill(ints2, 0);
+        r = new Random(seed);
+
+        for(int i=0;i<count;i++) {
+          if (i % 1000000 == 0) {
+            System.out.println(i + "...: ");
+          }
+          for(int j=10;j<ints2.length;j++) {
+            ints2[j] = r.nextInt(256);
+          }
+          assertEquals(NO_OUTPUT, Util.get(fst, input2));
+          nextInput(r, ints2);
+        }
+
+        System.out.println("\nTEST: enum all input/outputs");
+        IntsRefFSTEnum<Object> fstEnum = new IntsRefFSTEnum<Object>(fst);
+
+        Arrays.fill(ints2, 0);
+        r = new Random(seed);
+        int upto = 0;
+        while(true) {
+          IntsRefFSTEnum.InputOutput<Object> pair = fstEnum.next();
+          if (pair == null) {
+            break;
+          }
+          for(int j=10;j<ints2.length;j++) {
+            ints2[j] = r.nextInt(256);
+          }
+          assertEquals(input2, pair.input);
+          assertEquals(NO_OUTPUT, pair.output);
+          upto++;
+          nextInput(r, ints2);
+        }
+        assertEquals(count, upto);
+      }
+
+      // Build FST w/ ByteSequenceOutputs and stop when FST
+      // size = 3GB
+      {
+        System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=bytes");
+        Outputs<BytesRef> outputs = ByteSequenceOutputs.getSingleton();
+        final Builder<BytesRef> b = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
+                                                          null, doPack, PackedInts.COMPACT, true, 15);
+
+        byte[] outputBytes = new byte[20];
+        BytesRef output = new BytesRef(outputBytes);
+        Arrays.fill(ints, 0);
+        int count = 0;
+        Random r = new Random(seed);
+        while(true) {
+          r.nextBytes(outputBytes);
+          //System.out.println("add: " + input + " -> " + output);
+          b.add(input, BytesRef.deepCopyOf(output));
+          count++;
+          if (count % 1000000 == 0) {
+            System.out.println(count + "...: " + b.fstSizeInBytes() + " bytes");
+          }
+          if (b.fstSizeInBytes() > LIMIT) {
+            break;
+          }
+          nextInput(r, ints);
+        }
+
+        FST<BytesRef> fst = b.finish();
+
+        System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
+
+        r = new Random(seed);
+        Arrays.fill(ints, 0);
+
+        for(int i=0;i<count;i++) {
+          if (i % 1000000 == 0) {
+            System.out.println(i + "...: ");
+          }
+          r.nextBytes(outputBytes);
+          assertEquals(output, Util.get(fst, input));
+          nextInput(r, ints);
+        }
+
+        System.out.println("\nTEST: enum all input/outputs");
+        IntsRefFSTEnum<BytesRef> fstEnum = new IntsRefFSTEnum<BytesRef>(fst);
+
+        Arrays.fill(ints, 0);
+        r = new Random(seed);
+        int upto = 0;
+        while(true) {
+          IntsRefFSTEnum.InputOutput<BytesRef> pair = fstEnum.next();
+          if (pair == null) {
+            break;
+          }
+          assertEquals(input, pair.input);
+          r.nextBytes(outputBytes);
+          assertEquals(output, pair.output);
+          upto++;
+          nextInput(r, ints);
+        }
+        assertEquals(count, upto);
+      }
+
+      // Build FST w/ PositiveIntOutputs and stop when FST
+      // size = 3GB
+      {
+        System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=long");
+        Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
+        final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
+                                                  null, doPack, PackedInts.COMPACT, true, 15);
+
+        long output = 1;
+
+        Arrays.fill(ints, 0);
+        int count = 0;
+        Random r = new Random(seed);
+        while(true) {
+          //System.out.println("add: " + input + " -> " + output);
+          b.add(input, output);
+          output += 1+r.nextInt(10);
+          count++;
+          if (count % 1000000 == 0) {
+            System.out.println(count + "...: " + b.fstSizeInBytes() + " bytes");
+          }
+          if (b.fstSizeInBytes() > LIMIT) {
+            break;
+          }
+          nextInput(r, ints);
+        }
+
+        FST<Long> fst = b.finish();
+
+        System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
+
+        Arrays.fill(ints, 0);
+
+        output = 1;
+        r = new Random(seed);
+        for(int i=0;i<count;i++) {
+          if (i % 1000000 == 0) {
+            System.out.println(i + "...: ");
+          }
+
+          // forward lookup:
+          assertEquals(output, Util.get(fst, input).longValue());
+          // reverse lookup:
+          assertEquals(input, Util.getByOutput(fst, output));
+          output += 1 + r.nextInt(10);
+          nextInput(r, ints);
+        }
+
+        System.out.println("\nTEST: enum all input/outputs");
+        IntsRefFSTEnum<Long> fstEnum = new IntsRefFSTEnum<Long>(fst);
+
+        Arrays.fill(ints, 0);
+        r = new Random(seed);
+        int upto = 0;
+        output = 1;
+        while(true) {
+          IntsRefFSTEnum.InputOutput<Long> pair = fstEnum.next();
+          if (pair == null) {
+            break;
+          }
+          assertEquals(input, pair.input);
+          assertEquals(output, pair.output.longValue());
+          output += 1 + r.nextInt(10);
+          upto++;
+          nextInput(r, ints);
+        }
+        assertEquals(count, upto);
+      }
+    }
+  }
+
+  private void nextInput(Random r, int[] ints) {
+    int downTo = 6;
+    while(downTo >= 0) {
+      // Must add random amounts (and not just 1) because
+      // otherwise FST outsmarts us and remains tiny:
+      ints[downTo] += 1+r.nextInt(10);
+      if (ints[downTo] < 256) {
+        break;
+      } else {
+        ints[downTo] = 0;
+        downTo--;
+      }
+    }
+  }
+}

Modified: 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=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java Mon Jan 14 18:31:17 2013
@@ -321,10 +321,10 @@ public class TestBytesStore extends Luce
 
         if (reversed) {
           expectedPos = pos-numBytes;
-          left = r.getPosition();
+          left = (int) r.getPosition();
         } else {
           expectedPos = pos+numBytes;
-          left = totalLength - r.getPosition();
+          left = (int) (totalLength - r.getPosition());
         }
         assertEquals(expectedPos, r.getPosition());
 

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=1433026&r1=1433025&r2=1433026&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 Mon Jan 14 18:31:17 2013
@@ -310,7 +310,7 @@ public class TestFSTs extends LuceneTest
 
     final boolean doRewrite = random().nextBoolean();
 
-    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, true);
+    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, PackedInts.DEFAULT, true, 15);
 
     boolean storeOrd = random().nextBoolean();
     if (VERBOSE) {
@@ -453,7 +453,7 @@ public class TestFSTs extends LuceneTest
       this.outputs = outputs;
       this.doPack = doPack;
 
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, !noArcArrays);
+      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -1073,7 +1073,7 @@ public class TestFSTs extends LuceneTest
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
 
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
     builder.add(Util.toUTF32("station", new IntsRef()), 10L);
     final FST<Long> fst = builder.finish();
@@ -1088,7 +1088,7 @@ public class TestFSTs extends LuceneTest
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
     final boolean willRewrite = random().nextBoolean();
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();
@@ -1111,7 +1111,7 @@ public class TestFSTs extends LuceneTest
     final Long nothing = outputs.getNoOutput();
     final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
-    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true);
+    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true, 15);
 
     final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);
 

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Mon Jan 14 18:31:17 2013
@@ -24,6 +24,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.*;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Finite state automata based implementation of "autocomplete" functionality.
@@ -237,7 +238,8 @@ public class FSTCompletionBuilder {
     final Object empty = outputs.getNoOutput();
     final Builder<Object> builder = new Builder<Object>(
         FST.INPUT_TYPE.BYTE1, 0, 0, true, true, 
-        shareMaxTailLength, outputs, null, false, true);
+        shareMaxTailLength, outputs, null, false, 
+        PackedInts.DEFAULT, true, 15);
     
     BytesRef scratch = new BytesRef();
     BytesRef entry;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java?rev=1433026&r1=1433025&r2=1433026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java Mon Jan 14 18:31:17 2013
@@ -40,6 +40,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.packed.PackedInts;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -288,7 +289,9 @@ public class FSTTester<T> {
                                               outputs,
                                               null,
                                               willRewrite,
-                                              true);
+                                              PackedInts.DEFAULT,
+                                              true,
+                                              15);
     if (LuceneTestCase.VERBOSE) {
       if (willRewrite) {
         System.out.println("TEST: packed FST");