You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/04 04:01:24 UTC

svn commit: r1227035 - in /lucene/dev/branches/lucene3305: ./ lucene/ lucene/src/test-framework/java/org/apache/lucene/analysis/ modules/analysis/common/src/java/org/apache/lucene/analysis/util/ modules/analysis/common/src/test/org/apache/lucene/analys...

Author: rmuir
Date: Wed Jan  4 03:01:23 2012
New Revision: 1227035

URL: http://svn.apache.org/viewvc?rev=1227035&view=rev
Log:
LUCENE-3305: don't read whole doc into ram, use segmentingtokenizerbase + ja breakiterator, also reduces error rate wrt mecab a bit

Added:
    lucene/dev/branches/lucene3305/modules/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tanakaseg.zip   (with props)
Modified:
    lucene/dev/branches/lucene3305/   (props changed)
    lucene/dev/branches/lucene3305/lucene/   (props changed)
    lucene/dev/branches/lucene3305/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Tokenizer.java

Modified: lucene/dev/branches/lucene3305/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1227035&r1=1227034&r2=1227035&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene3305/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Jan  4 03:01:23 2012
@@ -249,7 +249,46 @@ public abstract class BaseTokenStreamTes
   // TODO: add a MockCharStream, and use it here too, to ensure that correctOffset etc is being done by tokenizers.
   public static void checkRandomData(Random random, Analyzer a, int iterations) throws IOException {
     checkRandomData(random, a, iterations, 20);
+    // now test with multiple threads
+    int numThreads = _TestUtil.nextInt(random, 4, 8);
+    Thread threads[] = new Thread[numThreads];
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new AnalysisThread(new Random(random.nextLong()), a, iterations);
+    }
+    for (int i = 0; i < threads.length; i++) {
+      threads[i].start();
+    }
+    for (int i = 0; i < threads.length; i++) {
+      try {
+        threads[i].join();
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
   }
+  
+  static class AnalysisThread extends Thread {
+    final int iterations;
+    final Random random;
+    final Analyzer a;
+    
+    AnalysisThread(Random random, Analyzer a, int iterations) {
+      this.random = random;
+      this.a = a;
+      this.iterations = iterations;
+    }
+    
+    @Override
+    public void run() {
+      try {
+        // see the part in checkRandomData where it replays the same text again
+        // to verify reproducability/reuse: hopefully this would catch thread hazards.
+        checkRandomData(random, a, iterations, 20);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  };
 
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
     for (int i = 0; i < iterations; i++) {

Added: lucene/dev/branches/lucene3305/modules/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java?rev=1227035&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java Wed Jan  4 03:01:23 2012
@@ -0,0 +1,180 @@
+package org.apache.lucene.analysis.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+
+import java.text.BreakIterator;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+
+/**
+ * Breaks text into sentences with a {@link BreakIterator} and
+ * allows subclasses to decompose these sentences into words.
+ * <p>
+ * This can be used by subclasses that need sentence context 
+ * for tokenization purposes, such as CJK segmenters.
+ * <p>
+ * Additionally it can be used by subclasses that want to mark
+ * sentence boundaries (with a custom attribute, extra token, position
+ * increment, etc) for downstream processing.
+ * 
+ * @lucene.experimental
+ */
+public abstract class SegmentingTokenizerBase extends Tokenizer {
+  protected static final int BUFFERMAX = 4096;
+  protected final char buffer[] = new char[BUFFERMAX];
+  /** true length of text in the buffer */
+  private int length = 0; 
+  /** length in buffer that can be evaluated safely, up to a safe end point */
+  private int usableLength = 0; 
+  /** accumulated offset of previous buffers for this reader, for offsetAtt */
+  protected int offset = 0;
+  
+  private final BreakIterator iterator;
+  private final CharArrayIterator wrapper = CharArrayIterator.newSentenceInstance();
+
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  /**
+   * Construct a new SegmenterBase from the given Reader, using
+   * the provided BreakIterator for sentence segmentation.
+   * <p>
+   * Note that you should never share BreakIterators across different
+   * TokenStreams, instead a newly created or cloned one should always
+   * be provided to this constructor.
+   */
+  public SegmentingTokenizerBase(Reader input, BreakIterator iterator) {
+    super(input);
+    this.iterator = iterator;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (length == 0 || !incrementWord()) {
+      while (!incrementSentence()) {
+        refill();
+        if (length <= 0) // no more bytes to read;
+          return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  @Override
+  public void reset() throws IOException {
+    wrapper.setText(buffer, 0, 0);
+    iterator.setText(wrapper);
+    length = usableLength = offset = 0;
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    this.input = input;
+    reset();
+  }
+  
+  @Override
+  public final void end() throws IOException {
+    final int finalOffset = correctOffset(length < 0 ? offset : offset + length);
+    offsetAtt.setOffset(finalOffset, finalOffset);
+  }  
+
+  /** Returns the last unambiguous break position in the text. */
+  private int findSafeEnd() {
+    for (int i = length - 1; i >= 0; i--)
+      if (isSafeEnd(buffer[i]))
+        return i + 1;
+    return -1;
+  }
+  
+  /** For sentence tokenization, these are the unambiguous break positions. */
+  protected boolean isSafeEnd(char ch) {
+    switch(ch) {
+      case 0x000D:
+      case 0x000A:
+      case 0x0085:
+      case 0x2028:
+      case 0x2029:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Refill the buffer, accumulating the offset and setting usableLength to the
+   * last unambiguous break position
+   */
+  private void refill() throws IOException {
+    offset += usableLength;
+    int leftover = length - usableLength;
+    System.arraycopy(buffer, usableLength, buffer, 0, leftover);
+    int requested = buffer.length - leftover;
+    int returned = input.read(buffer, leftover, requested);
+    length = returned < 0 ? leftover : returned + leftover;
+    if (returned < requested) /* reader has been emptied, process the rest */
+      usableLength = length;
+    else { /* still more data to be read, find a safe-stopping place */
+      usableLength = findSafeEnd();
+      if (usableLength < 0)
+        usableLength = length; /*
+                                * more than IOBUFFER of text without breaks,
+                                * gonna possibly truncate tokens
+                                */
+    }
+
+    wrapper.setText(buffer, 0, Math.max(0, usableLength));
+    iterator.setText(wrapper);
+  }
+
+  /**
+   * return true if there is a token from the buffer, or null if it is
+   * exhausted.
+   */
+  private boolean incrementSentence() throws IOException {
+    if (length == 0) // we must refill the buffer
+      return false;
+    
+    while (true) {
+      int start = iterator.current();
+
+      if (start == BreakIterator.DONE)
+        return false; // BreakIterator exhausted
+
+      // find the next set of boundaries
+      int end = iterator.next();
+
+      if (end == BreakIterator.DONE)
+        return false; // BreakIterator exhausted
+
+      setNextSentence(start, end);
+      if (incrementWord()) {
+        return true;
+      }
+    }
+  }
+  
+  /** Provides the next input sentence for analysis */
+  protected abstract void setNextSentence(int sentenceStart, int sentenceEnd);
+  /** Returns true if another word is available */
+  protected abstract boolean incrementWord();
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java?rev=1227035&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java Wed Jan  4 03:01:23 2012
@@ -0,0 +1,224 @@
+package org.apache.lucene.analysis.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.text.BreakIterator;
+import java.util.Arrays;
+import java.util.Locale;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/** Basic tests for {@link SegmentingTokenizerBase} */
+public class TestSegmentingTokenizerBase extends BaseTokenStreamTestCase {
+  private Analyzer sentence = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      Tokenizer tokenizer = new WholeSentenceTokenizer(reader);
+      return new TokenStreamComponents(tokenizer, tokenizer);
+    }
+  };
+  
+  private Analyzer sentenceAndWord = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+      Tokenizer tokenizer = new SentenceAndWordTokenizer(reader);
+      return new TokenStreamComponents(tokenizer, tokenizer);
+    }
+  };
+  
+  /** Some simple examples, just outputting the whole sentence boundaries as "terms" */
+  public void testBasics() throws IOException {
+    assertAnalyzesTo(sentence, "The acronym for United States is U.S. but this doesn't end a sentence",
+        new String[] { "The acronym for United States is U.S. but this doesn't end a sentence"}
+    );
+    assertAnalyzesTo(sentence, "He said, \"Are you going?\" John shook his head.",
+        new String[] { "He said, \"Are you going?\" ", 
+                       "John shook his head." }
+    );
+  }
+  
+  /** Test a subclass that sets some custom attribute values */
+  public void testCustomAttributes() throws IOException {
+    assertAnalyzesTo(sentenceAndWord, "He said, \"Are you going?\" John shook his head.",
+        new String[] { "He", "said", "Are", "you", "going", "John", "shook", "his", "head" },
+        new int[] { 0, 3, 10, 14, 18, 26, 31, 37, 41 },
+        new int[] { 2, 7, 13, 17, 23, 30, 36, 40, 45 },
+        new int[] { 1, 1,  1,  1,  1,  2,  1,  1,  1 }
+    );
+  }
+  
+  /** Tests tokenstream reuse */
+  public void testReuse() throws IOException {
+    assertAnalyzesToReuse(sentenceAndWord, "He said, \"Are you going?\"",
+        new String[] { "He", "said", "Are", "you", "going" },
+        new int[] { 0, 3, 10, 14, 18 },
+        new int[] { 2, 7, 13, 17, 23 },
+        new int[] { 1, 1,  1,  1,  1,}
+    );
+    assertAnalyzesToReuse(sentenceAndWord, "John shook his head.",
+        new String[] { "John", "shook", "his", "head" },
+        new int[] { 0,  5, 11, 15 },
+        new int[] { 4, 10, 14, 19 },
+        new int[] { 1,  1,  1,  1 }
+    );
+  }
+  
+  /** Tests TokenStream.end() */
+  public void testEnd() throws IOException {
+    // BaseTokenStreamTestCase asserts that end() is set to our StringReader's length for us here.
+    // we add some junk whitespace to the end just to test it.
+    assertAnalyzesTo(sentenceAndWord, "John shook his head          ",
+        new String[] { "John", "shook", "his", "head" }
+    );
+    assertAnalyzesTo(sentenceAndWord, "John shook his head.          ",
+        new String[] { "John", "shook", "his", "head" }
+    );
+  }
+  
+  /** Tests terms which span across boundaries */
+  public void testHugeDoc() throws IOException {
+    StringBuilder sb = new StringBuilder();
+    char whitespace[] = new char[4094];
+    Arrays.fill(whitespace, '\n');
+    sb.append(whitespace);
+    sb.append("testing 1234");
+    String input = sb.toString();
+    assertAnalyzesTo(sentenceAndWord, input, new String[] { "testing", "1234" });
+  }
+  
+  /** Tests the handling of binary/malformed data */
+  public void testHugeTerm() throws IOException {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 40960; i++) {
+      sb.append('a');
+    }
+    String input = sb.toString();
+    char token[] = new char[4096];
+    Arrays.fill(token, 'a');
+    String expectedToken = new String(token);
+    String expected[] = { 
+        expectedToken, expectedToken, expectedToken, 
+        expectedToken, expectedToken, expectedToken,
+        expectedToken, expectedToken, expectedToken,
+        expectedToken
+    };
+    assertAnalyzesTo(sentence, input, expected);
+  }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    checkRandomData(random, sentence, 10000*RANDOM_MULTIPLIER);
+    checkRandomData(random, sentenceAndWord, 10000*RANDOM_MULTIPLIER);
+  }
+
+  // some tokenizers for testing
+  
+  /** silly tokenizer that just returns whole sentences as tokens */
+  static class WholeSentenceTokenizer extends SegmentingTokenizerBase {
+    int sentenceStart, sentenceEnd;
+    boolean hasSentence;
+    
+    private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    
+    public WholeSentenceTokenizer(Reader input) {
+      super(input, BreakIterator.getSentenceInstance(new Locale("")));
+    }
+
+    @Override
+    protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+      this.sentenceStart = sentenceStart;
+      this.sentenceEnd = sentenceEnd;
+      hasSentence = true;
+    }
+
+    @Override
+    protected boolean incrementWord() {
+      if (hasSentence) {
+        hasSentence = false;
+        clearAttributes();
+        termAtt.copyBuffer(buffer, sentenceStart, sentenceEnd-sentenceStart);
+        offsetAtt.setOffset(offset+sentenceStart, offset+sentenceEnd);
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+  
+  /** 
+   * simple tokenizer, that bumps posinc + 1 for tokens after a 
+   * sentence boundary to inhibit phrase queries without slop.
+   */
+  static class SentenceAndWordTokenizer extends SegmentingTokenizerBase {
+    int sentenceStart, sentenceEnd;
+    int wordStart, wordEnd;
+    int posBoost = -1; // initially set to -1 so the first word in the document doesn't get a pos boost
+    
+    private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+    
+    public SentenceAndWordTokenizer(Reader input) {
+      super(input, BreakIterator.getSentenceInstance(new Locale("")));
+    }
+
+    @Override
+    protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+      this.wordStart = this.wordEnd = this.sentenceStart = sentenceStart;
+      this.sentenceEnd = sentenceEnd;
+      posBoost++;
+    }
+    
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      posBoost = -1;
+    }
+
+    @Override
+    protected boolean incrementWord() {
+      wordStart = wordEnd;
+      while (wordStart < sentenceEnd) {
+        if (Character.isLetterOrDigit(buffer[wordStart]))
+          break;
+        wordStart++;
+      }
+      
+      if (wordStart == sentenceEnd) return false;
+      
+      wordEnd = wordStart+1;
+      while (wordEnd < sentenceEnd && Character.isLetterOrDigit(buffer[wordEnd]))
+        wordEnd++;
+      
+      clearAttributes();
+      termAtt.copyBuffer(buffer, wordStart, wordEnd-wordStart);
+      offsetAtt.setOffset(offset+wordStart, offset+wordEnd);
+      posIncAtt.setPositionIncrement(posIncAtt.getPositionIncrement() + posBoost);
+      posBoost = 0;
+      return true;
+    }
+  }
+}

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java?rev=1227035&r1=1227034&r2=1227035&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java Wed Jan  4 03:01:23 2012
@@ -19,77 +19,55 @@ package org.apache.lucene.analysis.kurom
 
 import java.io.IOException;
 import java.io.Reader;
+import java.text.BreakIterator;
 import java.util.List;
+import java.util.Locale;
 
-import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
 
-public final class KuromojiTokenizer extends Tokenizer {
+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 TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
   private final org.apache.lucene.analysis.kuromoji.Tokenizer tokenizer;
   
-  private final StringBuilder str = new StringBuilder();
-  
-  private List<Token> tokens;
-  
+  private List<Token> tokens; 
   private int tokenIndex = 0;
+  private int sentenceStart = 0;
   
   public KuromojiTokenizer(org.apache.lucene.analysis.kuromoji.Tokenizer tokenizer, Reader input) throws IOException {
-    super(input);
+    super(input, (BreakIterator) proto.clone());
     this.tokenizer = tokenizer;
-    // nocommit: this won't really work for large docs.
-    // what kind of context does kuromoji need? just sentence maybe?
-    fillBuffer(str, input);
-    init();
   }
   
-  private void init() {
+  @Override
+  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+    this.sentenceStart = sentenceStart;
+    // TODO: allow the tokenizer, at least maybe doTokenize to take char[] or charsequence or characteriterator?
+    tokens = tokenizer.tokenize(new String(buffer, sentenceStart, sentenceEnd-sentenceStart));
     tokenIndex = 0;
-    tokens = tokenizer.tokenize(str.toString());
   }
-  
+
   @Override
-  public boolean incrementToken() {
-    if(tokenIndex == tokens.size()) {
+  protected boolean incrementWord() {
+    if (tokenIndex == tokens.size()) {
       return false;
     }
-    
     Token token = tokens.get(tokenIndex);
+    // TODO: we don't really need the surface form except for its length? (its in the buffer already)
     String surfaceForm = token.getSurfaceForm();
     int position = token.getPosition();
     int length = surfaceForm.length();
-    int end = position + length;
     clearAttributes();
-    termAtt.setEmpty().append(str, position, end);
-    offsetAtt.setOffset(correctOffset(position), correctOffset(end));
+    termAtt.copyBuffer(buffer, sentenceStart + position, length);
+    int startOffset = offset + sentenceStart + position;
+    offsetAtt.setOffset(correctOffset(startOffset), correctOffset(startOffset+length));
     typeAtt.setType(token.getPartOfSpeech());
     tokenIndex++;
     return true;
   }
-  
-  @Override
-  public void end() {
-    final int ofs = correctOffset(str.length());
-    offsetAtt.setOffset(ofs, ofs);
-  }
-  
-  @Override
-  public void reset(Reader input) throws IOException{
-    super.reset(input);
-    fillBuffer(str, input);
-    init();
-  }
-  
-  final char[] buffer = new char[8192];
-  private void fillBuffer(StringBuilder sb, Reader input) throws IOException {
-    int len;
-    sb.setLength(0);
-    while ((len = input.read(buffer)) > 0) {
-      sb.append(buffer, 0, len);
-    }
-  }
 }

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Tokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Tokenizer.java?rev=1227035&r1=1227034&r2=1227035&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Tokenizer.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Tokenizer.java Wed Jan  4 03:01:23 2012
@@ -168,7 +168,7 @@ public class Tokenizer {
     
     private Mode mode = Mode.NORMAL;
     
-    private boolean split = true;
+    private boolean split = false;
     
     private UserDictionary userDictionary = null;
     

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java?rev=1227035&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java Wed Jan  4 03:01:23 2012
@@ -0,0 +1,228 @@
+package org.apache.lucene.analysis.kuromoji;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.zip.ZipFile;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.kuromoji.Tokenizer.Mode;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.util.LuceneTestCase;
+
+// nocommit: we don't need this or its huge files i dont think?
+// just compares segmentation to some sentences pre-tokenized by mecab
+public class TestQuality extends LuceneTestCase {
+
+  public void test() throws Exception {
+    File datafile = getDataFile("tanakaseg.zip");
+    ZipFile zip = new ZipFile(datafile);
+    InputStream is = zip.getInputStream(zip.getEntry("sentences.txt"));
+    BufferedReader unseg = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+    InputStream is2 = zip.getInputStream(zip.getEntry("segmented.txt"));
+    BufferedReader seg = new BufferedReader(new InputStreamReader(is2, "UTF-8"));
+    Stats stats = new Stats();
+    /**
+     #words: 1578506
+     #chars: 4519246
+     #edits: 651
+     #sentences: 150122
+     sentence agreement?: 0.998161495317142
+     word agreement?: 0.999587584716181
+     */
+    final org.apache.lucene.analysis.kuromoji.Tokenizer tokenizer = 
+        org.apache.lucene.analysis.kuromoji.Tokenizer.builder().mode(Mode.NORMAL).build();
+    Analyzer testAnalyzer = new KuromojiAnalyzer(tokenizer);
+    
+    String line1 = null;
+    String line2 = null;
+    while ((line1 = unseg.readLine()) != null) {
+      line2 = seg.readLine();
+      evaluateLine(line1, line2, testAnalyzer, stats);
+    }
+    
+    System.out.println("#words: " + stats.numWords);
+    System.out.println("#chars: " + stats.numChars);
+    System.out.println("#edits: " + stats.numEdits);
+    System.out.println("#sentences: " + stats.numSentences);
+    System.out.println("sentence agreement?: " + (stats.numSentencesCorrect/(double)stats.numSentences));
+    System.out.println("word agreement?: " + (1D - (stats.numEdits / (double)stats.numWords)));
+    unseg.close();
+    seg.close();
+    zip.close();
+  }
+  
+  static class Stats {
+    long numWords = 0;
+    long numEdits = 0;
+    long numChars = 0;
+    long numSentences = 0;
+    long numSentencesCorrect = 0;
+  }
+  
+  public static void evaluateLine(String unseg, String seg, Analyzer analyzer, Stats stats) throws Exception {
+    List<String> tokens = new ArrayList<String>();
+    TokenStream stream = analyzer.tokenStream("bogus", new StringReader(unseg));
+    CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
+    stream.reset();
+    while (stream.incrementToken()) {
+      tokens.add(termAtt.toString());
+    }
+    stream.close();
+    
+    List<String> expectedTokens = Arrays.asList(seg.split("\\s+"));
+    tokens = normalize(tokens);
+    expectedTokens = normalize(expectedTokens);
+    
+    HashMap<String,Character> transformation = new HashMap<String,Character>();
+    CharRef charRef = new CharRef();
+    
+    String s1 = transform(tokens, transformation, charRef);
+    String s2 = transform(expectedTokens, transformation, charRef);
+    
+    int edits = getDistance(s2, s1);
+    //if (edits > 0) {
+    //  System.out.println("unseg: " + unseg);
+    //  System.out.println(tokens + " vs " + expectedTokens);
+    //}
+    stats.numChars += seg.length();
+    stats.numEdits += edits;
+    stats.numWords += expectedTokens.size();
+    stats.numSentences++;
+    if (edits == 0)
+      stats.numSentencesCorrect++;
+  }
+  
+  static class CharRef {
+    char c = 'a';
+  }
+  
+  static String transform(List<String> tokens, HashMap<String,Character> transformation, CharRef ref) {
+    StringBuilder builder = new StringBuilder();
+    for (String token : tokens) {
+      Character value = transformation.get(token);
+      
+      if (value == null) {
+        value = new Character(ref.c);
+        ref.c++;
+        transformation.put(token, value);
+      }
+      
+      builder.append(value.charValue());
+    }
+    return builder.toString();
+  }
+  
+  static List<String> normalize(List<String> tokens) {
+    List<String> newList = new ArrayList<String>();
+    Iterator<String> iterator = tokens.iterator();
+    while (iterator.hasNext()) {
+      String term = iterator.next();
+      if (Character.isLetterOrDigit(term.charAt(0)))
+        newList.add(term);
+    }
+    return newList;
+  }
+  
+  
+  //*****************************
+  // Compute Levenshtein distance: see org.apache.commons.lang.StringUtils#getLevenshteinDistance(String, String)
+  //*****************************
+  private static int getDistance (String target, String other) {
+    char[] sa;
+    int n;
+    int p[]; //'previous' cost array, horizontally
+    int d[]; // cost array, horizontally
+    int _d[]; //placeholder to assist in swapping p and d
+    
+      /*
+         The difference between this impl. and the previous is that, rather
+         than creating and retaining a matrix of size s.length()+1 by t.length()+1,
+         we maintain two single-dimensional arrays of length s.length()+1.  The first, d,
+         is the 'current working' distance array that maintains the newest distance cost
+         counts as we iterate through the characters of String s.  Each time we increment
+         the index of String t we are comparing, d is copied to p, the second int[].  Doing so
+         allows us to retain the previous cost counts as required by the algorithm (taking
+         the minimum of the cost count to the left, up one, and diagonally up and to the left
+         of the current cost count being calculated).  (Note that the arrays aren't really
+         copied anymore, just switched...this is clearly much better than cloning an array
+         or doing a System.arraycopy() each time  through the outer loop.)
+
+         Effectively, the difference between the two implementations is this one does not
+         cause an out of memory condition when calculating the LD over two very large strings.
+       */
+
+      sa = target.toCharArray();
+      n = sa.length;
+      p = new int[n+1]; 
+      d = new int[n+1]; 
+    
+      final int m = other.length();
+      if (n == 0 || m == 0) {
+        if (n == m) {
+          return 0;
+        }
+        else {
+          return Math.max(n, m);
+        }
+      } 
+
+
+      // indexes into strings s and t
+      int i; // iterates through s
+      int j; // iterates through t
+
+      char t_j; // jth character of t
+
+      int cost; // cost
+
+      for (i = 0; i<=n; i++) {
+          p[i] = i;
+      }
+
+      for (j = 1; j<=m; j++) {
+          t_j = other.charAt(j-1);
+          d[0] = j;
+
+          for (i=1; i<=n; i++) {
+              cost = sa[i-1]==t_j ? 0 : 1;
+              // minimum of cell to the left+1, to the top+1, diagonally left and up +cost
+              d[i] = Math.min(Math.min(d[i-1]+1, p[i]+1),  p[i-1]+cost);
+          }
+
+          // copy current distance counts to 'previous row' distance counts
+          _d = p;
+          p = d;
+          d = _d;
+      }
+
+      // our last action in the above loop was to switch d and p, so p now
+      // actually has the most recent cost counts
+      return Math.abs(p[n]);
+  }
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tanakaseg.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tanakaseg.zip?rev=1227035&view=auto
==============================================================================
Binary file - no diff available.