You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/02/15 00:53:07 UTC

svn commit: r1244282 - in /lucene/dev/branches/lucene3767: lucene/test-framework/src/java/org/apache/lucene/analysis/ modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/ modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/...

Author: mikemccand
Date: Tue Feb 14 23:53:06 2012
New Revision: 1244282

URL: http://svn.apache.org/viewvc?rev=1244282&view=rev
Log:
LUCENE-3767: more nocommits

Modified:
    lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt

Modified: lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Tue Feb 14 23:53:06 2012
@@ -330,7 +330,7 @@ public abstract class BaseTokenStreamTes
       }
 
       if (VERBOSE) {
-        System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
+        System.out.println(Thread.currentThread().getName() + ": NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
       }
 
       int remainder = random.nextInt(10);
@@ -364,7 +364,7 @@ public abstract class BaseTokenStreamTes
       // verify reusing is "reproducable" and also get the normal tokenstream sanity checks
       if (!tokens.isEmpty()) {
         if (VERBOSE) {
-          System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis");
+          System.out.println(Thread.currentThread().getName() + ": NOTE: BaseTokenStreamTestCase: re-run analysis; " + tokens.size() + " tokens");
         }
         reader = new StringReader(text);
         ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java Tue Feb 14 23:53:06 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.analysis.kurom
 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;
@@ -42,23 +43,20 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.fst.FST;
 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
 
-// nocommit needs assertAnalyzersTo to accept
-// posLength... then fix compound tests to verify it
-
 // nocomit explain how the 2nd best tokenization is
 // "contextual"...
 
-// nocommit -- should we use the sentence breakiterator
-// too..?  we can simply use it to slip an EOS/BOS token
-// in...
+// 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.
@@ -107,10 +105,6 @@ public final class KuromojiTokenizer2 ex
   // True once we've hit the EOF from the input reader:
   private boolean end;
 
-  // Used to determine if all extensions from a given node
-  // originated from the same backtrace point:
-  private int sameLeastIndex;
-
   // Last absolute position we backtraced from:
   private int lastBackTracePos;
 
@@ -184,6 +178,13 @@ public final class KuromojiTokenizer2 ex
   public void reset(Reader input) throws IOException {
     super.reset(input);
     buffer.reset(input);
+    // nocommit can we leave this out
+    //resetState();
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
     resetState();
   }
 
@@ -368,12 +369,6 @@ public final class KuromojiTokenizer2 ex
     //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);
-
-    if (sameLeastIndex == -1) {
-      sameLeastIndex = leastIDX;
-    } else if (sameLeastIndex != leastIDX) {
-      sameLeastIndex = -2;
-    }
   }
 
   @Override
@@ -397,6 +392,7 @@ public final class KuromojiTokenizer2 ex
     int position = token.getPosition();
     int length = token.getLength();
     clearAttributes();
+    //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);
@@ -411,14 +407,16 @@ public final class KuromojiTokenizer2 ex
       posIncAtt.setPositionIncrement(1);
       posLengthAtt.setPositionLength(1);
     }
-    if (VERBOSE) {
-      System.out.println("    incToken: return token=" + token);
+    if (true || 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];
 
@@ -543,10 +541,17 @@ public final class KuromojiTokenizer2 ex
       final Position posData = positions.get(pos);
       final boolean isFrontier = positions.getNextPos() == pos+1;
 
-      // nocommit if I change 40 below it changes number of
-      // tokens in TestQuality!
-      //if ((pos - lastBackTracePos >= 100) && posData.count == 1 && isFrontier) {
+      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
@@ -564,26 +569,39 @@ public final class KuromojiTokenizer2 ex
         }
       }
 
-      if (VERBOSE) {
-        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos));
-      }
+      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);
 
-      sameLeastIndex = -1;
+        // Re-base cost so we don't risk int overflow:
+        Arrays.fill(posData.costs, 0, posData.count, 0);
 
