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/14 18:34:07 UTC

svn commit: r1244134 - in /lucene/dev/branches/lucene3767: lucene/core/src/java/org/apache/lucene/analysis/ lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/lu...

Author: mikemccand
Date: Tue Feb 14 17:34:06 2012
New Revision: 1244134

URL: http://svn.apache.org/viewvc?rev=1244134&view=rev
Log:
LUCENE-3767: pull out RollingCharBuffer; use new PositionLengthAtt for pos length; fix some nocommits

Added:
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java   (with props)
    lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java   (with props)
Modified:
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/Token.java
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java
    lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java
    lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/TestToken.java
    lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
    lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java
    lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java

Modified: lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/Token.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/Token.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/Token.java Tue Feb 14 17:34:06 2012
@@ -128,7 +128,6 @@ public class Token extends CharTermAttri
   private int flags;
   private Payload payload;
   private int positionIncrement = 1;
-  private int positionLength = 1;
 
   /** Constructs a Token will null text. */
   public Token() {
@@ -271,20 +270,6 @@ public class Token extends CharTermAttri
     return positionIncrement;
   }
 
-  /** @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) {
-    this.positionLength = positionLength;
-  }
-
-  /** Returns the position length of this Token.
-   * @see #setPositionLength    
-   */
-  public int getPositionLength() {
-    return positionLength;
-  }
-
   /** Returns this Token's starting offset, the position of the first character
     corresponding to this token in the source text.
 
@@ -375,7 +360,6 @@ public class Token extends CharTermAttri
     super.clear();
     payload = null;
     positionIncrement = 1;
-    positionLength = 1;
     flags = 0;
     startOffset = endOffset = 0;
     type = DEFAULT_TYPE;
@@ -399,7 +383,6 @@ public class Token extends CharTermAttri
   public Token clone(char[] newTermBuffer, int newTermOffset, int newTermLength, int newStartOffset, int newEndOffset) {
     final Token t = new Token(newTermBuffer, newTermOffset, newTermLength, newStartOffset, newEndOffset);
     t.positionIncrement = positionIncrement;
-    t.positionLength = positionLength;
     t.flags = flags;
     t.type = type;
     if (payload != null)
@@ -418,7 +401,6 @@ public class Token extends CharTermAttri
           endOffset == other.endOffset && 
           flags == other.flags &&
           positionIncrement == other.positionIncrement &&
-          positionLength == other.positionLength &&
           (type == null ? other.type == null : type.equals(other.type)) &&
           (payload == null ? other.payload == null : payload.equals(other.payload)) &&
           super.equals(obj)
@@ -434,7 +416,6 @@ public class Token extends CharTermAttri
     code = code * 31 + endOffset;
     code = code * 31 + flags;
     code = code * 31 + positionIncrement;
-    code = code * 31 + positionLength;
     if (type != null)
       code = code * 31 + type.hashCode();
     if (payload != null)
@@ -446,7 +427,6 @@ public class Token extends CharTermAttri
   private void clearNoTermBuffer() {
     payload = null;
     positionIncrement = 1;
-    positionLength = 1;
     flags = 0;
     startOffset = endOffset = 0;
     type = DEFAULT_TYPE;
@@ -463,7 +443,6 @@ public class Token extends CharTermAttri
     copyBuffer(newTermBuffer, newTermOffset, newTermLength);
     payload = null;
     positionIncrement = 1;
-    positionLength = 1;
     startOffset = newStartOffset;
     endOffset = newEndOffset;
     type = newType;
@@ -552,7 +531,6 @@ public class Token extends CharTermAttri
   public void reinit(Token prototype) {
     copyBuffer(prototype.buffer(), 0, prototype.length());
     positionIncrement = prototype.positionIncrement;
-    positionLength = prototype.positionLength;
     flags = prototype.flags;
     startOffset = prototype.startOffset;
     endOffset = prototype.endOffset;
@@ -568,7 +546,6 @@ public class Token extends CharTermAttri
   public void reinit(Token prototype, String newTerm) {
     setEmpty().append(newTerm);
     positionIncrement = prototype.positionIncrement;
-    positionLength = prototype.positionLength;
     flags = prototype.flags;
     startOffset = prototype.startOffset;
     endOffset = prototype.endOffset;
@@ -586,7 +563,6 @@ public class Token extends CharTermAttri
   public void reinit(Token prototype, char[] newTermBuffer, int offset, int length) {
     copyBuffer(newTermBuffer, offset, length);
     positionIncrement = prototype.positionIncrement;
-    positionLength = prototype.positionLength;
     flags = prototype.flags;
     startOffset = prototype.startOffset;
     endOffset = prototype.endOffset;
@@ -607,7 +583,6 @@ public class Token extends CharTermAttri
       super.copyTo(target);
       ((OffsetAttribute) target).setOffset(startOffset, endOffset);
       ((PositionIncrementAttribute) target).setPositionIncrement(positionIncrement);
-      ((PositionIncrementAttribute) target).setPositionLength(positionLength);
       ((PayloadAttribute) target).setPayload((payload == null) ? null : (Payload) payload.clone());
       ((FlagsAttribute) target).setFlags(flags);
       ((TypeAttribute) target).setType(type);
@@ -620,7 +595,6 @@ public class Token extends CharTermAttri
     reflector.reflect(OffsetAttribute.class, "startOffset", startOffset);
     reflector.reflect(OffsetAttribute.class, "endOffset", endOffset);
     reflector.reflect(PositionIncrementAttribute.class, "positionIncrement", positionIncrement);
-    reflector.reflect(PositionIncrementAttribute.class, "positionLength", positionLength);
     reflector.reflect(PayloadAttribute.class, "payload", payload);
     reflector.reflect(FlagsAttribute.class, "flags", flags);
     reflector.reflect(TypeAttribute.class, "type", type);

Modified: lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java Tue Feb 14 17:34:06 2012
@@ -56,16 +56,4 @@ public interface PositionIncrementAttrib
    * @see #setPositionIncrement
    */
   public int getPositionIncrement();
