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 11:46:16 UTC

svn commit: r1244433 - in /lucene/dev/branches/lucene3767: lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/lucene/util/ lucene/test-framework/src/java/org/apache/lucene/analysis/ modules/analysis/common/src/java/org/apache/...

Author: mikemccand
Date: Wed Feb 15 10:46:15 2012
New Revision: 1244433

URL: http://svn.apache.org/viewvc?rev=1244433&view=rev
Log:
LUCENE-3767: add TokenStreamToDot; fix silly bug in RollingCharBuffer

Added:
    lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java   (with props)
Modified:
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java
    lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java
    lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/lucene3767/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
    lucene/dev/branches/lucene3767/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.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

Modified: lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java?rev=1244433&r1=1244432&r2=1244433&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java Wed Feb 15 10:46:15 2012
@@ -74,7 +74,7 @@ public final class RollingCharBuffer {
       if (count == buffer.length) {
         // Grow
         final char[] newBuffer = new char[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_CHAR)];
-        System.out.println(Thread.currentThread().getName() + ": cb grow " + newBuffer.length);
+        //System.out.println(Thread.currentThread().getName() + ": cb grow " + newBuffer.length);
         System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length - nextWrite);
         System.arraycopy(buffer, 0, newBuffer, buffer.length - nextWrite, nextWrite);
         nextWrite = buffer.length;
