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 2012/01/29 23:48:46 UTC

svn commit: r1237500 [2/2] - in /lucene/dev/trunk: lucene/ lucene/src/java/org/apache/lucene/codecs/ lucene/src/java/org/apache/lucene/codecs/simpletext/ lucene/src/java/org/apache/lucene/util/ lucene/src/java/org/apache/lucene/util/fst/ lucene/src/tes...

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/fst/Util.java Sun Jan 29 22:48:45 2012
@@ -37,23 +37,21 @@ 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);
+
     // Accumulate output as we go
-    final T NO_OUTPUT = fst.outputs.getNoOutput();
-    T output = NO_OUTPUT;
+    T output = fst.outputs.getNoOutput();
     for(int i=0;i<input.length;i++) {
-      if (fst.findTargetArc(input.ints[input.offset + i], arc, arc) == null) {
+      if (fst.findTargetArc(input.ints[input.offset + i], arc, arc, fstReader) == null) {
         return null;
-      } else if (arc.output != NO_OUTPUT) {
-        output = fst.outputs.add(output, arc.output);
       }
+      output = fst.outputs.add(output, arc.output);
     }
 
-    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
-      return null;
-    } else if (arc.output != NO_OUTPUT) {
-      return fst.outputs.add(output, arc.output);
+    if (arc.isFinal()) {
+      return fst.outputs.add(output, arc.nextFinalOutput);
     } else {
-      return output;
+      return null;
     }
   }
 
@@ -64,26 +62,24 @@ 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);
+
     // TODO: would be nice not to alloc this on every lookup
     final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
 
     // Accumulate output as we go
-    final T NO_OUTPUT = fst.outputs.getNoOutput();
-    T output = NO_OUTPUT;
+    T output = fst.outputs.getNoOutput();
     for(int i=0;i<input.length;i++) {
-      if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc) == null) {
+      if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc, fstReader) == null) {
         return null;
-      } else if (arc.output != NO_OUTPUT) {
-        output = fst.outputs.add(output, arc.output);
       }
+      output = fst.outputs.add(output, arc.output);
     }
 
-    if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
-      return null;
-    } else if (arc.output != NO_OUTPUT) {
-      return fst.outputs.add(output, arc.output);
+    if (arc.isFinal()) {
+      return fst.outputs.add(output, arc.nextFinalOutput);
     } else {
-      return output;
+      return null;
     }
   }
 
@@ -142,7 +138,7 @@ public final class Util {
           result.grow(1+upto);
         }
         
-        fst.readFirstRealArc(arc.target, arc, in);
+        fst.readFirstRealTargetArc(arc.target, arc, in);
 
         FST.Arc<Long> prevArc = null;
 
@@ -238,6 +234,7 @@ public final class Util {
     // A queue of transitions to consider when processing the next level.
     final List<FST.Arc<T>> nextLevelQueue = new ArrayList<FST.Arc<T>>();
     nextLevelQueue.add(startArc);
+    //System.out.println("toDot: startArc: " + startArc);
     
     // A list of states on the same level (for ranking).
     final List<Integer> sameLevelStates = new ArrayList<Integer>();
@@ -289,8 +286,11 @@ public final class Util {
 
     int level = 0;
 
+    final FST.BytesReader r = fst.getBytesReader(0);
+
     while (!nextLevelQueue.isEmpty()) {
       // we could double buffer here, but it doesn't matter probably.
+      //System.out.println("next level=" + level);
       thisLevelQueue.addAll(nextLevelQueue);
       nextLevelQueue.clear();
 
@@ -298,19 +298,19 @@ public final class Util {
       out.write("\n  // Transitions and states at level: " + level + "\n");
       while (!thisLevelQueue.isEmpty()) {
         final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
+        //System.out.println("  pop: " + arc);
         if (fst.targetHasArcs(arc)) {
-          // scan all arcs
+          // scan all target arcs
+          //System.out.println("  readFirstTarget...");
           final int node = arc.target;
-          fst.readFirstTargetArc(arc, arc);
 
-          if (arc.label == FST.END_LABEL) {
-            // Skip it -- prior recursion took this into account already
-            assert !arc.isLast();
-            fst.readNextArc(arc);
-          }
+          fst.readFirstRealTargetArc(arc.target, arc, r);
+
+          //System.out.println("    firstTarget: " + arc);
 
           while (true) {
 
+            //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)) {
 
@@ -329,7 +329,7 @@ public final class Util {
               if (fst.isExpandedTarget(arc)) {
                 stateColor = expandedNodeColor;
               } else {
-               stateColor = null;
+                stateColor = null;
               }
 
               final String finalOutput;
@@ -339,7 +339,9 @@ public final class Util {
                 finalOutput = "";
               }
 
-              emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput);
+              emitDotState(out, Integer.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);
               nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
               sameLevelStates.add(arc.target);
@@ -362,14 +364,22 @@ public final class Util {
               outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
             }
 
+            final String arcColor;
+            if (arc.flag(FST.BIT_TARGET_NEXT)) {
+              arcColor = "red";
+            } else {
+              arcColor = "black";
+            }
+
             assert arc.label != FST.END_LABEL;
-            out.write("  " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n");
+            out.write("  " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"" + (arc.isFinal() ? " style=\"bold\"" : "" ) + " color=\"" + arcColor + "\"]\n");
                    
             // Break the loop if we're on the last arc of this state.
             if (arc.isLast()) {
+              //System.out.println("    break");
               break;
             }
-            fst.readNextArc(arc);
+            fst.readNextRealArc(arc, r);
           }
         }
       }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java Sun Jan 29 22:48:45 2012
@@ -89,11 +89,11 @@ public class TestFSTs extends LuceneTest
     return br;
   }
 
-  private static IntsRef toIntsRef(String s, int inputMode) {
+  static IntsRef toIntsRef(String s, int inputMode) {
     return toIntsRef(s, inputMode, new IntsRef(10));
   }
 
-  private static IntsRef toIntsRef(String s, int inputMode, IntsRef ir) {
+  static IntsRef toIntsRef(String s, int inputMode, IntsRef ir) {
     if (inputMode == 0) {
       // utf8
       return toIntsRef(new BytesRef(s), ir);
@@ -103,7 +103,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  private static IntsRef toIntsRefUTF32(String s, IntsRef ir) {
+  static IntsRef toIntsRefUTF32(String s, IntsRef ir) {
     final int charLength = s.length();
     int charIdx = 0;
     int intIdx = 0;
@@ -120,7 +120,7 @@ public class TestFSTs extends LuceneTest
     return ir;
   }
 
-  private static IntsRef toIntsRef(BytesRef br, IntsRef ir) {
+  static IntsRef toIntsRef(BytesRef br, IntsRef ir) {
     if (br.length > ir.ints.length) {
       ir.grow(br.length);
     }
@@ -172,7 +172,7 @@ public class TestFSTs extends LuceneTest
         final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
         final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms2.length);
         for(int idx=0;idx<terms2.length;idx++) {
-          pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], outputs.get(idx)));
+          pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], (long) idx));
         }
         final FST<Long> fst = new FSTTester<Long>(random, dir, inputMode, pairs, outputs, true).doTest(0, 0, false);
         assertNotNull(fst);
@@ -230,7 +230,7 @@ public class TestFSTs extends LuceneTest
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
-        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], outputs.get(idx)));
+        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], (long) idx));
       }
       new FSTTester<Long>(random, dir, inputMode, pairs, outputs, true).doTest();
     }
