LUCENE-3767: get toDot working for Viterbi lattice; replace old tokenizer w/ new; leave default at Mode.SEARCH; no more nocommits

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3767@1245098 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
index 0904aab..384f586 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
@@ -67,7 +67,7 @@
       final boolean isFirst = pos == -1;
       int posInc = posIncAtt.getPositionIncrement();
       if (isFirst && posInc == 0) {
-        // nocommit hmm are TS's still allowed to do this...?
+        // TODO: hmm are TS's still allowed to do this...?
         System.err.println("WARNING: first posInc was 0; correcting to 1");
         posInc = 1;
       }
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java
new file mode 100644
index 0000000..67446e2
--- /dev/null
+++ b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java
@@ -0,0 +1,180 @@
+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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Position;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Type;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.WrappedPositionArray;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
+
+
+// TODO: would be nice to show 2nd best path in a diff't
+// color...
+
+public class GraphvizFormatter {
+  
+  private final static String BOS_LABEL = "BOS";
+  
+  private final static String EOS_LABEL = "EOS";
+  
+  private final static String FONT_NAME = "Helvetica";
+  
+  private final ConnectionCosts costs;
+  
+  private final Map<String, String> bestPathMap;
+  
+  private final StringBuilder sb = new StringBuilder();
+  
+  public GraphvizFormatter(ConnectionCosts costs) {
+    this.costs = costs;
+    this.bestPathMap = new HashMap<String, String>();
+    sb.append(formatHeader());
+    sb.append("  init [style=invis]\n");
+    sb.append("  init -> 0.0 [label=\"BOS\"]\n");
+  }
+
+  public String finish() {
+    sb.append(formatTrailer());
+    return sb.toString();
+  }
+
+  // Backtraces another incremental fragment:
+  void onBacktrace(KuromojiTokenizer tok, WrappedPositionArray positions, int lastBackTracePos, Position endPosData, int fromIDX, char[] fragment, boolean isEnd) {
+    setBestPathMap(positions, lastBackTracePos, endPosData, fromIDX);
+    sb.append(formatNodes(tok, positions, lastBackTracePos, endPosData, fragment));
+    if (isEnd) {
+      sb.append("  fini [style=invis]\n");
+      sb.append("  ");
+      sb.append(getNodeID(endPosData.pos, fromIDX));
+      sb.append(" -> fini [label=\"EOS\"]");
+    }
+  }
+
+  // Records which arcs make up the best bath:
+  private void setBestPathMap(WrappedPositionArray positions, int startPos, Position endPosData, int fromIDX) {
+    bestPathMap.clear();
+
+    int pos = endPosData.pos;
+    int bestIDX = fromIDX;
+    while (pos > startPos) {
+      final Position posData = positions.get(pos);
+
+      final int backPos = posData.backPos[bestIDX];
+      final int backIDX = posData.backIndex[bestIDX];
+
+      final String toNodeID = getNodeID(pos, bestIDX);
+      final String fromNodeID = getNodeID(backPos, backIDX);
+      
+      assert !bestPathMap.containsKey(fromNodeID);
+      assert !bestPathMap.containsValue(toNodeID);
+      bestPathMap.put(fromNodeID, toNodeID);
+      pos = backPos;
+      bestIDX = backIDX;
+    }
+  }
+  
+  private String formatNodes(KuromojiTokenizer tok, WrappedPositionArray positions, int startPos, Position endPosData, char[] fragment) {
+
+    StringBuilder sb = new StringBuilder();
+    // Output nodes
+    for (int pos = startPos+1; pos <= endPosData.pos; pos++) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        sb.append("  ");
+        sb.append(getNodeID(pos, idx));
+        sb.append(" [label=\"");
+        sb.append(pos);
+        sb.append(": ");
+        sb.append(posData.lastRightID[idx]);
+        sb.append("\"]\n");
+      }
+    }
+
+    // Output arcs
+    for (int pos = endPosData.pos; pos > startPos; pos--) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        final Position backPosData = positions.get(posData.backPos[idx]);
+        final String toNodeID = getNodeID(pos, idx);
+        final String fromNodeID = getNodeID(posData.backPos[idx], posData.backIndex[idx]);
+
+        sb.append("  ");
+        sb.append(fromNodeID);
+        sb.append(" -> ");
+        sb.append(toNodeID);
+
+        final String attrs;
+        if (toNodeID.equals(bestPathMap.get(fromNodeID))) {
+          // This arc is on best path
+          attrs = " color=\"#40e050\" fontcolor=\"#40a050\" penwidth=3 fontsize=20";
+        } else {
+          attrs = "";
+        }
+
+        final Dictionary dict = tok.getDict(posData.backType[idx]);
+        final int wordCost = dict.getWordCost(posData.backID[idx]);
+        final int bgCost = costs.get(backPosData.lastRightID[posData.backIndex[idx]],
+                                     dict.getLeftId(posData.backID[idx]));
+
+        final String surfaceForm = new String(fragment,
+                                              posData.backPos[idx] - startPos,
+                                              pos - posData.backPos[idx]);
+        
+        sb.append(" [label=\"");
+        sb.append(surfaceForm);
+        sb.append(' ');
+        sb.append(wordCost);
+        if (bgCost >= 0) {
+          sb.append('+');
+        }
+        sb.append(bgCost);
+        sb.append("\"");
+        sb.append(attrs);
+        sb.append("]\n");
+      }
+    }
+    return sb.toString();
+  }
+  
+  private String formatHeader() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("digraph viterbi {\n");
+    sb.append("  graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\"];\n");
+    //sb.append("  // A2 paper size\n");
+    //sb.append("  size = \"34.4,16.5\";\n");
+    //sb.append("  // try to fill paper\n");
+    //sb.append("  ratio = fill;\n");
+    sb.append("  edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]\n");
+    sb.append("  node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]\n");
+    
+    return sb.toString();
+  }
+  
+  private String formatTrailer() {
+    return "}";
+  }
+  
+  private String getNodeID(int pos, int idx) {
+    return pos + "." + idx;
+  }
+}
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
index 96bcdf1..cc6020f 100644
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
+++ b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
@@ -27,36 +27,26 @@
 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.KuromojiTokenizer.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);
+    this(matchVersion, null, KuromojiTokenizer.DEFAULT_MODE, DefaultSetHolder.DEFAULT_STOP_SET, DefaultSetHolder.DEFAULT_STOP_TAGS);
   }
   
