You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ry...@apache.org on 2012/03/06 19:13:50 UTC

svn commit: r1297628 [7/13] - in /lucene/dev/branches/solr_3159_jetty8: ./ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/contrib/demo/ dev-tools/maven/lucene/contrib/highlighter/ dev-tools/maven/lucene/contrib/memory/ dev-tools/maven/...

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java Tue Mar  6 18:13:38 2012
@@ -17,67 +17,1133 @@ package org.apache.lucene.analysis.kurom
  * 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;
+
+// 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, 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;
+
+  // For safety:
+  private static final int MAX_UNKNOWN_WORD_LENGTH = 1024;
+  private static final int MAX_BACKTRACE_GAP = 1024;
+
+  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>();
 
-public final class KuromojiTokenizer extends SegmentingTokenizerBase {
-  private static final BreakIterator proto = BreakIterator.getSentenceInstance(Locale.JAPAN);
   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(Segmenter segmenter, Reader input) {
-    super(input, (BreakIterator) proto.clone());
-    this.segmenter = segmenter;
+
+  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 = 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);
+  }
+
+  private GraphvizFormatter dotOut;
+
+  /** Expert: set this to produce graphviz (dot) output of
+   *  the Viterbi lattice */
+  public void setGraphvizFormatter(GraphvizFormatter dotOut) {
+    this.dotOut = dotOut;
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+    buffer.reset(input);
   }
-  
+
   @Override
-  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
-    this.sentenceStart = 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;
+  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
-  protected boolean incrementWord() {
-    if (tokenIndex == tokens.size()) {
-      return false;
+  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;
+        if (VERBOSE) {
+          System.out.println("        **");
+        }
+      }
+    }
+
+    leastCost += wordCost;
+
+    if (VERBOSE) {
+      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " leftID=" + leftID + " leastIDX=" + leastIDX + " toPos=" + endPos + " 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 >= MAX_BACKTRACE_GAP) {
+        // 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;unknownWordLength<MAX_UNKNOWN_WORD_LENGTH;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("BT: back pos=" + pos + " bestIDX=" + bestIDX);
+      final Position posData = positions.get(pos);
+      assert bestIDX < posData.count;
+
+      int backPos = posData.backPos[bestIDX];
+      assert backPos >= lastBackTracePos: "backPos=" + backPos + " vs lastBackTracePos=" + lastBackTracePos;
+      int length = pos - backPos;
+      Type backType = posData.backType[bestIDX];
+      int backID = posData.backID[bestIDX];
+      int nextBestIDX = posData.backIndex[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).
+        //System.out.println("    2nd best backPos=" + backPos + " pos=" + pos);
+
+        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;
+            nextBestIDX = posData.backIndex[bestIDX];
+
+            backPos = posData.backPos[bestIDX];
+            length = pos - backPos;
+            backType = posData.backType[bestIDX];
+            backID = posData.backID[bestIDX];
+            backCount = 0;
+            //System.out.println("  do alt token!");
+            
+          } 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:
+            //System.out.println("  no alt token! bestIDX=" + bestIDX);
+          }
+        }
+      }
+
+      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 = nextBestIDX;
+    }
+
+    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;
+    }
+  }
 }

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java Tue Mar  6 18:13:38 2012
@@ -17,8 +17,8 @@ package org.apache.lucene.analysis.kurom
  * 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;
@@ -30,6 +30,7 @@ public class Token {
   private final int length;
   
   private final int position;
+  private int positionLength;
   
   private final Type type;
   
@@ -40,8 +41,14 @@ public class Token {
     this.length = length;
     this.type = type;
     this.position = position;
+    this.positionLength = positionLength;
     this.dictionary = dictionary;
   }
+
+  @Override
+  public String toString() {
+    return "Token(\"" + new String(surfaceForm, offset, length) + "\" pos=" + position + " type=" + type + " wordId=" + wordId + " leftID=" + dictionary.getLeftId(wordId) + ")";
+  }
   
   /**
    * @return surfaceForm
@@ -144,4 +151,21 @@ public class Token {
   public int getPosition() {
     return position;
   }
+
+  /**
+   * Set the position length (in tokens) of this token.  For normal
+   * tokens this is 1; for compound tokens it's > 1.
+   */
+  public void setPositionLength(int positionLength) {
+    this.positionLength = positionLength;
+  }
+  
+  /**
+   * Get the length (in tokens) of this token.  For normal
+   * tokens this is 1; for compound tokens it's > 1.
+   * @return position length of token
+   */
+  public int getPositionLength() {
+    return positionLength;
+  }
 }

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

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java Tue Mar  6 18:13:38 2012
@@ -25,18 +25,17 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.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 KuromojiTokenizer(reader, null, true, Mode.EXTENDED);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java Tue Mar  6 18:13:38 2012
@@ -18,8 +18,11 @@ package org.apache.lucene.analysis.kurom
  */
 
 import java.io.IOException;