@@ -244,7 +244,7 @@ public class TestFSTs extends LuceneTest
       for(int idx=0;idx<terms.length;idx++) {
         final long value = lastOutput + _TestUtil.nextInt(random, 1, 1000);
         lastOutput = value;
-        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], outputs.get(value)));
+        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], value));
       }
       new FSTTester<Long>(random, dir, inputMode, pairs, outputs, doShare).doTest();
     }
@@ -254,7 +254,7 @@ public class TestFSTs extends LuceneTest
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random.nextBoolean());
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
-        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], outputs.get(random.nextLong()) & Long.MAX_VALUE));
+        pairs.add(new FSTTester.InputOutput<Long>(terms[idx], random.nextLong() & Long.MAX_VALUE));
       }
       new FSTTester<Long>(random, dir, inputMode, pairs, outputs, false).doTest();
     }
@@ -270,8 +270,7 @@ public class TestFSTs extends LuceneTest
         final long value = lastOutput + _TestUtil.nextInt(random, 1, 1000);
         lastOutput = value;
         pairs.add(new FSTTester.InputOutput<PairOutputs.Pair<Long,Long>>(terms[idx],
-                                                                         outputs.get(o1.get(idx),
-                                                                                     o2.get(value))));
+                                                                         outputs.newPair((long) idx, value)));
       }
       new FSTTester<PairOutputs.Pair<Long,Long>>(random, dir, inputMode, pairs, outputs, false).doTest();
     }
@@ -393,6 +392,7 @@ public class TestFSTs extends LuceneTest
       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);
 
       for(int i=0;i<=term.length;i++) {
         final int label;
@@ -401,8 +401,9 @@ public class TestFSTs extends LuceneTest
         } else {
           label = term.ints[term.offset+i];
         }