-  public KuromojiAnalyzer(Version matchVersion, Segmenter segmenter, CharArraySet stopwords, Set<String> stoptags) {
-    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(){
@@ -93,8 +83,7 @@
   
   @Override
   protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-    //Tokenizer tokenizer = new KuromojiTokenizer(this.segmenter, reader);
-    Tokenizer tokenizer = new KuromojiTokenizer2(reader, userDict, true, mode);
+    Tokenizer tokenizer = new KuromojiTokenizer(reader, userDict, true, mode);
     TokenStream stream = new KuromojiBaseFormFilter(tokenizer);
     stream = new KuromojiPartOfSpeechStopFilter(true, stream, stoptags);
     stream = new CJKWidthFilter(stream);
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
index 58bc59d..1a7f942 100644
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
+++ b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
@@ -17,68 +17,1125 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.io.Reader;
-import java.text.BreakIterator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
 import java.util.List;
-import java.util.Locale;
 
-import org.apache.lucene.analysis.kuromoji.tokenattributes.BaseFormAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.InflectionAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.PartOfSpeechAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.ReadingAttribute;
+import org.apache.lucene.analysis.Tokenizer;
+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.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.RollingCharBuffer;
+import org.apache.lucene.util.fst.FST;
 
-public final class KuromojiTokenizer extends SegmentingTokenizerBase {
-  private static final BreakIterator proto = BreakIterator.getSentenceInstance(Locale.JAPAN);
+// TODO: somehow factor out a reusable viterbi search here,
+// so other decompounders/tokenizers can reuse...
+
+/* Uses a rolling Viterbi search to find the least cost
+ * segmentation (path) of the incoming characters.  For
+ * tokens that appear to be compound (> length 2 for all
+ * Kanji, or > length 7 for non-Kanji), we see if there is a
+ * 2nd best segmentation of that token after applying
+ * penalties to the long tokens.  If so, and the Mode is
+ * SEARCH_WITH_COMPOUND, we output the alternate
+ * segmentation as well. */
+public final class KuromojiTokenizer extends Tokenizer {
+
+  public static enum Mode {
+    NORMAL, SEARCH, SEARCH_WITH_COMPOUNDS, EXTENDED
+  }
+
+  public static final Mode DEFAULT_MODE = Mode.SEARCH;
+
+  enum Type {
+    KNOWN,
+    UNKNOWN,
+    USER
+  }
+
+  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;
+
+  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 final RollingCharBuffer buffer = new RollingCharBuffer();
+
+  private final WrappedPositionArray positions = new WrappedPositionArray();
+
+  private final boolean discardPunctuation;
+  private final boolean searchMode;
+  private final boolean extendedMode;
+  private final boolean outputCompounds;
+
+  // Index of the last character of unknown word:
+  private int unknownWordEndIndex = -1;
+
+  // True once we've hit the EOF from the input reader:
+  private boolean end;
+
+  // Last absolute position we backtraced from:
+  private int lastBackTracePos;
+
+  // Position of last token we returned; we use this to
+  // figure out whether to set posIncr to 0 or 1:
+  private int lastTokenPos;
+
+  // Next absolute position to process:
+  private int pos;
+
+  // Already parsed, but not yet passed to caller, tokens:
+  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 PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.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);
-  private final Segmenter segmenter;
-  
-  private List<Token> tokens; 
-  private int tokenIndex = 0;
-  private int sentenceStart = 0;
-  
-  public KuromojiTokenizer(Reader input) {
-    this(new Segmenter(), input);
+
+  public KuromojiTokenizer(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;
+        outputCompounds = false;
+        break;
+      case SEARCH_WITH_COMPOUNDS:
+        searchMode = true;
+        extendedMode = false;
+        outputCompounds = true;
+        break;
+      case EXTENDED:
+        searchMode = true;
+        extendedMode = true;
+        outputCompounds = false;
+        break;
+      default:
+        searchMode = false;
+        extendedMode = false;
+        outputCompounds = false;
+        break;
+    }
+    buffer.reset(input);
+
+    resetState();
+
+    dictionaryMap.put(Type.KNOWN, dictionary);
+    dictionaryMap.put(Type.UNKNOWN, unkDictionary);
+    dictionaryMap.put(Type.USER, userDictionary);
   }
-  
-  public KuromojiTokenizer(Segmenter segmenter, Reader input) {
-    super(input, (BreakIterator) proto.clone());
-    this.segmenter = segmenter;
-  }
-  
-  @Override
-  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
-    this.sentenceStart = sentenceStart;
-    //System.out.println("\nNEXT SENTENCE: " + sentenceStart + " to " + sentenceEnd + ": " + new String(buffer, sentenceStart, sentenceEnd-sentenceStart));
-    // TODO: maybe don't pass 0 here, so kuromoji tracks offsets for us?
-    tokens = segmenter.doTokenize(0, buffer, sentenceStart, sentenceEnd-sentenceStart, true);
-    tokenIndex = 0;
+
+  private GraphvizFormatter dotOut;
+
+  /** Expert: set this to produce graphviz (dot) output of
+   *  the Viterbi lattice */
+  public void setGraphvizFormatter(GraphvizFormatter dotOut) {
+    this.dotOut = dotOut;
   }
 
   @Override
-  protected boolean incrementWord() {
-    if (tokenIndex == tokens.size()) {
-      return false;
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+    buffer.reset(input);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    resetState();
+  }
+
+  private void resetState() {
+    positions.reset();
+    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 void end() {
+    // Set final offset
+    offsetAtt.setOffset(correctOffset(pos), correctOffset(pos));
+  }
+
+  // Returns the added cost that a 2nd best segmentation is
+  // allowed to have.  Ie, if we see path with cost X,
+  // ending in a compound word, and this method returns
+  // threshold > 0, then we will also find the 2nd best
+  // segmentation and if its path score is within this
+  // threshold of X, we'll include it in the output:
+  private int computeSecondBestThreshold(int pos, int length) throws IOException {
+    // TODO: maybe we do something else here, instead of just
+    // using the penalty...?  EG we can be more aggressive on
+    // when to also test for 2nd best path
+    return computePenalty(pos, length);
+  }
+
+  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;
+        }				
+      }
+      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;								
+      }
     }
-    Token token = tokens.get(tokenIndex);
+    return 0;
+  }
+
+  // Holds all back pointers arriving to this position:
+  final static class Position {
+
+    int pos;
+
+    int count;
+
+    // maybe single int array * 5?
+    int[] costs = new int[8];
+    int[] lastRightID = new int[8];
+    int[] backPos = new int[8];
+    int[] backIndex = new int[8];
+    int[] backID = new int[8];
+    Type[] backType = new Type[8];
+
+    // Only used when finding 2nd best segmentation under a
+    // too-long token:
+    int forwardCount;
+    int[] forwardPos = new int[8];
+    int[] forwardID = new int[8];
+    int[] forwardIndex = new int[8];
+    Type[] forwardType = new Type[8];
+
+    public void grow() {
+      costs = ArrayUtil.grow(costs, 1+count);
+      lastRightID = ArrayUtil.grow(lastRightID, 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 lastRightID, int backPos, int backIndex, int backID, Type backType) {
+      // NOTE: this isn't quite a true Viterbit search,
+      // becase we should check if lastRightID is
+      // already present here, and only update if the new
+      // cost is less than the current cost, instead of
+      // simply appending.  However, that will likely hurt
+      // performance (usually we add a lastRightID only once),
+      // and it means we actually create the full graph
+      // intersection instead of a "normal" Viterbi lattice:
+      if (count == costs.length) {
+        grow();
+      }
+      this.costs[count] = cost;
+      this.lastRightID[count] = lastRightID;
+      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;
+    }
+  }
+
+  private void add(Dictionary dict, Position fromPosData, int endPos, int wordID, Type type, boolean addPenalty) throws IOException {
+    final int wordCost = dict.getWordCost(wordID);
+    final int leftID = dict.getLeftId(wordID);
+    int leastCost = Integer.MAX_VALUE;
+    int leastIDX = -1;
+    assert fromPosData.count > 0;
+    for(int idx=0;idx<fromPosData.count;idx++) {
+      // Cost is path cost so far, plus word cost (added at
+      // end of loop), plus bigram cost:
+      final int cost = fromPosData.costs[idx] + costs.get(fromPosData.lastRightID[idx], leftID);
+      if (VERBOSE) {
+        System.out.println("      fromIDX=" + idx + ": cost=" + cost + " (prevCost=" + fromPosData.costs[idx] + " wordCost=" + wordCost + " bgCost=" + costs.get(fromPosData.lastRightID[idx], leftID) + " leftID=" + leftID);
+      }
+      if (cost < leastCost) {
+        leastCost = cost;
+        leastIDX = idx;
+        //System.out.println("        **");
+      }
+    }
+
+    leastCost += wordCost;
+
+    if (VERBOSE) {
+      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " leftID=" + leftID + " leastIDX=" + leastIDX + " toPos.idx=" + positions.get(endPos).count);
+    }
+
+    if ((addPenalty || (!outputCompounds && searchMode)) && type != Type.USER) {
+      final int penalty = computePenalty(fromPosData.pos, endPos - fromPosData.pos);
+      if (VERBOSE) {
+        if (penalty > 0) {
+          System.out.println("        + penalty=" + penalty + " cost=" + (leastCost+penalty));
+        }
+      }
+      leastCost += penalty;
+    }
+
+    //positions.get(endPos).add(leastCost, dict.getRightId(wordID), fromPosData.pos, leastIDX, wordID, type);
+    assert leftID == dict.getRightId(wordID);
+    positions.get(endPos).add(leastCost, leftID, fromPosData.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(buffer, sentenceStart + position, length);
-    int startOffset = offset + sentenceStart + position;
-    offsetAtt.setOffset(correctOffset(startOffset), correctOffset(startOffset+length));
+    assert length > 0;
+    //System.out.println("off=" + token.getOffset() + " len=" + length + " vs " + token.getSurfaceForm().length);
+    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);
-    tokenIndex++;
+    if (token.getPosition() == lastTokenPos) {
+      posIncAtt.setPositionIncrement(0);
+      posLengthAtt.setPositionLength(token.getPositionLength());
+    } else {
+      assert token.getPosition() > lastTokenPos;
+      posIncAtt.setPositionIncrement(1);
+      posLengthAtt.setPositionLength(1);
+    }
+    if (VERBOSE) {
+      System.out.println(Thread.currentThread().getName() + ":    incToken: return token=" + token);
+    }
+    lastTokenPos = token.getPosition();
     return true;
   }