-
-  // nocommit break into new attr...
-  // nocommit better names...?  getLength?  getSpan?
-  /** @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);
-
-  /** Returns the position length of this Token.
-   * @see #setPositionLength
-   */
-  public int getPositionLength();
 }

Modified: lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java Tue Feb 14 17:34:06 2012
@@ -46,16 +46,16 @@ import org.apache.lucene.util.AttributeI
  */
 public class PositionIncrementAttributeImpl extends AttributeImpl implements PositionIncrementAttribute, Cloneable {
   private int positionIncrement = 1;
-  private int positionLength = 1;
   
   /** Set the position increment. The default value is one.
    *
    * @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: got " + positionIncrement);
+    }
     this.positionIncrement = positionIncrement;
   }
 
@@ -66,27 +66,9 @@ public class PositionIncrementAttributeI
     return positionIncrement;
   }
 
-  /** @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.positionIncrement = 1;
-    this.positionLength = 1;
   }
   
   @Override
@@ -97,8 +79,7 @@ public class PositionIncrementAttributeI
     
     if (other instanceof PositionIncrementAttributeImpl) {
       PositionIncrementAttributeImpl _other = (PositionIncrementAttributeImpl) other;
-      return positionIncrement ==  _other.positionIncrement &&
-        positionLength ==  _other.positionLength;
+      return positionIncrement ==  _other.positionIncrement;
     }
  
     return false;
@@ -106,14 +87,12 @@ public class PositionIncrementAttributeI
 
   @Override
   public int hashCode() {
-    return positionIncrement + positionLength*3947;
+    return positionIncrement;
   }
   
   @Override
   public void copyTo(AttributeImpl target) {
     PositionIncrementAttribute t = (PositionIncrementAttribute) target;
     t.setPositionIncrement(positionIncrement);
-    t.setPositionLength(positionLength);
   }  
-
 }

Added: 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=1244134&view=auto
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java (added)
+++ lucene/dev/branches/lucene3767/lucene/core/src/java/org/apache/lucene/util/RollingCharBuffer.java Tue Feb 14 17:34:06 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.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];
+    // nocommit what if entire buffer is requested...?
+    if (endIndex >= startIndex) {
+      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/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/TestToken.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/TestToken.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/TestToken.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/TestToken.java Tue Feb 14 17:34:06 2012
@@ -253,7 +253,6 @@ public class TestToken extends LuceneTes
         put(OffsetAttribute.class.getName() + "#startOffset", 6);
         put(OffsetAttribute.class.getName() + "#endOffset", 22);
         put(PositionIncrementAttribute.class.getName() + "#positionIncrement", 1);
-        put(PositionIncrementAttribute.class.getName() + "#positionLength", 1);
         put(PayloadAttribute.class.getName() + "#payload", null);
         put(TypeAttribute.class.getName() + "#type", TypeAttribute.DEFAULT_TYPE);
         put(FlagsAttribute.class.getName() + "#flags", 8);

Modified: lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java Tue Feb 14 17:34:06 2012
@@ -27,10 +27,10 @@ public class TestSimpleAttributeImpl ext
 
   // this checks using reflection API if the defaults are correct
   public void testAttributes() {
-    _TestUtil.assertAttributeReflection(new PositionIncrementAttributeImpl(), new HashMap<String,Object>() {{
-      put(PositionIncrementAttribute.class.getName()+"#positionIncrement", 1);
-      put(PositionIncrementAttribute.class.getName()+"#positionLength", 1);
-    }});
+    _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/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=1244134&view=auto
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java (added)
+++ lucene/dev/branches/lucene3767/lucene/core/src/test/org/apache/lucene/util/TestRollingCharBuffer.java Tue Feb 14 17:34:06 2012
@@ -0,0 +1,67 @@
+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++) {
+      String s = _TestUtil.randomUnicodeString(random, 10000);
+      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()) {
+          if (VERBOSE) {
+            System.out.println("    new char");
+          }
+          assertEquals(s.charAt(nextRead), buffer.get(nextRead));
+          nextRead++;
+          availCount++;
+        } else {
+          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));
+        }
+
+        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/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=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene3767/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Tue Feb 14 17:34:06 2012
@@ -83,7 +83,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 +107,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 +122,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 +137,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) {
@@ -145,6 +154,9 @@ public abstract class BaseTokenStreamTes
       if (posIncrAtt != null) {
         assertTrue("posIncrement must be >= 0", posIncrAtt.getPositionIncrement() >= 0);
       }
+      if (posLengthAtt != null) {
+        assertTrue("posLength must be >= 1", posLengthAtt.getPositionLength() >= 1);
+      }
     }
     assertFalse("TokenStream has more tokens than expected", ts.incrementToken());
     ts.end();
@@ -157,64 +169,68 @@ public abstract class BaseTokenStreamTes
   }
   
   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);
+  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());
+  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 {
@@ -316,7 +332,7 @@ public abstract class BaseTokenStreamTes
       if (VERBOSE) {
         System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
       }
-      
+
       int remainder = random.nextInt(10);
       Reader reader = new StringReader(text);
       TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
@@ -324,10 +340,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();
@@ -335,6 +353,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());
@@ -345,11 +364,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 useCharFilter=" + useCharFilter + " text.length()=" + text.length());
+          System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis");
         }
         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()]),
@@ -357,7 +386,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, 
@@ -366,6 +406,7 @@ public abstract class BaseTokenStreamTes
               toIntArray(endOffsets),
               null,
               toIntArray(positions),
+              null,
               text.length());
         } else if (offsetAtt != null) {
           // offset
@@ -375,6 +416,7 @@ public abstract class BaseTokenStreamTes
               toIntArray(endOffsets),
               null,
               null,
+              null,
               text.length());
         } else {
           // terms only

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=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer2.java Tue Feb 14 17:34:06 2012
@@ -38,10 +38,12 @@ import org.apache.lucene.analysis.kuromo
 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;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.RollingCharBuffer;
 
 // TODO: somehow factor out a reusable viterbi search here,
 // so other decompounders/tokenizers can reuse...
@@ -90,7 +92,7 @@ public final class KuromojiTokenizer2 ex
   private final FST.BytesReader userFSTReader;
   private final TokenInfoFST userFST;
 
-  private final WrappedCharArray buffer = new WrappedCharArray();
+  private final RollingCharBuffer buffer = new RollingCharBuffer();
 
   private final WrappedPositionArray positions = new WrappedPositionArray();
 
@@ -99,7 +101,7 @@ public final class KuromojiTokenizer2 ex
   private final boolean extendedMode;
   private final boolean outputCompounds;
 
-  // index of the last character of unknown word:
+  // Index of the last character of unknown word:
   private int unknownWordEndIndex = -1;
 
   // True once we've hit the EOF from the input reader:
@@ -125,6 +127,7 @@ public final class KuromojiTokenizer2 ex
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
   private final BaseFormAttribute basicFormAtt = addAttribute(BaseFormAttribute.class);
   private final PartOfSpeechAttribute posAtt = addAttribute(PartOfSpeechAttribute.class);
   private final ReadingAttribute readingAtt = addAttribute(ReadingAttribute.class);
@@ -199,7 +202,7 @@ public final class KuromojiTokenizer2 ex
 
   @Override
   public void end() {
-    // set final offset
+    // Set final offset
     offsetAtt.setOffset(correctOffset(pos), correctOffset(pos));
   }
 
@@ -244,25 +247,24 @@ public final class KuromojiTokenizer2 ex
     int count;
 
     // maybe single int array * 5?
-    int[] costs = new int[4];
-    // nommit rename to lastRightID or pathRightID or something:
-    int[] nodeID = new int[4];
-    int[] backPos = new int[4];
-    int[] backIndex = new int[4];
-    int[] backID = new int[4];
-    Type[] backType = new Type[4];
+    int[] costs = new int[8];
+    int[] lastRightID = new int[8];
+    int[] backPos = new int[8];
+    int[] backIndex = new int[8];
+    int[] backID = new int[8];
+    Type[] backType = new Type[8];
 
     // Only used when finding 2nd best segmentation under a
     // too-long token:
     int forwardCount;
-    int[] forwardPos = new int[4];
-    int[] forwardID = new int[4];
-    int[] forwardIndex = new int[4];
-    Type[] forwardType = new Type[4];
+    int[] forwardPos = new int[8];
+    int[] forwardID = new int[8];
+    int[] forwardIndex = new int[8];
+    Type[] forwardType = new Type[8];
 
     public void grow() {
       costs = ArrayUtil.grow(costs, 1+count);
-      nodeID = ArrayUtil.grow(nodeID, 1+count);
+      lastRightID = ArrayUtil.grow(lastRightID, 1+count);
       backPos = ArrayUtil.grow(backPos, 1+count);
       backIndex = ArrayUtil.grow(backIndex, 1+count);
       backID = ArrayUtil.grow(backID, 1+count);
@@ -288,20 +290,20 @@ public final class KuromojiTokenizer2 ex
       forwardType = newForwardType;
     }
 
-    public void add(int cost, int nodeID, int backPos, int backIndex, int backID, Type backType) {
+    public void add(int cost, int lastRightID, int backPos, int backIndex, int backID, Type backType) {
       // NOTE: this isn't quite a true Viterbit search,
-      // becase we should check if nodeID is
+      // becase we should check if lastRightID is
       // already present here, and only update if the new
       // cost is less than the current cost, instead of
       // simply appending.  However, that will likely hurt
-      // performance (usually we add a nodeID only once),
+      // performance (usually we add a lastRightID only once),
       // and it means we actually create the full graph
       // intersection instead of a "normal" Viterbi lattice:
       if (count == costs.length) {
         grow();
       }
       this.costs[count] = cost;
-      this.nodeID[count] = nodeID;
+      this.lastRightID[count] = lastRightID;
       this.backPos[count] = backPos;
       this.backIndex[count] = backIndex;
       this.backID[count] = backID;
@@ -336,9 +338,9 @@ public final class KuromojiTokenizer2 ex
     for(int idx=0;idx<fromPosData.count;idx++) {
       // Cost is path cost so far, plus word cost (added at
       // end of loop), plus bigram cost:
-      final int cost = fromPosData.costs[idx] + costs.get(fromPosData.nodeID[idx], leftID);
+      final int cost = fromPosData.costs[idx] + costs.get(fromPosData.lastRightID[idx], leftID);
       if (VERBOSE) {
-        System.out.println("      fromIDX=" + idx + ": cost=" + cost + " (prevCost=" + fromPosData.costs[idx] + " wordCost=" + wordCost + " bgCost=" + costs.get(fromPosData.nodeID[idx], leftID) + " leftID=" + leftID);
+        System.out.println("      fromIDX=" + idx + ": cost=" + cost + " (prevCost=" + fromPosData.costs[idx] + " wordCost=" + wordCost + " bgCost=" + costs.get(fromPosData.lastRightID[idx], leftID) + " leftID=" + leftID);
       }
       if (cost < leastCost) {
         leastCost = cost;
@@ -403,11 +405,11 @@ public final class KuromojiTokenizer2 ex
     inflectionAtt.setToken(token);
     if (token.getPosition() == lastTokenPos) {
       posIncAtt.setPositionIncrement(0);
-      posIncAtt.setPositionLength(token.getPositionLength());
+      posLengthAtt.setPositionLength(token.getPositionLength());
     } else {
       assert token.getPosition() > lastTokenPos;
       posIncAtt.setPositionIncrement(1);
-      posIncAtt.setPositionLength(1);
+      posLengthAtt.setPositionLength(1);
     }
     if (VERBOSE) {
       System.out.println("    incToken: return token=" + token);
@@ -416,129 +418,6 @@ public final class KuromojiTokenizer2 ex
     return true;
   }
 
-  // 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:
-  private static final class WrappedCharArray {
-
-    // TODO: pull out as standalone oal.util class?
-
-    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.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length - nextWrite);
-          System.arraycopy(buffer, 0, newBuffer, buffer.length - nextWrite, nextWrite);
-          nextWrite = buffer.length;
-          //System.out.println("buffer: grow from " + buffer.length + " to " + newBuffer.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 < 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];
-      // nocommit what if entire buffer is requested...?
-      if (endIndex >= startIndex) {
-        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 <= nextPos;
-      count = nextPos - pos;
-      assert count < buffer.length;
-    }
-  }
-
   // TODO: make generic'd version of this "circular array"?
   private static final class WrappedPositionArray {
     private Position[] positions = new Position[8];
@@ -849,7 +728,7 @@ public final class KuromojiTokenizer2 ex
       }
       for(int idx=0;idx<endPosData.count;idx++) {
         // Add EOS cost:
-        final int cost = endPosData.costs[idx] + costs.get(endPosData.nodeID[idx], 0);
+        final int cost = endPosData.costs[idx] + costs.get(endPosData.lastRightID[idx], 0);
         //System.out.println("    idx=" + idx + " cost=" + cost);
         if (cost < leastCost) {
           leastCost = cost;
@@ -961,7 +840,7 @@ public final class KuromojiTokenizer2 ex
         }
       } else {
         // On non-initial positions, we maximize score
-        // across all arriving nodeIDs:
+        // across all arriving lastRightIDs:
         for(int forwardArcIDX=0;forwardArcIDX<posData.forwardCount;forwardArcIDX++) {
           final Type forwardType = posData.forwardType[forwardArcIDX];
           final int toPos = posData.forwardPos[forwardArcIDX];

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=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestKuromojiAnalyzer.java Tue Feb 14 17:34:06 2012
@@ -103,40 +103,44 @@ public class TestKuromojiAnalyzer extend
     */
 
     // Senior software engineer:
-    assertAnalyzesTo(a, "シニアソフトウェアエンジニア",
-                     new String[] { "シニア",
-                                    "シニアソフトウェアエンジニア",
-                                    "ソフトウェア",
-                                    "エンジニア" },
-                     new int[] { 1, 0, 1, 1}
-                     );
+    assertAnalyzesToPositions(a, "シニアソフトウェアエンジニア",
+                              new String[] { "シニア",
+                                             "シニアソフトウェアエンジニア",
+                                             "ソフトウェア",
+                                             "エンジニア" },
+                              new int[] { 1, 0, 1, 1},
+                              new int[] { 1, 3, 1, 1}
+                              );
 
     // Kansai International Airport:
-    assertAnalyzesTo(a, "関西国際空港",
-                     new String[] { "関西",
-                                    "関西国際空港", // zero pos inc
-                                    "国際",
-                                    "空港" },
-                     new int[] {1, 0, 1, 1}
-                     );
+    assertAnalyzesToPositions(a, "関西国際空港",
+                              new String[] { "関西",
+                                             "関西国際空港", // zero pos inc
+                                             "国際",
+                                             "空港" },
+                              new int[] {1, 0, 1, 1},
+                              new int[] {1, 3, 1, 1}
+                              );
 
     // Konika Minolta Holdings; not quite the right
     // segmentation (see LUCENE-3726):
-    assertAnalyzesTo(a, "コニカミノルタホールディングス",
-                     new String[] { "コニカ",
-                                    "コニカミノルタホールディングス", // zero pos inc
-                                    "ミノルタ", 
-                                    "ホールディングス"},
-                     new int[] {1, 0, 1, 1}
-                     );
+    assertAnalyzesToPositions(a, "コニカミノルタホールディングス",
+                              new String[] { "コニカ",
+                                             "コニカミノルタホールディングス", // zero pos inc
+                                             "ミノルタ", 
+                                             "ホールディングス"},
+                              new int[] {1, 0, 1, 1},
+                              new int[] {1, 3, 1, 1}
+                              );
 
     // Narita Airport
-    assertAnalyzesTo(a, "成田空港",
-                     new String[] { "成田",
-                                    "成田空港",
-                                    "空港" },
-                     new int[] {1, 0, 1});
-
+    assertAnalyzesToPositions(a, "成田空港",
+                              new String[] { "成田",
+                                             "成田空港",
+                                             "空港" },
+                              new int[] {1, 0, 1},
+                              new int[] {1, 2, 1}
+                              );
   }
 
   

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=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestQuality.java Tue Feb 14 17:34:06 2012
@@ -33,6 +33,7 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -176,6 +177,7 @@ public class TestQuality extends LuceneT
     tokenizer.reset(new StringReader(unseg));
     CharTermAttribute termAtt = tokenizer.addAttribute(CharTermAttribute.class);
     PositionIncrementAttribute posIncAtt = tokenizer.addAttribute(PositionIncrementAttribute.class);
