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/02/11 21:29:24 UTC

svn commit: r1243149 [1/2] - in /lucene/dev/branches/lucene3767: lucene/core/src/java/org/apache/lucene/util/fst/ lucene/test-framework/src/java/org/apache/lucene/analysis/ modules/analysis/kuromoji/ modules/analysis/kuromoji/src/java/org/apache/lucene...

Author: mikemccand
Date: Sat Feb 11 20:29:23 2012
New Revision: 1243149

URL: http://svn.apache.org/viewvc?rev=1243149&view=rev
Log:
LUCENE-3767: commit current state; tests pass but are noisy!

Added:
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/Perf.java   (with props)
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java   (with props)
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java   (with props)
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tanakaseg.zip   (with props)
Modified:
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
    lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java

Modified: lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sat Feb 11 20:29:23 2012
@@ -840,6 +840,7 @@ public final class FST<T> {
   }
 
   public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
     final int address = getNodeAddress(node);
     in.pos = address;
     //System.out.println("  readFirstRealTargtArc address="
@@ -936,6 +937,7 @@ public final class FST<T> {
   /** Never returns null, but you should never call this if
    *  arc.isLast() is true. */
   public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
 
     // TODO: can't assert this because we call from readFirstArc
     // assert !flag(arc.flags, BIT_LAST_ARC);
@@ -1019,6 +1021,7 @@ public final class FST<T> {
    *  This returns null if the arc was not found, else the incoming arc. */
   public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
     assert cachedRootArcs != null;
+    assert in.bytes == bytes;
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1225,17 +1228,20 @@ public final class FST<T> {
 
   /** Expert */
   public static abstract class BytesReader extends DataInput {
-    int pos;
+    protected int pos;
+    protected final byte[] bytes;
+    protected BytesReader(byte[] bytes, int pos) {
+      this.bytes = bytes;
+      this.pos = pos;
+    }
     abstract void skip(int byteCount);
     abstract void skip(int base, int byteCount);
   }
 
   final static class ReverseBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ReverseBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override
@@ -1262,11 +1268,9 @@ public final class FST<T> {
   // TODO: can we use just ByteArrayDataInput...?  need to
   // add a .skipBytes to DataInput.. hmm and .setPosition
   final static class ForwardBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ForwardBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override

Modified: lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Sat Feb 11 20:29:23 2012
@@ -138,14 +138,15 @@ public abstract class BaseTokenStreamTes
         assertTrue("endOffset must be >= startOffset", offsetAtt.endOffset() >= offsetAtt.startOffset());
         if (finalOffset != null) {
           assertTrue("startOffset must be <= finalOffset", offsetAtt.startOffset() <= finalOffset.intValue());
-          assertTrue("endOffset must be <= finalOffset", offsetAtt.endOffset() <= finalOffset.intValue());
+          assertTrue("endOffset must be <= finalOffset: got endOffset=" + offsetAtt.endOffset() + " vs finalOffset=" + finalOffset.intValue(),
+                     offsetAtt.endOffset() <= finalOffset.intValue());
         }
       }
       if (posIncrAtt != null) {
         assertTrue("posIncrement must be >= 0", posIncrAtt.getPositionIncrement() >= 0);
       }
     }
-    assertFalse("end of stream", ts.incrementToken());
+    assertFalse("TokenStream has more tokens than expected", ts.incrementToken());
     ts.end();
     if (finalOffset != null)
       assertEquals("finalOffset ", finalOffset.intValue(), offsetAtt.endOffset());
@@ -315,7 +316,7 @@ public abstract class BaseTokenStreamTes
       if (VERBOSE) {
         System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
       }
-
+      
       int remainder = random.nextInt(10);
       Reader reader = new StringReader(text);
       TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
@@ -344,7 +345,7 @@ public abstract class BaseTokenStreamTes
       // verify reusing is "reproducable" and also get the normal tokenstream sanity checks
       if (!tokens.isEmpty()) {
         if (VERBOSE) {
-          System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis");
+          System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis useCharFilter=" + useCharFilter + " text.length()=" + text.length());
         }
         reader = new StringReader(text);
         ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);

Added: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/Perf.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/Perf.java?rev=1243149&view=auto
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/Perf.java (added)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/Perf.java Sat Feb 11 20:29:23 2012
@@ -0,0 +1,45 @@
+import java.io.Reader;
+import java.io.StringReader;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer2;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer;
+import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+
+// javac -cp ../build/kuromoji/classes/java:../../../lucene/build/classes/java:../../analysis/build/common/lucene-analyzers-common-4.0-SNAPSHOT.jar Perf.java
+
+// java -cp .:../build/kuromoji/classes/java:../../../lucene/build/classes/java:../../analysis/build/common/lucene-analyzers-common-4.0-SNAPSHOT.jar Perf
+public class Perf {
+
+  private final static Analyzer analyzer = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Mode.SEARCH);
+      //Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      return new TokenStreamComponents(tokenizer, tokenizer);
+    }
+  };
+  
+  final static String s0 = "本来は、貧困層の女性や子供に医療保護を提供するために創設された制度である、" +
+    "アメリカ低所得者医療援助制度が、今日では、その予算の約3分の1を老人に費やしている。";
+  final static String s = s0 + s0 + s0 + s0 + s0 + s0;
+
+  public static void main(String[] args) throws Exception {
+    for(int iter=0;iter<10;iter++) {
+      final long t0 = System.currentTimeMillis();
+      long count = 0;
+      final int ITERS = 3000;
+      for(int i=0;i<ITERS;i++) {
+        final TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
+        while(ts.incrementToken()) {
+          count++;
+        }
+      }
+      final long t1 = System.currentTimeMillis();
+      System.out.println((t1-t0) + " msec; " + (s0.length()*ITERS/((double) t1-t0)) + " chars/msec (" + count + " tokens)");
+    }
+  }
+}

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java Sat Feb 11 20:29:23 2012
@@ -27,13 +27,17 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.cjk.CJKWidthFilter;
 import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.core.StopFilter;