+
+  // TODO: make generic'd version of this "circular array"?
+  // It's a bit tricky because we do things to the Position
+  // (eg, set .pos = N on reuse)...
+  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.out.println("grow positions " + newPositions.length);
+          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);
+      final boolean isFrontier = positions.getNextPos() == pos+1;
+
+      if (posData.count == 0) {
+        // No arcs arrive here; move to next position:
+        pos++;
+        if (VERBOSE) {
+          System.out.println("    no arcs in; skip");
+        }
+        continue;
+      }
+
+      if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        //  if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        // 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 {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (pos - lastBackTracePos >= 2048) {
+        // Safety: if we've buffered too much, force a
+        // backtrace now:
+        int leastIDX = -1;
+        int leastCost = Integer.MAX_VALUE;
+        for(int idx=0;idx<posData.count;idx++) {
+          //System.out.println("    idx=" + idx + " cost=" + cost);
+          final int cost = posData.costs[idx];
+          if (cost < leastCost) {
+            leastCost = cost;
+            leastIDX = idx;
+          }
+        }
+        backtrace(posData, leastIDX);
+
+        // Re-base cost so we don't risk int overflow:
+        Arrays.fill(posData.costs, 0, posData.count, 0);
+
+        if (pending.size() != 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (VERBOSE) {
+        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos));
+      }
+
+      if (VERBOSE) {
+        System.out.println("    " + posData.count + " arcs in");
+      }
+
+      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()) {
+            if (VERBOSE) {
+              System.out.println("    USER word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1));
+            }
+            add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER, false);
+            anyMatches = true;
+          }
+        }
+      }
+
+      // TODO: we can be more aggressive about user
+      // matches?  if we are "under" a user match then don't
+      // extend KNOWN/UNKNOWN paths?
+
+      if (!anyMatches) {
+        // Next, try known dictionary matches
+        fst.getFirstArc(arc);
+        int output = 0;
+
+        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();
+
+          // Optimization: for known words that are too-long
+          // (compound), we should pre-compute the 2nd
+          // best segmentation and store it in the
+          // dictionary instead of recomputing it each time a
+          // match is found.
+
+          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, false);
+              anyMatches = true;
+            }
+          }
+        }
+      }
+
+      // In the case of normal mode, it doesn't process unknown word greedily.
+
+      if (!searchMode && unknownWordEndIndex > posData.pos) {
+        pos++;
+        continue;
+      }
+
+      final char firstCharacter = (char) buffer.get(pos);
+      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, false);
+        }
+
+        unknownWordEndIndex = posData.pos + unknownWordLength;
+      }
+
+      pos++;
+    }
+
+    end = true;
+
+    if (pos > 0) {
+
+      final Position endPosData = positions.get(pos);
+      int leastCost = Integer.MAX_VALUE;
+      int leastIDX = -1;
+      if (VERBOSE) {
+        System.out.println("  end: " + endPosData.count + " nodes");
+      }
+      for(int idx=0;idx<endPosData.count;idx++) {
+        // Add EOS cost:
+        final int cost = endPosData.costs[idx] + costs.get(endPosData.lastRightID[idx], 0);
+        //System.out.println("    idx=" + idx + " cost=" + cost + " (pathCost=" + endPosData.costs[idx] + " bgCost=" + costs.get(endPosData.lastRightID[idx], 0) + ") backPos=" + endPosData.backPos[idx]);
+        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);
+    }
+
+    // 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];
+        if (backPos >= startPos) {
+          // Keep this arc:
+          //System.out.println("      keep backPos=" + backPos);
+          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;
+      }
+    }
+
+    // Second pass: walk forward, re-scoring:
+    for(int pos=startPos; 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 int rightID;
+        if (startPos == 0) {
+          rightID = 0;
+        } else {
+          rightID = getDict(posData.backType[bestStartIDX]).getRightId(posData.backID[bestStartIDX]);
+        }
+        final int pathCost = posData.costs[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 toPos = posData.forwardPos[forwardArcIDX];
+          final int newCost = pathCost + dict2.getWordCost(wordID) + 
+            costs.get(rightID, dict2.getLeftId(wordID)) +
+            computePenalty(pos, toPos-pos);
+          if (VERBOSE) {
+            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos + " cost=" + newCost + " penalty=" + computePenalty(pos, toPos-pos) + " toPos.idx=" + positions.get(toPos).count);
+          }
+          positions.get(toPos).add(newCost,
+                                   dict2.getRightId(wordID),
+                                   pos,
+                                   bestStartIDX,
+                                   wordID,
+                                   forwardType);
+        }
+      } else {
+        // On non-initial positions, we maximize score
+        // across all arriving lastRightIDs:
+        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,
+              true);
+        }
+      }
+      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=" + pos + "; " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
+    }
+    final int endPos = endPosData.pos;
+
+    final char[] fragment = buffer.get(lastBackTracePos, endPos-lastBackTracePos);
+
+    if (dotOut != null) {
+      dotOut.onBacktrace(this, positions, lastBackTracePos, endPosData, fromIDX, fragment, end);
+    }
+
+    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;
+
+    int backCount = 0;
+
+    // TODO: sort of silly to make Token instances here; the
+    // back trace has all info needed to generate the
+    // token.  So, we could just directly set the attrs,
+    // from the backtrace, in incrementToken w/o ever
+    // creating Token; we'd have to defer calling freeBefore
+    // until after the bactrace was fully "consumed" by
+    // incrementToken.
+
+    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];
+      int backID = posData.backID[bestIDX];
+
+      if (outputCompounds && 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).
+
+        final int penalty = computeSecondBestThreshold(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 + " cost=" + posData.costs[bestIDX] + " bestIDX=" + bestIDX + " lastLeftID=" + lastLeftWordID);
+          }
+
+          // Use the penalty to set maxCost on the 2nd best
+          // segmentation:
+          int maxCost = posData.costs[bestIDX] + penalty;
+          if (lastLeftWordID != -1) {
+            maxCost += costs.get(getDict(backType).getRightId(backID), lastLeftWordID);
+          }
+
+          // Now, prune all too-long tokens from the graph:
+          pruneAndRescore(backPos, pos,
+                          posData.backIndex[bestIDX]);
+
+          // 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];
+            //System.out.println("    idx=" + idx + " prevCost=" + cost);
+            
+            if (lastLeftWordID != -1) {
+              cost += costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
+                                lastLeftWordID);
+              //System.out.println("      += bgCost=" + costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
+              //lastLeftWordID) + " -> " + cost);
+            }
+            //System.out.println("penalty " + posData.backPos[idx] + " to " + pos);
+            //cost += computePenalty(posData.backPos[idx], pos - posData.backPos[idx]);
+            if (cost < leastCost) {
+              //System.out.println("      ** ");
+              leastCost = cost;
+              leastIDX = idx;
+            }
+          }
+          //System.out.println("  leastIDX=" + leastIDX);
+
+          if (VERBOSE) {
+            System.out.println("  afterPrune: " + posData.count + " arcs arriving; leastCost=" + leastCost + " vs threshold=" + maxCost + " lastLeftWordID=" + lastLeftWordID);
+          }
+
+          if (leastIDX != -1 && leastCost <= maxCost && posData.backPos[leastIDX] != backPos) {
+            // 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];
+            backID = posData.backID[bestIDX];
+            backCount = 0;
+            
+          } 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;
+      assert offset >= 0;
+
+      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);
+        }
+        if (backCount > 0) {
+          backCount++;
+          altToken.setPositionLength(backCount);
+          pending.add(altToken);
+        } else {
+          // This means alt token was all punct tokens:
+          assert discardPunctuation;
+        }
+        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(backID);
+        int wordID = wordIDAndLength[0];
+        int current = 0;
+        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,
+                                current + offset,
+                                len,
+                                Type.USER,
+                                current + backPos,
+                                dict));
+          if (VERBOSE) {
+            System.out.println("    add USER token=" + pending.get(pending.size()-1));
+          }
+          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()));
+
+        backCount += wordIDAndLength.length-1;
+      } else {
+
+        if (extendedMode && backType == Type.UNKNOWN) {
+          // 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));
+            if (!discardPunctuation || !isPunctuation(fragment[offset+i])) {
+              pending.add(new Token(CharacterDefinition.NGRAM,
+                                    fragment,
+                                    offset + i,
+                                    charLen,
+                                    Type.UNKNOWN,
+                                    backPos + i,
+                                    unkDictionary));
+              unigramTokenCount++;
+            }
+          }
+          backCount += unigramTokenCount;
+          
+        } else if (!discardPunctuation || length == 0 || !isPunctuation(fragment[offset])) {
+          pending.add(new Token(backID,
+                                fragment,
+                                offset,
+                                length,
+                                backType,
+                                backPos,
+                                dict));
+          if (VERBOSE) {
+            System.out.println("    add token=" + pending.get(pending.size()-1));
+          }
+          backCount++;
+        } else {
+          if (VERBOSE) {
+            System.out.println("    skip punctuation token=" + new String(fragment, offset, length));
+          }
+        }
+      }
+
+      lastLeftWordID = dict.getLeftId(backID);
+      pos = backPos;
+      bestIDX = posData.backIndex[bestIDX];
+    }
+
+    lastBackTracePos = endPos;
+
+    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);
+  }
+
+  Dictionary getDict(Type type) {
+    return dictionaryMap.get(type);
+  }
+
+  private static boolean isPunctuation(char ch) {
+    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;
+    }
+  }
 }
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
deleted file mode 100644
index 8f52b51..0000000
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
+++ /dev/null
@@ -1,1123 +0,0 @@
-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.Arrays;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.List;
-
-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.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.RollingCharBuffer;
-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
-
-// nocomit explain how the 2nd best tokenization is
-// "contextual"...
-
-// nocommit beast test random data...
-
-// nocommit what default mode...?
-
-/* 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;
-
-  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 final RollingCharBuffer buffer = new RollingCharBuffer();
-
-  private final WrappedPositionArray positions = new WrappedPositionArray();
-
-  private final boolean discardPunctuation;
-  private final boolean searchMode;
-  private final boolean extendedMode;
-  private final boolean outputCompounds;
-
-  // Index of the last character of unknown word:
-  private int unknownWordEndIndex = -1;
-
-  // True once we've hit the EOF from the input reader:
-  private boolean end;
-
-  // Last absolute position we backtraced from:
-  private int lastBackTracePos;
-
-  // Position of last token we returned; we use this to
-  // figure out whether to set posIncr to 0 or 1:
-  private int lastTokenPos;
-
-  // Next absolute position to process:
-  private int pos;
-
-  // Already parsed, but not yet passed to caller, tokens:
-  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 PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.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;
-        outputCompounds = false;
-        break;
-      case SEARCH_WITH_COMPOUNDS:
-        searchMode = true;
-        extendedMode = false;
-        outputCompounds = true;
-        break;
-      case EXTENDED:
-        searchMode = true;
-        extendedMode = true;
-        outputCompounds = false;
-        break;
-      default:
-        searchMode = false;
-        extendedMode = false;
-        outputCompounds = 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);
-  }
-
-  @Override
-  public void reset() throws IOException {
-    super.reset();
-    resetState();
-  }
-
-  private void resetState() {
-    positions.reset();
-    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 void end() {
-    // Set final offset
-    offsetAtt.setOffset(correctOffset(pos), correctOffset(pos));
-  }
-
-  // Returns the added cost that a 2nd best segmentation is
-  // allowed to have.  Ie, if we see path with cost X,
-  // ending in a compound word, and this method returns
-  // threshold > 0, then we will also find the 2nd best
-  // segmentation and if its path score is within this
-  // threshold of X, we'll include it in the output:
-  private int computeSecondBestThreshold(int pos, int length) throws IOException {
-    // TODO: maybe we do something else here, instead of just
-    // using the penalty...?  EG we can be more aggressive on
-    // when to also test for 2nd best path
-    return computePenalty(pos, length);
-  }
-
-  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;
-        }				
-      }
-      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[8];
-    int[] lastRightID = new int[8];
-    int[] backPos = new int[8];
-    int[] backIndex = new int[8];
-    int[] backID = new int[8];
-    Type[] backType = new Type[8];
-
-    // Only used when finding 2nd best segmentation under a
-    // too-long token:
-    int forwardCount;
-    int[] forwardPos = new int[8];
-    int[] forwardID = new int[8];
-    int[] forwardIndex = new int[8];
-    Type[] forwardType = new Type[8];
-
-    public void grow() {
-      costs = ArrayUtil.grow(costs, 1+count);
-      lastRightID = ArrayUtil.grow(lastRightID, 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 lastRightID, int backPos, int backIndex, int backID, Type backType) {
-      // NOTE: this isn't quite a true Viterbit search,
-      // becase we should check if lastRightID is
-      // already present here, and only update if the new
-      // cost is less than the current cost, instead of
-      // simply appending.  However, that will likely hurt
-      // performance (usually we add a lastRightID only once),
-      // and it means we actually create the full graph
-      // intersection instead of a "normal" Viterbi lattice:
-      if (count == costs.length) {
-        grow();
-      }
-      this.costs[count] = cost;
-      this.lastRightID[count] = lastRightID;
-      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;
-    }
-  }
-
-  private void add(Dictionary dict, Position fromPosData, int endPos, int wordID, Type type, boolean addPenalty) throws IOException {
-    final int wordCost = dict.getWordCost(wordID);
-    final int leftID = dict.getLeftId(wordID);
-    int leastCost = Integer.MAX_VALUE;
-    int leastIDX = -1;
-    assert fromPosData.count > 0;
-    for(int idx=0;idx<fromPosData.count;idx++) {
-      // Cost is path cost so far, plus word cost (added at
-      // end of loop), plus bigram cost:
-      final int cost = fromPosData.costs[idx] + costs.get(fromPosData.lastRightID[idx], leftID);
-      if (VERBOSE) {
-        System.out.println("      fromIDX=" + idx + ": cost=" + cost + " (prevCost=" + fromPosData.costs[idx] + " wordCost=" + wordCost + " bgCost=" + costs.get(fromPosData.lastRightID[idx], leftID) + " leftID=" + leftID);
-      }
-      if (cost < leastCost) {
-        leastCost = cost;
-        leastIDX = idx;
-        //System.out.println("        **");
-      }
-    }
-
-    leastCost += wordCost;
-
-    if (VERBOSE) {
-      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " leftID=" + leftID + " leastIDX=" + leastIDX + " toPos.idx=" + positions.get(endPos).count);
-    }
-
-    if ((addPenalty || (!outputCompounds && searchMode)) && type != Type.USER) {
-      final int penalty = computePenalty(fromPosData.pos, endPos - fromPosData.pos);
-      if (VERBOSE) {
-        if (penalty > 0) {
-          System.out.println("        + penalty=" + penalty + " cost=" + (leastCost+penalty));
-        }
-      }
-      leastCost += penalty;
-    }
-
-    //positions.get(endPos).add(leastCost, dict.getRightId(wordID), fromPosData.pos, leastIDX, wordID, type);
-    assert leftID == dict.getRightId(wordID);
-    positions.get(endPos).add(leastCost, leftID, fromPosData.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();
-    assert length > 0;
-    //System.out.println("off=" + token.getOffset() + " len=" + length + " vs " + token.getSurfaceForm().length);
-    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);
-      posLengthAtt.setPositionLength(token.getPositionLength());
-    } else {
-      assert token.getPosition() > lastTokenPos;
-      posIncAtt.setPositionIncrement(1);
-      posLengthAtt.setPositionLength(1);
-    }
-    if (VERBOSE) {
-      System.out.println(Thread.currentThread().getName() + ":    incToken: return token=" + token);
-    }
-    lastTokenPos = token.getPosition();
-    return true;
-  }
-
-  // TODO: make generic'd version of this "circular array"?
-  // It's a bit tricky because we do things to the Position
-  // (eg, set .pos = N on reuse)...
-  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.out.println("grow positions " + newPositions.length);
-          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);
-      final boolean isFrontier = positions.getNextPos() == pos+1;
-
-      if (posData.count == 0) {
-        // No arcs arrive here; move to next position:
-        pos++;
-        if (VERBOSE) {
-          System.out.println("    no arcs in; skip");
-        }
-        continue;
-      }
-
-      if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
-        //  if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
-        // 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 {
-          // This means the backtrace only produced
-          // punctuation tokens, so we must keep parsing.
-        }
-      }
-
-      if (pos - lastBackTracePos >= 2048) {
-        // Safety: if we've buffered too much, force a
-        // backtrace now:
-        int leastIDX = -1;
-        int leastCost = Integer.MAX_VALUE;
-        for(int idx=0;idx<posData.count;idx++) {
-          //System.out.println("    idx=" + idx + " cost=" + cost);
-          final int cost = posData.costs[idx];
-          if (cost < leastCost) {
-            leastCost = cost;
-            leastIDX = idx;
-          }
-        }
-        backtrace(posData, leastIDX);
-
-        // Re-base cost so we don't risk int overflow:
-        Arrays.fill(posData.costs, 0, posData.count, 0);
-
-        if (pending.size() != 0) {
-          return;
-        } else {
-          // This means the backtrace only produced
-          // punctuation tokens, so we must keep parsing.
-        }
-      }
-
-      if (VERBOSE) {
-        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos));
-      }
-
-      if (VERBOSE) {
-        System.out.println("    " + posData.count + " arcs in");
-      }
-
-      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()) {
-            if (VERBOSE) {
-              System.out.println("    USER word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1));
-            }
-            add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER, false);
-            anyMatches = true;
-          }
-        }
-      }
-
-      // TODO: we can be more aggressive about user
-      // matches?  if we are "under" a user match then don't
-      // extend KNOWN/UNKNOWN paths?
-
-      if (!anyMatches) {
-        // Next, try known dictionary matches
-        fst.getFirstArc(arc);
-        int output = 0;
-
-        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();
-
-          // Optimization: for known words that are too-long
-          // (compound), we should pre-compute the 2nd
-          // best segmentation and store it in the
-          // dictionary instead of recomputing it each time a
-          // match is found.
-
-          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, false);
-              anyMatches = true;
-            }
-          }
-        }
-      }
-
-      // In the case of normal mode, it doesn't process unknown word greedily.
-
-      if (!searchMode && unknownWordEndIndex > posData.pos) {
-        continue;
-      }
-
-      final char firstCharacter = (char) buffer.get(pos);
-      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, false);
-        }
-
-        unknownWordEndIndex = posData.pos + unknownWordLength;
-      }
-
-      pos++;
-    }
-
-    end = true;
-
-    if (pos > 0) {
-
-      final Position endPosData = positions.get(pos);
-      int leastCost = Integer.MAX_VALUE;
-      int leastIDX = -1;
-      if (VERBOSE) {
-        System.out.println("  end: " + endPosData.count + " nodes");
-      }
-      for(int idx=0;idx<endPosData.count;idx++) {
-        // Add EOS cost:
-        final int cost = endPosData.costs[idx] + costs.get(endPosData.lastRightID[idx], 0);
-        //System.out.println("    idx=" + idx + " cost=" + cost + " (pathCost=" + endPosData.costs[idx] + " bgCost=" + costs.get(endPosData.lastRightID[idx], 0) + ") backPos=" + endPosData.backPos[idx]);
-        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);
-    }
-
-    // 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];
-        if (backPos >= startPos) {
-          // Keep this arc:
-          //System.out.println("      keep backPos=" + backPos);
-          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;
-      }
-    }
-
-    // Second pass: walk forward, re-scoring:
-    for(int pos=startPos; 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 int rightID;
-        if (startPos == 0) {
-          rightID = 0;
-        } else {
-          rightID = getDict(posData.backType[bestStartIDX]).getRightId(posData.backID[bestStartIDX]);
-        }
-        final int pathCost = posData.costs[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 toPos = posData.forwardPos[forwardArcIDX];
-          final int newCost = pathCost + dict2.getWordCost(wordID) + 
-            costs.get(rightID, dict2.getLeftId(wordID)) +
-            computePenalty(pos, toPos-pos);
-          if (VERBOSE) {
-            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos + " cost=" + newCost + " penalty=" + computePenalty(pos, toPos-pos) + " toPos.idx=" + positions.get(toPos).count);
-          }
-          positions.get(toPos).add(newCost,
-                                   dict2.getRightId(wordID),
-                                   pos,
-                                   bestStartIDX,
-                                   wordID,
-                                   forwardType);
-        }
-      } else {
-        // On non-initial positions, we maximize score
-        // across all arriving lastRightIDs:
-        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,
-              true);
-        }
-      }
-      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=" + pos + "; " + (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;
-
-    int backCount = 0;
-
-    // TODO: sort of silly to make Token instances here; the
-    // back trace has all info needed to generate the
-    // token.  So, we could just directly set the attrs,
-    // from the backtrace, in incrementToken w/o ever
-    // creating Token; we'd have to defer calling freeBefore
-    // until after the bactrace was fully "consumed" by
-    // incrementToken.
-
-    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];
-      int backID = posData.backID[bestIDX];
-
-      if (outputCompounds && 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).
-
-        final int penalty = computeSecondBestThreshold(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 + " cost=" + posData.costs[bestIDX] + " bestIDX=" + bestIDX + " lastLeftID=" + lastLeftWordID);
-          }
-
-          // Use the penalty to set maxCost on the 2nd best
-          // segmentation:
-          int maxCost = posData.costs[bestIDX] + penalty;
-          if (lastLeftWordID != -1) {
-            maxCost += costs.get(getDict(backType).getRightId(backID), lastLeftWordID);
-          }
-
-          // Now, prune all too-long tokens from the graph:
-          pruneAndRescore(backPos, pos,
-                          posData.backIndex[bestIDX]);
-
-          // 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];
-            //System.out.println("    idx=" + idx + " prevCost=" + cost);
-            
-            if (lastLeftWordID != -1) {
-              cost += costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
-                                lastLeftWordID);
-              //System.out.println("      += bgCost=" + costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
-              //lastLeftWordID) + " -> " + cost);
-            }
-            //System.out.println("penalty " + posData.backPos[idx] + " to " + pos);
-            //cost += computePenalty(posData.backPos[idx], pos - posData.backPos[idx]);
-            if (cost < leastCost) {
-              //System.out.println("      ** ");
-              leastCost = cost;
-              leastIDX = idx;
-            }
-          }
-          //System.out.println("  leastIDX=" + leastIDX);
-
-          if (VERBOSE) {
-            System.out.println("  afterPrune: " + posData.count + " arcs arriving; leastCost=" + leastCost + " vs threshold=" + maxCost + " lastLeftWordID=" + lastLeftWordID);
-          }
-
-          if (leastIDX != -1 && leastCost <= maxCost && posData.backPos[leastIDX] != backPos) {
-            // 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];
-            backID = posData.backID[bestIDX];
-            backCount = 0;
-            
-          } 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;
-      assert offset >= 0;
-
-      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);
-        }
-        if (backCount > 0) {
-          backCount++;
-          altToken.setPositionLength(backCount);
-          pending.add(altToken);
-        } else {
-          // This means alt token was all punct tokens:
-          assert discardPunctuation;
-        }
-        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(backID);
-        int wordID = wordIDAndLength[0];
-        int current = 0;
-        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,
-                                current + offset,
-                                len,
-                                Type.USER,
-                                current + backPos,
-                                dict));
-          if (VERBOSE) {
-            System.out.println("    add USER token=" + pending.get(pending.size()-1));
-          }
-          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()));
-
-        backCount += wordIDAndLength.length-1;
-      } else {
-
-        if (extendedMode && backType == Type.UNKNOWN) {
-          // 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));
-            if (!discardPunctuation || !isPunctuation(fragment[offset+i])) {
-              pending.add(new Token(CharacterDefinition.NGRAM,
-                                    fragment,
-                                    offset + i,
-                                    charLen,
-                                    Type.UNKNOWN,
-                                    backPos + i,
-                                    unkDictionary));
-              unigramTokenCount++;
-            }
-          }
-          backCount += unigramTokenCount;
-          
-        } else if (!discardPunctuation || length == 0 || !isPunctuation(fragment[offset])) {
-          pending.add(new Token(backID,
-                                fragment,
-                                offset,
-                                length,
-                                backType,
-                                backPos,
-                                dict));
-          if (VERBOSE) {
-            System.out.println("    add token=" + pending.get(pending.size()-1));
-          }
-          backCount++;
-        } else {
-          if (VERBOSE) {
-            System.out.println("    skip punctuation token=" + new String(fragment, offset, length));
-          }
-        }
-      }
-
-      lastLeftWordID = dict.getLeftId(backID);
-      pos = backPos;
-      bestIDX = posData.backIndex[bestIDX];
-    }
-
-    lastBackTracePos = endPos;
-
-    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 boolean isPunctuation(char ch) {
-    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;
-    }
-  }
-}
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
deleted file mode 100644
index b93190c..0000000
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
+++ /dev/null
@@ -1,218 +0,0 @@
-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.util.ArrayList;
-import java.util.EnumMap;
-import java.util.List;
-
-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.UnknownDictionary;
-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.Type;
-import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode;
-
-/**
- * Tokenizer main class.
- * Thread safe.
- */
-public class Segmenter {
-  public static enum Mode {
-    NORMAL, SEARCH, SEARCH_WITH_COMPOUNDS, EXTENDED
-  }
-  
-  public static final Mode DEFAULT_MODE = Mode.SEARCH;
-  
-  private final Viterbi viterbi;
-  
-  private final EnumMap<Type, Dictionary> dictionaryMap = new EnumMap<Type, Dictionary>(Type.class);
-  
-  private final boolean split;
-  
-  public Segmenter() {
-    this(null, DEFAULT_MODE, false);
-  }
-
-  public Segmenter(Mode mode) {
-    this(null, mode, false);
-  }
-
-  public Segmenter(UserDictionary userDictionary) {
-    this(userDictionary, DEFAULT_MODE, false);
-  }
-
-  public Segmenter(UserDictionary userDictionary, Mode mode) {
-    this(userDictionary, mode, false);
-  }
-
-  public Segmenter(UserDictionary userDictionary, Mode mode, boolean split) {
-    final TokenInfoDictionary dict = TokenInfoDictionary.getInstance();
-    final UnknownDictionary unknownDict = UnknownDictionary.getInstance();
-    this.viterbi = new Viterbi(dict, unknownDict, ConnectionCosts.getInstance(), userDictionary, mode);
-    this.split = split;
-    
-    dictionaryMap.put(Type.KNOWN, dict);
-    dictionaryMap.put(Type.UNKNOWN, unknownDict);
-    dictionaryMap.put(Type.USER, userDictionary);
-  }
-  
-  /**
-   * Tokenize input text
-   * @param text
-   * @return list of Token
-   */
-  public List<Token> tokenize(String text) {
-    
-    if (!split) {
-      return doTokenize(0, text);			
-    }
-    
-    List<Integer> splitPositions = getSplitPositions(text);
-    
-    if(splitPositions.size() == 0) {
-      return doTokenize(0, text);
-    }
-    
-    ArrayList<Token> result = new ArrayList<Token>();
-    int offset = 0;
-    for(int position : splitPositions) {
-      result.addAll(doTokenize(offset, text.substring(offset, position + 1)));
-      offset = position + 1;
-    }
-    
-    if(offset < text.length()) {
-      result.addAll(doTokenize(offset, text.substring(offset)));
-    }
-    
-    return result;
-  }
-  
-  /**
-   * Split input text at 句読点, which is 。 and 、
-   * @param text
-   * @return list of split position
-   */
-  private List<Integer> getSplitPositions(String text) {
-    ArrayList<Integer> splitPositions = new ArrayList<Integer>();
-    
-    int position = 0;
-    int currentPosition = 0;
-    
-    while(true) {
-      int indexOfMaru = text.indexOf("。", currentPosition);
-      int indexOfTen = text.indexOf("、", currentPosition);
-      
-      if(indexOfMaru < 0 || indexOfTen < 0) {
-        position = Math.max(indexOfMaru, indexOfTen);;
-      } else {
-        position = Math.min(indexOfMaru, indexOfTen);				
-      }
-
-      if(position >= 0) {
-        splitPositions.add(position);
-        currentPosition = position + 1;
-      } else {
-        break;
-      }
-    }
-    
-    return splitPositions;
-  }
-  
-  private List<Token> doTokenize(int offset, String sentence) {
-    char text[] = sentence.toCharArray();
-    return doTokenize(offset, text, 0, text.length, false);
-  }
-  
-  /**
-   * Tokenize input sentence.
-   * @param offset offset of sentence in original input text
-   * @param sentence sentence to tokenize
-   * @return list of Token
-   */
-  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);
-    } catch (IOException impossible) {
-      throw new RuntimeException(impossible);
-    }
-    List<ViterbiNode> bestPath = viterbi.search(lattice);
-    for (ViterbiNode node : bestPath) {
-      int wordId = node.getWordId();
-      if (node.getType() == Type.KNOWN && wordId == -1){ // Do not include BOS/EOS 
-        continue;
-      } else if (discardPunctuation && node.getLength() > 0 && isPunctuation(node.getSurfaceForm()[node.getOffset()])) {
-        continue; // Do not emit punctuation
-      }
-      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;
-  }
-  
-  /** returns a Graphviz String */
-  public String debugTokenize(String text) {
-    ViterbiNode[][][] lattice;
-    try {
-      lattice = this.viterbi.build(text.toCharArray(), 0, text.length());
-    } catch (IOException impossible) {
-      throw new RuntimeException(impossible);
-    }
-    List<ViterbiNode> bestPath = this.viterbi.search(lattice);
-    
-    return new GraphvizFormatter(ConnectionCosts.getInstance())
-      .format(lattice[0], lattice[1], bestPath);
-  }
-  
-  static final boolean isPunctuation(char ch) {
-    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;
-    }
-  }
-}
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
index 79a81fe..02c9adb 100644
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
+++ b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
@@ -17,8 +17,8 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Type;
 import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