-        //System.out.println("   loop i=" + i + " label=" + label + " output=" + fst.outputs.outputToString(output) + " curArc: target=" + arc.target + " isFinal?=" + arc.isFinal());
-        if (fst.findTargetArc(label, arc, arc) == null) {
+        // System.out.println("   loop i=" + i + " label=" + label + " output=" + fst.outputs.outputToString(output) + " curArc: target=" + arc.target + " isFinal?=" + arc.isFinal());
+        if (fst.findTargetArc(label, arc, arc, fstReader) == null) {
+          // System.out.println("    not found");
           if (prefixLength != null) {
             prefixLength[0] = i;
             return output;
@@ -462,16 +463,19 @@ public class TestFSTs extends LuceneTest
 
     FST<T> doTest(int prune1, int prune2, boolean allowRandomSuffixSharing) throws IOException {
       if (VERBOSE) {
-        System.out.println("TEST: prune1=" + prune1 + " prune2=" + prune2);
+        System.out.println("\nTEST: prune1=" + prune1 + " prune2=" + prune2);
       }
 
+      final boolean willRewrite = random.nextBoolean();
+
       final Builder<T> builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4,
                                                 prune1, prune2,
                                                 prune1==0 && prune2==0,
                                                 allowRandomSuffixSharing ? random.nextBoolean() : true,
                                                 allowRandomSuffixSharing ? _TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
                                                 outputs,
-                                                null);
+                                                null,
+                                                willRewrite);
 
       for(InputOutput<T> pair : pairs) {
         if (pair.output instanceof UpToTwoPositiveIntOutputs.TwoLongs) {
@@ -486,7 +490,7 @@ public class TestFSTs extends LuceneTest
       }
       FST<T> fst = builder.finish();
 
-      if (random.nextBoolean() && fst != null) {        
+      if (random.nextBoolean() && fst != null && !willRewrite) {
         TestFSTs t = new TestFSTs();
         IOContext context = t.newIOContext(random);
         IndexOutput out = dir.createOutput("fst.bin", context);
@@ -522,6 +526,21 @@ public class TestFSTs extends LuceneTest
         verifyPruned(inputMode, fst, prune1, prune2);
       }
 
+      if (willRewrite && fst != null) {
+        if (VERBOSE) {
+          System.out.println("TEST: now rewrite");
+        }
+        final FST<T> packed =fst.pack(_TestUtil.nextInt(random, 1, 10), _TestUtil.nextInt(random, 0, 10000000));
+        if (VERBOSE) {
+          System.out.println("TEST: now verify packed FST");
+        }
+        if (prune1 == 0 && prune2 == 0) {
+          verifyUnPruned(inputMode, packed);
+        } else {
+          verifyPruned(inputMode, packed, prune1, prune2);
+        }
+      }
+
       return fst;
     }
 
@@ -638,7 +657,7 @@ public class TestFSTs extends LuceneTest
       num = atLeast(100);
       for(int iter=0;iter<num;iter++) {
         if (VERBOSE) {
-          System.out.println("TEST: iter=" + iter);
+          System.out.println("  iter=" + iter);
         }
         if (random.nextBoolean()) {
           // seek to term that doesn't exist:
@@ -866,7 +885,15 @@ public class TestFSTs extends LuceneTest
             prefixes.put(IntsRef.deepCopyOf(scratch), cmo);
           } else {
             cmo.count++;
-            cmo.output = outputs.common(cmo.output, pair.output);
+            T output1 = cmo.output;
+            if (output1.equals(outputs.getNoOutput())) {
+              output1 = outputs.getNoOutput();
+            }
+            T output2 = pair.output;
+            if (output2.equals(outputs.getNoOutput())) {
+              output2 = outputs.getNoOutput();
+            }
+            cmo.output = outputs.common(output1, output2);
           }
           if (idx == pair.input.length) {
             cmo.isFinal = true;
@@ -992,7 +1019,7 @@ public class TestFSTs extends LuceneTest
 
   public void testRandomWords() throws IOException {
     testRandomWords(1000, atLeast(2));
-    //testRandomWords(20, 100);
+    //testRandomWords(100, 1);
   }
 
   String inputModeToString(int mode) {
@@ -1055,50 +1082,6 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  // NOTE: this test shows a case where our current builder
-  // fails to produce minimal FST:
-  /*
-  public void test3() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
-    IntsRef scratchIntsRef = new IntsRef();
-    builder.add(Util.toIntsRef(new BytesRef("aa$"), scratchIntsRef), outputs.get(0));
-    builder.add(Util.toIntsRef(new BytesRef("aab$"), scratchIntsRef), 1L);
-    builder.add(Util.toIntsRef(new BytesRef("bbb$"), scratchIntsRef), 2L);
-    final FST<Long> fst = builder.finish();
-    //System.out.println("NODES " + fst.getNodeCount() + " ARCS " + fst.getArcCount());
-    // NOTE: we produce 7 nodes today
-    assertEquals(6, fst.getNodeCount());
-    // NOTE: we produce 8 arcs today
-    assertEquals(7, fst.getNodeCount());
-    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
-    //Util.toDot(fst, w, false, false);
-    //w.close();
-  }
-  */
-
-  // NOTE: this test shows a case where our current builder
-  // fails to produce minimal FST:
-  /*
-  public void test4() throws Exception {
-    final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
-    Builder<BytesRef> builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, outputs);
-    IntsRef scratchIntsRef = new IntsRef();
-    builder.add(Util.toIntsRef(new BytesRef("aa$"), scratchIntsRef), outputs.getNoOutput());
-    builder.add(Util.toIntsRef(new BytesRef("aab$"), scratchIntsRef), new BytesRef("1"));
-    builder.add(Util.toIntsRef(new BytesRef("bbb$"), scratchIntsRef), new BytesRef("11"));
-    final FST<BytesRef> fst = builder.finish();
-    //System.out.println("NODES " + fst.getNodeCount() + " ARCS " + fst.getArcCount());
-    // NOTE: we produce 7 nodes today
-    assertEquals(6, fst.getNodeCount());
-    // NOTE: we produce 8 arcs today
-    assertEquals(7, fst.getNodeCount());
-    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
-    //Util.toDot(fst, w, false, false);
-    //w.close();
-  }
-  */
-
   // Build FST for all unique terms in the test line docs
   // file, up until a time limit
   public void testRealTerms() throws Exception {
@@ -1126,7 +1109,10 @@ public class TestFSTs extends LuceneTest
     IndexReader r = IndexReader.open(writer, true);
     writer.close();
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random.nextBoolean());
-    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
+
+    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);
 
     boolean storeOrd = random.nextBoolean();
     if (VERBOSE) {
@@ -1162,59 +1148,69 @@ public class TestFSTs extends LuceneTest
         } else {
           output = termsEnum.docFreq();
         }
-        builder.add(Util.toIntsRef(term, scratchIntsRef), outputs.get(output));
+        builder.add(Util.toIntsRef(term, scratchIntsRef), (long) output);
         ord++;
         if (VERBOSE && ord % 100000 == 0 && LuceneTestCase.TEST_NIGHTLY) {
           System.out.println(ord + " terms...");
         }
       }
-      final FST<Long> fst = builder.finish();
+      FST<Long> fst = builder.finish();
       if (VERBOSE) {
         System.out.println("FST: " + docCount + " docs; " + ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs;" + " " + fst.sizeInBytes() + " bytes");
       }
 
       if (ord > 0) {
-        // Now confirm BytesRefFSTEnum and TermsEnum act the
-        // same:
-        final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
-        int num = atLeast(1000);
-        for(int iter=0;iter<num;iter++) {
-          final BytesRef randomTerm = new BytesRef(getRandomString());
-        
-          if (VERBOSE) {
-            System.out.println("TEST: seek non-exist " + randomTerm.utf8ToString() + " " + randomTerm);
+        for(int rewriteIter=0;rewriteIter<2;rewriteIter++) {
+          if (rewriteIter == 1) {
+            if (doRewrite) {
+              // Verify again, with packed FST:
+              fst = fst.pack(_TestUtil.nextInt(random, 1, 10), _TestUtil.nextInt(random, 0, 10000000));
+            } else {
+              break;
+            }
           }
+          // Now confirm BytesRefFSTEnum and TermsEnum act the
+          // same:
+          final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
+          int num = atLeast(1000);
+          for(int iter=0;iter<num;iter++) {
+            final BytesRef randomTerm = new BytesRef(getRandomString());
+        
+            if (VERBOSE) {
+              System.out.println("TEST: seek non-exist " + randomTerm.utf8ToString() + " " + randomTerm);
+            }
 
-          final TermsEnum.SeekStatus seekResult = termsEnum.seekCeil(randomTerm);
-          final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.seekCeil(randomTerm);
+            final TermsEnum.SeekStatus seekResult = termsEnum.seekCeil(randomTerm);
+            final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.seekCeil(randomTerm);
 
-          if (seekResult == TermsEnum.SeekStatus.END) {
-            assertNull("got " + (fstSeekResult == null ? "null" : fstSeekResult.input.utf8ToString()) + " but expected null", fstSeekResult);
-          } else {
-            assertSame(termsEnum, fstEnum, storeOrd);
-            for(int nextIter=0;nextIter<10;nextIter++) {
-              if (VERBOSE) {
-                System.out.println("TEST: next");
-                if (storeOrd) {
-                  System.out.println("  ord=" + termsEnum.ord());
-                }
-              }
-              if (termsEnum.next() != null) {
-                if (VERBOSE) {
-                  System.out.println("  term=" + termsEnum.term().utf8ToString());
-                }
-                assertNotNull(fstEnum.next());
-                assertSame(termsEnum, fstEnum, storeOrd);
-              } else {
+            if (seekResult == TermsEnum.SeekStatus.END) {
+              assertNull("got " + (fstSeekResult == null ? "null" : fstSeekResult.input.utf8ToString()) + " but expected null", fstSeekResult);
+            } else {
+              assertSame(termsEnum, fstEnum, storeOrd);
+              for(int nextIter=0;nextIter<10;nextIter++) {
                 if (VERBOSE) {
-                  System.out.println("  end!");
+                  System.out.println("TEST: next");
+                  if (storeOrd) {
+                    System.out.println("  ord=" + termsEnum.ord());
+                  }
                 }
-                BytesRefFSTEnum.InputOutput<Long> nextResult = fstEnum.next();
-                if (nextResult != null) {
-                  System.out.println("expected null but got: input=" + nextResult.input.utf8ToString() + " output=" + outputs.outputToString(nextResult.output));
-                  fail();
+                if (termsEnum.next() != null) {
+                  if (VERBOSE) {
+                    System.out.println("  term=" + termsEnum.term().utf8ToString());
+                  }
+                  assertNotNull(fstEnum.next());
+                  assertSame(termsEnum, fstEnum, storeOrd);
+                } else {
+                  if (VERBOSE) {
+                    System.out.println("  end!");
+                  }
+                  BytesRefFSTEnum.InputOutput<Long> nextResult = fstEnum.next();
+                  if (nextResult != null) {
+                    System.out.println("expected null but got: input=" + nextResult.input.utf8ToString() + " output=" + outputs.outputToString(nextResult.output));
+                    fail();
+                  }
+                  break;
                 }
-                break;
               }
             }
           }
@@ -1248,14 +1244,17 @@ public class TestFSTs extends LuceneTest
     private int inputMode;
     private final Outputs<T> outputs;
     private final Builder<T> builder;
+    private final boolean doPack;
 
-    public VisitTerms(String dirOut, String wordsFileIn, int inputMode, int prune, Outputs<T> outputs) {
+    public VisitTerms(String dirOut, String wordsFileIn, int inputMode, int prune, Outputs<T> outputs, boolean doPack, boolean noArcArrays) {
       this.dirOut = dirOut;
       this.wordsFileIn = wordsFileIn;
       this.inputMode = inputMode;
       this.outputs = outputs;
-      
-      builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null);
+      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);
+      builder.setAllowArrayArcs(!noArcArrays);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -1287,14 +1286,15 @@ public class TestFSTs extends LuceneTest
         }
 
         assert builder.getTermCount() == ord;
-        final FST<T> fst = builder.finish();
+        FST<T> fst = builder.finish();
         if (fst == null) {
           System.out.println("FST was fully pruned!");
           System.exit(0);
         }
 
-        if (dirOut == null)
+        if (dirOut == null) {
           return;
+        }
 
         System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes());
         if (fst.getNodeCount() < 100) {
@@ -1304,12 +1304,17 @@ public class TestFSTs extends LuceneTest
           System.out.println("Wrote FST to out.dot");
         }
 
-        Directory dir = FSDirectory.open(new File(dirOut));
-        IndexOutput out = dir.createOutput("fst.bin", IOContext.DEFAULT);
-        fst.save(out);
-        out.close();
-
-        System.out.println("Saved FST to fst.bin.");
+        if (doPack) {
+          System.out.println("Pack...");
+          fst = fst.pack(4, 100000000);
+          System.out.println("New size " + fst.sizeInBytes() + " bytes");
+        } else {
+          Directory dir = FSDirectory.open(new File(dirOut));
+          IndexOutput out = dir.createOutput("fst.bin", IOContext.DEFAULT);
+          fst.save(out);
+          out.close();
+          System.out.println("Saved FST to fst.bin.");
+        }
 
         if (!verify) {
           return;
@@ -1317,37 +1322,42 @@ public class TestFSTs extends LuceneTest
 
         System.out.println("\nNow verify...");
 
-        is.close();
-        is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), "UTF-8"), 65536);
-
-        ord = 0;
-        tStart = System.currentTimeMillis();
         while(true) {
-          String w = is.readLine();
-          if (w == null) {
-            break;
-          }
-          toIntsRef(w, inputMode, intsRef);
-          T expected = getOutput(intsRef, ord);
-          T actual = Util.get(fst, intsRef);
-          if (actual == null) {
-            throw new RuntimeException("unexpected null output on input=" + w);
-          }
-          if (!actual.equals(expected)) {
-            throw new RuntimeException("wrong output (got " + outputs.outputToString(actual) + " but expected " + outputs.outputToString(expected) + ") on input=" + w);
-          }
+          is.close();
+          is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), "UTF-8"), 65536);
 
-          ord++;
-          if (ord % 500000 == 0) {
-            System.out.println(((System.currentTimeMillis()-tStart)/1000.0) + "s: " + ord + "...");
-          }
-          if (ord >= limit) {
-            break;
+          ord = 0;
+          tStart = System.currentTimeMillis();
+          while(true) {
+            String w = is.readLine();
+            if (w == null) {
+              break;
+            }
+            toIntsRef(w, inputMode, intsRef);
+            T expected = getOutput(intsRef, ord);
+            T actual = Util.get(fst, intsRef);
+            if (actual == null) {
+              throw new RuntimeException("unexpected null output on input=" + w);
+            }
+            if (!actual.equals(expected)) {
+              throw new RuntimeException("wrong output (got " + outputs.outputToString(actual) + " but expected " + outputs.outputToString(expected) + ") on input=" + w);
+            }
+
+            ord++;
+            if (ord % 500000 == 0) {
+              System.out.println(((System.currentTimeMillis()-tStart)/1000.0) + "s: " + ord + "...");
+            }
+            if (ord >= limit) {
+              break;
+            }
           }
-        }
 
-        double totSec = ((System.currentTimeMillis() - tStart)/1000.0);
-        System.out.println("Verify took " + totSec + " sec + (" + (int) ((totSec*1000000000/ord)) + " nsec per lookup)");
+          double totSec = ((System.currentTimeMillis() - tStart)/1000.0);
+          System.out.println("Verify took " + totSec + " sec + (" + (int) ((totSec*1000000000/ord)) + " nsec per lookup)");
+
+          // NOTE: comment out to profile lookup...
+          break;
+        }
 
       } finally {
         is.close();
@@ -1355,7 +1365,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.7.jar org.apache.lucene.util.automaton.fst.TestFSTs /x/tmp/allTerms3.txt out
+  // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.7.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
   public static void main(String[] args) throws IOException {
     int prune = 0;
     int limit = Integer.MAX_VALUE;
@@ -1363,7 +1373,8 @@ public class TestFSTs extends LuceneTest
     boolean storeOrds = false;
     boolean storeDocFreqs = false;
     boolean verify = true;
-    
+    boolean doPack = false;
+    boolean noArcArrays = false;
     String wordsFileIn = null;
     String dirOut = null;
 
@@ -1381,10 +1392,14 @@ public class TestFSTs extends LuceneTest
         inputMode = 1;
       } else if (args[idx].equals("-docFreq")) {
         storeDocFreqs = true;
+      } else if (args[idx].equals("-noArcArrays")) {
+        noArcArrays = true;
       } else if (args[idx].equals("-ords")) {
         storeOrds = true;
       } else if (args[idx].equals("-noverify")) {
         verify = false;
+      } else if (args[idx].equals("-pack")) {
+        doPack = true;
       } else if (args[idx].startsWith("-")) {
         System.err.println("Unrecognized option: " + args[idx]);
         System.exit(-1);
@@ -1413,44 +1428,44 @@ public class TestFSTs extends LuceneTest
       final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton(true);
       final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton(false);
       final PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(o1, o2);
-      new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs) {
+      new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
         @Override
         public PairOutputs.Pair<Long,Long> getOutput(IntsRef input, int ord) {
           if (ord == 0) {
             rand = new Random(17);
           }
-          return new PairOutputs.Pair<Long,Long>(o1.get(ord),
-                                                 o2.get(_TestUtil.nextInt(rand, 1, 5000)));
+          return outputs.newPair((long) ord,
+                                 (long) _TestUtil.nextInt(rand, 1, 5000));
         }
       }.run(limit, verify);
     } else if (storeOrds) {
       // Store only ords
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs) {
+      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         @Override
         public Long getOutput(IntsRef input, int ord) {
-          return outputs.get(ord);
+          return (long) ord;
         }
       }.run(limit, verify);
     } else if (storeDocFreqs) {
       // Store only docFreq
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(false);
-      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs) {
+      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
         @Override
         public Long getOutput(IntsRef input, int ord) {
           if (ord == 0) {
             rand = new Random(17);
           }
-          return outputs.get(_TestUtil.nextInt(rand, 1, 5000));
+          return (long) _TestUtil.nextInt(rand, 1, 5000);
         }
       }.run(limit, verify);
     } else {
       // Store nothing
       final NoOutputs outputs = NoOutputs.getSingleton();
       final Object NO_OUTPUT = outputs.getNoOutput();
-      new VisitTerms<Object>(dirOut, wordsFileIn, inputMode, prune, outputs) {
+      new VisitTerms<Object>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         @Override
         public Object getOutput(IntsRef input, int ord) {
           return NO_OUTPUT;
@@ -1468,6 +1483,46 @@ public class TestFSTs extends LuceneTest
     assertNull(fstEnum.seekCeil(new BytesRef("foobaz")));
   }
 
+  /*
+  public void testTrivial() throws Exception {
+
+    // Get outputs -- passing true means FST will share
+    // (delta code) the outputs.  This should result in
+    // smaller FST if the outputs grow monotonically.  But
+    // if numbers are "random", false should give smaller
+    // final size:
+    final NoOutputs outputs = NoOutputs.getSingleton();
+
+    String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation", "stat"};
+
+    final Builder<Object> builder = new Builder<Object>(FST.INPUT_TYPE.BYTE1,
+                                                        0, 0,
+                                                        true,
+                                                        true,
+                                                        Integer.MAX_VALUE,
+                                                        outputs,
+                                                        null,
+                                                        true);
+    Arrays.sort(strings);
+    final IntsRef scratch = new IntsRef();
+    for(String s : strings) {
+      builder.add(Util.toIntsRef(new BytesRef(s), scratch), outputs.getNoOutput());
+    }
+    final FST<Object> fst = builder.finish();
+    System.out.println("DOT before rewrite");
+    Writer w = new OutputStreamWriter(new FileOutputStream("/mnt/scratch/before.dot"));
+    Util.toDot(fst, w, false, false);
+    w.close();
+
+    final FST<Object> rewrite = new FST<Object>(fst, 1, 100);
+
+    System.out.println("DOT after rewrite");
+    w = new OutputStreamWriter(new FileOutputStream("/mnt/scratch/after.dot"));
+    Util.toDot(rewrite, w, false, false);
+    w.close();
+  }
+  */
+
   public void testSimple() throws Exception {
 
     // Get outputs -- passing true means FST will share
@@ -1484,9 +1539,9 @@ public class TestFSTs extends LuceneTest
     final BytesRef b = new BytesRef("b");
     final BytesRef c = new BytesRef("c");
 
-    builder.add(Util.toIntsRef(a, new IntsRef()), outputs.get(17));
-    builder.add(Util.toIntsRef(b, new IntsRef()), outputs.get(42));
-    builder.add(Util.toIntsRef(c, new IntsRef()), outputs.get(13824324872317238L));
+    builder.add(Util.toIntsRef(a, new IntsRef()), 17L);
+    builder.add(Util.toIntsRef(b, new IntsRef()), 42L);
+    builder.add(Util.toIntsRef(c, new IntsRef()), 13824324872317238L);
 
     final FST<Long> fst = builder.finish();
 
@@ -1795,11 +1850,11 @@ 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);
-    builder.add(Util.toUTF32("stat", new IntsRef()), outputs.get(17));
-    builder.add(Util.toUTF32("station", new IntsRef()), outputs.get(10));
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random.nextBoolean());
+    builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
+    builder.add(Util.toUTF32("station", new IntsRef()), 10L);
     final FST<Long> fst = builder.finish();
-    //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp3/out.dot"));
     StringWriter w = new StringWriter();
     Util.toDot(fst, w, false, false);
     w.close();
@@ -1809,8 +1864,8 @@ public class TestFSTs extends LuceneTest
 
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-
-    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
+    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);
     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();
@@ -1819,17 +1874,23 @@ public class TestFSTs extends LuceneTest
     Util.toDot(fst, w, false, false);
     w.close();
     //System.out.println(w.toString());
-    assertTrue(w.toString().indexOf("6 [shape=doublecircle") != -1);
+    final String expected;
+    if (willRewrite) {
+      expected = "4 -> 3 [label=\"t\" style=\"bold\"";
+    } else {
+      expected = "8 -> 6 [label=\"t\" style=\"bold\"";
+    }
+    assertTrue(w.toString().indexOf(expected) != -1);
   }
 
   // Make sure raw FST can differentiate between final vs
   // non-final end nodes
-  public void testNonFinalStopNodes() throws Exception {
+  public void testNonFinalStopNode() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
     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);
+    final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false);
 
     final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);
 
