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

svn commit: r1296805 [1/3] - in /lucene/dev/trunk: lucene/contrib/ lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/java/org/apache/lucene/util/fst/ lucene/core/src/test/org/a...

Author: cm
Date: Sun Mar  4 13:34:13 2012
New Revision: 1296805

URL: http://svn.apache.org/viewvc?rev=1296805&view=rev
Log:
Kuromoji now produces both compound words and the segmentation of those words in search mode (LUCENE-3767)

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java
Removed:
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/SegmenterTest.java
Modified:
    lucene/dev/trunk/lucene/contrib/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiBaseFormFilter.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiTokenizer.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/search-segmentation-tests.txt
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestKuromojiTokenizerFactory.java

Modified: lucene/dev/trunk/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/CHANGES.txt?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/contrib/CHANGES.txt Sun Mar  4 13:34:13 2012
@@ -154,6 +154,9 @@ New Features
  * LUCENE-3730: Refine Kuromoji search mode (Mode.SEARCH) decompounding
    heuristics.  (Christian Moen via Robert Muir)
 
+ * LUCENE-3767: Kuromoji tokenizer/analyzer produces both compound words 
+   and the segmentation of that compound in Mode.SEARCH. (Robert Muir, Mike McCandless via Christian Moen)
+
  * LUCENE-3685: Add ToChildBlockJoinQuery and renamed previous
    BlockJoinQuery to ToParentBlockJoinQuery, so that you can now do
    joins in both parent to child and child to parent directions.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java Sun Mar  4 13:34:13 2012
@@ -52,9 +52,10 @@ public class PositionIncrementAttributeI
    * @param positionIncrement the distance from the prior term
    */
   public void setPositionIncrement(int positionIncrement) {
-    if (positionIncrement < 0)
+    if (positionIncrement < 0) {
       throw new IllegalArgumentException
-        ("Increment must be zero or greater: " + positionIncrement);
+        ("Increment must be zero or greater: got " + positionIncrement);
+    }
     this.positionIncrement = positionIncrement;
   }
 
@@ -77,7 +78,8 @@ public class PositionIncrementAttributeI
     }
     
     if (other instanceof PositionIncrementAttributeImpl) {
-      return positionIncrement == ((PositionIncrementAttributeImpl) other).positionIncrement;
+      PositionIncrementAttributeImpl _other = (PositionIncrementAttributeImpl) other;
+      return positionIncrement ==  _other.positionIncrement;
     }
  
     return false;
@@ -93,5 +95,4 @@ public class PositionIncrementAttributeI
     PositionIncrementAttribute t = (PositionIncrementAttribute) target;
     t.setPositionIncrement(positionIncrement);
   }  
-
 }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java Sun Mar  4 13:34:13 2012