-import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
 
 public class Token {
   private final Dictionary dictionary;
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java
deleted file mode 100644
index 758efe9..0000000
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java
+++ /dev/null
@@ -1,226 +0,0 @@
-package org.apache.lucene.analysis.kuromoji.viterbi;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
-import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
-
-public class GraphvizFormatter {
-  
-  private final static String BOS_LABEL = "BOS";
-  
-  private final static String EOS_LABEL = "EOS";
-  
-  private final static String FONT_NAME = "Helvetica";
-  
-  private ConnectionCosts costs;
-  
-  private Map<String, ViterbiNode> nodeMap;
-  
-  private Map<String, String> bestPathMap;
-  
-  private boolean foundBOS;
-  
-  public GraphvizFormatter(ConnectionCosts costs) {
-    this.costs = costs;
-    this.nodeMap = new HashMap<String, ViterbiNode>();
-    this.bestPathMap = new HashMap<String, String>();
-  }
-  
-  public String format(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray) {
-    initBestPathMap(null);
-    
-    StringBuilder sb = new StringBuilder();
-    sb.append(formatHeader());
-    sb.append(formatNodes(startsArray, endsArray));
-    sb.append(formatTrailer());
-    return sb.toString();
-  }
-  
-  public String format(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray, List<ViterbiNode> bestPath) {
-    
-    //		List<ViterbiNode> bestPathWithBOSAndEOS = new ArrayList<ViterbiNode>(bastPath);
-    initBestPathMap(bestPath);
-    
-    StringBuilder sb = new StringBuilder();
-    sb.append(formatHeader());
-    sb.append(formatNodes(startsArray, endsArray));
-    sb.append(formatTrailer());
-    return sb.toString();
-    
-  }
-  
-  private void initBestPathMap(List<ViterbiNode> bestPath) {
-    this.bestPathMap.clear();
-    
-    if (bestPath == null){
-      return;
-    }
-    for (int i = 0; i < bestPath.size() - 1; i++) {
-      ViterbiNode from = bestPath.get(i);
-      ViterbiNode to = bestPath.get(i + 1);
-      
-      String fromId = getNodeId(from);
-      String toId = getNodeId(to);
-      
-      assert this.bestPathMap.containsKey(fromId) == false;
-      assert this.bestPathMap.containsValue(toId) == false;
-      this.bestPathMap.put(fromId, toId);
-    }
-  }
-  
-  private String formatNodes(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray) {
-    this.nodeMap.clear();
-    this.foundBOS = false;
-    
-    StringBuilder sb = new StringBuilder();
-    for (int i = 1; i < endsArray.length; i++) {
-      if(endsArray[i] == null || startsArray[i] == null) {
-        continue;
-      }
-      for (int j = 0; j < endsArray[i].length; j++) {
-        ViterbiNode from = endsArray[i][j];
-        if(from == null){
-          continue;
-        }
-        sb.append(formatNodeIfNew(from));
-        for (int k = 0; k < startsArray[i].length; k++) {
-          ViterbiNode to = startsArray[i][k];
-          if(to == null){
-            break;
-          }
-          sb.append(formatNodeIfNew(to));
-          sb.append(formatEdge(from, to));
-        }
-      }
-    }
-    return sb.toString();
-  }
-  
-  private String formatNodeIfNew(ViterbiNode node) {
-    String nodeId = getNodeId(node);
-    if (! this.nodeMap.containsKey(nodeId)) {
-      this.nodeMap.put(nodeId, node);
-      return formatNode(node);
-    } else {
-      return "";
-    }
-  }	
-  
-  private String formatHeader() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("digraph viterbi {\n");
-    sb.append("graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\" ];\n");
-    sb.append("# A2 paper size\n");
-    sb.append("size = \"34.4,16.5\";\n");
-    sb.append("# try to fill paper\n");
-    sb.append("ratio = fill;\n");
-    sb.append("edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]\n");
-    sb.append("node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]\n");
-    
-    return sb.toString();
-  }
-  
-  private String formatTrailer() {
-    return "}";
-  }
-  
-  
-  private String formatEdge(ViterbiNode from, ViterbiNode to) {
-    if (this.bestPathMap.containsKey(getNodeId(from)) &&
-        this.bestPathMap.get(getNodeId(from)).equals(getNodeId(to))) {
-      return formatEdge(from, to, "color=\"#40e050\" fontcolor=\"#40a050\" penwidth=3 fontsize=20 ");
-      
-    } else {
-      return formatEdge(from, to, "");
-    }
-  }
-  
-  
-  private String formatEdge(ViterbiNode from, ViterbiNode to, String attributes) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getNodeId(from));
-    sb.append(" -> ");
-    sb.append(getNodeId(to));
-    sb.append(" [ ");
-    sb.append("label=\"");
-    sb.append(getCost(from, to));
-    sb.append("\"");
-    sb.append(" ");
-    sb.append(attributes);
-    sb.append(" ");
-    sb.append(" ]");
-    sb.append("\n");
-    return sb.toString();
-  }
-  
-  private String formatNode(ViterbiNode node) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("\"");
-    sb.append(getNodeId(node));
-    sb.append("\"");
-    sb.append(" [ ");
-    sb.append("label=");
-    sb.append(formatNodeLabel(node));
-    sb.append(" ]");
-    return sb.toString();
-  }
-  
-  private String formatNodeLabel(ViterbiNode node) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("<<table border=\"0\" cellborder=\"0\">");
-    sb.append("<tr><td>");
-    sb.append(getNodeLabel(node));
-    sb.append("</td></tr>");
-    sb.append("<tr><td>");
-    sb.append("<font color=\"blue\">");
-    sb.append(node.getWordCost());
-    sb.append("</font>");
-    sb.append("</td></tr>");
-    //		sb.append("<tr><td>");
-    //		sb.append(this.dictionary.get(node.getWordId()).getPosInfo());
-    //		sb.append("</td></tr>");
-    sb.append("</table>>");
-    return sb.toString();
-  }
-  
-  private String getNodeId(ViterbiNode node) {
-    return String.valueOf(node.hashCode());
-  }
-  
-  private String getNodeLabel(ViterbiNode node) {
-    if (node.getType() == Type.KNOWN && node.getWordId() == 0) {
-      if (this.foundBOS) {
-        return EOS_LABEL;
-      } else {
-        this.foundBOS = true;
-        return BOS_LABEL;
-      }
-    } else {
-      return node.getSurfaceFormString();
-    }
-  }
-  
-  private int getCost(ViterbiNode from, ViterbiNode to) {
-    return this.costs.get(from.getLeftId(), to.getRightId());
-  }
-}
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
deleted file mode 100644
index d50fa9a..0000000
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
+++ /dev/null
@@ -1,382 +0,0 @@
-package org.apache.lucene.analysis.kuromoji.viterbi;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-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.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.viterbi.ViterbiNode.Type;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.fst.FST;
-
-public class Viterbi {
-  
-  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 boolean useUserDictionary;
-  
-  private final boolean searchMode;
-  
-  private final boolean extendedMode;
-  
-  private static final int DEFAULT_COST = 10000000;
-  
-  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;
-  
-  private static final char[] BOS = "BOS".toCharArray();
-  
-  private static final char[] EOS = "EOS".toCharArray();
-  
-  /**
-   * Constructor
-   */
-  public Viterbi(TokenInfoDictionary dictionary,
-      UnknownDictionary unkDictionary,
-      ConnectionCosts costs,
-      UserDictionary userDictionary,
-      Mode mode) {
-    this.dictionary = dictionary;
-    this.fst = dictionary.getFST();
-    this.unkDictionary = unkDictionary;
-    this.costs = costs;
-    this.userDictionary = userDictionary;
-    if(userDictionary == null) {
-      this.useUserDictionary = false;
-    } else {
-      this.useUserDictionary = true;
-    }
-    
-    switch(mode){
-      case SEARCH:
-        searchMode = true;
-        extendedMode = false;
-        break;
-      case EXTENDED:
-        searchMode = true;
-        extendedMode = true;
-        break;
-      default:
-        searchMode = false;
-        extendedMode = false;
-        break;
-    }
-    
-    this.characterDefinition = unkDictionary.getCharacterDefinition();
-  }
-  
-  /**
-   * Find best path from input lattice.
-   * @param lattice the result of build method
-   * @return	List of ViterbiNode which consist best path 
-   */
-  public List<ViterbiNode> search(ViterbiNode[][][] lattice) {
-    ViterbiNode[][] startIndexArr = lattice[0];
-    ViterbiNode[][] endIndexArr = lattice[1];
-    
-    for (int i = 1; i < startIndexArr.length; i++){
-      
-      if (startIndexArr[i] == null || endIndexArr[i] == null){	// continue since no array which contains ViterbiNodes exists. Or no previous node exists.
-        continue;
-      }
-      //System.out.println("\npos=" + (i-1));
-
-      // 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;
-        }
-
-        //System.out.println("  leaving node.wordID=" + node.getWordId() + " leftID=" + node.getLeftId() + " toPos=" + (node.getOffset()+node.getLength()));
-        
-        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;
-          }
-          
-          //System.out.println("    arriving node.wordID=" + leftNode.getWordId() + " rightID=" + leftNode.getRightId());
-          int pathCost = leftNode.getPathCost() + costs.get(leftNode.getRightId(), backwardConnectionId) + wordCost;	// cost = [total cost from BOS to previous node] + [connection cost between previous node and current node] + [word cost]
-
-          //System.out.println("      pathCost=" + pathCost);
-          // "Search mode". Add extra costs if it is long node.
-          if (searchMode) {
-            //						System.out.print(""); // If this line exists, kuromoji runs faster for some reason when searchMode == false.
-            char[] surfaceForm = node.getSurfaceForm();
-            int offset = node.getOffset();
-            int length = node.getLength();
-            if (length > SEARCH_MODE_KANJI_LENGTH) {
-              boolean allKanji = true;
-              // check if node consists of only kanji
-              for (int pos = 0; pos < length; pos++) {
-                if (!characterDefinition.isKanji(surfaceForm[offset+pos])){
-                  allKanji = false;
-                  break;
-                }				
-              }
-              
-              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);
-                //System.out.println("        after penalty pathCost=" + 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);
-                //System.out.println("        after penalty pathCost=" + pathCost);
-              }
-            }
-          }
-          
-          if (pathCost < leastPathCost){	// If total cost is lower than before, set current previous node as best left node (previous means left).
-            //System.out.println("        **");
-            leastPathCost = pathCost;
-            node.setPathCost(leastPathCost);
-            node.setLeftNode(leftNode);
-          }					
-        }
-      }
-    }
-    
-    // track best path
-    ViterbiNode node = endIndexArr[0][0];	// EOS
-    LinkedList<ViterbiNode> result = new LinkedList<ViterbiNode>();
-    result.add(node);
-    while (true) {
-      ViterbiNode leftNode = node.getLeftNode();
-      if (leftNode == null) {
-        break;
-      }
-      
-      // EXTENDED mode convert unknown word into unigram node
-      if (extendedMode && leftNode.getType() == Type.UNKNOWN) {
-        byte unigramWordId = CharacterDefinition.NGRAM;
-        int unigramLeftId = unkDictionary.getLeftId(unigramWordId); // isn't required
-        int unigramRightId = unkDictionary.getLeftId(unigramWordId); // isn't required
-        int unigramWordCost = unkDictionary.getWordCost(unigramWordId); // isn't required
-        char[] surfaceForm = leftNode.getSurfaceForm();
-        int offset = leftNode.getOffset();
-        int length = leftNode.getLength();
-        for (int i = length - 1; i >= 0; i--) {
-          int charLen = 1;
-          if (i > 0 && Character.isLowSurrogate(surfaceForm[offset+i])) {
-            i--;
-            charLen = 2;
-          }
-          ViterbiNode uniGramNode = new ViterbiNode(unigramWordId, surfaceForm, offset + i, charLen, unigramLeftId, unigramRightId, unigramWordCost, leftNode.getStartIndex() + i, Type.UNKNOWN);
-          result.addFirst(uniGramNode);
-        }
-      } else {
-        result.addFirst(leftNode);		
-      }
-      node = leftNode;
-    }
-    
-    return result;
-  }
-
-  /**
-   * Build lattice from input text
-   * @param text
-   */
-  public ViterbiNode[][][] build(char text[], int offset, int length) throws IOException {
-    ViterbiNode[][] startIndexArr = new ViterbiNode[length + 2][];  // text length + BOS and EOS
-    ViterbiNode[][] endIndexArr = new ViterbiNode[length + 2][];  // text length + BOS and EOS
-    int[] startSizeArr = new int[length + 2]; // array to keep ViterbiNode count in startIndexArr
-    int[] endSizeArr = new int[length + 2];   // array to keep ViterbiNode count in endIndexArr
-    FST.Arc<Long> arc = new FST.Arc<Long>();
-    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);
-    }
-    
-    int unknownWordEndIndex = -1;	// index of the last character of unknown word
-    
-    final IntsRef wordIdRef = new IntsRef();
-    
-    for (int startIndex = 0; startIndex < length; startIndex++) {
-      // If no token ends where current token starts, skip this index
-      if (endSizeArr[startIndex + 1] == 0) {
-        continue;
-      }
-      
-      int suffixStart = offset + startIndex;
-      int suffixLength = length - startIndex;
-      
-      boolean found = false;
-      arc = fst.getFirstArc(arc);
-      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
-        }
-        output += arc.output.intValue();
-
-        if (arc.isFinal()) {
-          final int finalOutput = output + arc.nextFinalOutput.intValue();
-          found = true; // Don't produce unknown word starting from this index
-          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);
-          }
-        }
-      }
-      
-      // In the case of normal mode, it doesn't process unknown word greedily.
-      if(!searchMode && unknownWordEndIndex > startIndex){
-        continue;
-      }
-      
-      // Process Unknown Word: hmm what is this isInvoke logic (same no matter what)
-      int unknownWordLength = 0;
-      char firstCharacter = text[suffixStart];
-      boolean isInvoke = characterDefinition.isInvoke(firstCharacter);
-      if (isInvoke){	// Process "invoke"
-        unknownWordLength = unkDictionary.lookup(text, suffixStart, suffixLength);
-      } else if (found == false){	// Process not "invoke"
-        unknownWordLength = unkDictionary.lookup(text, suffixStart, suffixLength);				
-      }
-      
-      if (unknownWordLength > 0) {      // found unknown word
-        final int characterId = characterDefinition.getCharacterClass(firstCharacter);
-        unkDictionary.lookupWordIds(characterId, wordIdRef); // characters in input text are supposed to be the same
-        for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
-          final int wordId = wordIdRef.ints[wordIdRef.offset + ofs];
-          ViterbiNode node = new ViterbiNode(wordId, text, suffixStart, unknownWordLength, unkDictionary.getLeftId(wordId), unkDictionary.getRightId(wordId), unkDictionary.getWordCost(wordId), startIndex, Type.UNKNOWN);
-          addToArrays(node, startIndex + 1, startIndex + 1 + unknownWordLength, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
-        }
-        unknownWordEndIndex = startIndex + unknownWordLength;
-      }
-    }
-    
-    ViterbiNode eosNode = new ViterbiNode(-1, EOS, 0, EOS.length, 0, 0, 0, length + 1, Type.KNOWN);
-    addToArrays(eosNode, length + 1, 0, startIndexArr, endIndexArr, startSizeArr, endSizeArr); //Add EOS node to endIndexArr at index 0
-    
-    ViterbiNode[][][] result = new ViterbiNode[][][]{startIndexArr, endIndexArr};
-    
-    return result;
-  }
-  
-  /**
-   * Find token(s) in input text and set found token(s) in arrays as normal tokens
-   * @param text	
-   * @param startIndexArr
-   * @param endIndexArr
-   * @param startSizeArr
-   * @param endSizeArr
-   */
-  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];
-      ViterbiNode node = new ViterbiNode(wordId, text, offset + index, length, userDictionary.getLeftId(wordId), userDictionary.getRightId(wordId), userDictionary.getWordCost(wordId), index, Type.USER);
-      addToArrays(node, index + 1, index + 1 + length, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
-    }
-  }
-  
-  /**
-   * Add node to arrays and increment count in size array
-   * @param node
-   * @param startIndex
-   * @param endIndex
-   * @param startIndexArr
-   * @param endIndexArr
-   * @param startSizeArr
-   * @param endSizeArr
-   */
-  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];
-    }
-    
-    if (endNodesCount == 0) {
-      endIndexArr[endIndex] = new ViterbiNode[10];
-    }
-    
-    if (startIndexArr[startIndex].length <= startNodesCount){
-      startIndexArr[startIndex] = extendArray(startIndexArr[startIndex]);
-    }
-    
-    if (endIndexArr[endIndex].length <= endNodesCount){
-      endIndexArr[endIndex] = extendArray(endIndexArr[endIndex]);
-    }
-    
-    startIndexArr[startIndex][startNodesCount] = node;
-    endIndexArr[endIndex][endNodesCount] = node;
-    
-    startSizeArr[startIndex] = startNodesCount + 1;
-    endSizeArr[endIndex] = endNodesCount + 1;
-  }
-  
-  
-  /**
-   * Return twice as big array which contains value of input array
-   * @param array
-   * @return
-   */
-  private ViterbiNode[] extendArray(ViterbiNode[] array) {
-    //extend array
-    ViterbiNode[] newArray = new ViterbiNode[array.length * 2];
-    System.arraycopy(array, 0, newArray, 0, array.length);
-    return newArray;
-  }
-}
diff --git a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java b/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java
deleted file mode 100644
index 06ae4cd..0000000
--- a/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java
+++ /dev/null
@@ -1,147 +0,0 @@
-package org.apache.lucene.analysis.kuromoji.viterbi;
-
-/**
- * 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.
- */
-
-public final class ViterbiNode {
-  public enum Type {
-    KNOWN,
-    UNKNOWN,
-    USER
-  }
-  
-  private final int wordId;
-  
-  private final char[] surfaceForm;
-  private final int offset;
-  private final int length;
-  
-  private final int leftId;
-  
-  private final int rightId;
-  
-  /** word cost for this node */
-  private final int wordCost;
-  
-  /** minimum path cost found thus far */
-  private int pathCost;
-  
-  private ViterbiNode leftNode;
-  
-  private final Type type;
-  
-  private final int startIndex;
-  
-  public ViterbiNode(int wordId, char[] surfaceForm, int offset, int length, int leftId, int rightId, int wordCost, int startIndex, Type type) {
-    this.wordId = wordId;
-    this.surfaceForm = surfaceForm;
-    this.offset = offset;
-    this.length = length;
-    this.leftId = leftId;
-    this.rightId = rightId;
-    this.wordCost = wordCost;
-    this.startIndex = startIndex;
-    this.type = type;
-  }
-  
-  
-  /**
-   * @return the wordId
-   */
-  public int getWordId() {
-    return wordId;
-  }
-  
-  /**
-   * @return the surfaceForm
-   */
-  public char[] getSurfaceForm() {
-    return surfaceForm;
-  }
-  
-  /**
-   * @return start offset into surfaceForm
-   */
-  public int getOffset() {
-    return offset;
-  }
-  
-  /**
-   * @return length of surfaceForm
-   */
-  public int getLength() {
-    return length;
-  }
-  
-  /**
-   * @return the surfaceForm as a String
-   */
-  public String getSurfaceFormString() {
-    return new String(surfaceForm, offset, length);
-  }
-  
-  /**
-   * @return the leftId
-   */
-  public int getLeftId() {
-    return leftId;
-  }
-  
-  /**
-   * @return the rightId
-   */
-  public int getRightId() {
-    return rightId;
-  }
-  
-  /**
-   * @return the cost
-   */
-  public int getWordCost() {
-    return wordCost;
-  }
-  
-  /**
-   * @return the cost
-   */
-  public int getPathCost() {
-    return pathCost;
-  }
-  
-  /**
-   * param cost minimum path cost found this far
-   */
-  public void setPathCost(int pathCost) {
-    this.pathCost = pathCost;
-  }
-  
-  public void setLeftNode(ViterbiNode node) {
-    leftNode = node;
-  }
-  
-  public ViterbiNode getLeftNode() {
-    return leftNode;
-  }
-  
-  public int getStartIndex() {
-    return startIndex;
-  }
-  
-  public Type getType() {
-    return type;
-  }
-}
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java
deleted file mode 100644
index e0c35a0..0000000
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java
+++ /dev/null
@@ -1,231 +0,0 @@
-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.InputStreamReader;
-import java.io.LineNumberReader;
-import java.util.List;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class SegmenterTest extends LuceneTestCase {
-  
-  private static Segmenter segmenter;
-  
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    segmenter = new Segmenter();
-  }
-  
-  @AfterClass
-  public static void afterClass() throws Exception {
-    segmenter = null;
-  }
-  
-  @Test
-  public void testSegmentation() {
-    // Skip tests for Michelle Kwan -- UniDic segments Kwan as ク ワン
-    //		String input = "ミシェル・クワンが優勝しました。スペースステーションに行きます。うたがわしい。";
-    //		String[] surfaceForms = {
-    //				"ミシェル", "・", "クワン", "が", "優勝", "し", "まし", "た", "。",
-    //				"スペース", "ステーション", "に", "行き", "ます", "。",
-    //				"うたがわしい", "。"
-    //		};
-    String input = "スペースステーションに行きます。うたがわしい。";
-    String[] surfaceForms = {
-        "スペース", "ステーション", "に", "行き", "ます", "。",
-        "うたがわしい", "。"
-    };
-    List<Token> tokens = segmenter.tokenize(input);
-    assertTrue(tokens.size() == surfaceForms.length);
-    for (int i = 0; i < tokens.size(); i++) {
-      assertEquals(surfaceForms[i], tokens.get(i).getSurfaceFormString());
-    }
-  }
-  
-  @Test
-  public void testReadings() {
-    List<Token> tokens = segmenter.tokenize("寿司が食べたいです。");
-    assertEquals(6, tokens.size());
-    assertEquals("スシ", tokens.get(0).getReading());
-    assertEquals("ガ",    tokens.get(1).getReading());
-    assertEquals("タベ", tokens.get(2).getReading());
-    assertEquals("タイ",  tokens.get(3).getReading());
-    assertEquals("デス", tokens.get(4).getReading());
-    assertEquals("。", tokens.get(5).getReading());
-  }
-  
-  @Test
-  public void testReadings2() {
-    List<Token> tokens = segmenter.tokenize("多くの学生が試験に落ちた。");
-    assertEquals(9, tokens.size());
-    assertEquals("オオク", tokens.get(0).getReading());
-    assertEquals("ノ", tokens.get(1).getReading());
-    assertEquals("ガクセイ", tokens.get(2).getReading());
-    assertEquals("ガ", tokens.get(3).getReading());
-    assertEquals("シケン", tokens.get(4).getReading());
-    assertEquals("ニ", tokens.get(5).getReading());
-    assertEquals("オチ", tokens.get(6).getReading());
-    assertEquals("タ", tokens.get(7).getReading());
-    assertEquals("。", tokens.get(8).getReading());
-  }
-  
-  @Test
-  public void testPronunciations() {
-    List<Token> tokens = segmenter.tokenize("寿司が食べたいです。");
-    assertEquals(6, tokens.size());
-    assertEquals("スシ", tokens.get(0).getPronunciation());
-    assertEquals("ガ",    tokens.get(1).getPronunciation());
-    assertEquals("タベ", tokens.get(2).getPronunciation());
-    assertEquals("タイ",  tokens.get(3).getPronunciation());
-    assertEquals("デス", tokens.get(4).getPronunciation());
-    assertEquals("。", tokens.get(5).getPronunciation());
-  }
-  
-  @Test
-  public void testPronunciations2() {
-    List<Token> tokens = segmenter.tokenize("多くの学生が試験に落ちた。");
-    assertEquals(9, tokens.size());
-    // pronunciation differs from reading here
-    assertEquals("オーク", tokens.get(0).getPronunciation());
-    assertEquals("ノ", tokens.get(1).getPronunciation());
-    assertEquals("ガクセイ", tokens.get(2).getPronunciation());
-    assertEquals("ガ", tokens.get(3).getPronunciation());
-    assertEquals("シケン", tokens.get(4).getPronunciation());
-    assertEquals("ニ", tokens.get(5).getPronunciation());
-    assertEquals("オチ", tokens.get(6).getPronunciation());
-    assertEquals("タ", tokens.get(7).getPronunciation());
-    assertEquals("。", tokens.get(8).getPronunciation());
-  }
-  
-  @Test
-  public void testBasicForms() {
-    List<Token> tokens = segmenter.tokenize("それはまだ実験段階にあります。");
-    assertEquals(9, tokens.size());
-    assertNull(tokens.get(0).getBaseForm());
-    assertNull(tokens.get(1).getBaseForm());
-    assertNull(tokens.get(2).getBaseForm());
-    assertNull(tokens.get(3).getBaseForm());
-    assertNull(tokens.get(4).getBaseForm());
-    assertNull(tokens.get(5).getBaseForm());
-    assertEquals(tokens.get(6).getBaseForm(), "ある");
-    assertNull(tokens.get(7).getBaseForm());
-    assertNull(tokens.get(8).getBaseForm());
-  }
-  
-  @Test
-  public void testInflectionTypes() {
-    List<Token> tokens = segmenter.tokenize("それはまだ実験段階にあります。");
-    assertEquals(9, tokens.size());
-    assertNull(tokens.get(0).getInflectionType());
-    assertNull(tokens.get(1).getInflectionType());
-    assertNull(tokens.get(2).getInflectionType());
-    assertNull(tokens.get(3).getInflectionType());
-    assertNull(tokens.get(4).getInflectionType());
-    assertNull(tokens.get(5).getInflectionType());
-    assertEquals("五段・ラ行", tokens.get(6).getInflectionType());
-    assertEquals("特殊・マス", tokens.get(7).getInflectionType());
-    assertNull(tokens.get(8).getInflectionType());
-  }
-  
-  @Test
-  public void testInflectionForms() {
-    List<Token> tokens = segmenter.tokenize("それはまだ実験段階にあります。");
-    assertEquals(9, tokens.size());
-    assertNull(tokens.get(0).getInflectionForm());
-    assertNull(tokens.get(1).getInflectionForm());
-    assertNull(tokens.get(2).getInflectionForm());
-    assertNull(tokens.get(3).getInflectionForm());
-    assertNull(tokens.get(4).getInflectionForm());
-    assertNull(tokens.get(5).getInflectionForm());
-    assertEquals("連用形", tokens.get(6).getInflectionForm());
-    assertEquals("基本形", tokens.get(7).getInflectionForm());
-    assertNull(tokens.get(8).getInflectionForm());
-  }
-  
-  @Test
-  public void testPartOfSpeech() {
-    List<Token> tokens = segmenter.tokenize("それはまだ実験段階にあります。");
-    assertEquals(9, tokens.size());
-    assertEquals("名詞-代名詞-一般",  tokens.get(0).getPartOfSpeech());
-    assertEquals("助詞-係助詞",    tokens.get(1).getPartOfSpeech());
-    assertEquals("副詞-助詞類接続", tokens.get(2).getPartOfSpeech());
-    assertEquals("名詞-サ変接続",   tokens.get(3).getPartOfSpeech());
-    assertEquals("名詞-一般",      tokens.get(4).getPartOfSpeech());
-    assertEquals("助詞-格助詞-一般",  tokens.get(5).getPartOfSpeech());
-    assertEquals("動詞-自立",      tokens.get(6).getPartOfSpeech());
-    assertEquals("助動詞",       tokens.get(7).getPartOfSpeech());
-    assertEquals("記号-句点",      tokens.get(8).getPartOfSpeech());
-  }
-
-  // TODO: the next 2 tests are no longer using the first/last word ids, maybe lookup the words and fix?
-  // do we have a possibility to actually lookup the first and last word from dictionary?
-  public void testYabottai() {
-    List<Token> tokens = segmenter.tokenize("やぼったい");
-    assertEquals(1, tokens.size());
-    assertEquals("やぼったい", tokens.get(0).getSurfaceFormString());
-  }
-
-  public void testTsukitosha() {
-    List<Token> tokens = segmenter.tokenize("突き通しゃ");
-    assertEquals(1, tokens.size());
-    assertEquals("突き通しゃ", tokens.get(0).getSurfaceFormString());
-  }
-
-  public void testBocchan() throws Exception {
-    doTestBocchan(1);
-  }
-  
-  @Test @Nightly
-  public void testBocchanBig() throws Exception {
-    doTestBocchan(100);
-  }
-  
-  private void doTestBocchan(int numIterations) throws Exception {
-    LineNumberReader reader = new LineNumberReader(new InputStreamReader(
-        this.getClass().getResourceAsStream("bocchan.utf-8")));
-    
-    String line = reader.readLine();
-    reader.close();
-    
-    if (VERBOSE) {
-      System.out.println("Test for Bocchan without pre-splitting sentences");
-    }
-    long totalStart = System.currentTimeMillis();
-    for (int i = 0; i < numIterations; i++){
-      segmenter.tokenize(line);
-    }
-    if (VERBOSE) {
-      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
-      System.out.println("Test for Bocchan with pre-splitting sentences");
-    }
-    String[] sentences = line.split("、|。");
-    totalStart = System.currentTimeMillis();
-    for (int i = 0; i < numIterations; i++) {
-      for (String sentence: sentences) {
-        segmenter.tokenize(sentence);       
-      }
-    }
-    if (VERBOSE) {
-      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
-    }
-  }
-}
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
index daea1fb..e66556e 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
@@ -25,19 +25,17 @@
 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.KuromojiTokenizer.Mode;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 
 public class TestExtendedMode extends BaseTokenStreamTestCase {
-  private final Segmenter segmenter = new Segmenter(Mode.EXTENDED);
   private final Analyzer analyzer = new Analyzer() {
     
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      //Tokenizer tokenizer = new KuromojiTokenizer(segmenter, reader);
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Mode.EXTENDED);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, Mode.EXTENDED);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
index 568bd79..897a744 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
@@ -23,7 +23,7 @@
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 
 public class TestKuromojiAnalyzer extends BaseTokenStreamTestCase {
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
index 79167c9..ca0d454 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
@@ -28,8 +28,7 @@
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      //Tokenizer tokenizer = new KuromojiTokenizer(reader);
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Segmenter.DEFAULT_MODE);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, KuromojiTokenizer.DEFAULT_MODE);
       return new TokenStreamComponents(tokenizer, new KuromojiBaseFormFilter(tokenizer));
     }
   };
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
index 49f8cbd..dc642e9 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
@@ -21,6 +21,7 @@
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.LineNumberReader;
+import java.io.PrintWriter;
 import java.io.Reader;
 import java.io.StringReader;
 