+import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.util.Version;
 
 public class KuromojiAnalyzer extends StopwordAnalyzerBase {
   private final Segmenter segmenter;
+  private final Mode mode;
   private final Set<String> stoptags;
+  private final UserDictionary userDict;
   
   public KuromojiAnalyzer(Version matchVersion) {
     this(matchVersion, new Segmenter(), DefaultSetHolder.DEFAULT_STOP_SET, DefaultSetHolder.DEFAULT_STOP_TAGS);
@@ -43,6 +47,16 @@ public class KuromojiAnalyzer extends St
     super(matchVersion, stopwords);
     this.segmenter = segmenter;
     this.stoptags = stoptags;
+    userDict = null;
+    mode = Segmenter.DEFAULT_MODE;
+  }
+
+  public KuromojiAnalyzer(Version matchVersion, UserDictionary userDict, Mode mode, CharArraySet stopwords, Set<String> stoptags) {
+    super(matchVersion, stopwords);
+    this.userDict = userDict;
+    this.mode = mode;
+    this.stoptags = stoptags;
+    this.segmenter = null;
   }
   
   public static CharArraySet getDefaultStopSet(){
@@ -79,7 +93,8 @@ public class KuromojiAnalyzer extends St
   
   @Override
   protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-    Tokenizer tokenizer = new KuromojiTokenizer(this.segmenter, reader);
+    //Tokenizer tokenizer = new KuromojiTokenizer(this.segmenter, reader);
+    Tokenizer tokenizer = new KuromojiTokenizer2(reader, userDict, true, mode);
     TokenStream stream = new KuromojiBaseFormFilter(tokenizer);
     stream = new KuromojiPartOfSpeechStopFilter(true, stream, stoptags);
     stream = new CJKWidthFilter(stream);

Added: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java?rev=1243149&view=auto
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java (added)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java Sat Feb 11 20:29:23 2012
@@ -0,0 +1,1182 @@
+package org.apache.lucene.analysis.kuromoji;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.analysis.CharStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoFST;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.apache.lucene.analysis.kuromoji.tokenattributes.*;
+import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.fst.FST;
+
+// TODO: somehow factor out a reusable viterbi search here,
+// so other decompounders/tokenizers can reuse...
+
+// nocommit add toDot and look at 1st pass intersection
+
+// nocommit add comment explaining this isn't quite a real viterbi
+
+// nocomit explain how the 2nd best tokenization is
+// "contextual"...
+
+// nocommit -- need a test that doesn't pre-split by
+// sentence... ie, we don't BOS/EOS on each sentence
+// break any more... so this can change the results
+// depending on whether ipadic was "trained" with sentence
+// breaks?
+
+/* Uses a rolling Viterbi search to find the least cost
+ * segmentation (path) of the incoming characters.
+ *
+ * @lucene.experimental */
+public final class KuromojiTokenizer2 extends Tokenizer {
+
+  private static final boolean VERBOSE = false;
+
+  private static final int SEARCH_MODE_KANJI_LENGTH = 2;
+
+  private static final int SEARCH_MODE_OTHER_LENGTH = 7; // Must be >= SEARCH_MODE_KANJI_LENGTH
+
+  private static final int SEARCH_MODE_KANJI_PENALTY = 3000;
+
+  private static final int SEARCH_MODE_OTHER_PENALTY = 1700;
+
+  // nocommit
+  public static boolean DO_OUTPUT_COMPOUND = false;
+
+  private final EnumMap<Type, Dictionary> dictionaryMap = new EnumMap<Type, Dictionary>(Type.class);
+
+  private final TokenInfoFST fst;
+  private final TokenInfoDictionary dictionary;
+  private final UnknownDictionary unkDictionary;
+  private final ConnectionCosts costs;
+  private final UserDictionary userDictionary;
+  private final CharacterDefinition characterDefinition;
+
+  private final FST.Arc<Long> arc = new FST.Arc<Long>();
+  private final FST.BytesReader fstReader;
+  private final IntsRef wordIdRef = new IntsRef();
+
+  private final FST.BytesReader userFSTReader;
+  private final TokenInfoFST userFST;
+
+  private Reader reader;
+
+  // Next absolute position to process:
+  private int pos;
+
+  private WrappedCharArray buffer = new WrappedCharArray();
+
+  // index of the last character of unknown word:
+  // nocommit put back:
+  // int unknownWordEndIndex = -1;
+
+  private WrappedPositionArray positions = new WrappedPositionArray();
+
+  private boolean end;
+  private final boolean discardPunctuation;
+  private final boolean searchMode;
+  private final boolean extendedMode;
+
+  private int lastBackTracePos;
+  private int lastTokenPos;
+
+  // Already parsed but not yet passed to caller:
+  private final List<Token> pending = new ArrayList<Token>();
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final BaseFormAttribute basicFormAtt = addAttribute(BaseFormAttribute.class);
+  private final PartOfSpeechAttribute posAtt = addAttribute(PartOfSpeechAttribute.class);
+  private final ReadingAttribute readingAtt = addAttribute(ReadingAttribute.class);
+  private final InflectionAttribute inflectionAtt = addAttribute(InflectionAttribute.class);
+
+  public KuromojiTokenizer2(Reader input, UserDictionary userDictionary, boolean discardPunctuation, Mode mode) {
+    super(input);
+    dictionary = TokenInfoDictionary.getInstance();
+    fst = dictionary.getFST();
+    unkDictionary = UnknownDictionary.getInstance();
+    characterDefinition = unkDictionary.getCharacterDefinition();
+    this.userDictionary = userDictionary;
+    costs = ConnectionCosts.getInstance();
+    fstReader = fst.getBytesReader(0);
+    if (userDictionary != null) {
+      userFST = userDictionary.getFST();
+      userFSTReader = userFST.getBytesReader(0);
+    } else {
+      userFST = null;
+      userFSTReader = null;
+    }
+    this.discardPunctuation = discardPunctuation;
+    switch(mode){
+      case SEARCH:
+        searchMode = true;
+        extendedMode = false;
+        break;
+      case EXTENDED:
+        searchMode = true;
+        extendedMode = true;
+        break;
+      default:
+        searchMode = false;
+        extendedMode = false;
+        break;
+    }
+    buffer.reset(input);
+    resetState();
+
+    dictionaryMap.put(Type.KNOWN, dictionary);
+    dictionaryMap.put(Type.UNKNOWN, unkDictionary);
+    dictionaryMap.put(Type.USER, userDictionary);
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+    buffer.reset(input);
+    resetState();
+  }
+
+  private void resetState() {
+    positions.reset();
+    // nocommit put back
+    //unknownWordEndIndex = -1;
+    pos = 0;
+    end = false;
+    lastBackTracePos = 0;
+    lastTokenPos = -1;
+    pending.clear();
+
+    // Add BOS:
+    positions.get(0).add(0, 0, -1, -1, -1, Type.KNOWN);
+  }
+
+  @Override
+  public final void end() {
+    // set final offset
+    offsetAtt.setOffset(correctOffset(pos), correctOffset(pos));
+  }
+
+  private int computePenalty(int pos, int length) throws IOException {
+    if (length > SEARCH_MODE_KANJI_LENGTH) {
+      boolean allKanji = true;
+      // check if node consists of only kanji
+      final int endPos = pos + length;
+      for (int pos2 = pos; pos2 < endPos; pos2++) {
+        if (!characterDefinition.isKanji((char) buffer.get(pos2))) {
+          allKanji = false;
+          break;
+        }				
+      }
+      final int penalty;
+      if (allKanji) {	// Process only Kanji keywords
+        return (length - SEARCH_MODE_KANJI_LENGTH) * SEARCH_MODE_KANJI_PENALTY;
+      } else if (length > SEARCH_MODE_OTHER_LENGTH) {
+        return (length - SEARCH_MODE_OTHER_LENGTH) * SEARCH_MODE_OTHER_PENALTY;								
+      }
+    }
+    return 0;
+  }
+
+  // Holds all back pointers arriving to this position:
+  private final static class Position {
+
+    int pos;
+
+    int count;
+
+    // maybe single int array * 5?
+    int[] costs = new int[4];
+    // nommit rename to lastRightID or pathRightID or something:
+    int[] nodeID = new int[4];
+    int[] backPos = new int[4];
+    int[] backIndex = new int[4];
+    int[] backID = new int[4];
+    Type[] backType = new Type[4];
+
+    // Only used when finding 2nd best segmentation under a
+    // too-long token:
+    int forwardCount;
+    int[] forwardPos = new int[4];
+    int[] forwardID = new int[4];
+    int[] forwardIndex = new int[4];
+    Type[] forwardType = new Type[4];
+
+    public void grow() {
+      costs = ArrayUtil.grow(costs, 1+count);
+      nodeID = ArrayUtil.grow(nodeID, 1+count);
+      backPos = ArrayUtil.grow(backPos, 1+count);
+      backIndex = ArrayUtil.grow(backIndex, 1+count);
+      backID = ArrayUtil.grow(backID, 1+count);
+
+      // NOTE: sneaky: grow separately because
+      // ArrayUtil.grow will otherwise pick a different
+      // length than the int[]s we just grew:
+      final Type[] newBackType = new Type[backID.length];
+      System.arraycopy(backType, 0, newBackType, 0, backType.length);
+      backType = newBackType;
+    }
+
+    public void growForward() {
+      forwardPos = ArrayUtil.grow(forwardPos, 1+forwardCount);
+      forwardID = ArrayUtil.grow(forwardID, 1+forwardCount);
+      forwardIndex = ArrayUtil.grow(forwardIndex, 1+forwardCount);
+
+      // NOTE: sneaky: grow separately because
+      // ArrayUtil.grow will otherwise pick a different
+      // length than the int[]s we just grew:
+      final Type[] newForwardType = new Type[forwardPos.length];
+      System.arraycopy(forwardType, 0, newForwardType, 0, forwardType.length);
+      forwardType = newForwardType;
+    }
+
+    public void add(int cost, int nodeID, int backPos, int backIndex, int backID, Type backType) {
+      // nocommit in theory, we should check if nodeID is
+      // already present here, and update it if
+      // so... instead of just always adding:
+      if (count == costs.length) {
+        grow();
+      }
+      this.costs[count] = cost;
+      this.nodeID[count] = nodeID;
+      this.backPos[count] = backPos;
+      this.backIndex[count] = backIndex;
+      this.backID[count] = backID;
+      this.backType[count] = backType;
+      count++;
+    }
+
+    public void addForward(int forwardPos, int forwardIndex, int forwardID, Type forwardType) {
+      if (forwardCount == this.forwardID.length) {
+        growForward();
+      }
+      this.forwardPos[forwardCount] = forwardPos;
+      this.forwardIndex[forwardCount] = forwardIndex;
+      this.forwardID[forwardCount] = forwardID;
+      this.forwardType[forwardCount] = forwardType;
+      forwardCount++;
+    }
+
+    public void reset() {
+      count = 0;
+      // forwardCount naturally resets after it runs:
+      assert forwardCount == 0: "pos=" + pos + " forwardCount=" + forwardCount;
+    }
+
+    // nocommit maybe?
+    // public void update(int cost, int nodeID, int backPos, int backIndex, int backID)
+  }
+
+  // nocommit absorb into add...?
+  private void add(Dictionary dict, Position posData, int endPos, int wordID, Type type) throws IOException {
+    final int wordCost = dict.getWordCost(wordID);
+    final int leftID = dict.getLeftId(wordID);
+    int leastCost = Integer.MAX_VALUE;
+    int leastIDX = -1;
+    assert posData.count > 0;
+    for(int idx=0;idx<posData.count;idx++) {
+      // Cost is path cost so far, plus word cost, plus
+      // bigram cost:
+      final int cost = posData.costs[idx] + wordCost + costs.get(posData.nodeID[idx], leftID);
+      if (cost < leastCost) {
+        leastCost = cost;
+        leastIDX = idx;
+      }
+    }
+
+    if (VERBOSE) {
+      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " wordCat=" + leftID + " tok=" + new String(buffer.get(posData.pos, endPos-posData.pos)));
+    }
+
+    // nocommit ideally we don't have to do this ... just
+    // putting it here to confirm same results as current
+    // segmenter:
+    if (!DO_OUTPUT_COMPOUND && searchMode && type != Type.USER) {
+      final int penalty = computePenalty(posData.pos, endPos - posData.pos);
+      if (VERBOSE) {
+        if (penalty > 0) {
+          System.out.println("        + penalty=" + penalty + " cost=" + (leastCost+penalty));
+        }
+      }
+      leastCost += penalty;
+    }
+
+    positions.get(endPos).add(leastCost, dict.getRightId(wordID), posData.pos, leastIDX, wordID, type);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+
+    // parse() is able to return w/o producing any new
+    // tokens, when the tokens it had produced were entirely
+    // punctuation.  So we loop here until we get a real
+    // token or we end:
+    while (pending.size() == 0) {
+      if (end) {
+        return false;
+      }
+
+      // Push Viterbi forward some more:
+      parse();
+    }
+
+    final Token token = pending.remove(pending.size()-1);
+
+    int position = token.getPosition();
+    int length = token.getLength();
+    clearAttributes();
+    termAtt.copyBuffer(token.getSurfaceForm(), token.getOffset(), length);
+    offsetAtt.setOffset(correctOffset(position), correctOffset(position+length));
+    basicFormAtt.setToken(token);
+    posAtt.setToken(token);
+    readingAtt.setToken(token);
+    inflectionAtt.setToken(token);
+    if (token.getPosition() == lastTokenPos) {
+      posIncAtt.setPositionIncrement(0);
+    } else {
+      assert token.getPosition() > lastTokenPos;
+      posIncAtt.setPositionIncrement(1);
+    }
+    if (VERBOSE) {
+      System.out.println("    incToken: return token=" + token);
+    }
+    lastTokenPos = token.getPosition();
+    return true;
+  }
+
+  // Acts like a forever growing char[] as you read
+  // characters into it from the provided reader, but
+  // internally it uses a circular buffer to only hold the
+  // characters that haven't been freed yet:
+  private static final class WrappedCharArray {
+
+    // TODO: pull out as standalone oal.util class?
+
+    private Reader reader;
+
+    private char[] buffer = new char[32];
+
+    // Next array index to write to in buffer:
+    private int nextWrite;
+
+    // Next absolute position to read from reader:
+    private int nextPos;
+
+    // How many valid chars (wrapped) are in the buffer:
+    private int count;
+
+    // True if we hit EOF
+    private boolean end;
+    
+    /** Clear array and switch to new reader. */
+    public void reset(Reader reader) {
+      this.reader = reader;
+      nextPos = 0;
+      nextWrite = 0;
+      count = 0;
+      end = false;
+    }
+
+    /* Absolute position read.  NOTE: pos must not jump
+     * ahead by more than 1!  Ie, it's OK to read arbitarily
+     * far back (just not prior to the last {@link
+     * #freeBefore}), but NOT ok to read arbitrarily far
+     * ahead.  Returns -1 if you hit EOF. */
+    public int get(int pos) throws IOException {
+      //System.out.println("    get pos=" + pos + " nextPos=" + nextPos + " count=" + count);
+      if (pos == nextPos) {
+        if (end) {
+          return -1;
+        }
+        final int ch = reader.read();
+        if (ch == -1) {
+          end = true;
+          return -1;
+        }
+        if (count == buffer.length) {
+          // Grow
+          final char[] newBuffer = new char[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_CHAR)];
+          System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length - nextWrite);
+          System.arraycopy(buffer, 0, newBuffer, buffer.length - nextWrite, nextWrite);
+          nextWrite = buffer.length;
+          //System.out.println("buffer: grow from " + buffer.length + " to " + newBuffer.length);
+          buffer = newBuffer;
+        }
+        if (nextWrite == buffer.length) {
+          nextWrite = 0;
+        }
+        buffer[nextWrite++] = (char) ch;
+        count++;
+        nextPos++;
+        return ch;
+      } else {
+        // Cannot read from future (except by 1):
+        assert pos < nextPos;
+
+        // Cannot read from already freed past:
+        assert nextPos - pos <= count;
+
+        final int index = getIndex(pos);
+        return buffer[index];
+      }
+    }
+
+    // For assert:
+    private boolean inBounds(int pos) {
+      return pos < nextPos && pos >= nextPos - count;
+    }
+
+    private int getIndex(int pos) {
+      int index = nextWrite - (nextPos - pos);
+      if (index < 0) {
+        // Wrap:
+        index += buffer.length;
+        assert index >= 0;
+      }
+      return index;
+    }
+
+    public char[] get(int posStart, int length) {
+      assert length > 0;
+      assert inBounds(posStart): "posStart=" + posStart + " length=" + length;
+      //System.out.println("    buffer.get posStart=" + posStart + " len=" + length);
+      
+      final int startIndex = getIndex(posStart);
+      final int endIndex = getIndex(posStart + length);
+      //System.out.println("      startIndex=" + startIndex + " endIndex=" + endIndex);
+
+      final char[] result = new char[length];
+      // nocommit what if entire buffer is requested...?
+      if (endIndex >= startIndex) {
+        System.arraycopy(buffer, startIndex, result, 0, endIndex-startIndex);
+      } else {
+        // Wrapped:
+        final int part1 = buffer.length-startIndex;
+        System.arraycopy(buffer, startIndex, result, 0, part1);
+        System.arraycopy(buffer, 0, result, buffer.length-startIndex, length-part1);
+      }
+      return result;
+    }
+
+    /** Call this to notify us that no chars before this
+     *  absolute position are needed anymore. */
+    public void freeBefore(int pos) {
+      assert pos <= nextPos;
+      count = nextPos - pos;
+      assert count < buffer.length;
+    }
+  }
+
+  // TODO: make generic'd version of this "circular array"?
+  private static final class WrappedPositionArray {
+    private Position[] positions = new Position[8];
+
+    public WrappedPositionArray() {
+      for(int i=0;i<positions.length;i++) {
+        positions[i] = new Position();
+      }
+    }
+
+    // Next array index to write to in positions:
+    private int nextWrite;
+
+    // Next position to write:
+    private int nextPos;
+    
+    // How many valid Position instances are held in the
+    // positions array:
+    private int count;
+
+    public void reset() {
+      nextWrite--;
+      while(count > 0) {
+        if (nextWrite == -1) {
+          nextWrite = positions.length - 1;
+        }
+        positions[nextWrite--].reset();
+        count--;
+      }
+      nextWrite = 0;
+      nextPos = 0;
+      count = 0;
+    }
+
+    /** Get Position instance for this absolute position;
+     *  this is allowed to be arbitrarily far "in the
+     *  future" but cannot be before the last freeBefore. */
+    public Position get(int pos) {
+      while(pos >= nextPos) {
+        //System.out.println("count=" + count + " vs len=" + positions.length);
+        if (count == positions.length) {
+          Position[] newPositions = new Position[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(positions, nextWrite, newPositions, 0, positions.length-nextWrite);
+          System.arraycopy(positions, 0, newPositions, positions.length-nextWrite, nextWrite);
+          for(int i=positions.length;i<newPositions.length;i++) {
+            newPositions[i] = new Position();
+          }
+          nextWrite = positions.length;
+          positions = newPositions;
+        }
+        if (nextWrite == positions.length) {
+          nextWrite = 0;
+        }
+        // Should have already been reset:
+        assert positions[nextWrite].count == 0;
+        positions[nextWrite++].pos = nextPos++;
+        count++;
+      }
+      assert inBounds(pos);
+      final int index = getIndex(pos);
+      assert positions[index].pos == pos;
+      return positions[index];
+    }
+
+    public int getNextPos() {
+      return nextPos;
+    }
+
+    // For assert:
+    private boolean inBounds(int pos) {
+      return pos < nextPos && pos >= nextPos - count;
+    }
+
+    private int getIndex(int pos) {
+      int index = nextWrite - (nextPos - pos);
+      if (index < 0) {
+        index += positions.length;
+      }
+      return index;
+    }
+
+    public void freeBefore(int pos) {
+      final int toFree = count - (nextPos - pos);
+      assert toFree >= 0;
+      assert toFree <= count;
+      int index = nextWrite - count;
+      if (index < 0) {
+        index += positions.length;
+      }
+      for(int i=0;i<toFree;i++) {
+        if (index == positions.length) {
+          index = 0;
+        }
+        //System.out.println("  fb idx=" + index);
+        positions[index].reset();
+        index++;
+      }
+      count -= toFree;
+    }
+  }
+
+  /* Incrementally parse some more characters.  This runs
+   * the viterbi search forwards "enough" so that we
+   * generate some more tokens.  How much forward depends on
+   * the chars coming in, since some chars could cause
+   * longer-lasting ambiguity in the parsing.  Once the
+   * ambiguity is resolved, then we back trace, produce
+   * the pending tokens, and return. */
+  private void parse() throws IOException {
+    if (VERBOSE) {
+      System.out.println("\nPARSE");
+    }
+
+    // Advances over each position (character):
+    while (true) {
+
+      if (buffer.get(pos) == -1) {
+        // End
+        break;
+      }
+
+      final Position posData = positions.get(pos);
+
+      if (pos != lastBackTracePos && posData.count == 1 && positions.getNextPos() == pos+1) {
+        // We are at a "frontier", and only one node is
+        // alive, so whatever the eventual best path is must
+        // come through this node.  So we can safely commit
+        // to the prefix of the best path at this point:
+        backtrace(posData, 0);
+
+        // Re-base cost so we don't risk int overflow:
+        posData.costs[0] = 0;
+
+        if (pending.size() != 0) {
+          return;
+        } else {
+          // nocommit: make punctuation-only testcase
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (VERBOSE) {
+        System.out.println("\n  extend @ pos=" + pos);
+      }
+
+      if (posData.count == 0) {
+        // No arcs arrive here; move to next position:
+        pos++;
+        if (VERBOSE) {
+          System.out.println("    no arcs in; skip");
+        }
+        continue;
+      }
+
+      if (VERBOSE) {
+        System.out.println("    " + posData.count + " arcs in");
+      }
+
+      // nocommit must also 1) detect a
+      // less-obvious-yet-still-committable backtrace op,
+      // when, even though N > 1 states are/were alive, they
+      // all back through a single state, and also 2) maybe
+      // need to "force" a "when all else fails" backtrace
+
+      boolean anyMatches = false;
+
+      // First try user dict:
+      if (userFST != null) {
+        userFST.getFirstArc(arc);
+        int output = 0;
+        for(int posAhead=posData.pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          if (userFST.findTargetArc(ch, arc, arc, posAhead == posData.pos, userFSTReader) == null) {
+            break;
+          }
+          output += arc.output.intValue();
+          if (arc.isFinal()) {
+            add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER);
+            anyMatches = true;
+          }
+        }
+      }
+
+      // nocommit we can be more aggressive about user
+      // matches?  if we are "under" a user match then don't
+      // extend these paths?
+
+      if (!anyMatches) {
+        // Next, try known dictionary matches
+        fst.getFirstArc(arc);
+        int output = 0;
+
+        boolean allKanji = true;
+        for(int posAhead=posData.pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          //System.out.println("    match " + (char) ch + " posAhead=" + posAhead);
+          
+          if (fst.findTargetArc(ch, arc, arc, posAhead == posData.pos, fstReader) == null) {
+            break;
+          }
+
+          output += arc.output.intValue();
+
+          // NOTE: for known words that are too-long
+          // (compound), we should pre-compute the 2nd
+          // best segmentation and store it in the
+          // dictionary.  This is just a time/space opto...
+
+          if (arc.isFinal()) {
+            dictionary.lookupWordIds(output + arc.nextFinalOutput.intValue(), wordIdRef);
+            if (VERBOSE) {
+              System.out.println("    KNOWN word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1) + " " + wordIdRef.length + " wordIDs");
+            }
+            for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+              add(dictionary, posData, posAhead+1, wordIdRef.ints[wordIdRef.offset + ofs], Type.KNOWN);
+              anyMatches = true;
+            }
+          }
+        }
+      }
+
+      // In the case of normal mode, it doesn't process unknown word greedily.
+
+      // nocommit: fix
+      /*
+      if (!searchMode && unknownWordEndIndex > posData.pos) {
+        continue;
+      }
+      */
+      final char firstCharacter = (char) buffer.get(pos);
+      // nocommit -- can't we NOT pursue unk if a known
+      // token "covers" us...?
+      if (!anyMatches || characterDefinition.isInvoke(firstCharacter)) {
+
+        // Find unknown match:
+        final int characterId = characterDefinition.getCharacterClass(firstCharacter);
+
+        // NOTE: copied from UnknownDictionary.lookup:
+        int unknownWordLength;
+        if (!characterDefinition.isGroup(firstCharacter)) {
+          unknownWordLength = 1;
+        } else {
+          // Extract unknown word. Characters with the same character class are considered to be part of unknown word
+          unknownWordLength = 1;
+          for (int posAhead=pos+1;;posAhead++) {
+            final int ch = buffer.get(posAhead);
+            if (ch == -1) {
+              break;
+            }
+            if (characterId == characterDefinition.getCharacterClass((char) ch)) {
+              unknownWordLength++;    			
+            } else {
+              break;
+            }
+          }
+        }
+
+        unkDictionary.lookupWordIds(characterId, wordIdRef); // characters in input text are supposed to be the same
+        if (VERBOSE) {
+          System.out.println("    UNKNOWN word len=" + unknownWordLength + " " + wordIdRef.length + " wordIDs");
+        }
+        for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+          add(unkDictionary, posData, posData.pos + unknownWordLength, wordIdRef.ints[wordIdRef.offset + ofs], Type.UNKNOWN);
+        }
+
+        // nocommit fixme
+        //unknownWordEndIndex = posData.pos + unknownWordLength;
+      }
+
+      pos++;
+    }
+
+    end = true;
+
+    if (pos > 0) {
+
+      final Position endPosData = positions.get(pos);
+      int leastCost = Integer.MAX_VALUE;
+      int leastIDX = 0;
+      for(int idx=0;idx<endPosData.count;idx++) {
+        // Add EOS cost:
+        final int cost = endPosData.costs[idx] + costs.get(endPosData.nodeID[idx], 0);
+        if (cost < leastCost) {
+          leastCost = cost;
+          leastIDX = idx;
+        }
+      }
+
+      backtrace(endPosData, leastIDX);
+    } else {
+      // No characters in the input string; return no tokens!
+    }
+  }
+
+  // Eliminates arcs from the lattice that are compound
+  // tokens (have a penalty) or are not congruent with the
+  // compound token we've matched (ie, span across the
+  // startPos).  This should be fairly efficient, because we
+  // just keep the already intersected structure of the
+  // graph, eg we don't have to consult the FSTs again:
+
+  private void pruneAndRescore(int startPos, int endPos, int bestStartIDX) throws IOException {
+    if (VERBOSE) {
+      System.out.println("  pruneAndRescore startPos=" + startPos + " endPos=" + endPos + " bestStartIDX=" + bestStartIDX);
+    }
+
+    int minBackPos = Integer.MAX_VALUE;
+
+    final int compoundLength = endPos - startPos;
+    // First pass: walk backwards, building up the forward
+    // arcs and pruning inadmissible arcs:
+    for(int pos=endPos; pos >= startPos; pos--) {
+      final Position posData = positions.get(pos);
+      if (VERBOSE) {
+        System.out.println("    back pos=" + pos);
+      }
+      for(int arcIDX=0;arcIDX<posData.count;arcIDX++) {
+        final int backPos = posData.backPos[arcIDX];
+        // nocommit: O(N^2) cost here!!  adversary could
+        // exploit this i think...?  how to protect?  hmm
+        // maybe i need to track lastKanjiCount at each
+        // position..
+        // nocommit hacky:
+        // if (backPos >= startPos && computePenalty(backPos, pos-backPos) == 0) {
+        if (backPos >= startPos && (pos-backPos) <= 2*compoundLength/3) {
+          //if (backPos != -1 && (pos-backPos) <= 2*compoundLength/3) {
+          // Keep this arc:
+          //System.out.println("      keep backPos=" + backPos);
+          minBackPos = Math.min(backPos, minBackPos);
+          positions.get(backPos).addForward(pos,
+                                            arcIDX,
+                                            posData.backID[arcIDX],
+                                            posData.backType[arcIDX]);
+        } else {
+          if (VERBOSE) {
+            System.out.println("      prune");
+          }
+        }
+      }
+      if (pos != startPos) {
+        posData.count = 0;
+      }
+    }
+
+    // nocommit what if minBackPos not set...?
+
+    // Second pass: walk forward, re-scoring:
+    //for(int pos=startPos; pos < endPos; pos++) {
+    for(int pos=minBackPos; pos < endPos; pos++) {
+      final Position posData = positions.get(pos);
+      if (VERBOSE) {
+        System.out.println("    forward pos=" + pos + " count=" + posData.forwardCount);
+      }
+      if (posData.count == 0) {
+        // No arcs arrive here...
+        if (VERBOSE) {
+          System.out.println("      skip");
+        }
+        posData.forwardCount = 0;
+        continue;
+      }
+      if (pos == startPos) {
+        // On the initial position, only consider the best
+        // path so we "force congruence":  the
+        // sub-segmentation is "in context" of what the best
+        // path (compound token) had matched:
+        final Dictionary dict = getDict(posData.backType[bestStartIDX]);
+        final int rightID = startPos == 0 ? 0 : dict.getRightId(posData.backID[bestStartIDX]);
+        final int pathCost = posData.backID[bestStartIDX];
+        for(int forwardArcIDX=0;forwardArcIDX<posData.forwardCount;forwardArcIDX++) {
+          final Type forwardType = posData.forwardType[forwardArcIDX];
+          final Dictionary dict2 = getDict(forwardType);
+          final int wordID = posData.forwardID[forwardArcIDX];
+          final int newCost = pathCost + dict2.getWordCost(wordID) + 
+            costs.get(rightID, dict2.getLeftId(wordID));
+          final int toPos = posData.forwardPos[forwardArcIDX];
+          if (VERBOSE) {
+            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos);
+          }
+          positions.get(toPos).add(newCost,
+                                   dict2.getRightId(wordID),
+                                   pos,
+                                   bestStartIDX,
+                                   wordID,
+                                   forwardType);
+        }
+      } else {
+        // On non-initial positions, we maximize score
+        // across all arriving nodeIDs:
+        for(int forwardArcIDX=0;forwardArcIDX<posData.forwardCount;forwardArcIDX++) {
+          final Type forwardType = posData.forwardType[forwardArcIDX];
+          final int toPos = posData.forwardPos[forwardArcIDX];
+          if (VERBOSE) {
+            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos);
+          }
+          add(getDict(forwardType),
+              posData,
+              toPos,
+              posData.forwardID[forwardArcIDX],
+              forwardType);
+        }
+      }
+      posData.forwardCount = 0;
+    }
+  }
+
+  // Backtrace from the provided position, back to the last
+  // time we back-traced, accumulating the resulting tokens to
+  // the pending list.  The pending list is then in-reverse
+  // (last token should be returned first).
+  private void backtrace(final Position endPosData, final int fromIDX) throws IOException {
+    if (VERBOSE) {
+      System.out.println("\n  backtrace: " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
+    }
+    final int endPos = endPosData.pos;
+
+    final char[] fragment = buffer.get(lastBackTracePos, endPos-lastBackTracePos);
+
+    int pos = endPos;
+    int bestIDX = fromIDX;
+    Token altToken = null;
+
+    // We trace backwards, so this will be the leftWordID of
+    // the token after the one we are now on:
+    int lastLeftWordID = -1;
+    
+    // nocommit: don't use intermediate Token instance
+    // here... change this to just hold raw back trace info,
+    // then in incrementToken we pull the necessary char[],
+    // and only call freeBefore once we're done iterating
+    // these tokens:
+    while (pos > lastBackTracePos) {
+      //System.out.println("back pos=" + pos);
+      final Position posData = positions.get(pos);
+
+      int backPos = posData.backPos[bestIDX];
+      int length = pos - backPos;
+      Type backType = posData.backType[bestIDX];
+
+      if (DO_OUTPUT_COMPOUND && searchMode && altToken == null && backType != Type.USER) {
+        
+        // In searchMode, if best path had picked a too-long
+        // token, we use the "penalty" to compute the allowed
+        // max cost of an alternate back-trace.  If we find an
+        // alternate back trace with cost below that
+        // threshold, we pursue it instead (but also output
+        // the long token).
+
+        // nocommit -- maybe re-tune this "penalty", now
+        // that it means something very different ("output
+        // smaller segmentation"):
+        final int penalty = computePenalty(backPos, pos-backPos);
+
+        if (penalty > 0) {
+          if (VERBOSE) {
+            System.out.println("  compound=" + new String(buffer.get(backPos, pos-backPos)) + " backPos=" + backPos + " pos=" + pos + " penalty=" + penalty);
+          }
+
+          final int backID = posData.backID[bestIDX];
+
+          // Now, prune all too-long tokens from the graph:
+          pruneAndRescore(backPos, pos,
+                          posData.backIndex[bestIDX]);
+
+          if (VERBOSE) {
+            System.out.println("  afterPrune: " + posData.count + " arcs arriving");
+          }
+
+          // Finally, find 2nd best back-trace and resume
+          // backtrace there:
+          int leastCost = Integer.MAX_VALUE;
+          int leastIDX = -1;
+          for(int idx=0;idx<posData.count;idx++) {
+            int cost = posData.costs[idx];
+            if (lastLeftWordID != -1) {
+              cost += costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
+                                lastLeftWordID);
+            }
+            if (cost < leastCost) {
+              leastCost = cost;
+              leastIDX = idx;
+            }
+          }
+          //System.out.println("  leastIDX=" + leastIDX);
+
+          // nocommit -- must also check whether 2nd best score falls w/in threshold?
+
+          if (leastIDX != -1) {
+            // We should have pruned the altToken from the graph:
+            assert posData.backPos[leastIDX] != backPos;
+
+            // Save the current compound token, to output when
+            // this alternate path joins back:
+            altToken = new Token(backID,
+                                 fragment,
+                                 backPos - lastBackTracePos,
+                                 length,
+                                 backType,
+                                 backPos,
+                                 getDict(backType));
+
+            // Redirect our backtrace to 2nd best:
+            bestIDX = leastIDX;
+
+            backPos = posData.backPos[bestIDX];
+            length = pos - backPos;
+            backType = posData.backType[bestIDX];
+            
+          } else {
+            // I think in theory it's possible there is no
+            // 2nd best path, which is fine; in this case we
+            // only output the compound token:
+          }
+        }
+      }
+
+      final int offset = backPos - lastBackTracePos;
+
+      // nocommit -- how come TestQuality doesn't change if
+      // i output the altToken!
+      // nocommit
+      if (altToken != null && altToken.getPosition() >= backPos) {
+
+        // We've backtraced to the position where the
+        // compound token starts; add it now:
+
+        // The pruning we did when we created the altToken
+        // ensures that the back trace will align back with
+        // the start of the altToken:
+        // cannot assert...
+        //assert altToken.getPosition() == backPos: altToken.getPosition() + " vs " + backPos;
+
+        if (VERBOSE) {
+          System.out.println("    add altToken=" + altToken);
+        }
+        // nocommit
+        pending.add(altToken);
+        altToken = null;
+      }
+
+      final Dictionary dict = getDict(backType);
+
+      if (backType == Type.USER) {
+
+        // Expand the phraseID we recorded into the actual
+        // segmentation:
+        final int[] wordIDAndLength = userDictionary.lookupSegmentation(posData.backID[bestIDX]);
+        int wordID = wordIDAndLength[0];
+        int current = backPos;
+        for(int j=1; j < wordIDAndLength.length; j++) {
+          final int len = wordIDAndLength[j];
+          //System.out.println("    add user: len=" + len);
+          pending.add(new Token(wordID+j-1,
+                                fragment,
+                                backPos + current - lastBackTracePos,
+                                len,
+                                Type.USER,
+                                backPos + current,
+                                dict));
+          current += len;
+        }
+
+        // Reverse the tokens we just added, because when we
+        // serve them up from incrementToken we serve in
+        // reverse:
+        Collections.reverse(pending.subList(pending.size() - (wordIDAndLength.length - 1),
+                                            pending.size()));
+
+      } else {
+
+        if (extendedMode && posData.backType[bestIDX] == Type.UNKNOWN) {
+          // nocommit what if the altToken is unknonwn?  
+          // In EXTENDED mode we convert unknown word into
+          // unigrams:
+          int unigramTokenCount = 0;
+          for(int i=length-1;i>=0;i--) {
+            int charLen = 1;
+            if (i > 0 && Character.isLowSurrogate(fragment[offset+i])) {
+              i--;
+              charLen = 2;
+            }
+            //System.out.println("    extended tok offset=" + (offset + i));
+            pending.add(new Token(CharacterDefinition.NGRAM,
+                                  fragment,
+                                  offset + i,
+                                  charLen,
+                                  Type.UNKNOWN,
+                                  backPos + i,
+                                  unkDictionary));
+            unigramTokenCount++;
+          }
+          
+        } else if (!discardPunctuation || length == 0 || !isPunctuation(fragment[offset])) {
+          //System.out.println("backPos=" + backPos);
+          pending.add(new Token(posData.backID[bestIDX],
+                                fragment,
+                                offset,
+                                length,
+                                backType,
+                                backPos,
+                                dict));
+          if (VERBOSE) {
+            System.out.println("    add token=" + pending.get(pending.size()-1));
+          }
+        } else {
+          if (VERBOSE) {
+            System.out.println("    skip punctuation token=" + new String(fragment, offset, length));
+          }
+        }
+      }
+
+      // nocommit -- accuracy drops a bit if we do this.... weird
+      lastLeftWordID = dict.getLeftId(posData.backID[bestIDX]);
+      pos = backPos;
+      bestIDX = posData.backIndex[bestIDX];
+    }
+
+    lastBackTracePos = endPos;
+    // nocommit explain & justify...:
+    if (endPosData.count == 0) {
+      endPosData.count = 1;
+    }
+
+    if (VERBOSE) {
+      System.out.println("  freeBefore pos=" + endPos);
+    }
+    // Notify the circular buffers that we are done with
+    // these positions:
+    buffer.freeBefore(endPos);
+    positions.freeBefore(endPos);
+  }
+
+  private Dictionary getDict(Type type) {
+    return dictionaryMap.get(type);
+  }
+
+  private static final boolean isPunctuation(char ch) {
+    // TODO: somehow this is slowish.. takes ~5% off
+    // chars/msec from Perf.java; maybe we
+    // can spend RAM...
+
+    // nocommit can we call this only when token is len
+    // 1... or it's unknown...?
+    switch(Character.getType(ch)) {
+      case Character.SPACE_SEPARATOR:
+      case Character.LINE_SEPARATOR:
+      case Character.PARAGRAPH_SEPARATOR:
+      case Character.CONTROL:
+      case Character.FORMAT:
+      case Character.DASH_PUNCTUATION:
+      case Character.START_PUNCTUATION:
+      case Character.END_PUNCTUATION:
+      case Character.CONNECTOR_PUNCTUATION:
+      case Character.OTHER_PUNCTUATION:
+      case Character.MATH_SYMBOL:
+      case Character.CURRENCY_SYMBOL:
+      case Character.MODIFIER_SYMBOL:
+      case Character.OTHER_SYMBOL:
+      case Character.INITIAL_QUOTE_PUNCTUATION:
+      case Character.FINAL_QUOTE_PUNCTUATION:
+        return true;
+      default:
+        return false;
+    }
+  }
+}

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java Sat Feb 11 20:29:23 2012
@@ -29,8 +29,8 @@ import org.apache.lucene.analysis.kuromo
 import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.analysis.kuromoji.viterbi.GraphvizFormatter;
 import org.apache.lucene.analysis.kuromoji.viterbi.Viterbi;