@@ -124,8 +124,7 @@ public final class RollingCharBuffer {
     //System.out.println("      startIndex=" + startIndex + " endIndex=" + endIndex);
 
     final char[] result = new char[length];
-    // nocommit what if entire buffer is requested...?
-    if (endIndex >= startIndex) {
+    if (endIndex >= startIndex && length < buffer.length) {
       System.arraycopy(buffer, startIndex, result, 0, endIndex-startIndex);
     } else {
       // Wrapped:

Modified: lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java?rev=1244433&r1=1244432&r2=1244433&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java Wed Feb 15 10:46:15 2012
@@ -27,7 +27,13 @@ public class TestRollingCharBuffer exten
     RollingCharBuffer buffer = new RollingCharBuffer();
 
     for(int iter=0;iter<ITERS;iter++) {
-      String s = _TestUtil.randomUnicodeString(random, 10000);
+      final int stringLen = random.nextBoolean() ? random.nextInt(50) : random.nextInt(20000);
+      final String s;
+      if (stringLen == 0) {
+        s = "";
+      } else {
+        s = _TestUtil.randomUnicodeString(random, stringLen);
+      }
       if (VERBOSE) {
         System.out.println("\nTEST: iter=" + iter + " s.length()=" + s.length());
       }
@@ -39,18 +45,39 @@ public class TestRollingCharBuffer exten
           System.out.println("  cycle nextRead=" + nextRead + " avail=" + availCount);
         }
         if (availCount == 0 || random.nextBoolean()) {
+          // Read next char
           if (VERBOSE) {
             System.out.println("    new char");
           }
           assertEquals(s.charAt(nextRead), buffer.get(nextRead));
           nextRead++;
           availCount++;
-        } else {
+        } else if (random.nextBoolean()) {
+          // Read previous char
           int pos = _TestUtil.nextInt(random, nextRead-availCount, nextRead-1);
           if (VERBOSE) {
             System.out.println("    old char pos=" + pos);
           }
           assertEquals(s.charAt(pos), buffer.get(pos));
+        } else {
+          // Read slice
+          int length;
+          if (availCount == 1) {
+            length = 1;
+          } else {
+            length = _TestUtil.nextInt(random, 1, availCount);
+          }
+          int start;
+          if (length == availCount) {
+            start = nextRead - availCount;
+          } else {
+            start = nextRead - availCount + random.nextInt(availCount-length);
+          }
+          if (VERBOSE) {
+            System.out.println("    slice start=" + start + " length=" + length);
+          }
+          assertEquals(s.substring(start, start+length),
+                       new String(buffer.get(start, length)));
         }
 
         if (availCount > 0 && random.nextInt(20) == 17) {

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=1244433&r1=1244432&r2=1244433&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 Wed Feb 15 10:46:15 2012
@@ -17,13 +17,18 @@ package org.apache.lucene.analysis;
  * limitations under the License.
  */
 
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
 import java.io.Reader;
 import java.io.StringReader;
-import java.io.IOException;
+import java.io.StringWriter;
+import java.io.Writer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
- 
+
 import org.apache.lucene.analysis.tokenattributes.*;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
@@ -438,6 +443,22 @@ public abstract class BaseTokenStreamTes
       }
     }
   }
+
+  protected String toDot(Analyzer a, String inputText) throws IOException {
+    final StringWriter sw = new StringWriter();
+    final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
+    ts.reset();
+    new TokenStreamToDot(inputText, ts, new PrintWriter(sw)).toDot();
+    return sw.toString();
+  }
+
+  protected void toDotFile(Analyzer a, String inputText, String localFileName) throws IOException {
+    Writer w = new OutputStreamWriter(new FileOutputStream(localFileName), "UTF-8");
+    final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
+    ts.reset();
+    new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot();
+    w.close();
+  }
   
   static int[] toIntArray(List<Integer> list) {
     int ret[] = new int[list.size()];

Added: lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java?rev=1244433&view=auto
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java (added)
+++ lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java Wed Feb 15 10:46:15 2012
@@ -0,0 +1,159 @@
+package org.apache.lucene.analysis;
+
+/**
+ * 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.PrintWriter;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+
+/** Consumes a TokenStream and outputs the dot (graphviz) string (graph). */
+public class TokenStreamToDot {
+
+  private final TokenStream in;
+  private final CharTermAttribute termAtt;
+  private final PositionIncrementAttribute posIncAtt;
+  private final PositionLengthAttribute posLengthAtt;
+  private final OffsetAttribute offsetAtt;
+  private final String inputText;
+  protected final PrintWriter out;
+
+  /** If inputText is non-null, and the TokenStream has
+   *  offsets, we include the surface form in each arc's
+   *  label. */
+  public TokenStreamToDot(String inputText, TokenStream in, PrintWriter out) {
+    this.in = in;
+    this.out = out;
+    this.inputText = inputText;
+    termAtt = in.addAttribute(CharTermAttribute.class);
+    posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
+    posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
+    if (in.hasAttribute(OffsetAttribute.class)) {
+      offsetAtt = in.addAttribute(OffsetAttribute.class);
+    } else {
+      offsetAtt = null;
+    }
+  }
+
+  public void toDot() throws IOException {
+    in.reset();
+    writeHeader();
+
+    // TODO: is there some way to tell dot that it should
+    // make the "main path" a straight line and have the
+    // non-sausage arcs not affect node placement...
+
+    int pos = -1;
+    int lastEndPos = -1;
+    while (in.incrementToken()) {
+      final boolean isFirst = pos == -1;
+      int posInc = posIncAtt.getPositionIncrement();
+      if (isFirst && posInc == 0) {
+        // nocommit hmm are TS's still allowed to do this...?
+        System.err.println("WARNING: first posInc was 0; correcting to 1");
+        posInc = 1;
+      }
+
+      if (posInc > 0) {
+        // New node:
+        pos += posInc;
+        writeNode(pos, Integer.toString(pos));
+      }
+
+      if (posInc > 1) {
+        // Gap!
+        writeArc(lastEndPos, pos, null, "dotted");
+      }
+
+      if (isFirst) {
+        writeNode(-1, null);
+        writeArc(-1, pos, null, null);
+      }
+
+      String arcLabel = termAtt.toString();
+      if (offsetAtt != null) {
+        final int startOffset = offsetAtt.startOffset();
+        final int endOffset = offsetAtt.endOffset();
+        //System.out.println("start=" + startOffset + " end=" + endOffset + " len=" + inputText.length());
+        if (inputText != null) {
+          arcLabel += "  / " + inputText.substring(startOffset, endOffset);
+        } else {
+          arcLabel += " / " + startOffset + "-" + endOffset;
+        }
+      }
+
+      writeArc(pos, pos + posLengthAtt.getPositionLength(), arcLabel, null);
+      lastEndPos = pos + posLengthAtt.getPositionLength();
+    }
+
+    in.end();
+
+    if (lastEndPos != -1) {
+      // TODO: should we output any final text (from end
+      // offsets) on this arc...?
+      writeNode(-2, null);
+      writeArc(lastEndPos, -2, null, null);
+    }
+
+    writeTrailer();
+  }
+
+  protected void writeArc(int fromNode, int toNode, String label, String style) {
+    out.print("  " + fromNode + " -> " + toNode + " [");
+    if (label != null) {
+      out.print(" label=\"" + label + "\"");
+    }
+    if (style != null) {
+      out.print(" style=\"" + style + "\"");
+    }
+    out.println("]");
+  }
+
+  protected void writeNode(int name, String label) {
+    out.print("  " + name);
+    if (label != null) {
+      out.print(" [label=\"" + label + "\"]");
+    } else {
+      out.print(" [shape=point color=white]");
+    }
+    out.println();
+  }
+
+  private final static String FONT_NAME = "Helvetica";
+
+  /** Override to customize. */
+  protected void writeHeader() {
+    out.println("digraph tokens {");
+    out.println("  graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\" ];");
+    out.println("  // A2 paper size");
+    out.println("  size = \"34.4,16.5\";");
+    //out.println("  // try to fill paper");
+    //out.println("  ratio = fill;");
+    out.println("  edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]");
+    out.println("  node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]");
+    out.println();
+  }
+
+  /** Override to customize. */
+  protected void writeTrailer() {
+    out.println("}");
+  }
+}

Modified: lucene/dev/branches/lucene3767/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java?rev=1244433&r1=1244432&r2=1244433&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java Wed Feb 15 10:46:15 2012
@@ -112,6 +112,8 @@ public final class SynonymFilter extends
 
   private int captureCount;
 
+  // TODO: we should set PositionLengthAttr too...
+
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
   private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);

Modified: lucene/dev/branches/lucene3767/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java?rev=1244433&r1=1244432&r2=1244433&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java Wed Feb 15 10:46:15 2012
@@ -117,7 +117,7 @@ public class TestStandardAnalyzer extend
     BaseTokenStreamTestCase.assertAnalyzesTo(a, "我是中国人。 1234 Tests ",
         new String[] { "我", "是", "中", "国", "人", "1234", "Tests"});
   }