+    PositionLengthAttribute posLengthAtt = tokenizer.addAttribute(PositionLengthAttribute.class);
     List<Path> paths = new ArrayList<Path>();
     paths.add(new Path());
     
@@ -183,7 +185,7 @@ public class TestQuality extends LuceneT
     int numTokens = 0;
     while(tokenizer.incrementToken()) {
       final int posInc = posIncAtt.getPositionIncrement();
-      final int posLength = posIncAtt.getPositionLength();
+      final int posLength = posLengthAtt.getPositionLength();
       final String token = termAtt.toString();
 
       //System.out.println("  tok=" + token + " numPaths=" + paths.size() + " posLen=" + posLength);

Modified: lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java?rev=1244134&r1=1244133&r2=1244134&view=diff
==============================================================================
--- lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java (original)
+++ lucene/dev/branches/lucene3767/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestSearchMode.java Tue Feb 14 17:34:06 2012
@@ -104,11 +104,14 @@ public class TestSearchMode extends Base
         String sourceText = fields[0];
         String[] expectedTokens = fields[1].split("\\s+");
         int[] expectedPosIncrs = new int[expectedTokens.length];
+        int[] expectedPosLengths = new int[expectedTokens.length];
         for(int tokIDX=0;tokIDX<expectedTokens.length;tokIDX++) {
           if (expectedTokens[tokIDX].endsWith("/0")) {
             expectedTokens[tokIDX] = expectedTokens[tokIDX].replace("/0", "");
+            expectedPosLengths[tokIDX] = expectedTokens.length-1;
           } else {
             expectedPosIncrs[tokIDX] = 1;
+            expectedPosLengths[tokIDX] = 1;
           }
         }
         assertAnalyzesTo(analyzerWithCompounds, sourceText, expectedTokens, expectedPosIncrs);