-import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode;
 import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
+import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode;
 
 /**
  * Tokenizer main class.
@@ -127,8 +127,9 @@ public class Segmenter {
       } else {
         position = Math.min(indexOfMaru, indexOfTen);				
       }
-      
-      if(position >= 0) {
+
+      // nocommit
+      if(false && position >= 0) {
         splitPositions.add(position);
         currentPosition = position + 1;
       } else {
@@ -152,7 +153,6 @@ public class Segmenter {
    */
   public List<Token> doTokenize(int offset, char[] sentence, int sentenceOffset, int sentenceLength, boolean discardPunctuation) {
     ArrayList<Token> result = new ArrayList<Token>();
-    
     ViterbiNode[][][] lattice;
     try {
       lattice = viterbi.build(sentence, sentenceOffset, sentenceLength);
@@ -170,7 +170,12 @@ public class Segmenter {
       Token token = new Token(wordId, node.getSurfaceForm(), node.getOffset(), node.getLength(), node.getType(), offset + node.getStartIndex(), dictionaryMap.get(node.getType()));	// Pass different dictionary based on the type of node
       result.add(token);
     }
-    
+    /*
+    System.out.println("result:");
+    for(Token token : result) {
+      System.out.println("  " + token);
+    }
+    */
     return result;
   }
   

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java Sat Feb 11 20:29:23 2012
@@ -42,6 +42,11 @@ public class Token {
     this.position = position;
     this.dictionary = dictionary;
   }
+
+  @Override
+  public String toString() {
+    return "Token(\"" + new String(surfaceForm, offset, length) + "\" pos=" + position + " type=" + type + " wordId=" + wordId + ")";
+  }
   
   /**
    * @return surfaceForm

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java Sat Feb 11 20:29:23 2012
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
 import org.apache.lucene.analysis.kuromoji.util.CSVUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.Builder;
@@ -159,6 +160,10 @@ public final class UserDictionary implem
     return found ? toIndexArray(result) : EMPTY_RESULT;
   }
   
+  public TokenInfoFST getFST() {
+    return fst;
+  }
+
   private static final int[][] EMPTY_RESULT = new int[0][];
   
   /**
@@ -181,6 +186,10 @@ public final class UserDictionary implem
     }
     return result.toArray(new int[result.size()][]);
   }
+
+  public int[] lookupSegmentation(int phraseID) {
+    return segmentations[phraseID];
+  }
   
   @Override
   public int getLeftId(int wordId) {

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java Sat Feb 11 20:29:23 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.analysis.kurom
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -117,7 +118,8 @@ public class Viterbi {
       if (startIndexArr[i] == null || endIndexArr[i] == null){	// continue since no array which contains ViterbiNodes exists. Or no previous node exists.
         continue;
       }
-      
+
+      // For each arc leaving...
       for (ViterbiNode node : startIndexArr[i]) {
         if (node == null){	// If array doesn't contain ViterbiNode any more, continue to next index
           break;
@@ -126,6 +128,7 @@ public class Viterbi {
         int backwardConnectionId = node.getLeftId();
         int wordCost = node.getWordCost();
         int leastPathCost = DEFAULT_COST;
+        // For each arc arriving...
         for (ViterbiNode leftNode : endIndexArr[i]) {
           if (leftNode == null){ // If array doesn't contain ViterbiNode any more, continue to next index
             break;
@@ -151,8 +154,10 @@ public class Viterbi {
               
               if (allKanji) {	// Process only Kanji keywords
                 pathCost += (length - SEARCH_MODE_KANJI_LENGTH) * SEARCH_MODE_KANJI_PENALTY;
+                //System.out.println("    + kanji penalty=" + (length - SEARCH_MODE_KANJI_LENGTH) * SEARCH_MODE_KANJI_PENALTY + " cost=" + pathCost);
               } else if (length > SEARCH_MODE_OTHER_LENGTH) {
                 pathCost += (length - SEARCH_MODE_OTHER_LENGTH) * SEARCH_MODE_OTHER_PENALTY;								
+                //System.out.println("    + non-kanji penalty=" + (length - SEARCH_MODE_OTHER_LENGTH) * SEARCH_MODE_OTHER_PENALTY + " cost=" + pathCost);
               }
             }
           }
@@ -241,7 +246,7 @@ public class Viterbi {
       int output = 0;
       for (int endIndex = 1; endIndex < suffixLength + 1; endIndex++) {
         int ch = text[suffixStart + endIndex - 1];
-        
+        //System.out.println("    match " + (char) ch);
         if (fst.findTargetArc(ch, arc, arc, endIndex == 1, fstReader) == null) {
           break; // continue to next position
         }
@@ -253,6 +258,7 @@ public class Viterbi {
           dictionary.lookupWordIds(finalOutput, wordIdRef);
           for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
             final int wordId = wordIdRef.ints[wordIdRef.offset + ofs];
+            //System.out.println("output=" + finalOutput + " wid=" + wordId);
             ViterbiNode node = new ViterbiNode(wordId, text, suffixStart, endIndex, dictionary.getLeftId(wordId), dictionary.getRightId(wordId), dictionary.getWordCost(wordId), startIndex, Type.KNOWN);
             addToArrays(node, startIndex + 1, startIndex + 1 + endIndex, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
           }
@@ -305,6 +311,7 @@ public class Viterbi {
   private void processUserDictionary(char text[], int offset, int len, ViterbiNode[][] startIndexArr, ViterbiNode[][] endIndexArr, int[] startSizeArr, int[] endSizeArr) throws IOException {
     int[][] result = userDictionary.lookup(text, offset, len);
     for(int[] segmentation : result) {
+      System.out.println("SEG=" + Arrays.toString(segmentation));
       int wordId = segmentation[0];
       int index = segmentation[1];
       int length = segmentation[2];
@@ -326,6 +333,8 @@ public class Viterbi {
   private void addToArrays(ViterbiNode node, int startIndex, int endIndex, ViterbiNode[][] startIndexArr, ViterbiNode[][] endIndexArr, int[] startSizeArr, int[] endSizeArr ) {
     int startNodesCount = startSizeArr[startIndex];
     int endNodesCount = endSizeArr[endIndex];
+
+    //System.out.println("  + " + startIndex + " to " + endIndex);
     
     if (startNodesCount == 0) {
       startIndexArr[startIndex] = new ViterbiNode[10];

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java Sat Feb 11 20:29:23 2012
@@ -26,6 +26,8 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+// nocommit cut these over to KT2
+
 public class SegmenterTest extends LuceneTestCase {
   
   private static Segmenter segmenter;

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java Sat Feb 11 20:29:23 2012
@@ -36,7 +36,8 @@ public class TestExtendedMode extends Ba
     
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(segmenter, reader);
+      //Tokenizer tokenizer = new KuromojiTokenizer(segmenter, reader);
+      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Mode.EXTENDED);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java Sat Feb 11 20:29:23 2012
@@ -18,8 +18,12 @@ package org.apache.lucene.analysis.kurom
  */
 
 import java.io.IOException;
+import java.io.StringReader;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 
 public class TestKuromojiAnalyzer extends BaseTokenStreamTestCase {
   /** This test fails with NPE when the 
@@ -41,15 +45,107 @@ public class TestKuromojiAnalyzer extend
         new int[] { 1, 2, 2,  2 }
       );
   }
-  
+
   /**
    * Test that search mode is enabled and working by default
    */
   public void testDecomposition() throws IOException {
-    assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "シニアソフトウェアエンジニア",
-        new String[] { "シニア", "ソフトウェア", "エンジニア" }
-    );
+
+    // nocommit
+    TokenStream ts = new KuromojiAnalyzer(TEST_VERSION_CURRENT)
+        .tokenStream("foo",
+                     new StringReader("マイケルジャクソン	マイケル ジャクソン"));
+    while(ts.incrementToken());
+
+    // Senior software engineer:
+    if (KuromojiTokenizer2.DO_OUTPUT_COMPOUND) {
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "シニアソフトウェアエンジニア",
+                       new String[] { "シニア",
+                                      "シニアソフトウェアエンジニア",
+                                      "ソフトウェア",
+                                      "エンジニア" },
+                       new int[] { 1, 0, 1, 1}
+                       );
+
+      // Kansai International Airport:
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "関西国際空港",
+                       new String[] { "関西",
+                                      "関西国際空港", // zero pos inc
+                                      "国際",
+                                      "空港" },
+                       new int[] {1, 0, 1, 1}
+                       );
+
+      // Konika Minolta Holdings; not quite the right
+      // segmentation (see LUCENE-3726):
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "コニカミノルタホールディングス",
+                       new String[] { "コニカ",
+                                      "コニカミノルタホールディングス", // zero pos inc
+                                      "ミノルタ", 
+                                      "ホールディングス"},
+                       new int[] {1, 0, 1, 1}
+                       );
+
+      // Narita Airport
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "成田空港",
+                       new String[] { "成田",
+                                      "成田空港",
+                                      "空港" },
+                       new int[] {1, 0, 1});
+
+      // Kyoto University Baseball Club
+      // nocommit --segments differently but perhaps OK
+      /*
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "京都大学硬式野球部",
+                       new String[] { "京都",
+                                      "京都大学硬式野球部",
+                                      "大学",
+                                      "硬式",
+                                      "野球",
+                                      "部" },
+                       new int[] {1, 0, 1, 1, 1, 1});
+      */
+    } else {
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "シニアソフトウェアエンジニア",
+                       new String[] { "シニア",
+                                      "ソフトウェア",
+                                      "エンジニア" },
+                       new int[] { 1, 1, 1}
+                       );
+      // Kansai International Airport:
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "関西国際空港",
+                       new String[] { "関西",
+                                      "国際",
+                                      "空港" },
+                       new int[] {1, 1, 1}
+                       );
+
+      // Konika Minolta Holdings; not quite the right
+      // segmentation (see LUCENE-3726):
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "コニカミノルタホールディングス",
+                       new String[] { "コニカ",
+                                      "ミノルタ", 
+                                      "ホールディングス"},
+                       new int[] {1, 1, 1}
+                       );
+
+      // Narita Airport
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "成田空港",
+                       new String[] { "成田",
+                                      "空港" },
+                       new int[] {1, 1});
+
+      // Kyoto University Baseball Club
+      assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "京都大学硬式野球部",
+                       new String[] { "京都",
+                                      "大学",
+                                      "硬式",
+                                      "野球",
+                                      "部" },
+                       new int[] {1, 1, 1, 1, 1});
+    }
   }