@@ -0,0 +1,41 @@
+package org.apache.lucene.analysis.tokenattributes;
+
+/**
+ * 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 org.apache.lucene.util.Attribute;
+
+/** The positionLength determines how many positions this
+ *  token spans.  Very few analyzer components actually
+ *  produce this attribute, and indexing ignores it, but
+ *  it's useful to express the graph structure naturally
+ *  produced by decompounding, word splitting/joining,
+ *  synonym filtering, etc.
+ *
+ * <p>The default value is one. */
+
+public interface PositionLengthAttribute extends Attribute {
+  /** @param positionLength how many positions this token
+   *  spans. */
+  public void setPositionLength(int positionLength);
+
+  /** Returns the position length of this Token.
+   * @see #setPositionLength
+   */
+  public int getPositionLength();
+}
+

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java Sun Mar  4 13:34:13 2012
@@ -0,0 +1,74 @@
+package org.apache.lucene.analysis.tokenattributes;
+
+/**
+ * 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 org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.util.AttributeImpl;
+
+/** See {@link PositionLengthAttribute}. */
+public class PositionLengthAttributeImpl extends AttributeImpl implements PositionLengthAttribute, Cloneable {
+  private int positionLength = 1;
+  
+  /** @param positionLength how many positions this token
+   *  spans.  NOTE: this is optional, and most analyzers
+   *  don't change the default value (1). */
+  public void setPositionLength(int positionLength) {
+    if (positionLength < 1) {
+      throw new IllegalArgumentException
+        ("Position length must be 1 or greater: got " + positionLength);
+    }
+    this.positionLength = positionLength;
+  }
+
+  /** Returns the position length of this Token.
+   * @see #setPositionLength    
+   */
+  public int getPositionLength() {
+    return positionLength;
+  }
+
+  @Override
+  public void clear() {
+    this.positionLength = 1;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other == this) {
+      return true;
+    }
+    
+    if (other instanceof PositionLengthAttributeImpl) {
+      PositionLengthAttributeImpl _other = (PositionLengthAttributeImpl) other;
+      return positionLength ==  _other.positionLength;
+    }
+ 
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return positionLength;
+  }
+  
+  @Override
+  public void copyTo(AttributeImpl target) {
+    PositionLengthAttribute t = (PositionLengthAttribute) target;
+    t.setPositionLength(positionLength);
+  }  
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java Sun Mar  4 13:34:13 2012
@@ -0,0 +1,148 @@
+package org.apache.lucene.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;
+
+/** Acts like a forever growing char[] as you read
+ *  characters into it from the provided reader, but
+ *  internally it uses a circular buffer to only hold the
+ *  characters that haven't been freed yet.  This is like a
+ *  PushbackReader, except you don't have to specify
+ *  up-front the max size of the buffer, but you do have to
+ *  periodically call {@link #freeBefore}. */
+
+public final class RollingCharBuffer {
+
+  private Reader reader;
+
+  private char[] buffer = new char[32];
+
+  // Next array index to write to in buffer:
+  private int nextWrite;
+
+  // Next absolute position to read from reader:
+  private int nextPos;
+
+  // How many valid chars (wrapped) are in the buffer:
+  private int count;
+
+  // True if we hit EOF
+  private boolean end;
+    
+  /** Clear array and switch to new reader. */
+  public void reset(Reader reader) {
+    this.reader = reader;
+    nextPos = 0;
+    nextWrite = 0;
+    count = 0;
+    end = false;
+  }
+
+  /* Absolute position read.  NOTE: pos must not jump
+   * ahead by more than 1!  Ie, it's OK to read arbitarily
+   * far back (just not prior to the last {@link
+   * #freeBefore}), but NOT ok to read arbitrarily far
+   * ahead.  Returns -1 if you hit EOF. */
+  public int get(int pos) throws IOException {
+    //System.out.println("    get pos=" + pos + " nextPos=" + nextPos + " count=" + count);
+    if (pos == nextPos) {
+      if (end) {
+        return -1;
+      }
+      final int ch = reader.read();
+      if (ch == -1) {
+        end = true;
+        return -1;
+      }
+      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.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length - nextWrite);
+        System.arraycopy(buffer, 0, newBuffer, buffer.length - nextWrite, nextWrite);
+        nextWrite = buffer.length;
+        buffer = newBuffer;
+      }
+      if (nextWrite == buffer.length) {
+        nextWrite = 0;
+      }
+      buffer[nextWrite++] = (char) ch;
+      count++;
+      nextPos++;
+      return ch;
+    } else {
+      // Cannot read from future (except by 1):
+      assert pos < nextPos;
+
+      // Cannot read from already freed past:
+      assert nextPos - pos <= count;
+
+      final int index = getIndex(pos);
+      return buffer[index];
+    }
+  }
+
+  // For assert:
+  private boolean inBounds(int pos) {
+    return pos >= 0 && pos < nextPos && pos >= nextPos - count;
+  }
+
+  private int getIndex(int pos) {
+    int index = nextWrite - (nextPos - pos);
+    if (index < 0) {
+      // Wrap:
+      index += buffer.length;
+      assert index >= 0;
+    }
+    return index;
+  }
+
+  public char[] get(int posStart, int length) {
+    assert length > 0;
+    assert inBounds(posStart): "posStart=" + posStart + " length=" + length;
+    //System.out.println("    buffer.get posStart=" + posStart + " len=" + length);
+      
+    final int startIndex = getIndex(posStart);
+    final int endIndex = getIndex(posStart + length);
+    //System.out.println("      startIndex=" + startIndex + " endIndex=" + endIndex);
+
+    final char[] result = new char[length];
+    if (endIndex >= startIndex && length < buffer.length) {
+      System.arraycopy(buffer, startIndex, result, 0, endIndex-startIndex);
+    } else {
+      // Wrapped:
+      final int part1 = buffer.length-startIndex;
+      System.arraycopy(buffer, startIndex, result, 0, part1);
+      System.arraycopy(buffer, 0, result, buffer.length-startIndex, length-part1);
+    }
+    return result;
+  }
+
+  /** Call this to notify us that no chars before this
+   *  absolute position are needed anymore. */
+  public void freeBefore(int pos) {
+    assert pos >= 0;
+    assert pos <= nextPos;
+    final int newCount = nextPos - pos;
+    assert newCount <= count: "newCount=" + newCount + " count=" + count;
+    assert newCount <= buffer.length: "newCount=" + newCount + " buf.length=" + buffer.length;
+    count = newCount;
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sun Mar  4 13:34:13 2012
@@ -840,6 +840,7 @@ public final class FST<T> {
   }
 
   public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
     final int address = getNodeAddress(node);
     in.pos = address;
     //System.out.println("  readFirstRealTargtArc address="
@@ -936,6 +937,7 @@ public final class FST<T> {
   /** Never returns null, but you should never call this if
    *  arc.isLast() is true. */
   public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
 
     // TODO: can't assert this because we call from readFirstArc
     // assert !flag(arc.flags, BIT_LAST_ARC);
@@ -1019,6 +1021,7 @@ public final class FST<T> {
    *  This returns null if the arc was not found, else the incoming arc. */
   public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
     assert cachedRootArcs != null;
+    assert in.bytes == bytes;
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1225,17 +1228,20 @@ public final class FST<T> {
 
   /** Expert */
   public static abstract class BytesReader extends DataInput {
-    int pos;
+    protected int pos;
+    protected final byte[] bytes;
+    protected BytesReader(byte[] bytes, int pos) {
+      this.bytes = bytes;
+      this.pos = pos;
+    }
     abstract void skip(int byteCount);
     abstract void skip(int base, int byteCount);
   }
 
   final static class ReverseBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ReverseBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override
@@ -1262,11 +1268,9 @@ public final class FST<T> {
   // TODO: can we use just ByteArrayDataInput...?  need to
   // add a .skipBytes to DataInput.. hmm and .setPosition
   final static class ForwardBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ForwardBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java Sun Mar  4 13:34:13 2012
@@ -29,6 +29,8 @@ public class TestSimpleAttributeImpl ext
   public void testAttributes() {
     _TestUtil.assertAttributeReflection(new PositionIncrementAttributeImpl(),
       Collections.singletonMap(PositionIncrementAttribute.class.getName()+"#positionIncrement", 1));
+    _TestUtil.assertAttributeReflection(new PositionLengthAttributeImpl(),
+      Collections.singletonMap(PositionLengthAttribute.class.getName()+"#positionLength", 1));
     _TestUtil.assertAttributeReflection(new FlagsAttributeImpl(),
       Collections.singletonMap(FlagsAttribute.class.getName()+"#flags", 0));
     _TestUtil.assertAttributeReflection(new TypeAttributeImpl(),

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java Sun Mar  4 13:34:13 2012
@@ -0,0 +1,94 @@
+package org.apache.lucene.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.StringReader;
+
+public class TestRollingCharBuffer extends LuceneTestCase {
+
+  public void test() throws Exception {
+    final int ITERS = atLeast(1000);
+    
+    RollingCharBuffer buffer = new RollingCharBuffer();
+
+    for(int iter=0;iter<ITERS;iter++) {
+      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());
+      }
+      buffer.reset(new StringReader(s));
+      int nextRead = 0;
+      int availCount = 0;
+      while(nextRead < s.length()) {
+        if (VERBOSE) {
+          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 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) {
+          final int toFree = random.nextInt(availCount);
+          if (VERBOSE) {
+            System.out.println("    free " + toFree + " (avail=" + (availCount-toFree) + ")");
+          }
+          buffer.freeBefore(nextRead-(availCount-toFree));
+          availCount -= toFree;
+        }
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Sun Mar  4 13:34:13 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;
@@ -83,7 +88,7 @@ public abstract class BaseTokenStreamTes
     }
   }
 
-  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], Integer finalOffset) throws IOException {
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset) throws IOException {
     assertNotNull(output);
     CheckClearAttributesAttribute checkClearAtt = ts.addAttribute(CheckClearAttributesAttribute.class);
     
@@ -107,6 +112,12 @@ public abstract class BaseTokenStreamTes
       assertTrue("has no PositionIncrementAttribute", ts.hasAttribute(PositionIncrementAttribute.class));
       posIncrAtt = ts.getAttribute(PositionIncrementAttribute.class);
     }
+
+    PositionLengthAttribute posLengthAtt = null;
+    if (posLengths != null) {
+      assertTrue("has no PositionLengthAttribute", ts.hasAttribute(PositionLengthAttribute.class));
+      posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
+    }
     
     ts.reset();
     for (int i = 0; i < output.length; i++) {
@@ -116,6 +127,7 @@ public abstract class BaseTokenStreamTes
       if (offsetAtt != null) offsetAtt.setOffset(14584724,24683243);
       if (typeAtt != null) typeAtt.setType("bogusType");
       if (posIncrAtt != null) posIncrAtt.setPositionIncrement(45987657);
+      if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
       
       checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
       assertTrue("token "+i+" does not exist", ts.incrementToken());
@@ -130,6 +142,8 @@ public abstract class BaseTokenStreamTes
         assertEquals("type "+i, types[i], typeAtt.type());
       if (posIncrements != null)
         assertEquals("posIncrement "+i, posIncrements[i], posIncrAtt.getPositionIncrement());
+      if (posLengths != null)
+        assertEquals("posLength "+i, posLengths[i], posLengthAtt.getPositionLength());
       
       // we can enforce some basic things about a few attributes even if the caller doesn't check:
       if (offsetAtt != null) {
@@ -138,14 +152,18 @@ public abstract class BaseTokenStreamTes
         assertTrue("endOffset must be >= startOffset", offsetAtt.endOffset() >= offsetAtt.startOffset());
         if (finalOffset != null) {
           assertTrue("startOffset must be <= finalOffset", offsetAtt.startOffset() <= finalOffset.intValue());
-          assertTrue("endOffset must be <= finalOffset", offsetAtt.endOffset() <= finalOffset.intValue());
+          assertTrue("endOffset must be <= finalOffset: got endOffset=" + offsetAtt.endOffset() + " vs finalOffset=" + finalOffset.intValue(),
+                     offsetAtt.endOffset() <= finalOffset.intValue());
         }
       }
       if (posIncrAtt != null) {
         assertTrue("posIncrement must be >= 0", posIncrAtt.getPositionIncrement() >= 0);
       }
+      if (posLengthAtt != null) {
+        assertTrue("posLength must be >= 1", posLengthAtt.getPositionLength() >= 1);
+      }
     }
-    assertFalse("end of stream", ts.incrementToken());
+    assertFalse("TokenStream has more tokens than expected", ts.incrementToken());
     ts.end();
     if (finalOffset != null)
       assertEquals("finalOffset ", finalOffset.intValue(), offsetAtt.endOffset());
@@ -155,65 +173,81 @@ public abstract class BaseTokenStreamTes
     ts.close();
   }
   
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], Integer finalOffset) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null, finalOffset);
+  }
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null, null);
   }
 
   public static void assertTokenStreamContents(TokenStream ts, String[] output) throws IOException {
-    assertTokenStreamContents(ts, output, null, null, null, null, null);
+    assertTokenStreamContents(ts, output, null, null, null, null, null, null);
   }
   
   public static void assertTokenStreamContents(TokenStream ts, String[] output, String[] types) throws IOException {
-    assertTokenStreamContents(ts, output, null, null, types, null, null);
+    assertTokenStreamContents(ts, output, null, null, types, null, null, null);
   }
   
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int[] posIncrements) throws IOException {
-    assertTokenStreamContents(ts, output, null, null, null, posIncrements, null);
+    assertTokenStreamContents(ts, output, null, null, null, posIncrements, null, null);
   }
   
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[]) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, null);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, null, null);
   }
   
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], Integer finalOffset) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, finalOffset);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, null, finalOffset);
   }
   
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, null);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, null, null);
   }
 
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements, Integer finalOffset) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, finalOffset);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, null, finalOffset);
+  }
+  
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements, int[] posLengths, Integer finalOffset) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, posLengths, finalOffset);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
+  }
+  
+  public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
-    assertAnalyzesTo(a, input, output, null, null, null, null);
+    assertAnalyzesTo(a, input, output, null, null, null, null, null);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, String[] types) throws IOException {
-    assertAnalyzesTo(a, input, output, null, null, types, null);
+    assertAnalyzesTo(a, input, output, null, null, types, null, null);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int[] posIncrements) throws IOException {
-    assertAnalyzesTo(a, input, output, null, null, null, posIncrements);
+    assertAnalyzesTo(a, input, output, null, null, null, posIncrements, null);
+  }
+
+  public static void assertAnalyzesToPositions(Analyzer a, String input, String[] output, int[] posIncrements, int[] posLengths) throws IOException {
+    assertAnalyzesTo(a, input, output, null, null, null, posIncrements, posLengths);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[]) throws IOException {
-    assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, null);
+    assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, null, null);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements) throws IOException {
-    assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, posIncrements);
+    assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, posIncrements, null);
   }
   
 
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
   }
   
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output) throws IOException {
@@ -326,7 +360,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);
@@ -336,10 +370,12 @@ public abstract class BaseTokenStreamTes
       CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
       OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
       PositionIncrementAttribute posIncAtt = ts.hasAttribute(PositionIncrementAttribute.class) ? ts.getAttribute(PositionIncrementAttribute.class) : null;