+import java.io.StringReader;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 
 public class TestKuromojiAnalyzer extends BaseTokenStreamTestCase {
   /** This test fails with NPE when the 
@@ -41,20 +44,103 @@ public class TestKuromojiAnalyzer extend
         new int[] { 1, 2, 2,  2 }
       );
   }
-  
+
   /**
    * Test that search mode is enabled and working by default
    */
   public void testDecomposition() throws IOException {
-    assertAnalyzesTo(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "シニアソフトウェアエンジニア",
-        new String[] { "シニア", "ソフトウェア", "エンジニア" }
-    );
+
+    final Analyzer a = new KuromojiAnalyzer(TEST_VERSION_CURRENT, null, Mode.SEARCH,
+                                            KuromojiAnalyzer.getDefaultStopSet(),
+                                            KuromojiAnalyzer.getDefaultStopTags());
+
+    /*
+    //TokenStream ts = a.tokenStream("foo", new StringReader("妹の咲子です。俺と年子で、今受験生です。"));
+    TokenStream ts = a.tokenStream("foo", new StringReader("&#x250cdf66<!--\"<!--#<!--;?><!--#<!--#><!---->?>-->;"));
+    ts.reset();
+    CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+    while(ts.incrementToken()) {
+      System.out.println("  " + termAtt.toString());
+    }
+    System.out.println("DONE PARSE\n\n");
+    */
+
+    // Senior software engineer:
+    assertAnalyzesToPositions(a, "シニアソフトウェアエンジニア",
+                              new String[] { "シニア",
+                                             "シニアソフトウェアエンジニア",
+                                             "ソフトウェア",
+                                             "エンジニア" },
+                              new int[] { 1, 0, 1, 1},
+                              new int[] { 1, 3, 1, 1}
+                              );
+
+    // Kansai International Airport:
+    assertAnalyzesToPositions(a, "関西国際空港",
+                              new String[] { "関西",
+                                             "関西国際空港", // zero pos inc
+                                             "国際",
+                                             "空港" },
+                              new int[] {1, 0, 1, 1},
+                              new int[] {1, 3, 1, 1}
+                              );
+
+    // Konika Minolta Holdings; not quite the right
+    // segmentation (see LUCENE-3726):
+    assertAnalyzesToPositions(a, "コニカミノルタホールディングス",
+                              new String[] { "コニカ",
+                                             "コニカミノルタホールディングス", // zero pos inc
+                                             "ミノルタ", 
+                                             "ホールディングス"},
+                              new int[] {1, 0, 1, 1},
+                              new int[] {1, 3, 1, 1}
+                              );
+
+    // Narita Airport
+    assertAnalyzesToPositions(a, "成田空港",
+                              new String[] { "成田",
+                                             "成田空港",
+                                             "空港" },
+                              new int[] {1, 0, 1},
+                              new int[] {1, 2, 1}
+                              );
+
+    // Kyoto University Baseball Club
+    assertAnalyzesToPositions(new KuromojiAnalyzer(TEST_VERSION_CURRENT), "京都大学硬式野球部",
+                     new String[] { "京都大",
+                                    "å­¦",
+                                    "硬式",
+                                    "野球",
+                                    "部" },
+                              new int[] {1, 1, 1, 1, 1},
+                              new int[] {1, 1, 1, 1, 1});
+    // toDotFile(a, "成田空港", "/mnt/scratch/out.dot");
   }
+
   
   /**
    * blast random strings against the analyzer
    */
   public void testRandom() throws IOException {
-    checkRandomData(random, new KuromojiAnalyzer(TEST_VERSION_CURRENT), atLeast(10000));
+    final Analyzer a = new KuromojiAnalyzer(TEST_VERSION_CURRENT, null, Mode.SEARCH,
+                                            KuromojiAnalyzer.getDefaultStopSet(),
+                                            KuromojiAnalyzer.getDefaultStopTags());
+    checkRandomData(random, a, atLeast(10000));
+  }
+
+  // Copied from TestKuromojiTokenizer, to make sure passing
+  // user dict to analyzer works:
+  public void testUserDict3() throws Exception {
+    // Test entry that breaks into multiple tokens:
+    final Analyzer a = new KuromojiAnalyzer(TEST_VERSION_CURRENT, TestKuromojiTokenizer.readDict(),
+                                            Mode.SEARCH,
+                                            KuromojiAnalyzer.getDefaultStopSet(),
+                                            KuromojiAnalyzer.getDefaultStopTags());
+    assertTokenStreamContents(a.tokenStream("foo", new StringReader("abcd")),
+                              new String[] { "a", "b", "cd"  },
+                              new int[] { 0, 1, 2 },
+                              new int[] { 1, 2, 4 },
+                              new Integer(4)
+    );
   }
 }

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java Tue Mar  6 18:13:38 2012
@@ -28,7 +28,7 @@ public class TestKuromojiBaseFormFilter 
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, KuromojiTokenizer.DEFAULT_MODE);
       return new TokenStreamComponents(tokenizer, new KuromojiBaseFormFilter(tokenizer));
     }
   };

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java Tue Mar  6 18:13:38 2012
@@ -17,7 +17,13 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
+import java.io.BufferedReader;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+import java.io.PrintWriter;
 import java.io.Reader;
 import java.io.StringReader;
 