@@ -28,7 +29,8 @@
 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.KuromojiTokenizer.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
 import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.analysis.kuromoji.tokenattributes.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -58,7 +60,15 @@
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, readDict(), false, Mode.SEARCH);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), false, Mode.SEARCH);
+      return new TokenStreamComponents(tokenizer, tokenizer);
+    }
+  };
+
+  private Analyzer analyzerNormal = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), false, Mode.NORMAL);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
@@ -66,7 +76,7 @@
   private Analyzer analyzerNoPunct = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, readDict(), true, Mode.SEARCH);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), true, Mode.SEARCH);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
@@ -74,7 +84,7 @@
   private Analyzer analyzerWithCompounds = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, readDict(), false, Mode.SEARCH_WITH_COMPOUNDS);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), false, Mode.SEARCH_WITH_COMPOUNDS);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
@@ -82,11 +92,17 @@
   private Analyzer extendedModeAnalyzerNoPunct = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, readDict(), true, Mode.EXTENDED);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), true, Mode.EXTENDED);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
-  
+
+  public void testNormalMode() throws Exception {
+    assertAnalyzesTo(analyzerNormal,
+                     "シニアソフトウェアエンジニア",
+                     new String[] {"シニアソフトウェアエンジニア"});
+  }
+
   public void testDecomposition1() throws Exception {
     assertAnalyzesTo(analyzerNoPunct, "本来は、貧困層の女性や子供に医療保護を提供するために創設された制度である、" +
                          "アメリカ低所得者医療援助制度が、今日では、その予算の約3分の1を老人に費やしている。",
@@ -312,6 +328,29 @@
                      surfaceForms);
   }
 