@@ -1839,8 +1900,8 @@ public class TestFSTs extends LuceneTest
       node.isFinal = true;
       rootNode.addArc('a', node);
       final Builder.CompiledNode frozen = new Builder.CompiledNode();
-      frozen.address = fst.addNode(node);
-      rootNode.arcs[0].nextFinalOutput = outputs.get(17);
+      frozen.node = fst.addNode(node);
+      rootNode.arcs[0].nextFinalOutput = 17L;
       rootNode.arcs[0].isFinal = true;
       rootNode.arcs[0].output = nothing;
       rootNode.arcs[0].target = frozen;
@@ -1851,13 +1912,18 @@ public class TestFSTs extends LuceneTest
       final Builder.UnCompiledNode<Long> node = new Builder.UnCompiledNode<Long>(b, 0);
       rootNode.addArc('b', node);
       final Builder.CompiledNode frozen = new Builder.CompiledNode();
-      frozen.address = fst.addNode(node);
+      frozen.node = fst.addNode(node);
       rootNode.arcs[1].nextFinalOutput = nothing;
-      rootNode.arcs[1].output = outputs.get(42);
+      rootNode.arcs[1].output = 42L;
       rootNode.arcs[1].target = frozen;
     }
 
     fst.finish(fst.addNode(rootNode));