-  
+
   public void testEmpty() throws Exception {
     BaseTokenStreamTestCase.assertAnalyzesTo(a, "", new String[] {});
     BaseTokenStreamTestCase.assertAnalyzesTo(a, ".", new String[] {});

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=1244433&r1=1244432&r2=1244433&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 Wed Feb 15 10:46:15 2012
@@ -167,6 +167,7 @@ public final class KuromojiTokenizer2 ex
         break;
     }
     buffer.reset(input);
+
     resetState();
 
     dictionaryMap.put(Type.KNOWN, dictionary);
@@ -390,6 +391,7 @@ public final class KuromojiTokenizer2 ex
     int position = token.getPosition();
     int length = token.getLength();
     clearAttributes();
+    assert length > 0;
     //System.out.println("off=" + token.getOffset() + " len=" + length + " vs " + token.getSurfaceForm().length);
     termAtt.copyBuffer(token.getSurfaceForm(), token.getOffset(), length);
     offsetAtt.setOffset(correctOffset(position), correctOffset(position+length));
@@ -405,7 +407,7 @@ public final class KuromojiTokenizer2 ex
       posIncAtt.setPositionIncrement(1);
       posLengthAtt.setPositionLength(1);
     }
-    if (true || VERBOSE) {
+    if (VERBOSE) {
       System.out.println(Thread.currentThread().getName() + ":    incToken: return token=" + token);
     }
     lastTokenPos = token.getPosition();

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=1244433&r1=1244432&r2=1244433&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 Wed Feb 15 10:46:15 2012
@@ -148,6 +148,7 @@ public class TestKuromojiAnalyzer extend
                               new int[] {1, 0, 1},
                               new int[] {1, 2, 1}
                               );
+    // toDotFile(a, "成田空港", "/mnt/scratch/out.dot");
   }
 
   

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=1244433&r1=1244432&r2=1244433&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 Wed Feb 15 10:46:15 2012
@@ -175,8 +175,7 @@ public class TestKuromojiTokenizer exten
   /** blast some random strings through the analyzer */
   public void testRandomStrings() throws Exception {
     checkRandomData(random, analyzer, 10000*RANDOM_MULTIPLIER);
-    // nocommit put back
-    //checkRandomData(random, analyzerNoPunct, 10000*RANDOM_MULTIPLIER);
+    checkRandomData(random, analyzerNoPunct, 10000*RANDOM_MULTIPLIER);
   }
   
   public void testLargeDocReliability() throws Exception {

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=1244433&r1=1244432&r2=1244433&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 Wed Feb 15 10:46:15 2012
@@ -48,9 +48,7 @@ public class TestQuality extends LuceneT
     BufferedReader unseg = new BufferedReader(new InputStreamReader(is, IOUtils.CHARSET_UTF_8));
     InputStream is2 = zip.getInputStream(zip.getEntry("segmented.txt"));
     BufferedReader seg = new BufferedReader(new InputStreamReader(is2, IOUtils.CHARSET_UTF_8));
-    Stats stats = new Stats();
 
-    // nocommit
     final boolean ONE_TIME = true;
 
     /**
@@ -65,7 +63,6 @@ public class TestQuality extends LuceneT
     StringBuilder sb = new StringBuilder();
 
     String line1 = null;
-    String line2 = null;
     int maxLen = 0;
     int count = 0;
     while ((line1 = unseg.readLine()) != null) {
@@ -85,6 +82,7 @@ public class TestQuality extends LuceneT
 
     final Tokenizer tokenizer = new KuromojiTokenizer2(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
     //final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""));
+    tokenizer.reset();
     final String all = sb.toString();
     final int ITERS = 20;
     CharTermAttribute termAtt = tokenizer.addAttribute(CharTermAttribute.class); 
@@ -127,7 +125,6 @@ public class TestQuality extends LuceneT
     //final Tokenizer tokenizer = new KuromojiTokenizer(new StringReader(""));
     //final Tokenizer tokenizer = new KuromojiTokenizer(new Segmenter(Mode.NORMAL), new StringReader(""));
     final Tokenizer tokenizer = new KuromojiTokenizer2(new StringReader(""), null, true, Mode.SEARCH_WITH_COMPOUNDS);
-    
     String line1 = null;
     String line2 = null;
     int count = 0;
@@ -176,6 +173,7 @@ public class TestQuality extends LuceneT
       System.out.println("\nTEST " + lineCount + ": input " + unseg);
     }
     tokenizer.reset(new StringReader(unseg));
+    tokenizer.reset();
     CharTermAttribute termAtt = tokenizer.addAttribute(CharTermAttribute.class);
     PositionIncrementAttribute posIncAtt = tokenizer.addAttribute(PositionIncrementAttribute.class);
     PositionLengthAttribute posLengthAtt = tokenizer.addAttribute(PositionLengthAttribute.class);