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

svn commit: r1435141 [2/2] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/analysis/ lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/ lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/ lucene/analysis/kurom...

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

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

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

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (from r1432459, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java&r1=1432459&r2=1435141&rev=1435141&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java Fri Jan 18 14:05:37 2013
@@ -42,7 +42,7 @@ public class TestBytesStore extends Luce
 
       int pos = 0;
       while(pos < numBytes) {
-        int op = random().nextInt(7);
+        int op = random().nextInt(8);
         if (VERBOSE) {
           System.out.println("  cycle pos=" + pos);
         }
@@ -97,21 +97,21 @@ public class TestBytesStore extends Luce
         case 3:
           {
             // reverse bytes
-            if (pos > 0) {
-              int len = _TestUtil.nextInt(random(), 1, Math.min(100, pos));
+            if (pos > 1) {
+              int len = _TestUtil.nextInt(random(), 2, Math.min(100, pos));
               int start;
               if (len == pos) {
                 start = 0;
               } else {
                 start = random().nextInt(pos - len);
               }
-              int end = start + len;
+              int end = start + len - 1;
               if (VERBOSE) {
-                System.out.println("    reverse start=" + start + " end=" + end + " len=" + len);
+                System.out.println("    reverse start=" + start + " end=" + end + " len=" + len + " pos=" + pos);
               }
               bytes.reverse(start, end);
 
-              while(start < end) {
+              while(start <= end) {
                 byte b = expected[end];
                 expected[end] = expected[start];
                 expected[start] = b;
@@ -159,17 +159,49 @@ public class TestBytesStore extends Luce
           {
             // skip
             int len = random().nextInt(Math.min(100, numBytes - pos));
-            pos += len;
-            bytes.skip(len);
+
             if (VERBOSE) {
               System.out.println("    skip len=" + len);
             }
+
+            pos += len;
+            bytes.skipBytes(len);
+
+            // NOTE: must fill in zeros in case truncate was
+            // used, else we get false fails:
+            if (len > 0) {
+              byte[] zeros = new byte[len];
+              bytes.writeBytes(pos-len, zeros, 0, len);
+            }
           }
           break;
+
+        case 7:
+          {
+            // absWriteByte
+            if (pos > 0) {
+              int dest = random().nextInt(pos);
+              byte b = (byte) random().nextInt(256);
+              expected[dest] = b;
+              bytes.writeByte(dest, b);
+            }
+            break;
+          }
         }
 
         assertEquals(pos, bytes.getPosition());
 
+        if (pos > 0 && random().nextInt(50) == 17) {
+          // truncate
+          int len = _TestUtil.nextInt(random(), 1, Math.min(pos, 100));
+          bytes.truncate(pos - len);
+          pos -= len;
+          Arrays.fill(expected, pos, pos+len, (byte) 0);
+          if (VERBOSE) {
+            System.out.println("    truncate len=" + len + " newPos=" + pos);
+          }
+        }
+
         if ((pos > 0 && random().nextInt(200) == 17)) {
           verify(bytes, expected, pos);
         }
@@ -289,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/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Fri Jan 18 14:05:37 2013
@@ -310,7 +310,7 @@ public class TestFSTs extends LuceneTest
 
     final boolean doRewrite = random().nextBoolean();
 
-    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, true);
+    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, PackedInts.DEFAULT, true, 15);
 
     boolean storeOrd = random().nextBoolean();
     if (VERBOSE) {
@@ -453,7 +453,7 @@ public class TestFSTs extends LuceneTest
       this.outputs = outputs;
       this.doPack = doPack;
 
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, !noArcArrays);
+      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -484,8 +484,13 @@ public class TestFSTs extends LuceneTest
           }
         }
 
+        long tMid = System.currentTimeMillis();
+        System.out.println(((tMid-tStart) / 1000.0) + " sec to add all terms");
+
         assert builder.getTermCount() == ord;
         FST<T> fst = builder.finish();
+        long tEnd = System.currentTimeMillis();
+        System.out.println(((tEnd-tMid) / 1000.0) + " sec to finish/pack");
         if (fst == null) {
           System.out.println("FST was fully pruned!");
           System.exit(0);
@@ -513,6 +518,12 @@ public class TestFSTs extends LuceneTest
           return;
         }
 
+        /*
+        IndexInput in = dir.openInput("fst.bin", IOContext.DEFAULT);
+        fst = new FST<T>(in, outputs);
+        in.close();
+        */
+
         System.out.println("\nNow verify...");
 
         while(true) {
@@ -576,7 +587,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
+  // java -cp ../build/codecs/classes/java:../test-framework/lib/randomizedtesting-runner-2.0.8.jar:../build/core/classes/test:../build/core/classes/test-framework:../build/core/classes/java:../build/test-framework/classes/java:../test-framework/lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /xold/tmp/allTerms3.txt out
   public static void main(String[] args) throws IOException {
     int prune = 0;
     int limit = Integer.MAX_VALUE;
@@ -1022,7 +1033,7 @@ public class TestFSTs extends LuceneTest
         throws IOException {
         if (FST.targetHasArcs(arc)) {
           int childCount = 0;
-          FST.BytesReader fstReader = fst.getBytesReader(0);
+          BytesReader fstReader = fst.getBytesReader();
           for (arc = fst.readFirstTargetArc(arc, arc, fstReader);; 
                arc = fst.readNextArc(arc, fstReader), childCount++)
           {
@@ -1062,7 +1073,7 @@ public class TestFSTs extends LuceneTest
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
 
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
     builder.add(Util.toUTF32("station", new IntsRef()), 10L);
     final FST<Long> fst = builder.finish();
@@ -1077,7 +1088,7 @@ public class TestFSTs extends LuceneTest
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
     final boolean willRewrite = random().nextBoolean();
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();
@@ -1100,7 +1111,7 @@ public class TestFSTs extends LuceneTest
     final Long nothing = outputs.getNoOutput();
     final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
-    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true);
+    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true, 15);
 
     final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);
 
@@ -1157,12 +1168,12 @@ public class TestFSTs extends LuceneTest
     assertEquals(nothing, startArc.nextFinalOutput);
 
     FST.Arc<Long> arc = fst.readFirstTargetArc(startArc, new FST.Arc<Long>(),
-                                               fst.getBytesReader(0));
+                                               fst.getBytesReader());
     assertEquals('a', arc.label);
     assertEquals(17, arc.nextFinalOutput.longValue());
     assertTrue(arc.isFinal());
 
-    arc = fst.readNextArc(arc, fst.getBytesReader(0));
+    arc = fst.readNextArc(arc, fst.getBytesReader());
     assertEquals('b', arc.label);
     assertFalse(arc.isFinal());
     assertEquals(42, arc.output.longValue());
@@ -1292,7 +1303,7 @@ public class TestFSTs extends LuceneTest
     //Util.toDot(fst, w, false, false);
     //w.close();
     
-    BytesReader reader = fst.getBytesReader(0);
+    BytesReader reader = fst.getBytesReader();
     
     //System.out.println("testing: " + allPrefixes.size() + " prefixes");
     for (String prefix : allPrefixes) {
@@ -1413,7 +1424,7 @@ public class TestFSTs extends LuceneTest
     //Util.toDot(fst, w, false, false);
     //w.close();
     
-    BytesReader reader = fst.getBytesReader(0);
+    BytesReader reader = fst.getBytesReader();
     
     //System.out.println("testing: " + allPrefixes.size() + " prefixes");
     for (String prefix : allPrefixes) {

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Fri Jan 18 14:05:37 2013
@@ -587,7 +587,7 @@ public class AnalyzingSuggester extends 
 
       //System.out.println("  prefixPaths: " + prefixPaths.size());
 
-      BytesReader bytesReader = fst.getBytesReader(0);
+      BytesReader bytesReader = fst.getBytesReader();
 
       FST.Arc<Pair<Long,BytesRef>> scratchArc = new FST.Arc<Pair<Long,BytesRef>>();
 

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Fri Jan 18 14:05:37 2013
@@ -77,7 +77,7 @@ public class FSTUtil {
         new IntsRef()));
     
     final FST.Arc<T> scratchArc = new FST.Arc<T>();
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
     
     while (queue.size() != 0) {
       final Path<T> path = queue.remove(queue.size() - 1);

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java Fri Jan 18 14:05:37 2013
@@ -139,7 +139,7 @@ public class FSTCompletion {
     try {
       List<Arc<Object>> rootArcs = new ArrayList<Arc<Object>>();
       Arc<Object> arc = automaton.getFirstArc(new Arc<Object>());
-      FST.BytesReader fstReader = automaton.getBytesReader(0);
+      FST.BytesReader fstReader = automaton.getBytesReader();
       automaton.readFirstTargetArc(arc, arc, fstReader);
       while (true) {
         rootArcs.add(new Arc<Object>().copyFrom(arc));
@@ -173,7 +173,7 @@ public class FSTCompletion {
     // Get the UTF-8 bytes representation of the input key.
     try {
       final FST.Arc<Object> scratch = new FST.Arc<Object>();
-      FST.BytesReader fstReader = automaton.getBytesReader(0);
+      FST.BytesReader fstReader = automaton.getBytesReader();
       for (; rootArcIndex < rootArcs.length; rootArcIndex++) {
         final FST.Arc<Object> rootArc = rootArcs[rootArcIndex];
         final FST.Arc<Object> arc = scratch.copyFrom(rootArc);
@@ -338,7 +338,7 @@ public class FSTCompletion {
     final int max = utf8.offset + utf8.length;
     // Cannot save as instance var since multiple threads
     // can use FSTCompletion at once...
-    final FST.BytesReader fstReader = automaton.getBytesReader(0);
+    final FST.BytesReader fstReader = automaton.getBytesReader();
     for (int i = utf8.offset; i < max; i++) {
       if (automaton.findTargetArc(utf8.bytes[i] & 0xff, arc, arc, fstReader) == null) {
         // No matching prefixes, return an empty result.
@@ -362,7 +362,7 @@ public class FSTCompletion {
     }
     assert output.offset == 0;
     output.bytes[output.length++] = (byte) arc.label;
-    FST.BytesReader fstReader = automaton.getBytesReader(0);
+    FST.BytesReader fstReader = automaton.getBytesReader();
     automaton.readFirstTargetArc(arc, arc, fstReader);
     while (true) {
       if (arc.label == FST.END_LABEL) {

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Fri Jan 18 14:05:37 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/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Fri Jan 18 14:05:37 2013
@@ -200,7 +200,7 @@ public class WFSTCompletionLookup extend
   private Long lookupPrefix(BytesRef scratch, Arc<Long> arc) throws /*Bogus*/IOException {
     assert 0 == fst.outputs.getNoOutput().longValue();
     long output = 0;
-    BytesReader bytesReader = fst.getBytesReader(0);
+    BytesReader bytesReader = fst.getBytesReader();
     
     fst.getFirstArc(arc);
     

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java Fri Jan 18 14:05:37 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;
@@ -203,7 +204,7 @@ public class FSTTester<T> {
     final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
     final T NO_OUTPUT = fst.outputs.getNoOutput();
     T output = NO_OUTPUT;
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
 
     for(int i=0;i<=term.length;i++) {
       final int label;
@@ -240,7 +241,7 @@ public class FSTTester<T> {
     in.offset = 0;
     final T NO_OUTPUT = fst.outputs.getNoOutput();
     T output = NO_OUTPUT;
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
 
     while(true) {
       // read all arcs:
@@ -288,7 +289,16 @@ public class FSTTester<T> {
                                               outputs,
                                               null,
                                               willRewrite,
-                                              true);
+                                              PackedInts.DEFAULT,
+                                              true,
+                                              15);
+    if (LuceneTestCase.VERBOSE) {
+      if (willRewrite) {
+        System.out.println("TEST: packed FST");
+      } else {
+        System.out.println("TEST: non-packed FST");
+      }
+    }
 
     for(InputOutput<T> pair : pairs) {
       if (pair.output instanceof List) {