+
+    StringWriter w = new StringWriter();
+    //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp3/out.dot"));
+    Util.toDot(fst, w, false, false);
+    w.close();
     
     checkStopNodes(fst, outputs);
 

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java Sun Jan 29 22:48:45 2012
@@ -226,6 +226,9 @@ public final class SynonymFilter extends
 
   private final FST<BytesRef> fst;
 
+  private final FST.BytesReader fstReader;
+
+
   private final BytesRef scratchBytes = new BytesRef();
   private final CharsRef scratchChars = new CharsRef();
 
@@ -241,7 +244,7 @@ public final class SynonymFilter extends
     this.synonyms = synonyms;
     this.ignoreCase = ignoreCase;
     this.fst = synonyms.fst;
-
+    this.fstReader = fst.getBytesReader(0);
     if (fst == null) {
       throw new IllegalArgumentException("fst must be non-null");
     }
@@ -366,7 +369,7 @@ public final class SynonymFilter extends
       int bufUpto = 0;
       while(bufUpto < bufferLen) {
         final int codePoint = Character.codePointAt(buffer, bufUpto, bufferLen);
-        if (fst.findTargetArc(ignoreCase ? Character.toLowerCase(codePoint) : codePoint, scratchArc, scratchArc) == null) {
+        if (fst.findTargetArc(ignoreCase ? Character.toLowerCase(codePoint) : codePoint, scratchArc, scratchArc, fstReader) == null) {
           //System.out.println("    stop");
           break byToken;
         }
@@ -388,7 +391,7 @@ public final class SynonymFilter extends
 
       // See if the FST wants to continue matching (ie, needs to
       // see the next input token):
-      if (fst.findTargetArc(SynonymMap.WORD_SEPARATOR, scratchArc, scratchArc) == null) {
+      if (fst.findTargetArc(SynonymMap.WORD_SEPARATOR, scratchArc, scratchArc, fstReader) == null) {
         // No further rules can match here; we're done
         // searching for matching rules starting at the
         // current input position.

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java Sun Jan 29 22:48:45 2012
@@ -47,16 +47,17 @@ public final class TokenInfoFST {
     FST.Arc<Long> firstArc = new FST.Arc<Long>();
     fst.getFirstArc(firstArc);
     FST.Arc<Long> arc = new FST.Arc<Long>();
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
     // TODO: jump to 3040, readNextRealArc to ceiling? (just be careful we don't add bugs)
     for (int i = 0; i < rootCache.length; i++) {
-      if (fst.findTargetArc(0x3040 + i, firstArc, arc) != null) {
+      if (fst.findTargetArc(0x3040 + i, firstArc, arc, fstReader) != null) {
         rootCache[i] = new FST.Arc<Long>().copyFrom(arc);
       }
     }
     return rootCache;
   }
   
-  public FST.Arc<Long> findTargetArc(int ch, FST.Arc<Long> follow, FST.Arc<Long> arc, boolean useCache) throws IOException {
+  public FST.Arc<Long> findTargetArc(int ch, FST.Arc<Long> follow, FST.Arc<Long> arc, boolean useCache, FST.BytesReader fstReader) throws IOException {
     if (useCache && ch >= 0x3040 && ch <= cacheCeiling) {
       assert ch != FST.END_LABEL;
       final Arc<Long> result = rootCache[ch - 0x3040];
@@ -67,13 +68,17 @@ public final class TokenInfoFST {
         return arc;
       }
     } else {
-      return fst.findTargetArc(ch, follow, arc);
+      return fst.findTargetArc(ch, follow, arc, fstReader);
     }
   }
   
   public Arc<Long> getFirstArc(FST.Arc<Long> arc) {
     return fst.getFirstArc(arc);
   }
+
+  public FST.BytesReader getBytesReader(int pos) {
+    return fst.getBytesReader(pos);
+  }
   
   /** @lucene.internal for testing only */
   FST<Long> getInternalFST() {

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java Sun Jan 29 22:48:45 2012
@@ -113,7 +113,7 @@ public final class UserDictionary implem
       for (int i = 0; i < token.length(); i++) {
         scratch.ints[i] = (int) token.charAt(i);
       }
-      fstBuilder.add(scratch, fstOutput.get(ord));
+      fstBuilder.add(scratch, ord);
       segmentations.add(wordIdAndLength);
       ord++;
     }
@@ -134,6 +134,8 @@ public final class UserDictionary implem
     TreeMap<Integer, int[]> result = new TreeMap<Integer, int[]>(); // index, [length, length...]
     boolean found = false; // true if we found any results
 
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
+
     FST.Arc<Long> arc = new FST.Arc<Long>();
     int end = off + len;
     for (int startOffset = off; startOffset < end; startOffset++) {
@@ -142,7 +144,7 @@ public final class UserDictionary implem
       int remaining = end - startOffset;
       for (int i = 0; i < remaining; i++) {
         int ch = chars[startOffset+i];
-        if (fst.findTargetArc(ch, arc, arc, i == 0) == null) {
+        if (fst.findTargetArc(ch, arc, arc, i == 0, fstReader) == null) {
           break; // continue to next position
         }
         output += arc.output.intValue();

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java Sun Jan 29 22:48:45 2012
@@ -35,7 +35,7 @@ import org.apache.lucene.util.fst.FST;
 public class Viterbi {
   
   private final TokenInfoFST fst;
-  
+
   private final TokenInfoDictionary dictionary;
   
   private final UnknownDictionary unkDictionary;
@@ -214,6 +214,8 @@ public class Viterbi {
     ViterbiNode bosNode = new ViterbiNode(-1, BOS, 0, BOS.length, 0, 0, 0, -1, Type.KNOWN);
     addToArrays(bosNode, 0, 1, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
     
+    final FST.BytesReader fstReader = fst.getBytesReader(0);
+
     // Process user dictionary;
     if (useUserDictionary) {
       processUserDictionary(text, offset, length, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
@@ -238,7 +240,7 @@ public class Viterbi {
       for (int endIndex = 1; endIndex < suffixLength + 1; endIndex++) {
         int ch = text[suffixStart + endIndex - 1];
         
-        if (fst.findTargetArc(ch, arc, arc, endIndex == 1) == null) {
+        if (fst.findTargetArc(ch, arc, arc, endIndex == 1, fstReader) == null) {
           break; // continue to next position
         }
         output += arc.output.intValue();

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$fst.dat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary%24fst.dat?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java Sun Jan 29 22:48:45 2012
@@ -131,7 +131,7 @@ public class TokenInfoDictionaryBuilder 
     System.out.println("  encode...");
 
     PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton(true);
-    Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, fstOutput);
+    Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true);
     IntsRef scratch = new IntsRef();
     long ord = -1; // first ord will be 0
     String lastValue = null;
@@ -155,13 +155,14 @@ public class TokenInfoDictionaryBuilder 
         for (int i = 0; i < token.length(); i++) {
           scratch.ints[i] = (int) token.charAt(i);
         }
-        fstBuilder.add(scratch, fstOutput.get(ord));
+        fstBuilder.add(scratch, ord);
       }
       dictionary.addMapping((int)ord, offset);
       offset = next;
     }
     
-    FST<Long> fst = fstBuilder.finish();
+    final FST<Long> fst = fstBuilder.finish().pack(2, 100000);
+    
     System.out.print("  " + fst.getNodeCount() + " nodes, " + fst.getArcCount() + " arcs, " + fst.sizeInBytes() + " bytes...  ");
     dictionary.setFST(fst);
     System.out.println(" done");

Modified: lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java (original)
+++ lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java Sun Jan 29 22:48:45 2012
@@ -329,8 +329,11 @@ public class FSTCompletion {
   private boolean descendWithPrefix(Arc<Object> arc, BytesRef utf8)
       throws IOException {
     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);
     for (int i = utf8.offset; i < max; i++) {
-      if (automaton.findTargetArc(utf8.bytes[i] & 0xff, arc, arc) == null) {
+      if (automaton.findTargetArc(utf8.bytes[i] & 0xff, arc, arc, fstReader) == null) {
         // No matching prefixes, return an empty result.
         return false;
       }

Modified: lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1237500&r1=1237499&r2=1237500&view=diff
==============================================================================
--- lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Sun Jan 29 22:48:45 2012
@@ -234,7 +234,7 @@ 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);
+        shareMaxTailLength, outputs, null, false);
     
     BytesRef scratch = new BytesRef();
     final IntsRef scratchIntsRef = new IntsRef();