+  public void testLatticeToDot() throws Exception {
+    final GraphvizFormatter gv2 = new GraphvizFormatter(ConnectionCosts.getInstance());
+    final Analyzer analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        KuromojiTokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), false, Mode.SEARCH);
+        tokenizer.setGraphvizFormatter(gv2);
+        return new TokenStreamComponents(tokenizer, tokenizer);
+      }
+    };
+
+    String input = "スペースステーションに行きます。うたがわしい。";
+    String[] surfaceForms = {
+        "スペース", "ステーション", "に", "行き", "ます", "。",
+        "うたがわしい", "。"
+    };
+    assertAnalyzesTo(analyzer,
+                     input,
+                     surfaceForms);
+    
+    assertTrue(gv2.finish().indexOf("22.0") != -1);
+  }
+
   private void assertReadings(String input, String... readings) throws IOException {
     TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
     ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
index 95edbb1..4972f3d 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
@@ -30,7 +30,7 @@
 import java.util.zip.ZipFile;
 
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
@@ -80,14 +80,15 @@
     }
     //System.out.println("maxLen=" + maxLen);
 
-    final Tokenizer tokenizer = new KuromojiTokenizer2(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
-    //final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""));
+    final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
     tokenizer.reset();
     final String all = sb.toString();
+    System.out.println("all.len=" + all.length());
     final int ITERS = 20;
     CharTermAttribute termAtt = tokenizer.addAttribute(CharTermAttribute.class); 
     for(int iter=0;iter<ITERS;iter++) {
       tokenizer.reset(new StringReader(all));
+      tokenizer.reset();
       count = 0;
       long t0 = System.currentTimeMillis();
       while(tokenizer.incrementToken()) {
@@ -122,9 +123,7 @@
      word agreement?: 0.999587584716181
      */
     
-    //final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""));
-    //final Tokenizer tokenizer = new KuromojiTokenizer(new Segmenter(Mode.NORMAL), new StringReader(""));
-    final Tokenizer tokenizer = new KuromojiTokenizer2(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
+    final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
     String line1 = null;
     String line2 = null;
     int count = 0;
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
index e46c287..d06956f 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
@@ -28,7 +28,7 @@
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 import org.apache.lucene.util.IOUtils;
 
 public class TestSearchMode extends BaseTokenStreamTestCase {
@@ -37,7 +37,7 @@
   private final 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, null, true, Mode.SEARCH);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
@@ -45,7 +45,7 @@
   private final Analyzer analyzerWithCompounds = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer2(reader, null, true, Mode.SEARCH_WITH_COMPOUNDS);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, Mode.SEARCH_WITH_COMPOUNDS);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
diff --git a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
index 4e94284..8992342 100644
--- a/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
+++ b/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
@@ -23,29 +23,17 @@
 import java.io.Reader;
 import java.io.IOException;
 
-import org.apache.lucene.analysis.kuromoji.SegmenterTest;
 import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.apache.lucene.analysis.kuromoji.TestKuromojiTokenizer;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
 public class UserDictionaryTest extends LuceneTestCase {
 
-  private UserDictionary readDict() throws IOException {
-    InputStream is = SegmenterTest.class.getResourceAsStream("userdict.txt");
-    if (is == null)
-      throw new FileNotFoundException("Cannot find userdict.txt in test classpath!");
-    try {
-      Reader reader = new InputStreamReader(is, IOUtils.CHARSET_UTF_8);
-      return new UserDictionary(reader);
-    } finally {
-      is.close();
-    }
-  }
-  
   @Test
   public void testLookup() throws IOException {
-    UserDictionary dictionary = readDict();
+    UserDictionary dictionary = TestKuromojiTokenizer.readDict();
     String s = "関西国際空港に行った";
     int[][] dictionaryEntryResult = dictionary.lookup(s.toCharArray(), 0, s.length());
     // Length should be three 関西, 国際, 空港
@@ -69,7 +57,7 @@
   
   @Test
   public void testReadings() throws IOException {
-    UserDictionary dictionary = readDict();
+    UserDictionary dictionary = TestKuromojiTokenizer.readDict();
     int[][] result = dictionary.lookup("日本経済新聞".toCharArray(), 0, 6);
     assertEquals(3, result.length);
     int wordIdNihon = result[0][0]; // wordId of 日本 in 日本経済新聞
@@ -83,7 +71,7 @@
   
   @Test
   public void testPartOfSpeech() throws IOException {
-    UserDictionary dictionary = readDict();
+    UserDictionary dictionary = TestKuromojiTokenizer.readDict();
     int[][] result = dictionary.lookup("日本経済新聞".toCharArray(), 0, 6);
     assertEquals(3, result.length);
     int wordIdKeizai = result[1][0]; // wordId of 経済 in 日本経済新聞
@@ -92,7 +80,7 @@
   
   @Test
   public void testRead() throws IOException {
-    UserDictionary dictionary = readDict();
+    UserDictionary dictionary = TestKuromojiTokenizer.readDict();
     assertNotNull(dictionary);		
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java b/solr/core/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java
index ef1face..6ec97f7 100644
--- a/solr/core/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java
+++ b/solr/core/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java
@@ -28,8 +28,7 @@
 
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer;
-import org.apache.lucene.analysis.kuromoji.Segmenter;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.analysis.BaseTokenizerFactory;
@@ -88,7 +87,7 @@
   
   @Override
   public Tokenizer create(Reader input) {
-    return new KuromojiTokenizer(new Segmenter(userDictionary, mode), input);
+    return new KuromojiTokenizer(input, userDictionary, true, mode);
   }
   
   private Mode getMode(Map<String, String> args) {
@@ -96,7 +95,7 @@
     if (mode != null) {
       return Mode.valueOf(mode.toUpperCase(Locale.ENGLISH));
     } else {
-      return Segmenter.DEFAULT_MODE;
+      return KuromojiTokenizer.DEFAULT_MODE;
     }
   }
 }