+      PositionLengthAttribute posLengthAtt = ts.hasAttribute(PositionLengthAttribute.class) ? ts.getAttribute(PositionLengthAttribute.class) : null;
       TypeAttribute typeAtt = ts.hasAttribute(TypeAttribute.class) ? ts.getAttribute(TypeAttribute.class) : null;
       List<String> tokens = new ArrayList<String>();
       List<String> types = new ArrayList<String>();
       List<Integer> positions = new ArrayList<Integer>();
+      List<Integer> positionLengths = new ArrayList<Integer>();
       List<Integer> startOffsets = new ArrayList<Integer>();
       List<Integer> endOffsets = new ArrayList<Integer>();
       ts.reset();
@@ -347,6 +383,7 @@ public abstract class BaseTokenStreamTes
         tokens.add(termAtt.toString());
         if (typeAtt != null) types.add(typeAtt.type());
         if (posIncAtt != null) positions.add(posIncAtt.getPositionIncrement());
+        if (posLengthAtt != null) positionLengths.add(posLengthAtt.getPositionLength());
         if (offsetAtt != null) {
           startOffsets.add(offsetAtt.startOffset());
           endOffsets.add(offsetAtt.endOffset());
@@ -357,11 +394,21 @@ 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);
-        if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
+        if (typeAtt != null && posIncAtt != null && posLengthAtt != null && offsetAtt != null) {
+          // offset + pos + posLength + type
+          assertTokenStreamContents(ts, 
+            tokens.toArray(new String[tokens.size()]),
+            toIntArray(startOffsets),
+            toIntArray(endOffsets),
+            types.toArray(new String[types.size()]),
+            toIntArray(positions),
+            toIntArray(positionLengths),
+            text.length());
+        } else if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
           // offset + pos + type
           assertTokenStreamContents(ts, 
             tokens.toArray(new String[tokens.size()]),
@@ -369,7 +416,18 @@ public abstract class BaseTokenStreamTes
             toIntArray(endOffsets),
             types.toArray(new String[types.size()]),
             toIntArray(positions),
+            null,
             text.length());