+
   
   /**
    * blast random strings against the analyzer

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java Sat Feb 11 20:29:23 2012
@@ -28,7 +28,8 @@ public class TestKuromojiBaseFormFilter 
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      //Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Segmenter.DEFAULT_MODE);
       return new TokenStreamComponents(tokenizer, new KuromojiBaseFormFilter(tokenizer));
     }
   };

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java?rev=1243149&r1=1243148&r2=1243149&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java Sat Feb 11 20:29:23 2012
@@ -17,7 +17,10 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.io.Reader;
 import java.io.StringReader;
 
@@ -25,15 +28,39 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 
 public class TestKuromojiTokenizer extends BaseTokenStreamTestCase {
+
+  private static UserDictionary readDict() {
+    InputStream is = SegmenterTest.class.getResourceAsStream("userdict.txt");
+    if (is == null) {
+      throw new RuntimeException("Cannot find userdict.txt in test classpath!");
+    }
+    try {
+      try {
+        Reader reader = new InputStreamReader(is, IOUtils.CHARSET_UTF_8);
+        return new UserDictionary(reader);
+      } finally {
+        is.close();
+      }
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe);
+    }
+  }
+
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      //Tokenizer tokenizer = new KuromojiTokenizer(new Segmenter(readDict()), reader);
+      //Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      Tokenizer tokenizer = new KuromojiTokenizer2(reader, readDict(), true, Mode.SEARCH);
+      //Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, true);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
@@ -91,6 +118,18 @@ public class TestKuromojiTokenizer exten
     ts.close();
   }
 
+  /*
+    // NOTE: intentionally fails!  Just trying to debug this
+    // one input...
+  public void testDecomposition6() throws Exception {
+    assertAnalyzesTo(analyzer, "奈良先端科学技術大学院大学",
+      new String[] { "これ", "は", "本", "で", "は", "ない" },
+      new int[] { 0, 2, 3, 4, 5, 6 },
+      new int[] { 2, 3, 4, 5, 6, 8 }
+                     );
+  }
+  */
+
   /** Tests that sentence offset is incorporated into the resulting offsets */
   public void testTwoSentences() throws Exception {
     assertAnalyzesTo(analyzer, "魔女狩大将マシュー・ホプキンス。 魔女狩大将マシュー・ホプキンス。",
@@ -125,6 +164,9 @@ public class TestKuromojiTokenizer exten
   public void testSurrogates2() throws IOException {
     int numIterations = atLeast(10000);
     for (int i = 0; i < numIterations; i++) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + i);
+      }
       String s = _TestUtil.randomUnicodeString(random, 100);
       TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
       CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
@@ -152,4 +194,32 @@ public class TestKuromojiTokenizer exten
         new Integer(12)
     );
   }
+
+  public void testUserDict() throws Exception {
+    // Not a great test because w/o userdict.txt the
+    // segmentation is the same:
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("関西国際空港に行った")),
+                              new String[] { "関西", "国際", "空港", "に", "行っ", "た"  },
+                              new int[] { 0, 2, 4, 6, 7, 9 },
+                              new int[] { 2, 4, 6, 7, 9, 10 },
+                              new Integer(10)
+    );
+  }
+
+  public void testUserDict2() throws Exception {
+    // Better test: w/o userdict the segmentation is different:
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("朝青龍")),
+                              new String[] { "朝青龍"  },
+                              new int[] { 0 },
+                              new int[] { 3 },
+                              new Integer(3)
+    );
+  }
+
+  // nocommit need a still better test, where an input is
+  // tokenized to multiple tokens that would be different
+  // w/o the user dict...
+
+  // nocommit need test where userdict has multiple matches
+  // from one spot (ie, some entries are prefixes)...
 }