-      if (posData.count == 0) {
-        // No arcs arrive here; move to next position:
-        pos++;
-        if (VERBOSE) {
-          System.out.println("    no arcs in; skip");
+        if (pending.size() != 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
         }
-        continue;
       }
 
       if (VERBOSE) {
-        System.out.println("    " + posData.count + " arcs in");
+        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos));
       }
 
-      // nocommit may need "force" a "when all else fails" backtrace?
+      if (VERBOSE) {
+        System.out.println("    " + posData.count + " arcs in");
+      }
 
       boolean anyMatches = false;
 
@@ -601,6 +619,9 @@ public final class KuromojiTokenizer2 ex
           }
           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;
           }
@@ -655,8 +676,6 @@ public final class KuromojiTokenizer2 ex
       }
 
       final char firstCharacter = (char) buffer.get(pos);
-      // nocommit -- can't we NOT pursue unk if a known
-      // token "covers" us...?
       if (!anyMatches || characterDefinition.isInvoke(firstCharacter)) {
 
         // Find unknown match:
@@ -693,26 +712,6 @@ public final class KuromojiTokenizer2 ex
         unknownWordEndIndex = posData.pos + unknownWordLength;
       }
 
-      // nocommit explainme:
-      if ((pos - lastBackTracePos >= 100) && sameLeastIndex >= 0 && isFrontier) {
-
-        final int sav = sameLeastIndex;
-
-        //System.out.println("**SAME: " + sameLeastIndex + " vs " + posData.count);
-        backtrace(posData, sameLeastIndex);
-
-        // Re-base cost so we don't risk int overflow:
-        // nocommit: this does nothing: arcs were already extended
-        posData.costs[sav] = 0;
-
-        if (pending.size() != 0) {
-          return;
-        } else {
-          // This means the backtrace only produced
-          // punctuation tokens, so we must keep parsing.
-        }
-      }
-
       pos++;
     }
 
@@ -722,14 +721,14 @@ public final class KuromojiTokenizer2 ex
 
       final Position endPosData = positions.get(pos);
       int leastCost = Integer.MAX_VALUE;
-      int leastIDX = 0;
+      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);
+        //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;
@@ -754,8 +753,6 @@ public final class KuromojiTokenizer2 ex
       System.out.println("  pruneAndRescore startPos=" + startPos + " endPos=" + endPos + " bestStartIDX=" + bestStartIDX);
     }
 