@@ -25,21 +31,76 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.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;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 
 public class TestKuromojiTokenizer extends BaseTokenStreamTestCase {
+
+  public static UserDictionary readDict() {
+    InputStream is = TestKuromojiTokenizer.class.getResourceAsStream("userdict.txt");
+    if (is == null) {
+      throw new RuntimeException("Cannot find userdict.txt in test classpath!");
+    }
+    try {
+      try {
+        Reader reader = new InputStreamReader(is, IOUtils.CHARSET_UTF_8);
+        return new UserDictionary(reader);
+      } finally {
+        is.close();
+      }
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe);
+    }
+  }
+
   private Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(reader);
+      Tokenizer tokenizer = new KuromojiTokenizer(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);
+    }
+  };
+
+  private Analyzer analyzerNoPunct = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, readDict(), true, Mode.SEARCH);
+      return new TokenStreamComponents(tokenizer, tokenizer);
+    }
+  };
+
+  private Analyzer extendedModeAnalyzerNoPunct = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      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(analyzer, "本来は、貧困層の女性や子供に医療保護を提供するために創設された制度である、" +
+    assertAnalyzesTo(analyzerNoPunct, "本来は、貧困層の女性や子供に医療保護を提供するために創設された制度である、" +
                          "アメリカ低所得者医療援助制度が、今日では、その予算の約3分の1を老人に費やしている。",
      new String[] { "本来", "は",  "貧困", "層", "の", "女性", "や", "子供", "に", "医療", "保護", "を",      
                     "提供", "する", "ため", "に", "創設", "さ", "れ", "た", "制度", "で", "ある",  "アメリカ", 
@@ -55,7 +116,7 @@ public class TestKuromojiTokenizer exten
   }
   
   public void testDecomposition2() throws Exception {
-    assertAnalyzesTo(analyzer, "麻薬の密売は根こそぎ絶やさなければならない",
+    assertAnalyzesTo(analyzerNoPunct, "麻薬の密売は根こそぎ絶やさなければならない",
       new String[] { "麻薬", "の", "密売", "は", "根こそぎ", "絶やさ", "なけれ", "ば", "なら", "ない" },
       new int[] { 0, 2, 3, 5, 6,  10, 13, 16, 17, 19 },
       new int[] { 2, 3, 5, 6, 10, 13, 16, 17, 19, 21 }
@@ -63,7 +124,7 @@ public class TestKuromojiTokenizer exten
   }
   
   public void testDecomposition3() throws Exception {
-    assertAnalyzesTo(analyzer, "魔女狩大将マシュー・ホプキンス。",
+    assertAnalyzesTo(analyzerNoPunct, "魔女狩大将マシュー・ホプキンス。",
       new String[] { "魔女", "狩", "大将", "マシュー",  "ホプキンス" },
       new int[] { 0, 2, 3, 5, 10 },
       new int[] { 2, 3, 5, 9, 15 }
@@ -91,9 +152,32 @@ public class TestKuromojiTokenizer exten
     ts.close();
   }
 
+  /*
+    // NOTE: intentionally fails!  Just trying to debug this
+    // one input...
+  public void testDecomposition6() throws Exception {
+    assertAnalyzesTo(analyzer, "奈良先端科学技術大学院大学",
+      new String[] { "これ", "は", "本", "で", "は", "ない" },
+      new int[] { 0, 2, 3, 4, 5, 6 },
+      new int[] { 2, 3, 4, 5, 6, 8 }
+                     );
+  }
+  */
+
   /** Tests that sentence offset is incorporated into the resulting offsets */
   public void testTwoSentences() throws Exception {
-    assertAnalyzesTo(analyzer, "魔女狩大将マシュー・ホプキンス。 魔女狩大将マシュー・ホプキンス。",
+    /*
+    //TokenStream ts = a.tokenStream("foo", new StringReader("妹の咲子です。俺と年子で、今受験生です。"));
+    TokenStream ts = analyzer.tokenStream("foo", new StringReader("&#x250cdf66<!--\"<!--#<!--;?><!--#<!--#><!---->?>-->;"));
+    ts.reset();
+    CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+    while(ts.incrementToken()) {
+      System.out.println("  " + termAtt.toString());
+    }
+    System.out.println("DONE PARSE\n\n");
+    */
+
+    assertAnalyzesTo(analyzerNoPunct, "魔女狩大将マシュー・ホプキンス。 魔女狩大将マシュー・ホプキンス。",
       new String[] { "魔女", "狩", "大将", "マシュー", "ホプキンス",  "魔女", "狩", "大将", "マシュー",  "ホプキンス"  },
       new int[] { 0, 2, 3, 5, 10, 17, 19, 20, 22, 27 },
       new int[] { 2, 3, 5, 9, 15, 19, 20, 22, 26, 32 }
@@ -103,6 +187,7 @@ public class TestKuromojiTokenizer exten
   /** blast some random strings through the analyzer */
   public void testRandomStrings() throws Exception {
     checkRandomData(random, analyzer, 10000*RANDOM_MULTIPLIER);
+    checkRandomData(random, analyzerNoPunct, 10000*RANDOM_MULTIPLIER);
   }
   
   public void testLargeDocReliability() throws Exception {
@@ -125,6 +210,9 @@ public class TestKuromojiTokenizer exten
   public void testSurrogates2() throws IOException {
     int numIterations = atLeast(10000);
     for (int i = 0; i < numIterations; i++) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + i);
+      }
       String s = _TestUtil.randomUnicodeString(random, 100);
       TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
       CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
@@ -134,22 +222,410 @@ public class TestKuromojiTokenizer exten
       }
     }
   }
+
+  public void testOnlyPunctuation() throws IOException {
+    TokenStream ts = analyzerNoPunct.tokenStream("foo", new StringReader("。、。。"));
+    ts.reset();
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+
+  public void testOnlyPunctuationExtended() throws IOException {
+    TokenStream ts = extendedModeAnalyzerNoPunct.tokenStream("foo", new StringReader("......"));
+    ts.reset();
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
   
   // note: test is kinda silly since kuromoji emits punctuation tokens.
   // but, when/if we filter these out it will be useful.
   public void testEnd() throws Exception {
-    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("これは本ではない")),
+    assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない")),
         new String[] { "これ", "は", "本", "で", "は", "ない" },
         new int[] { 0, 2, 3, 4, 5, 6 },
         new int[] { 2, 3, 4, 5, 6, 8 },
         new Integer(8)
     );
-    
-    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("これは本ではない    ")),
+
+    assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない    ")),
         new String[] { "これ", "は", "本", "で", "は", "ない"  },
         new int[] { 0, 2, 3, 4, 5, 6, 8 },
         new int[] { 2, 3, 4, 5, 6, 8, 9 },
         new Integer(12)
     );
   }
+
+  public void testUserDict() throws Exception {
+    // Not a great test because w/o userdict.txt the
+    // segmentation is the same:
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("関西国際空港に行った")),
+                              new String[] { "関西", "国際", "空港", "に", "行っ", "た"  },
+                              new int[] { 0, 2, 4, 6, 7, 9 },
+                              new int[] { 2, 4, 6, 7, 9, 10 },
+                              new Integer(10)
+    );
+  }
+
+  public void testUserDict2() throws Exception {
+    // Better test: w/o userdict the segmentation is different:
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("朝青龍")),
+                              new String[] { "朝青龍"  },
+                              new int[] { 0 },
+                              new int[] { 3 },
+                              new Integer(3)
+    );
+  }
+
+  public void testUserDict3() throws Exception {
+    // Test entry that breaks into multiple tokens:
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("abcd")),
+                              new String[] { "a", "b", "cd"  },
+                              new int[] { 0, 1, 2 },
+                              new int[] { 1, 2, 4 },
+                              new Integer(4)
+    );
+  }
+
+  // HMM: fails (segments as a/b/cd/efghij)... because the
+  // two paths have exactly equal paths (1 KNOWN + 1
+  // UNKNOWN) and we don't seem to favor longer KNOWN /
+  // shorter UNKNOWN matches:
+
+  /*
+  public void testUserDict4() throws Exception {
+    // Test entry that has another entry as prefix
+    assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("abcdefghij")),
+                              new String[] { "ab", "cd", "efg", "hij"  },
+                              new int[] { 0, 2, 4, 7 },
+                              new int[] { 2, 4, 7, 10 },
+                              new Integer(10)
+    );
+  }
+  */
+  
+  public void testSegmentation() throws Exception {
+    // Skip tests for Michelle Kwan -- UniDic segments Kwan as ク ワン
+    //		String input = "ミシェル・クワンが優勝しました。スペースステーションに行きます。うたがわしい。";
+    //		String[] surfaceForms = {
+    //				"ミシェル", "・", "クワン", "が", "優勝", "し", "まし", "た", "。",
+    //				"スペース", "ステーション", "に", "行き", "ます", "。",
+    //				"うたがわしい", "。"
+    //		};
+    String input = "スペースステーションに行きます。うたがわしい。";
+    String[] surfaceForms = {
+        "スペース", "ステーション", "に", "行き", "ます", "。",
+        "うたがわしい", "。"
+    };
+    assertAnalyzesTo(analyzer,
+                     input,
+                     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);
+    ts.reset();
+    for(String reading : readings) {
+      assertTrue(ts.incrementToken());
+      assertEquals(reading, readingAtt.getReading());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+
+  private void assertPronunciations(String input, String... pronunciations) throws IOException {
+    TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
+    ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
+    ts.reset();
+    for(String pronunciation : pronunciations) {
+      assertTrue(ts.incrementToken());
+      assertEquals(pronunciation, readingAtt.getPronunciation());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+  
+  private void assertBaseForms(String input, String... baseForms) throws IOException {
+    TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
+    BaseFormAttribute baseFormAtt = ts.addAttribute(BaseFormAttribute.class);
+    ts.reset();
+    for(String baseForm : baseForms) {
+      assertTrue(ts.incrementToken());
+      assertEquals(baseForm, baseFormAtt.getBaseForm());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+
+  private void assertInflectionTypes(String input, String... inflectionTypes) throws IOException {
+    TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
+    InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
+    ts.reset();
+    for(String inflectionType : inflectionTypes) {
+      assertTrue(ts.incrementToken());
+      assertEquals(inflectionType, inflectionAtt.getInflectionType());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+
+  private void assertInflectionForms(String input, String... inflectionForms) throws IOException {
+    TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
+    InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
+    ts.reset();
+    for(String inflectionForm : inflectionForms) {
+      assertTrue(ts.incrementToken());
+      assertEquals(inflectionForm, inflectionAtt.getInflectionForm());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+  
+  private void assertPartsOfSpeech(String input, String... partsOfSpeech) throws IOException {
+    TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
+    PartOfSpeechAttribute partOfSpeechAtt = ts.addAttribute(PartOfSpeechAttribute.class);
+    ts.reset();
+    for(String partOfSpeech : partsOfSpeech) {
+      assertTrue(ts.incrementToken());
+      assertEquals(partOfSpeech, partOfSpeechAtt.getPartOfSpeech());
+    }
+    assertFalse(ts.incrementToken());
+    ts.end();
+  }
+  
+  public void testReadings() throws Exception {
+    assertReadings("寿司が食べたいです。",
+                   "スシ",
+                   "ガ",
+                   "タベ",
+                   "タイ",
+                   "デス",
+                   "。");
+  }
+  
+  public void testReadings2() throws Exception {
+    assertReadings("多くの学生が試験に落ちた。",
+                   "オオク",
+                   "ノ",
+                   "ガクセイ",
+                   "ガ",
+                   "シケン",
+                   "ニ",
+                   "オチ",
+                   "タ",
+                   "。");
+  }
+  
+  public void testPronunciations() throws Exception {
+    assertPronunciations("寿司が食べたいです。",
+                         "スシ",
+                         "ガ",
+                         "タベ",
+                         "タイ",
+                         "デス",
+                         "。");
+  }
+  
+  public void testPronunciations2() throws Exception {
+    // pronunciation differs from reading here
+    assertPronunciations("多くの学生が試験に落ちた。",
+                         "オーク",
+                         "ノ",
+                         "ガクセイ",
+                         "ガ",
+                         "シケン",
+                         "ニ",
+                         "オチ",
+                         "タ",
+                         "。");
+  }
+  
+  public void testBasicForms() throws Exception {
+    assertBaseForms("それはまだ実験段階にあります。",
+                    null,
+                    null,
+                    null,
+                    null,
+                    null,
+                    null,
+                    "ある",
+                    null,
+                    null);
+  }
+  
+  public void testInflectionTypes() throws Exception {
+    assertInflectionTypes("それはまだ実験段階にあります。",
+                          null,
+                          null,
+                          null,
+                          null,
+                          null,
+                          null,
+                          "五段・ラ行",
+                          "特殊・マス",
+                          null);
+  }
+  
+  public void testInflectionForms() throws Exception {
+    assertInflectionForms("それはまだ実験段階にあります。",
+                          null,
+                          null,
+                          null,
+                          null,
+                          null,
+                          null,
+                          "連用形",
+                          "基本形",
+                          null);
+  }
+  
+  public void testPartOfSpeech() throws Exception {
+    assertPartsOfSpeech("それはまだ実験段階にあります。",
+                        "名詞-代名詞-一般",
+                        "助詞-係助詞",
+                        "副詞-助詞類接続",
+                        "名詞-サ変接続",
+                        "名詞-一般",
+                        "助詞-格助詞-一般",
+                        "動詞-自立",
+                        "助動詞",
+                        "記号-句点");
+  }
+
+  // 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() throws Exception {
+    assertAnalyzesTo(analyzer, "やぼったい",
+                     new String[] {"やぼったい"});
+  }
+
+  public void testTsukitosha() throws Exception {
+    assertAnalyzesTo(analyzer, "突き通しゃ",
+                     new String[] {"突き通しゃ"});
+  }
+
+  public void testBocchan() throws Exception {
+    doTestBocchan(1);
+  }
+
+  @Nightly
+  public void testBocchanBig() throws Exception {
+    doTestBocchan(100);
+  }
+
+  /*
+  public void testWikipedia() throws Exception {
+    final FileInputStream fis = new FileInputStream("/q/lucene/jawiki-20120220-pages-articles.xml");
+    final Reader r = new BufferedReader(new InputStreamReader(fis, "UTF-8"));
+
+    final long startTimeNS = System.nanoTime();
+    boolean done = false;
+    long compoundCount = 0;
+    long nonCompoundCount = 0;
+    long netOffset = 0;
+    while (!done) {
+      final TokenStream ts = analyzer.tokenStream("ignored", r);
+      ts.reset();
+      final PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+      final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+      int count = 0;
+      while (true) {
+        if (!ts.incrementToken()) {
+          done = true;
+          break;
+        }
+        count++;
+        if (posIncAtt.getPositionIncrement() == 0) {
+          compoundCount++;
+        } else {
+          nonCompoundCount++;
+          if (nonCompoundCount % 1000000 == 0) {
+            System.out.println(String.format("%.2f msec [pos=%d, %d, %d]",
+                                             (System.nanoTime()-startTimeNS)/1000000.0,
+                                             netOffset + offsetAtt.startOffset(),
+                                             nonCompoundCount,
+                                             compoundCount));
+          }
+        }
+        if (count == 100000000) {
+          System.out.println("  again...");
+          break;
+        }
+      }
+      ts.end();
+      netOffset += offsetAtt.endOffset();
+    }
+    System.out.println("compoundCount=" + compoundCount + " nonCompoundCount=" + nonCompoundCount);
+    r.close();
+  }
+  */
+
+  
+  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");
+    }
+
+    /*
+    if (numIterations > 1) {
+      // warmup
+      for (int i = 0; i < numIterations; i++) {
+        final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(line));
+        ts.reset();
+        while(ts.incrementToken());
+      }
+    }
+    */
+
+    long totalStart = System.currentTimeMillis();
+    for (int i = 0; i < numIterations; i++) {
+      final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(line));
+      ts.reset();
+      while(ts.incrementToken());
+    }
+    String[] sentences = line.split("、|。");
+    if (VERBOSE) {
+      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+      System.out.println("Test for Bocchan with pre-splitting sentences (" + sentences.length + " sentences)");
+    }
+    totalStart = System.currentTimeMillis();
+    for (int i = 0; i < numIterations; i++) {
+      for (String sentence: sentences) {
+        final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(sentence));
+        ts.reset();
+        while(ts.incrementToken());
+      }
+    }
+    if (VERBOSE) {
+      System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+    }
+  }
 }

Modified: lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java?rev=1297628&r1=1297627&r2=1297628&view=diff
==============================================================================
--- lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java (original)
+++ lucene/dev/branches/solr_3159_jetty8/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java Tue Mar  6 18:13:38 2012
@@ -27,20 +27,19 @@ import java.io.Reader;
 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 {
   private final static String SEGMENTATION_FILENAME = "search-segmentation-tests.txt";
-  private final Segmenter segmenter = new Segmenter(Mode.SEARCH);
   private final Analyzer analyzer = new Analyzer() {
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(segmenter, reader);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, Mode.SEARCH);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };
-  
+
   /** Test search mode segmentation */
   public void testSearchSegmentation() throws IOException {
     InputStream is = TestSearchMode.class.getResourceAsStream(SEGMENTATION_FILENAME);
@@ -63,7 +62,18 @@ public class TestSearchMode extends Base
         String[] fields = line.split("\t", 2);
         String sourceText = fields[0];
         String[] expectedTokens = fields[1].split("\\s+");
-        assertAnalyzesTo(analyzer, sourceText, expectedTokens);
+        int[] expectedPosIncrs = new int[expectedTokens.length];
+        int[] expectedPosLengths = new int[expectedTokens.length];
+        for(int tokIDX=0;tokIDX<expectedTokens.length;tokIDX++) {
+          if (expectedTokens[tokIDX].endsWith("/0")) {
+            expectedTokens[tokIDX] = expectedTokens[tokIDX].replace("/0", "");
+            expectedPosLengths[tokIDX] = expectedTokens.length-1;
+          } else {
+            expectedPosIncrs[tokIDX] = 1;
+            expectedPosLengths[tokIDX] = 1;
+          }
+        }
+        assertAnalyzesTo(analyzer, sourceText, expectedTokens, expectedPosIncrs);
       }
     } finally {
       is.close();