+        } else if (posIncAtt != null && posLengthAtt != null && offsetAtt != null) {
+          // offset + pos + posLength
+          assertTokenStreamContents(ts, 
+              tokens.toArray(new String[tokens.size()]),
+              toIntArray(startOffsets),
+              toIntArray(endOffsets),
+              null,
+              toIntArray(positions),
+              toIntArray(positionLengths),
+              text.length());
         } else if (posIncAtt != null && offsetAtt != null) {
           // offset + pos
           assertTokenStreamContents(ts, 
@@ -378,6 +436,7 @@ public abstract class BaseTokenStreamTes
               toIntArray(endOffsets),
               null,
               toIntArray(positions),
+              null,
               text.length());
         } else if (offsetAtt != null) {
           // offset
@@ -387,6 +446,7 @@ public abstract class BaseTokenStreamTes
               toIntArray(endOffsets),
               null,
               null,
+              null,
               text.length());
         } else {
           // terms only
@@ -396,6 +456,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/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/TokenStreamToDot.java Sun Mar  4 13:34:13 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) {
+        // TODO: 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/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java Sun Mar  4 13:34:13 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);

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java?rev=1296805&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/GraphvizFormatter.java Sun Mar  4 13:34:13 2012
@@ -0,0 +1,180 @@
+package org.apache.lucene.analysis.kuromoji;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Position;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Type;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.WrappedPositionArray;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
+
+
+// TODO: would be nice to show 2nd best path in a diff't
+// color...
+
+public class GraphvizFormatter {
+  
+  private final static String BOS_LABEL = "BOS";
+  
+  private final static String EOS_LABEL = "EOS";
+  
+  private final static String FONT_NAME = "Helvetica";
+  
+  private final ConnectionCosts costs;
+  
+  private final Map<String, String> bestPathMap;
+  
+  private final StringBuilder sb = new StringBuilder();
+  
+  public GraphvizFormatter(ConnectionCosts costs) {
+    this.costs = costs;
+    this.bestPathMap = new HashMap<String, String>();
+    sb.append(formatHeader());
+    sb.append("  init [style=invis]\n");
+    sb.append("  init -> 0.0 [label=\"" + BOS_LABEL + "\"]\n");
+  }
+
+  public String finish() {
+    sb.append(formatTrailer());
+    return sb.toString();
+  }
+
+  // Backtraces another incremental fragment:
+  void onBacktrace(KuromojiTokenizer tok, WrappedPositionArray positions, int lastBackTracePos, Position endPosData, int fromIDX, char[] fragment, boolean isEnd) {
+    setBestPathMap(positions, lastBackTracePos, endPosData, fromIDX);
+    sb.append(formatNodes(tok, positions, lastBackTracePos, endPosData, fragment));
+    if (isEnd) {
+      sb.append("  fini [style=invis]\n");
+      sb.append("  ");
+      sb.append(getNodeID(endPosData.pos, fromIDX));
+      sb.append(" -> fini [label=\"" + EOS_LABEL + "\"]");
+    }
+  }
+
+  // Records which arcs make up the best bath:
+  private void setBestPathMap(WrappedPositionArray positions, int startPos, Position endPosData, int fromIDX) {
+    bestPathMap.clear();
+
+    int pos = endPosData.pos;
+    int bestIDX = fromIDX;
+    while (pos > startPos) {
+      final Position posData = positions.get(pos);
+
+      final int backPos = posData.backPos[bestIDX];
+      final int backIDX = posData.backIndex[bestIDX];
+
+      final String toNodeID = getNodeID(pos, bestIDX);
+      final String fromNodeID = getNodeID(backPos, backIDX);
+      
+      assert !bestPathMap.containsKey(fromNodeID);
+      assert !bestPathMap.containsValue(toNodeID);
+      bestPathMap.put(fromNodeID, toNodeID);
+      pos = backPos;
+      bestIDX = backIDX;
+    }
+  }
+  
+  private String formatNodes(KuromojiTokenizer tok, WrappedPositionArray positions, int startPos, Position endPosData, char[] fragment) {
+
+    StringBuilder sb = new StringBuilder();
+    // Output nodes
+    for (int pos = startPos+1; pos <= endPosData.pos; pos++) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        sb.append("  ");
+        sb.append(getNodeID(pos, idx));
+        sb.append(" [label=\"");
+        sb.append(pos);
+        sb.append(": ");
+        sb.append(posData.lastRightID[idx]);
+        sb.append("\"]\n");
+      }
+    }
+
+    // Output arcs
+    for (int pos = endPosData.pos; pos > startPos; pos--) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        final Position backPosData = positions.get(posData.backPos[idx]);
+        final String toNodeID = getNodeID(pos, idx);
+        final String fromNodeID = getNodeID(posData.backPos[idx], posData.backIndex[idx]);
+
+        sb.append("  ");
+        sb.append(fromNodeID);
+        sb.append(" -> ");
+        sb.append(toNodeID);
+
+        final String attrs;
+        if (toNodeID.equals(bestPathMap.get(fromNodeID))) {
+          // This arc is on best path
+          attrs = " color=\"#40e050\" fontcolor=\"#40a050\" penwidth=3 fontsize=20";
+        } else {
+          attrs = "";
+        }
+
+        final Dictionary dict = tok.getDict(posData.backType[idx]);
+        final int wordCost = dict.getWordCost(posData.backID[idx]);
+        final int bgCost = costs.get(backPosData.lastRightID[posData.backIndex[idx]],
+                                     dict.getLeftId(posData.backID[idx]));
+
+        final String surfaceForm = new String(fragment,
+                                              posData.backPos[idx] - startPos,
+                                              pos - posData.backPos[idx]);
+        
+        sb.append(" [label=\"");
+        sb.append(surfaceForm);
+        sb.append(' ');
+        sb.append(wordCost);
+        if (bgCost >= 0) {
+          sb.append('+');
+        }
+        sb.append(bgCost);
+        sb.append("\"");
+        sb.append(attrs);
+        sb.append("]\n");
+      }
+    }
+    return sb.toString();
+  }
+  
+  private String formatHeader() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("digraph viterbi {\n");
+    sb.append("  graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\"];\n");
+    //sb.append("  // A2 paper size\n");
+    //sb.append("  size = \"34.4,16.5\";\n");
+    //sb.append("  // try to fill paper\n");
+    //sb.append("  ratio = fill;\n");
+    sb.append("  edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]\n");
+    sb.append("  node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]\n");
+    
+    return sb.toString();
+  }
+  
+  private String formatTrailer() {
+    return "}";
+  }
+  
+  private String getNodeID(int pos, int idx) {
+    return pos + "." + idx;
+  }
+}

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java?rev=1296805&r1=1296804&r2=1296805&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java Sun Mar  4 13:34:13 2012
@@ -27,21 +27,25 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.cjk.CJKWidthFilter;
 import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.core.StopFilter;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.util.Version;
 
 public class KuromojiAnalyzer extends StopwordAnalyzerBase {
-  private final Segmenter segmenter;
+  private final Mode mode;
   private final Set<String> stoptags;
+  private final UserDictionary userDict;
   
   public KuromojiAnalyzer(Version matchVersion) {
-    this(matchVersion, new Segmenter(), DefaultSetHolder.DEFAULT_STOP_SET, DefaultSetHolder.DEFAULT_STOP_TAGS);
+    this(matchVersion, null, KuromojiTokenizer.DEFAULT_MODE, DefaultSetHolder.DEFAULT_STOP_SET, DefaultSetHolder.DEFAULT_STOP_TAGS);
   }
   
-  public KuromojiAnalyzer(Version matchVersion, Segmenter segmenter, CharArraySet stopwords, Set<String> stoptags) {
+  public KuromojiAnalyzer(Version matchVersion, UserDictionary userDict, Mode mode, CharArraySet stopwords, Set<String> stoptags) {
     super(matchVersion, stopwords);
-    this.segmenter = segmenter;
+    this.userDict = userDict;
+    this.mode = mode;
     this.stoptags = stoptags;
   }
   
@@ -79,7 +83,7 @@ public class KuromojiAnalyzer extends St
   
   @Override
   protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-    Tokenizer tokenizer = new KuromojiTokenizer(this.segmenter, reader);
+    Tokenizer tokenizer = new KuromojiTokenizer(reader, userDict, true, mode);
     TokenStream stream = new KuromojiBaseFormFilter(tokenizer);
     stream = new KuromojiPartOfSpeechStopFilter(true, stream, stoptags);
     stream = new CJKWidthFilter(stream);