-    int minBackPos = Integer.MAX_VALUE;
-
     // First pass: walk backwards, building up the forward
     // arcs and pruning inadmissible arcs:
     for(int pos=endPos; pos >= startPos; pos--) {
@@ -768,7 +765,6 @@ public final class KuromojiTokenizer2 ex
         if (backPos >= startPos) {
           // Keep this arc:
           //System.out.println("      keep backPos=" + backPos);
-          minBackPos = Math.min(backPos, minBackPos);
           positions.get(backPos).addForward(pos,
                                             arcIDX,
                                             posData.backID[arcIDX],
@@ -784,17 +780,8 @@ public final class KuromojiTokenizer2 ex
       }
     }
 
-    // nocommit shouldn't we just "run w/ penalties" in this
-    // pass...?
-
-    // nocommit what if minBackPos not set...?
-
-    // nocommit it must be startPos...?
-    assert minBackPos == startPos;
-
     // Second pass: walk forward, re-scoring:
-    //for(int pos=startPos; pos < endPos; pos++) {
-    for(int pos=minBackPos; pos < endPos; pos++) {
+    for(int pos=startPos; pos < endPos; pos++) {
       final Position posData = positions.get(pos);
       if (VERBOSE) {
         System.out.println("    forward pos=" + pos + " count=" + posData.forwardCount);
@@ -813,12 +800,12 @@ public final class KuromojiTokenizer2 ex
         // path so we "force congruence":  the
         // sub-segmentation is "in context" of what the best
         // path (compound token) had matched:
-
-        // nocommit could be -1?:  don't i need to check
-        // startPos==0 too...?
-        final Dictionary dict = getDict(posData.backType[bestStartIDX]);
-
-        final int rightID = startPos == 0 ? 0 : dict.getRightId(posData.backID[bestStartIDX]);
+        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];
@@ -865,7 +852,7 @@ public final class KuromojiTokenizer2 ex
   // (last token should be returned first).
   private void backtrace(final Position endPosData, final int fromIDX) throws IOException {
     if (VERBOSE) {
-      System.out.println("\n  backtrace: " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
+      System.out.println("\n  backtrace: pos=" + pos + "; " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
     }
     final int endPos = endPosData.pos;
 
@@ -881,11 +868,14 @@ public final class KuromojiTokenizer2 ex
 
     int backCount = 0;
 
-    // nocommit: don't use intermediate Token instance
-    // here... change this to just hold raw back trace info,
-    // then in incrementToken we pull the necessary char[],
-    // and only call freeBefore once we're done iterating
-    // these tokens:
+    // 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);
@@ -908,7 +898,7 @@ public final class KuromojiTokenizer2 ex
         
         if (penalty > 0) {
           if (VERBOSE) {
-            System.out.println("  compound=" + new String(buffer.get(backPos, pos-backPos)) + " backPos=" + backPos + " pos=" + pos + " penalty=" + penalty);
+            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
@@ -927,7 +917,6 @@ public final class KuromojiTokenizer2 ex
           int leastCost = Integer.MAX_VALUE;
           int leastIDX = -1;
           for(int idx=0;idx<posData.count;idx++) {
-            // nocommit needs wordCost added here?
             int cost = posData.costs[idx];
             //System.out.println("    idx=" + idx + " prevCost=" + cost);
             
@@ -985,7 +974,6 @@ public final class KuromojiTokenizer2 ex
       final int offset = backPos - lastBackTracePos;
       assert offset >= 0;
 
-      // nocommit
       if (altToken != null && altToken.getPosition() >= backPos) {
 
         // We've backtraced to the position where the
@@ -1019,17 +1007,20 @@ public final class KuromojiTokenizer2 ex
         // segmentation:
         final int[] wordIDAndLength = userDictionary.lookupSegmentation(backID);
         int wordID = wordIDAndLength[0];
-        int current = backPos;
+        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,
-                                backPos + current - lastBackTracePos,
+                                current + offset,
                                 len,
                                 Type.USER,
-                                backPos + current,
+                                current + backPos,
                                 dict));
+          if (VERBOSE) {
+            System.out.println("    add USER token=" + pending.get(pending.size()-1));
+          }
           current += len;
         }
 
@@ -1043,7 +1034,6 @@ public final class KuromojiTokenizer2 ex
       } else {
 
         if (extendedMode && backType == Type.UNKNOWN) {
-          // nocommit what if the altToken is unknonwn?  
           // In EXTENDED mode we convert unknown word into
           // unigrams:
           int unigramTokenCount = 0;
@@ -1069,7 +1059,6 @@ public final class KuromojiTokenizer2 ex
           backCount += unigramTokenCount;
           
         } else if (!discardPunctuation || length == 0 || !isPunctuation(fragment[offset])) {
-          //System.out.println("backPos=" + backPos);
           pending.add(new Token(backID,
                                 fragment,
                                 offset,
@@ -1094,10 +1083,6 @@ public final class KuromojiTokenizer2 ex
     }
 
     lastBackTracePos = endPos;
-    // nocommit explain & justify...:
-    if (endPosData.count == 0) {
-      endPosData.count = 1;
-    }
 
     if (VERBOSE) {
       System.out.println("  freeBefore pos=" + endPos);
@@ -1113,12 +1098,6 @@ public final class KuromojiTokenizer2 ex
   }
 
   private static boolean isPunctuation(char ch) {
-    // TODO: somehow this is slowish.. takes ~5% off
-    // chars/msec from Perf.java; maybe we
-    // can spend RAM...
-
-    // nocommit can we call this only when token is len
-    // 1... or it's unknown...?
     switch(Character.getType(ch)) {
       case Character.SPACE_SEPARATOR:
       case Character.LINE_SEPARATOR:

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java Tue Feb 14 23:53:06 2012
@@ -24,6 +24,7 @@ import org.apache.lucene.analysis.Analyz
 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.tokenattributes.CharTermAttribute;
 
 public class TestKuromojiAnalyzer extends BaseTokenStreamTestCase {
   /** This test fails with NPE when the 
@@ -98,8 +99,14 @@ public class TestKuromojiAnalyzer extend
                                             KuromojiAnalyzer.getDefaultStopTags());
 
     /*
-    TokenStream ts = a.tokenStream("foo", new StringReader("京都大学硬式野球部"));
-    while(ts.incrementToken());
+    //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:
@@ -160,4 +167,20 @@ public class TestKuromojiAnalyzer extend
                                             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_WITH_COMPOUNDS,
+                                            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/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java Tue Feb 14 23:53:06 2012
@@ -38,8 +38,8 @@ import org.apache.lucene.util._TestUtil;
 
 public class TestKuromojiTokenizer extends BaseTokenStreamTestCase {
 
-  private static UserDictionary readDict() {
-    InputStream is = SegmenterTest.class.getResourceAsStream("userdict.txt");
+  public static UserDictionary readDict() {
+    InputStream is = TestKuromojiTokenizer.class.getResourceAsStream("userdict.txt");
     if (is == null) {
       throw new RuntimeException("Cannot find userdict.txt in test classpath!");
     }
@@ -154,6 +154,17 @@ public class TestKuromojiTokenizer exten
 
   /** Tests that sentence offset is incorporated into the resulting offsets */
   public void testTwoSentences() throws Exception {
+    /*
+    //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 },
@@ -164,6 +175,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 {
@@ -222,7 +234,7 @@ public class TestKuromojiTokenizer exten
         new int[] { 2, 3, 4, 5, 6, 8 },
         new Integer(8)
     );
-    
+
     assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない    ")),
         new String[] { "これ", "は", "本", "で", "は", "ない"  },
         new int[] { 0, 2, 3, 4, 5, 6, 8 },
@@ -251,6 +263,33 @@ public class TestKuromojiTokenizer exten
                               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 ク ワン
@@ -517,12 +556,4 @@ public class TestKuromojiTokenizer exten
       System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
     }
   }
-
-
-  // nocommit need a still better test, where an input is
-  // tokenized to multiple tokens that would be different
-  // w/o the user dict...
-
-  // nocommit need test where userdict has multiple matches
-  // from one spot (ie, some entries are prefixes)...
 }

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java Tue Feb 14 23:53:06 2012
@@ -50,6 +50,7 @@ public class TestQuality extends LuceneT
     BufferedReader seg = new BufferedReader(new InputStreamReader(is2, IOUtils.CHARSET_UTF_8));
     Stats stats = new Stats();
 
+    // nocommit
     final boolean ONE_TIME = true;
 
     /**
@@ -230,7 +231,7 @@ public class TestQuality extends LuceneT
     }
 
     if (VERBOSE) {
-      System.out.println("  " + paths.size() + " paths");
+      System.out.println("  " + paths.size() + " paths; " + numTokens + " tokens");
     }
 
     List<String> expectedTokens = Arrays.asList(seg.split("\\s+"));

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java Tue Feb 14 23:53:06 2012
@@ -31,8 +31,6 @@ import org.junit.Test;
 
 public class UserDictionaryTest extends LuceneTestCase {
 
-  // nocommit do we test passing userDict to analyzer...?
-
   private UserDictionary readDict() throws IOException {
     InputStream is = SegmenterTest.class.getResourceAsStream("userdict.txt");
     if (is == null)

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt?rev=1244282&r1=1244281&r2=1244282&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt Tue Feb 14 23:53:06 2012
@@ -4,3 +4,7 @@
 
 # Custom reading for sumo wrestler
 朝青龍,朝青龍,アサショウリュウ,カスタム人名
+
+# Silly entry:
+abcd,a b cd,foo1 foo2 foo3,bar
+abcdefg,ab cd efg,foo1 foo2 foo4,bar