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

svn commit: r1297785 [7/16] - in /lucene/dev/branches/lucene3795_lsp_spatial_module: ./ dev-tools/eclipse/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/contrib/demo/ dev-tools/maven/lucene/contrib/highlighter/ dev-tools/maven/lucene/...

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilter.java Tue Mar  6 23:17:08 2012
@@ -18,12 +18,13 @@
 package org.apache.lucene.analysis.pattern;
 
 import java.io.IOException;
-import java.util.LinkedList;
+import java.io.Reader;
+import java.io.StringReader;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.lucene.analysis.charfilter.BaseCharFilter;
 import org.apache.lucene.analysis.CharStream;
+import org.apache.lucene.analysis.charfilter.BaseCharFilter;
 
 /**
  * CharFilter that uses a regular expression for the target of replace string.
@@ -48,147 +49,88 @@ import org.apache.lucene.analysis.CharSt
  * @since Solr 1.5
  */
 public class PatternReplaceCharFilter extends BaseCharFilter {
+  @Deprecated
+  public static final int DEFAULT_MAX_BLOCK_CHARS = 10000;
 
   private final Pattern pattern;
   private final String replacement;
-  private final int maxBlockChars;
-  private final String blockDelimiters;
-  public static final int DEFAULT_MAX_BLOCK_CHARS = 10000;
+  private Reader transformedInput;
 
-  private LinkedList<Character> buffer;
-  private int nextCharCounter;
-  private char[] blockBuffer;
-  private int blockBufferLength;
-  private String replaceBlockBuffer;
-  private int replaceBlockBufferOffset;
-  
-  public PatternReplaceCharFilter( Pattern pattern, String replacement, CharStream in ){
-    this( pattern, replacement, DEFAULT_MAX_BLOCK_CHARS, null, in );
-  }
-
-  public PatternReplaceCharFilter( Pattern pattern, String replacement,
-      int maxBlockChars, CharStream in ){
-    this( pattern, replacement, maxBlockChars, null, in );
-  }
-
-  public PatternReplaceCharFilter( Pattern pattern, String replacement,
-      String blockDelimiters, CharStream in ){
-    this( pattern, replacement, DEFAULT_MAX_BLOCK_CHARS, blockDelimiters, in );
-  }
-
-  public PatternReplaceCharFilter( Pattern pattern, String replacement,
-      int maxBlockChars, String blockDelimiters, CharStream in ){
-    super( in );
+  public PatternReplaceCharFilter(Pattern pattern, String replacement, CharStream in) {
+    super(in);
     this.pattern = pattern;
     this.replacement = replacement;
-    if( maxBlockChars < 1 )
-      throw new IllegalArgumentException( "maxBlockChars should be greater than 0, but it is " + maxBlockChars );
-    this.maxBlockChars = maxBlockChars;
-    this.blockDelimiters = blockDelimiters;
-    blockBuffer = new char[maxBlockChars];
-  }
-  
-  private boolean prepareReplaceBlock() throws IOException {
-    while( true ){
-      if( replaceBlockBuffer != null && replaceBlockBuffer.length() > replaceBlockBufferOffset )
-        return true;
-      // prepare block buffer
-      blockBufferLength = 0;
-      while( true ){
-        int c = nextChar();
-        if( c == -1 ) break;
-        blockBuffer[blockBufferLength++] = (char)c;
-        // end of block?
-        boolean foundDelimiter =
-          ( blockDelimiters != null ) &&
-          ( blockDelimiters.length() > 0 ) &&
-          blockDelimiters.indexOf( c ) >= 0;
-        if( foundDelimiter ||
-            blockBufferLength >= maxBlockChars ) break;
-      }
-      // block buffer available?
-      if( blockBufferLength == 0 ) return false;
-      replaceBlockBuffer = getReplaceBlock( blockBuffer, 0, blockBufferLength );
-      replaceBlockBufferOffset = 0;
-    }
   }
 
-  @Override
-  public int read() throws IOException {
-    while( prepareReplaceBlock() ){
-      return replaceBlockBuffer.charAt( replaceBlockBufferOffset++ );
-    }
-    return -1;
+  @Deprecated
+  public PatternReplaceCharFilter(Pattern pattern, String replacement, 
+      int maxBlockChars, String blockDelimiter, CharStream in) {
+    this(pattern, replacement, in);
   }
 
   @Override
   public int read(char[] cbuf, int off, int len) throws IOException {
-    char[] tmp = new char[len];
-    int l = input.read(tmp, 0, len);
-    if (l != -1) {
-      for(int i = 0; i < l; i++)
-        pushLastChar(tmp[i]);
-    }
-    l = 0;
-    for(int i = off; i < off + len; i++) {
-      int c = read();
-      if (c == -1) break;
-      cbuf[i] = (char) c;
-      l++;
+    // Buffer all input on the first call.
+    if (transformedInput == null) {
+      StringBuilder buffered = new StringBuilder();
+      char [] temp = new char [1024];
+      for (int cnt = input.read(temp); cnt > 0; cnt = input.read(temp)) {
+        buffered.append(temp, 0, cnt);
+      }
+      transformedInput = new StringReader(processPattern(buffered).toString());
     }
-    return l == 0 ? -1 : l;
-  }
 
-  private int nextChar() throws IOException {
-    if (buffer != null && !buffer.isEmpty()) {
-      nextCharCounter++;
-      return buffer.removeFirst().charValue();
-    }
-    int c = input.read();
-    if( c != -1 )
-      nextCharCounter++;
-    return c;
+    return transformedInput.read(cbuf, off, len);
   }
 
-  private void pushLastChar(int c) {
-    if (buffer == null) {
-      buffer = new LinkedList<Character>();
-    }
-    buffer.addLast(new Character((char) c));
-  }
-  
-  String getReplaceBlock( String block ){
-    char[] blockChars = block.toCharArray();
-    return getReplaceBlock( blockChars, 0, blockChars.length );
+  @Override
+  protected int correct(int currentOff) {
+    return Math.max(0,  super.correct(currentOff));
   }
-    
-  String getReplaceBlock( char block[], int offset, int length ){
-    StringBuffer replaceBlock = new StringBuffer();
-    String sourceBlock = new String( block, offset, length );
-    Matcher m = pattern.matcher( sourceBlock );
-    int lastMatchOffset = 0, lastDiff = 0;
-    while( m.find() ){
-      m.appendReplacement( replaceBlock, replacement );
-      // record cumulative diff for the offset correction
-      int diff = replaceBlock.length() - lastMatchOffset - lastDiff - ( m.end( 0 ) - lastMatchOffset );
-      if (diff != 0) {
-        int prevCumulativeDiff = getLastCumulativeDiff();
-        if (diff > 0) {
-          for(int i = 0; i < diff; i++){
-            addOffCorrectMap(nextCharCounter - length + m.end( 0 ) + i - prevCumulativeDiff,
-                prevCumulativeDiff - 1 - i);
-          }
+
+  /**
+   * Replace pattern in input and mark correction offsets. 
+   */
+  CharSequence processPattern(CharSequence input) {
+    final Matcher m = pattern.matcher(input);
+
+    final StringBuffer cumulativeOutput = new StringBuffer();
+    int cumulative = 0;
+    int lastMatchEnd = 0;
+    while (m.find()) {
+      final int groupSize = m.end() - m.start();
+      final int skippedSize = m.start() - lastMatchEnd;
+      lastMatchEnd = m.end();
+
+      final int lengthBeforeReplacement = cumulativeOutput.length() + skippedSize;
+      m.appendReplacement(cumulativeOutput, replacement);
+      // Matcher doesn't tell us how many characters have been appended before the replacement.
+      // So we need to calculate it. Skipped characters have been added as part of appendReplacement.
+      final int replacementSize = cumulativeOutput.length() - lengthBeforeReplacement;
+
+      if (groupSize != replacementSize) {
+        if (replacementSize < groupSize) {
+          // The replacement is smaller. 
+          // Add the 'backskip' to the next index after the replacement (this is possibly 
+          // after the end of string, but it's fine -- it just means the last character 
+          // of the replaced block doesn't reach the end of the original string.
+          cumulative += groupSize - replacementSize;
+          int atIndex = lengthBeforeReplacement + replacementSize;
+          // System.err.println(atIndex + "!" + cumulative);
+          addOffCorrectMap(atIndex, cumulative);
         } else {
-          addOffCorrectMap(nextCharCounter - length + m.end( 0 ) + diff - prevCumulativeDiff,
-              prevCumulativeDiff - diff);
+          // The replacement is larger. Every new index needs to point to the last
+          // element of the original group (if any).
+          for (int i = groupSize; i < replacementSize; i++) {
+            addOffCorrectMap(lengthBeforeReplacement + i, --cumulative);
+            // System.err.println((lengthBeforeReplacement + i) + " " + cumulative);
+          }
         }
       }
-      // save last offsets
-      lastMatchOffset = m.end( 0 );
-      lastDiff = diff;
     }
-    // copy remaining of the part of source block
-    m.appendTail( replaceBlock );
-    return replaceBlock.toString();
+
+    // Append the remaining output, no further changes to indices.
+    m.appendTail(cumulativeOutput);
+    return cumulativeOutput;    
   }
 }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java Tue Mar  6 23:17:08 2012
@@ -112,6 +112,8 @@ public final class SynonymFilter extends
 
   private int captureCount;
 
+  // TODO: we should set PositionLengthAttr too...
+
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
   private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/util/CharArrayMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/util/CharArrayMap.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/util/CharArrayMap.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/java/org/apache/lucene/analysis/util/CharArrayMap.java Tue Mar  6 23:17:08 2012
@@ -376,7 +376,7 @@ public class CharArrayMap<V> extends Abs
 
   /** Returns an {@link CharArraySet} view on the map's keys.
    * The set will use the same {@code matchVersion} as this map. */
-  @Override @SuppressWarnings("unchecked")
+  @Override @SuppressWarnings({"unchecked","rawtypes"})
   public final CharArraySet keySet() {
     if (keySet == null) {
       // prevent adding of entries
@@ -508,10 +508,11 @@ public class CharArrayMap<V> extends Abs
     }
     
     @Override
+    @SuppressWarnings("unchecked")
     public boolean contains(Object o) {
       if (!(o instanceof Map.Entry))
         return false;
-      final Map.Entry e = (Map.Entry)o;
+      final Map.Entry<Object,V> e = (Map.Entry<Object,V>)o;
       final Object key = e.getKey();
       final Object val = e.getValue();
       final Object v = get(key);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java Tue Mar  6 23:17:08 2012
@@ -12,6 +12,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Version;
@@ -285,7 +286,7 @@ public class TestClassicAnalyzer extends
                                                                 "content",
                                                                 new BytesRef("another"),
                                                                 false);
-    assertTrue(tps.nextDoc() != DocsEnum.NO_MORE_DOCS);
+    assertTrue(tps.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(1, tps.freq());
     assertEquals(3, tps.nextPosition());
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java Tue Mar  6 23:17:08 2012
@@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
@@ -102,7 +103,7 @@ public class TestKeywordAnalyzer extends
                                  MultiFields.getLiveDocs(reader),
                                  null,
                                  false);
-    assertTrue(td.nextDoc() != DocsEnum.NO_MORE_DOCS);
+    assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     td = _TestUtil.docs(random,
                         reader,
                         "partnum",
@@ -110,7 +111,7 @@ public class TestKeywordAnalyzer extends
                         MultiFields.getLiveDocs(reader),
                         null,
                         false);
-    assertTrue(td.nextDoc() != DocsEnum.NO_MORE_DOCS);
+    assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
   }
 
   // LUCENE-1441

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/pattern/TestPatternReplaceCharFilter.java Tue Mar  6 23:17:08 2012
@@ -20,7 +20,9 @@ package org.apache.lucene.analysis.patte
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
@@ -29,12 +31,108 @@ import org.apache.lucene.analysis.CharSt
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.util._TestUtil;
+import org.junit.Ignore;
 
 /**
  * Tests {@link PatternReplaceCharFilter}
  */
 public class TestPatternReplaceCharFilter extends BaseTokenStreamTestCase {
-  
+  public void testFailingDot() throws IOException {
+    checkOutput(
+        "A. .B.", "\\.[\\s]*", ".",
+        "A..B.",
+        "A..B.");
+  }
+
+  public void testLongerReplacement() throws IOException {
+    checkOutput(
+        "XXabcZZabcYY", "abc", "abcde",
+        "XXabcdeZZabcdeYY",
+        "XXabcccZZabcccYY");
+    checkOutput(
+        "XXabcabcYY", "abc", "abcde",
+        "XXabcdeabcdeYY",
+        "XXabcccabcccYY");
+    checkOutput(
+        "abcabcYY", "abc", "abcde",
+        "abcdeabcdeYY",
+        "abcccabcccYY");
+    checkOutput(
+        "YY", "^", "abcde",
+        "abcdeYY",
+        // Should be: "-----YY" but we're enforcing non-negative offsets.
+        "YYYYYYY");
+    checkOutput(
+        "YY", "$", "abcde",
+        "YYabcde",
+        "YYYYYYY");
+    checkOutput(
+        "XYZ", ".", "abc",
+        "abcabcabc",
+        "XXXYYYZZZ");
+    checkOutput(
+        "XYZ", ".", "$0abc",
+        "XabcYabcZabc",
+        "XXXXYYYYZZZZ");
+  }
+
+  public void testShorterReplacement() throws IOException {
+    checkOutput(
+        "XXabcZZabcYY", "abc", "xy",
+        "XXxyZZxyYY",
+        "XXabZZabYY");
+    checkOutput(
+        "XXabcabcYY", "abc", "xy",
+        "XXxyxyYY",
+        "XXababYY");
+    checkOutput(
+        "abcabcYY", "abc", "xy",
+        "xyxyYY",
+        "ababYY");
+    checkOutput(
+        "abcabcYY", "abc", "",
+        "YY",
+        "YY");
+    checkOutput(
+        "YYabcabc", "abc", "",
+        "YY",
+        "YY");
+  }
+
+  private void checkOutput(String input, String pattern, String replacement,
+      String expectedOutput, String expectedIndexMatchedOutput) throws IOException {
+    CharStream cs = new PatternReplaceCharFilter(pattern(pattern), replacement,
+        CharReader.get(new StringReader(input)));
+
+    StringBuilder output = new StringBuilder();
+    for (int chr = cs.read(); chr > 0; chr = cs.read()) {
+      output.append((char) chr);
+    }
+
+    StringBuilder indexMatched = new StringBuilder();
+    for (int i = 0; i < output.length(); i++) {
+      indexMatched.append((cs.correctOffset(i) < 0 ? "-" : input.charAt(cs.correctOffset(i))));
+    }
+
+    boolean outputGood = expectedOutput.equals(output.toString());
+    boolean indexMatchedGood = expectedIndexMatchedOutput.equals(indexMatched.toString());
+
+    if (!outputGood || !indexMatchedGood || false) {
+      System.out.println("Pattern : " + pattern);
+      System.out.println("Replac. : " + replacement);
+      System.out.println("Input   : " + input);
+      System.out.println("Output  : " + output);
+      System.out.println("Expected: " + expectedOutput);
+      System.out.println("Output/i: " + indexMatched);
+      System.out.println("Expected: " + expectedIndexMatchedOutput);
+      System.out.println();
+    }
+
+    assertTrue("Output doesn't match.", outputGood);
+    assertTrue("Index-matched output doesn't match.", indexMatchedGood);
+  }
+
   //           1111
   // 01234567890123
   // this is test.
@@ -142,9 +240,13 @@ public class TestPatternReplaceCharFilte
   // 012345678901234567890123456789012345678
   //   aa bb cc --- aa bb aa. bb aa   bb cc
   //   aa##bb cc --- aa##bb aa. bb aa##bb cc
+
+  //   aa bb cc --- aa bbbaa. bb aa   b cc
+  
   public void test2blocksMultiMatches() throws IOException {
     final String BLOCK = "  aa bb cc --- aa bb aa. bb aa   bb cc";
-    CharStream cs = new PatternReplaceCharFilter( pattern("(aa)\\s+(bb)"), "$1##$2", ".",
+
+    CharStream cs = new PatternReplaceCharFilter( pattern("(aa)\\s+(bb)"), "$1##$2",
           CharReader.get( new StringReader( BLOCK ) ) );
     TokenStream ts = new MockTokenizer(cs, MockTokenizer.WHITESPACE, false);
     assertTokenStreamContents(ts,
@@ -160,10 +262,10 @@ public class TestPatternReplaceCharFilte
   //  aa b - c . --- b aa . c c b
   public void testChain() throws IOException {
     final String BLOCK = " a bb - ccc . --- bb a . ccc ccc bb";
-    CharStream cs = new PatternReplaceCharFilter( pattern("a"), "aa", ".",
+    CharStream cs = new PatternReplaceCharFilter( pattern("a"), "aa",
         CharReader.get( new StringReader( BLOCK ) ) );
-    cs = new PatternReplaceCharFilter( pattern("bb"), "b", ".", cs );
-    cs = new PatternReplaceCharFilter( pattern("ccc"), "c", ".", cs );
+    cs = new PatternReplaceCharFilter( pattern("bb"), "b", cs );
+    cs = new PatternReplaceCharFilter( pattern("ccc"), "c", cs );
     TokenStream ts = new MockTokenizer(cs, MockTokenizer.WHITESPACE, false);
     assertTokenStreamContents(ts,
         new String[] { "aa", "b", "-", "c", ".", "---", "b", "aa", ".", "c", "c", "b" },
@@ -175,21 +277,60 @@ public class TestPatternReplaceCharFilte
   private Pattern pattern( String p ){
     return Pattern.compile( p );
   }
-  
+
+  /**
+   * A demonstration of how backtracking regular expressions can lead to relatively 
+   * easy DoS attacks.
+   * 
+   * @see "http://swtch.com/~rsc/regexp/regexp1.html"
+   */
+  @Ignore
+  public void testNastyPattern() throws Exception {
+    Pattern p = Pattern.compile("(c.+)*xy");
+    String input = "[;<!--aecbbaa--><    febcfdc fbb = \"fbeeebff\" fc = dd   >\\';<eefceceaa e= babae\" eacbaff =\"fcfaccacd\" = bcced>>><  bccaafe edb = ecfccdff\"   <?</script><    edbd ebbcd=\"faacfcc\" aeca= bedbc ceeaac =adeafde aadccdaf = \"afcc ffda=aafbe &#x16921ed5\"1843785582']";
+    for (int i = 0; i < input.length(); i++) {
+      Matcher matcher = p.matcher(input.substring(0, i));
+      long t = System.currentTimeMillis();
+      if (matcher.find()) {
+        System.out.println(matcher.group());
+      }
+      System.out.println(i + " > " + (System.currentTimeMillis() - t) / 1000.0);
+    }
+  }
+
   /** blast some random strings through the analyzer */
   public void testRandomStrings() throws Exception {
-    Analyzer a = new Analyzer() {
-      @Override
-      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
-        return new TokenStreamComponents(tokenizer, tokenizer);
-      }
+    int numPatterns = atLeast(100);
+    long start = System.currentTimeMillis();
+    long maxTime = 1000 * 2;
+    for (int i = 0; i < numPatterns && start + maxTime > System.currentTimeMillis(); i++) {
+      final Pattern p = randomPattern();
+      final String replacement = _TestUtil.randomSimpleString(random);
+      Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, tokenizer);
+        }
 
-      @Override
-      protected Reader initReader(Reader reader) {
-        return new PatternReplaceCharFilter(Pattern.compile("a"), "b", CharReader.get(reader));
+        @Override
+        protected Reader initReader(Reader reader) {
+          return new PatternReplaceCharFilter(p, replacement, CharReader.get(reader));
+        }
+      };
+      checkRandomData(random, a, 1000 * RANDOM_MULTIPLIER, 
+          /* max input length. don't make it longer -- exponential processing
+           * time for certain patterns. */ 40, true); // only ascii
+    }
+  }
+  
+  public static Pattern randomPattern() {
+    while (true) {
+      try {
+        return Pattern.compile(_TestUtil.randomRegexpishString(random));
+      } catch (PatternSyntaxException ignored) {
+        // if at first you don't succeed...
       }
-    };
-    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+    }
   }
-}
+ }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleFilterTest.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleFilterTest.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleFilterTest.java Tue Mar  6 23:17:08 2012
@@ -59,7 +59,7 @@ public class ShingleFilterTest extends B
         termAtt.copyBuffer(t.buffer(), 0, t.length());
         offsetAtt.setOffset(t.startOffset(), t.endOffset());
         posIncrAtt.setPositionIncrement(t.getPositionIncrement());
-        typeAtt.setType(TypeAttributeImpl.DEFAULT_TYPE);
+        typeAtt.setType(TypeAttribute.DEFAULT_TYPE);
         return true;
       } else {
         return false;
@@ -1018,14 +1018,14 @@ public class ShingleFilterTest extends B
     assertTokenStreamContents(filter,
       new String[]{"please","please divide","divide","divide this","this","this sentence","sentence"},
       new int[]{0,0,7,7,14,14,19}, new int[]{6,13,13,18,18,27,27},
-      new String[]{TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE},
+      new String[]{TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE},
       new int[]{1,0,1,0,1,0,1}
     );
     wsTokenizer.reset(new StringReader("please divide this sentence"));
     assertTokenStreamContents(filter,
       new String[]{"please","please divide","divide","divide this","this","this sentence","sentence"},
       new int[]{0,0,7,7,14,14,19}, new int[]{6,13,13,18,18,27,27},
-      new String[]{TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE,"shingle",TypeAttributeImpl.DEFAULT_TYPE},
+      new String[]{TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE,"shingle",TypeAttribute.DEFAULT_TYPE},
       new int[]{1,0,1,0,1,0,1}
     );
   }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java Tue Mar  6 23:17:08 2012
@@ -35,6 +35,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.English;
@@ -110,7 +111,7 @@ public class TestTeeSinkTokenFilter exte
     termsEnum.next();
     assertEquals(2, termsEnum.totalTermFreq());
     DocsAndPositionsEnum positions = termsEnum.docsAndPositions(null, null, true);
-    assertTrue(positions.nextDoc() != DocsEnum.NO_MORE_DOCS);
+    assertTrue(positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(2, positions.freq());
     positions.nextPosition();
     assertEquals(0, positions.startOffset());
@@ -118,7 +119,7 @@ public class TestTeeSinkTokenFilter exte
     positions.nextPosition();
     assertEquals(8, positions.startOffset());
     assertEquals(12, positions.endOffset());
-    assertEquals(DocsEnum.NO_MORE_DOCS, positions.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, positions.nextDoc());
     r.close();
     dir.close();
   }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ScriptIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ScriptIterator.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ScriptIterator.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ScriptIterator.java Tue Mar  6 23:17:08 2012
@@ -29,6 +29,7 @@ package org.apache.lucene.analysis.icu.s
  */
 
 import com.ibm.icu.lang.UCharacter;
+import com.ibm.icu.lang.UCharacterEnums.ECharacterCategory;
 import com.ibm.icu.lang.UScript;
 import com.ibm.icu.text.UTF16;
 
@@ -110,7 +111,7 @@ final class ScriptIterator {
        * value — should inherit the script value of its base character.
        */
       if (isSameScript(scriptCode, sc)
-          || UCharacter.getType(ch) == UCharacter.NON_SPACING_MARK) {
+          || UCharacter.getType(ch) == ECharacterCategory.NON_SPACING_MARK) {
         index += UTF16.getCharCount(ch);
 
         /*

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiAnalyzer.java Tue Mar  6 23:17:08 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);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/KuromojiTokenizer.java Tue Mar  6 23:17:08 2012
@@ -17,67 +17,1133 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.io.Reader;
-import java.text.BreakIterator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
 import java.util.List;
-import java.util.Locale;
 
-import org.apache.lucene.analysis.kuromoji.tokenattributes.BaseFormAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.InflectionAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.PartOfSpeechAttribute;
-import org.apache.lucene.analysis.kuromoji.tokenattributes.ReadingAttribute;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoFST;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.apache.lucene.analysis.kuromoji.tokenattributes.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+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.RollingCharBuffer;
+import org.apache.lucene.util.fst.FST;
+
+// TODO: somehow factor out a reusable viterbi search here,
+// so other decompounders/tokenizers can reuse...
+
+/* Uses a rolling Viterbi search to find the least cost
+ * segmentation (path) of the incoming characters.  For
+ * tokens that appear to be compound (> length 2 for all
+ * Kanji, or > length 7 for non-Kanji), we see if there is a
+ * 2nd best segmentation of that token after applying
+ * penalties to the long tokens.  If so, and the Mode is
+ * SEARCH_WITH_COMPOUND, we output the alternate
+ * segmentation as well. */
+public final class KuromojiTokenizer extends Tokenizer {
+
+  public static enum Mode {
+    NORMAL, SEARCH, EXTENDED
+  }
+
+  public static final Mode DEFAULT_MODE = Mode.SEARCH;
+
+  enum Type {
+    KNOWN,
+    UNKNOWN,
+    USER
+  }
+
+  private static final boolean VERBOSE = false;
+
+  private static final int SEARCH_MODE_KANJI_LENGTH = 2;
+
+  private static final int SEARCH_MODE_OTHER_LENGTH = 7; // Must be >= SEARCH_MODE_KANJI_LENGTH
+
+  private static final int SEARCH_MODE_KANJI_PENALTY = 3000;
+
+  private static final int SEARCH_MODE_OTHER_PENALTY = 1700;
+
+  // For safety:
+  private static final int MAX_UNKNOWN_WORD_LENGTH = 1024;
+  private static final int MAX_BACKTRACE_GAP = 1024;
+
+  private final EnumMap<Type, Dictionary> dictionaryMap = new EnumMap<Type, Dictionary>(Type.class);
+
+  private final TokenInfoFST fst;
+  private final TokenInfoDictionary dictionary;
+  private final UnknownDictionary unkDictionary;
+  private final ConnectionCosts costs;
+  private final UserDictionary userDictionary;
+  private final CharacterDefinition characterDefinition;
+
+  private final FST.Arc<Long> arc = new FST.Arc<Long>();
+  private final FST.BytesReader fstReader;
+  private final IntsRef wordIdRef = new IntsRef();
+
+  private final FST.BytesReader userFSTReader;
+  private final TokenInfoFST userFST;
+
+  private final RollingCharBuffer buffer = new RollingCharBuffer();
+
+  private final WrappedPositionArray positions = new WrappedPositionArray();
+
+  private final boolean discardPunctuation;
+  private final boolean searchMode;
+  private final boolean extendedMode;
+  private final boolean outputCompounds;
+
+  // Index of the last character of unknown word:
+  private int unknownWordEndIndex = -1;
+
+  // True once we've hit the EOF from the input reader:
+  private boolean end;
+
+  // Last absolute position we backtraced from:
+  private int lastBackTracePos;
+
+  // Position of last token we returned; we use this to
+  // figure out whether to set posIncr to 0 or 1:
+  private int lastTokenPos;
+
+  // Next absolute position to process:
+  private int pos;
+
+  // Already parsed, but not yet passed to caller, tokens:
+  private final List<Token> pending = new ArrayList<Token>();
 
-public final class KuromojiTokenizer extends SegmentingTokenizerBase {
-  private static final BreakIterator proto = BreakIterator.getSentenceInstance(Locale.JAPAN);
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final 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);
   private final InflectionAttribute inflectionAtt = addAttribute(InflectionAttribute.class);
-  private final Segmenter segmenter;
-  
-  private List<Token> tokens; 
-  private int tokenIndex = 0;
-  private int sentenceStart = 0;
-  
-  public KuromojiTokenizer(Reader input) {
-    this(new Segmenter(), input);
-  }
-  
-  public KuromojiTokenizer(Segmenter segmenter, Reader input) {
-    super(input, (BreakIterator) proto.clone());
-    this.segmenter = segmenter;
+
+  public KuromojiTokenizer(Reader input, UserDictionary userDictionary, boolean discardPunctuation, Mode mode) {
+    super(input);
+    dictionary = TokenInfoDictionary.getInstance();
+    fst = dictionary.getFST();
+    unkDictionary = UnknownDictionary.getInstance();
+    characterDefinition = unkDictionary.getCharacterDefinition();
+    this.userDictionary = userDictionary;
+    costs = ConnectionCosts.getInstance();
+    fstReader = fst.getBytesReader(0);
+    if (userDictionary != null) {
+      userFST = userDictionary.getFST();
+      userFSTReader = userFST.getBytesReader(0);
+    } else {
+      userFST = null;
+      userFSTReader = null;
+    }
+    this.discardPunctuation = discardPunctuation;
+    switch(mode){
+      case SEARCH:
+        searchMode = true;
+        extendedMode = false;
+        outputCompounds = true;
+        break;
+      case EXTENDED:
+        searchMode = true;
+        extendedMode = true;
+        outputCompounds = false;
+        break;
+      default:
+        searchMode = false;
+        extendedMode = false;
+        outputCompounds = false;
+        break;
+    }
+    buffer.reset(input);
+
+    resetState();
+
+    dictionaryMap.put(Type.KNOWN, dictionary);
+    dictionaryMap.put(Type.UNKNOWN, unkDictionary);
+    dictionaryMap.put(Type.USER, userDictionary);
+  }
+
+  private GraphvizFormatter dotOut;
+
+  /** Expert: set this to produce graphviz (dot) output of
+   *  the Viterbi lattice */
+  public void setGraphvizFormatter(GraphvizFormatter dotOut) {
+    this.dotOut = dotOut;
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+    buffer.reset(input);
   }
-  
+
   @Override
-  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
-    this.sentenceStart = sentenceStart;
-    // TODO: maybe don't pass 0 here, so kuromoji tracks offsets for us?
-    tokens = segmenter.doTokenize(0, buffer, sentenceStart, sentenceEnd-sentenceStart, true);
-    tokenIndex = 0;
+  public void reset() throws IOException {
+    super.reset();
+    resetState();
+  }
+
+  private void resetState() {
+    positions.reset();
+    unknownWordEndIndex = -1;
+    pos = 0;
+    end = false;
+    lastBackTracePos = 0;
+    lastTokenPos = -1;
+    pending.clear();
+
+    // Add BOS:
+    positions.get(0).add(0, 0, -1, -1, -1, Type.KNOWN);
   }
 
   @Override
-  protected boolean incrementWord() {
-    if (tokenIndex == tokens.size()) {
-      return false;
+  public void end() {
+    // Set final offset
+    offsetAtt.setOffset(correctOffset(pos), correctOffset(pos));
+  }
+
+  // Returns the added cost that a 2nd best segmentation is
+  // allowed to have.  Ie, if we see path with cost X,
+  // ending in a compound word, and this method returns
+  // threshold > 0, then we will also find the 2nd best
+  // segmentation and if its path score is within this
+  // threshold of X, we'll include it in the output:
+  private int computeSecondBestThreshold(int pos, int length) throws IOException {
+    // TODO: maybe we do something else here, instead of just
+    // using the penalty...?  EG we can be more aggressive on
+    // when to also test for 2nd best path
+    return computePenalty(pos, length);
+  }
+
+  private int computePenalty(int pos, int length) throws IOException {
+    if (length > SEARCH_MODE_KANJI_LENGTH) {
+      boolean allKanji = true;
+      // check if node consists of only kanji
+      final int endPos = pos + length;
+      for (int pos2 = pos; pos2 < endPos; pos2++) {
+        if (!characterDefinition.isKanji((char) buffer.get(pos2))) {
+          allKanji = false;
+          break;
+        }				
+      }
+      if (allKanji) {	// Process only Kanji keywords
+        return (length - SEARCH_MODE_KANJI_LENGTH) * SEARCH_MODE_KANJI_PENALTY;
+      } else if (length > SEARCH_MODE_OTHER_LENGTH) {
+        return (length - SEARCH_MODE_OTHER_LENGTH) * SEARCH_MODE_OTHER_PENALTY;								
+      }
     }
-    Token token = tokens.get(tokenIndex);
+    return 0;
+  }
+
+  // Holds all back pointers arriving to this position:
+  final static class Position {
+
+    int pos;
+
+    int count;
+
+    // maybe single int array * 5?
+    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[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);
+      lastRightID = ArrayUtil.grow(lastRightID, 1+count);
+      backPos = ArrayUtil.grow(backPos, 1+count);
+      backIndex = ArrayUtil.grow(backIndex, 1+count);
+      backID = ArrayUtil.grow(backID, 1+count);
+
+      // NOTE: sneaky: grow separately because
+      // ArrayUtil.grow will otherwise pick a different
+      // length than the int[]s we just grew:
+      final Type[] newBackType = new Type[backID.length];
+      System.arraycopy(backType, 0, newBackType, 0, backType.length);
+      backType = newBackType;
+    }
+
+    public void growForward() {
+      forwardPos = ArrayUtil.grow(forwardPos, 1+forwardCount);
+      forwardID = ArrayUtil.grow(forwardID, 1+forwardCount);
+      forwardIndex = ArrayUtil.grow(forwardIndex, 1+forwardCount);
+
+      // NOTE: sneaky: grow separately because
+      // ArrayUtil.grow will otherwise pick a different
+      // length than the int[]s we just grew:
+      final Type[] newForwardType = new Type[forwardPos.length];
+      System.arraycopy(forwardType, 0, newForwardType, 0, forwardType.length);
+      forwardType = newForwardType;
+    }
+
+    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 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 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.lastRightID[count] = lastRightID;
+      this.backPos[count] = backPos;
+      this.backIndex[count] = backIndex;
+      this.backID[count] = backID;
+      this.backType[count] = backType;
+      count++;
+    }
+
+    public void addForward(int forwardPos, int forwardIndex, int forwardID, Type forwardType) {
+      if (forwardCount == this.forwardID.length) {
+        growForward();
+      }
+      this.forwardPos[forwardCount] = forwardPos;
+      this.forwardIndex[forwardCount] = forwardIndex;
+      this.forwardID[forwardCount] = forwardID;
+      this.forwardType[forwardCount] = forwardType;
+      forwardCount++;
+    }
+
+    public void reset() {
+      count = 0;
+      // forwardCount naturally resets after it runs:
+      assert forwardCount == 0: "pos=" + pos + " forwardCount=" + forwardCount;
+    }
+  }
+
+  private void add(Dictionary dict, Position fromPosData, int endPos, int wordID, Type type, boolean addPenalty) throws IOException {
+    final int wordCost = dict.getWordCost(wordID);
+    final int leftID = dict.getLeftId(wordID);
+    int leastCost = Integer.MAX_VALUE;
+    int leastIDX = -1;
+    assert fromPosData.count > 0;
+    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.lastRightID[idx], leftID);
+      if (VERBOSE) {
+        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;
+        leastIDX = idx;
+        if (VERBOSE) {
+          System.out.println("        **");
+        }
+      }
+    }
+
+    leastCost += wordCost;
+
+    if (VERBOSE) {
+      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " leftID=" + leftID + " leastIDX=" + leastIDX + " toPos=" + endPos + " toPos.idx=" + positions.get(endPos).count);
+    }
+
+    if ((addPenalty || (!outputCompounds && searchMode)) && type != Type.USER) {
+      final int penalty = computePenalty(fromPosData.pos, endPos - fromPosData.pos);
+      if (VERBOSE) {
+        if (penalty > 0) {
+          System.out.println("        + penalty=" + penalty + " cost=" + (leastCost+penalty));
+        }
+      }
+      leastCost += penalty;
+    }
+
+    //positions.get(endPos).add(leastCost, dict.getRightId(wordID), fromPosData.pos, leastIDX, wordID, type);
+    assert leftID == dict.getRightId(wordID);
+    positions.get(endPos).add(leastCost, leftID, fromPosData.pos, leastIDX, wordID, type);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+
+    // parse() is able to return w/o producing any new
+    // tokens, when the tokens it had produced were entirely
+    // punctuation.  So we loop here until we get a real
+    // token or we end:
+    while (pending.size() == 0) {
+      if (end) {
+        return false;
+      }
+
+      // Push Viterbi forward some more:
+      parse();
+    }
+
+    final Token token = pending.remove(pending.size()-1);
+
     int position = token.getPosition();
     int length = token.getLength();
     clearAttributes();
-    termAtt.copyBuffer(buffer, sentenceStart + position, length);
-    int startOffset = offset + sentenceStart + position;
-    offsetAtt.setOffset(correctOffset(startOffset), correctOffset(startOffset+length));
+    assert length > 0;
+    //System.out.println("off=" + token.getOffset() + " len=" + length + " vs " + token.getSurfaceForm().length);
+    termAtt.copyBuffer(token.getSurfaceForm(), token.getOffset(), length);
+    offsetAtt.setOffset(correctOffset(position), correctOffset(position+length));
     basicFormAtt.setToken(token);
     posAtt.setToken(token);
     readingAtt.setToken(token);
     inflectionAtt.setToken(token);
-    tokenIndex++;
+    if (token.getPosition() == lastTokenPos) {
+      posIncAtt.setPositionIncrement(0);
+      posLengthAtt.setPositionLength(token.getPositionLength());
+    } else {
+      assert token.getPosition() > lastTokenPos;
+      posIncAtt.setPositionIncrement(1);
+      posLengthAtt.setPositionLength(1);
+    }
+    if (VERBOSE) {
+      System.out.println(Thread.currentThread().getName() + ":    incToken: return token=" + token);
+    }
+    lastTokenPos = token.getPosition();
     return true;
   }
+
+  // TODO: make generic'd version of this "circular array"?
+  // It's a bit tricky because we do things to the Position
+  // (eg, set .pos = N on reuse)...
+  static final class WrappedPositionArray {
+    private Position[] positions = new Position[8];
+
+    public WrappedPositionArray() {
+      for(int i=0;i<positions.length;i++) {
+        positions[i] = new Position();
+      }
+    }
+
+    // Next array index to write to in positions:
+    private int nextWrite;
+
+    // Next position to write:
+    private int nextPos;
+    
+    // How many valid Position instances are held in the
+    // positions array:
+    private int count;
+
+    public void reset() {
+      nextWrite--;
+      while(count > 0) {
+        if (nextWrite == -1) {
+          nextWrite = positions.length - 1;
+        }
+        positions[nextWrite--].reset();
+        count--;
+      }
+      nextWrite = 0;
+      nextPos = 0;
+      count = 0;
+    }
+
+    /** Get Position instance for this absolute position;
+     *  this is allowed to be arbitrarily far "in the
+     *  future" but cannot be before the last freeBefore. */
+    public Position get(int pos) {
+      while(pos >= nextPos) {
+        //System.out.println("count=" + count + " vs len=" + positions.length);
+        if (count == positions.length) {
+          Position[] newPositions = new Position[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          //System.out.println("grow positions " + newPositions.length);
+          System.arraycopy(positions, nextWrite, newPositions, 0, positions.length-nextWrite);
+          System.arraycopy(positions, 0, newPositions, positions.length-nextWrite, nextWrite);
+          for(int i=positions.length;i<newPositions.length;i++) {
+            newPositions[i] = new Position();
+          }
+          nextWrite = positions.length;
+          positions = newPositions;
+        }
+        if (nextWrite == positions.length) {
+          nextWrite = 0;
+        }
+        // Should have already been reset:
+        assert positions[nextWrite].count == 0;
+        positions[nextWrite++].pos = nextPos++;
+        count++;
+      }
+      assert inBounds(pos);
+      final int index = getIndex(pos);
+      assert positions[index].pos == pos;
+      return positions[index];
+    }
+
+    public int getNextPos() {
+      return nextPos;
+    }
+
+    // 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) {
+        index += positions.length;
+      }
+      return index;
+    }
+
+    public void freeBefore(int pos) {
+      final int toFree = count - (nextPos - pos);
+      assert toFree >= 0;
+      assert toFree <= count;
+      int index = nextWrite - count;
+      if (index < 0) {
+        index += positions.length;
+      }
+      for(int i=0;i<toFree;i++) {
+        if (index == positions.length) {
+          index = 0;
+        }
+        //System.out.println("  fb idx=" + index);
+        positions[index].reset();
+        index++;
+      }
+      count -= toFree;
+    }
+  }
+
+  /* Incrementally parse some more characters.  This runs
+   * the viterbi search forwards "enough" so that we
+   * generate some more tokens.  How much forward depends on
+   * the chars coming in, since some chars could cause
+   * longer-lasting ambiguity in the parsing.  Once the
+   * ambiguity is resolved, then we back trace, produce
+   * the pending tokens, and return. */
+  private void parse() throws IOException {
+    if (VERBOSE) {
+      System.out.println("\nPARSE");
+    }
+
+    // Advances over each position (character):
+    while (true) {
+
+      if (buffer.get(pos) == -1) {
+        // End
+        break;
+      }
+
+      final Position posData = positions.get(pos);
+      final boolean isFrontier = positions.getNextPos() == pos+1;
+
+      if (posData.count == 0) {
+        // No arcs arrive here; move to next position:
+        pos++;
+        if (VERBOSE) {
+          System.out.println("    no arcs in; skip");
+        }
+        continue;
+      }
+
+      if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        //  if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        // We are at a "frontier", and only one node is
+        // alive, so whatever the eventual best path is must
+        // come through this node.  So we can safely commit
+        // to the prefix of the best path at this point:
+        backtrace(posData, 0);
+
+        // Re-base cost so we don't risk int overflow:
+        posData.costs[0] = 0;
+
+        if (pending.size() != 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (pos - lastBackTracePos >= MAX_BACKTRACE_GAP) {
+        // Safety: if we've buffered too much, force a
+        // backtrace now:
+        int leastIDX = -1;
+        int leastCost = Integer.MAX_VALUE;
+        for(int idx=0;idx<posData.count;idx++) {
+          //System.out.println("    idx=" + idx + " cost=" + cost);
+          final int cost = posData.costs[idx];
+          if (cost < leastCost) {
+            leastCost = cost;
+            leastIDX = idx;
+          }
+        }
+        backtrace(posData, leastIDX);
+
+        // Re-base cost so we don't risk int overflow:
+        Arrays.fill(posData.costs, 0, posData.count, 0);
+
+        if (pending.size() != 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (VERBOSE) {
+        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos));
+      }
+
+      if (VERBOSE) {
+        System.out.println("    " + posData.count + " arcs in");
+      }
+
+      boolean anyMatches = false;
+
+      // First try user dict:
+      if (userFST != null) {
+        userFST.getFirstArc(arc);
+        int output = 0;
+        for(int posAhead=posData.pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          if (userFST.findTargetArc(ch, arc, arc, posAhead == posData.pos, userFSTReader) == null) {
+            break;
+          }
+          output += arc.output.intValue();
+          if (arc.isFinal()) {
+            if (VERBOSE) {
+              System.out.println("    USER word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1));
+            }
+            add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER, false);
+            anyMatches = true;
+          }
+        }
+      }
+
+      // TODO: we can be more aggressive about user
+      // matches?  if we are "under" a user match then don't
+      // extend KNOWN/UNKNOWN paths?
+
+      if (!anyMatches) {
+        // Next, try known dictionary matches
+        fst.getFirstArc(arc);
+        int output = 0;
+
+        for(int posAhead=posData.pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          //System.out.println("    match " + (char) ch + " posAhead=" + posAhead);
+          
+          if (fst.findTargetArc(ch, arc, arc, posAhead == posData.pos, fstReader) == null) {
+            break;
+          }
+
+          output += arc.output.intValue();
+
+          // Optimization: for known words that are too-long
+          // (compound), we should pre-compute the 2nd
+          // best segmentation and store it in the
+          // dictionary instead of recomputing it each time a
+          // match is found.
+
+          if (arc.isFinal()) {
+            dictionary.lookupWordIds(output + arc.nextFinalOutput.intValue(), wordIdRef);
+            if (VERBOSE) {
+              System.out.println("    KNOWN word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1) + " " + wordIdRef.length + " wordIDs");
+            }
+            for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+              add(dictionary, posData, posAhead+1, wordIdRef.ints[wordIdRef.offset + ofs], Type.KNOWN, false);
+              anyMatches = true;
+            }
+          }
+        }
+      }
+
+      // In the case of normal mode, it doesn't process unknown word greedily.
+
+      if (!searchMode && unknownWordEndIndex > posData.pos) {
+        pos++;
+        continue;
+      }
+
+      final char firstCharacter = (char) buffer.get(pos);
+      if (!anyMatches || characterDefinition.isInvoke(firstCharacter)) {
+
+        // Find unknown match:
+        final int characterId = characterDefinition.getCharacterClass(firstCharacter);
+
+        // NOTE: copied from UnknownDictionary.lookup:
+        int unknownWordLength;
+        if (!characterDefinition.isGroup(firstCharacter)) {
+          unknownWordLength = 1;
+        } else {
+          // Extract unknown word. Characters with the same character class are considered to be part of unknown word
+          unknownWordLength = 1;
+          for (int posAhead=pos+1;unknownWordLength<MAX_UNKNOWN_WORD_LENGTH;posAhead++) {
+            final int ch = buffer.get(posAhead);
+            if (ch == -1) {
+              break;
+            }
+            if (characterId == characterDefinition.getCharacterClass((char) ch)) {
+              unknownWordLength++;    			
+            } else {
+              break;
+            }
+          }
+        }
+
+        unkDictionary.lookupWordIds(characterId, wordIdRef); // characters in input text are supposed to be the same
+        if (VERBOSE) {
+          System.out.println("    UNKNOWN word len=" + unknownWordLength + " " + wordIdRef.length + " wordIDs");
+        }
+        for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+          add(unkDictionary, posData, posData.pos + unknownWordLength, wordIdRef.ints[wordIdRef.offset + ofs], Type.UNKNOWN, false);
+        }
+
+        unknownWordEndIndex = posData.pos + unknownWordLength;
+      }
+
+      pos++;
+    }
+
+    end = true;
+
+    if (pos > 0) {
+
+      final Position endPosData = positions.get(pos);
+      int leastCost = Integer.MAX_VALUE;
+      int leastIDX = -1;
+      if (VERBOSE) {
+        System.out.println("  end: " + endPosData.count + " nodes");
+      }
+      for(int idx=0;idx<endPosData.count;idx++) {
+        // Add EOS cost:
+        final int cost = endPosData.costs[idx] + costs.get(endPosData.lastRightID[idx], 0);
+        //System.out.println("    idx=" + idx + " cost=" + cost + " (pathCost=" + endPosData.costs[idx] + " bgCost=" + costs.get(endPosData.lastRightID[idx], 0) + ") backPos=" + endPosData.backPos[idx]);
+        if (cost < leastCost) {
+          leastCost = cost;
+          leastIDX = idx;
+        }
+      }
+
+      backtrace(endPosData, leastIDX);
+    } else {
+      // No characters in the input string; return no tokens!
+    }
+  }
+
+  // Eliminates arcs from the lattice that are compound
+  // tokens (have a penalty) or are not congruent with the
+  // compound token we've matched (ie, span across the
+  // startPos).  This should be fairly efficient, because we
+  // just keep the already intersected structure of the
+  // graph, eg we don't have to consult the FSTs again:
+
+  private void pruneAndRescore(int startPos, int endPos, int bestStartIDX) throws IOException {
+    if (VERBOSE) {
+      System.out.println("  pruneAndRescore startPos=" + startPos + " endPos=" + endPos + " bestStartIDX=" + bestStartIDX);
+    }
+
+    // First pass: walk backwards, building up the forward
+    // arcs and pruning inadmissible arcs:
+    for(int pos=endPos; pos > startPos; pos--) {
+      final Position posData = positions.get(pos);
+      if (VERBOSE) {
+        System.out.println("    back pos=" + pos);
+      }
+      for(int arcIDX=0;arcIDX<posData.count;arcIDX++) {
+        final int backPos = posData.backPos[arcIDX];
+        if (backPos >= startPos) {
+          // Keep this arc:
+          //System.out.println("      keep backPos=" + backPos);
+          positions.get(backPos).addForward(pos,
+                                            arcIDX,
+                                            posData.backID[arcIDX],
+                                            posData.backType[arcIDX]);
+        } else {
+          if (VERBOSE) {
+            System.out.println("      prune");
+          }
+        }
+      }
+      if (pos != startPos) {
+        posData.count = 0;
+      }
+    }
+
+    // Second pass: walk forward, re-scoring:
+    for(int pos=startPos; pos < endPos; pos++) {
+      final Position posData = positions.get(pos);
+      if (VERBOSE) {
+        System.out.println("    forward pos=" + pos + " count=" + posData.forwardCount);
+      }
+      if (posData.count == 0) {
+        // No arcs arrive here...
+        if (VERBOSE) {
+          System.out.println("      skip");
+        }
+        posData.forwardCount = 0;
+        continue;
+      }
+
+      if (pos == startPos) {
+        // On the initial position, only consider the best
+        // path so we "force congruence":  the
+        // sub-segmentation is "in context" of what the best
+        // path (compound token) had matched:
+        final int rightID;
+        if (startPos == 0) {
+          rightID = 0;
+        } else {
+          rightID = getDict(posData.backType[bestStartIDX]).getRightId(posData.backID[bestStartIDX]);
+        }
+        final int pathCost = posData.costs[bestStartIDX];
+        for(int forwardArcIDX=0;forwardArcIDX<posData.forwardCount;forwardArcIDX++) {
+          final Type forwardType = posData.forwardType[forwardArcIDX];
+          final Dictionary dict2 = getDict(forwardType);
+          final int wordID = posData.forwardID[forwardArcIDX];
+          final int toPos = posData.forwardPos[forwardArcIDX];
+          final int newCost = pathCost + dict2.getWordCost(wordID) + 
+            costs.get(rightID, dict2.getLeftId(wordID)) +
+            computePenalty(pos, toPos-pos);
+          if (VERBOSE) {
+            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos + " cost=" + newCost + " penalty=" + computePenalty(pos, toPos-pos) + " toPos.idx=" + positions.get(toPos).count);
+          }
+          positions.get(toPos).add(newCost,
+                                   dict2.getRightId(wordID),
+                                   pos,
+                                   bestStartIDX,
+                                   wordID,
+                                   forwardType);
+        }
+      } else {
+        // On non-initial positions, we maximize score
+        // across all arriving lastRightIDs:
+        for(int forwardArcIDX=0;forwardArcIDX<posData.forwardCount;forwardArcIDX++) {
+          final Type forwardType = posData.forwardType[forwardArcIDX];
+          final int toPos = posData.forwardPos[forwardArcIDX];
+          if (VERBOSE) {
+            System.out.println("      + " + forwardType + " word " + new String(buffer.get(pos, toPos-pos)) + " toPos=" + toPos);
+          }
+          add(getDict(forwardType),
+              posData,
+              toPos,
+              posData.forwardID[forwardArcIDX],
+              forwardType,
+              true);
+        }
+      }
+      posData.forwardCount = 0;
+    }
+  }
+
+  // Backtrace from the provided position, back to the last
+  // time we back-traced, accumulating the resulting tokens to
+  // the pending list.  The pending list is then in-reverse
+  // (last token should be returned first).
+  private void backtrace(final Position endPosData, final int fromIDX) throws IOException {
+    if (VERBOSE) {
+      System.out.println("\n  backtrace: pos=" + pos + "; " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
+    }
+    final int endPos = endPosData.pos;
+
+    final char[] fragment = buffer.get(lastBackTracePos, endPos-lastBackTracePos);
+
+    if (dotOut != null) {
+      dotOut.onBacktrace(this, positions, lastBackTracePos, endPosData, fromIDX, fragment, end);
+    }
+
+    int pos = endPos;
+    int bestIDX = fromIDX;
+    Token altToken = null;
+
+    // We trace backwards, so this will be the leftWordID of
+    // the token after the one we are now on:
+    int lastLeftWordID = -1;
+
+    int backCount = 0;
+
+    // TODO: sort of silly to make Token instances here; the
+    // back trace has all info needed to generate the
+    // token.  So, we could just directly set the attrs,
+    // from the backtrace, in incrementToken w/o ever
+    // creating Token; we'd have to defer calling freeBefore
+    // until after the bactrace was fully "consumed" by
+    // incrementToken.
+
+    while (pos > lastBackTracePos) {
+      //System.out.println("BT: back pos=" + pos + " bestIDX=" + bestIDX);
+      final Position posData = positions.get(pos);
+      assert bestIDX < posData.count;
+
+      int backPos = posData.backPos[bestIDX];
+      assert backPos >= lastBackTracePos: "backPos=" + backPos + " vs lastBackTracePos=" + lastBackTracePos;
+      int length = pos - backPos;
+      Type backType = posData.backType[bestIDX];
+      int backID = posData.backID[bestIDX];
+      int nextBestIDX = posData.backIndex[bestIDX];
+
+      if (outputCompounds && searchMode && altToken == null && backType != Type.USER) {
+        
+        // In searchMode, if best path had picked a too-long
+        // token, we use the "penalty" to compute the allowed
+        // max cost of an alternate back-trace.  If we find an
+        // alternate back trace with cost below that
+        // threshold, we pursue it instead (but also output
+        // the long token).
+        //System.out.println("    2nd best backPos=" + backPos + " pos=" + pos);
+
+        final int penalty = computeSecondBestThreshold(backPos, pos-backPos);
+        
+        if (penalty > 0) {
+          if (VERBOSE) {
+            System.out.println("  compound=" + new String(buffer.get(backPos, pos-backPos)) + " backPos=" + backPos + " pos=" + pos + " penalty=" + penalty + " cost=" + posData.costs[bestIDX] + " bestIDX=" + bestIDX + " lastLeftID=" + lastLeftWordID);
+          }
+
+          // Use the penalty to set maxCost on the 2nd best
+          // segmentation:
+          int maxCost = posData.costs[bestIDX] + penalty;
+          if (lastLeftWordID != -1) {
+            maxCost += costs.get(getDict(backType).getRightId(backID), lastLeftWordID);
+          }
+
+          // Now, prune all too-long tokens from the graph:
+          pruneAndRescore(backPos, pos,
+                          posData.backIndex[bestIDX]);
+
+          // Finally, find 2nd best back-trace and resume
+          // backtrace there:
+          int leastCost = Integer.MAX_VALUE;
+          int leastIDX = -1;
+          for(int idx=0;idx<posData.count;idx++) {
+            int cost = posData.costs[idx];
+            //System.out.println("    idx=" + idx + " prevCost=" + cost);
+            
+            if (lastLeftWordID != -1) {
+              cost += costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
+                                lastLeftWordID);
+              //System.out.println("      += bgCost=" + costs.get(getDict(posData.backType[idx]).getRightId(posData.backID[idx]),
+              //lastLeftWordID) + " -> " + cost);
+            }
+            //System.out.println("penalty " + posData.backPos[idx] + " to " + pos);
+            //cost += computePenalty(posData.backPos[idx], pos - posData.backPos[idx]);
+            if (cost < leastCost) {
+              //System.out.println("      ** ");
+              leastCost = cost;
+              leastIDX = idx;
+            }
+          }
+          //System.out.println("  leastIDX=" + leastIDX);
+
+          if (VERBOSE) {
+            System.out.println("  afterPrune: " + posData.count + " arcs arriving; leastCost=" + leastCost + " vs threshold=" + maxCost + " lastLeftWordID=" + lastLeftWordID);
+          }
+
+          if (leastIDX != -1 && leastCost <= maxCost && posData.backPos[leastIDX] != backPos) {
+            // We should have pruned the altToken from the graph:
+            assert posData.backPos[leastIDX] != backPos;
+
+            // Save the current compound token, to output when
+            // this alternate path joins back:
+            altToken = new Token(backID,
+                                 fragment,
+                                 backPos - lastBackTracePos,
+                                 length,
+                                 backType,
+                                 backPos,
+                                 getDict(backType));
+
+            // Redirect our backtrace to 2nd best:
+            bestIDX = leastIDX;
+            nextBestIDX = posData.backIndex[bestIDX];
+
+            backPos = posData.backPos[bestIDX];
+            length = pos - backPos;
+            backType = posData.backType[bestIDX];
+            backID = posData.backID[bestIDX];
+            backCount = 0;
+            //System.out.println("  do alt token!");
+            
+          } else {
+            // I think in theory it's possible there is no
+            // 2nd best path, which is fine; in this case we
+            // only output the compound token:
+            //System.out.println("  no alt token! bestIDX=" + bestIDX);
+          }
+        }
+      }
+
+      final int offset = backPos - lastBackTracePos;
+      assert offset >= 0;
+
+      if (altToken != null && altToken.getPosition() >= backPos) {
+
+        // We've backtraced to the position where the
+        // compound token starts; add it now:
+
+        // The pruning we did when we created the altToken
+        // ensures that the back trace will align back with
+        // the start of the altToken:
+        // cannot assert...
+        //assert altToken.getPosition() == backPos: altToken.getPosition() + " vs " + backPos;
+
+        if (VERBOSE) {
+          System.out.println("    add altToken=" + altToken);
+        }
+        if (backCount > 0) {
+          backCount++;
+          altToken.setPositionLength(backCount);
+          pending.add(altToken);
+        } else {
+          // This means alt token was all punct tokens:
+          assert discardPunctuation;
+        }
+        altToken = null;
+      }
+
+      final Dictionary dict = getDict(backType);
+
+      if (backType == Type.USER) {
+
+        // Expand the phraseID we recorded into the actual
+        // segmentation:
+        final int[] wordIDAndLength = userDictionary.lookupSegmentation(backID);
+        int wordID = wordIDAndLength[0];
+        int current = 0;
+        for(int j=1; j < wordIDAndLength.length; j++) {
+          final int len = wordIDAndLength[j];
+          //System.out.println("    add user: len=" + len);
+          pending.add(new Token(wordID+j-1,
+                                fragment,
+                                current + offset,
+                                len,
+                                Type.USER,
+                                current + backPos,
+                                dict));
+          if (VERBOSE) {
+            System.out.println("    add USER token=" + pending.get(pending.size()-1));
+          }
+          current += len;
+        }
+
+        // Reverse the tokens we just added, because when we
+        // serve them up from incrementToken we serve in
+        // reverse:
+        Collections.reverse(pending.subList(pending.size() - (wordIDAndLength.length - 1),
+                                            pending.size()));
+
+        backCount += wordIDAndLength.length-1;
+      } else {
+
+        if (extendedMode && backType == Type.UNKNOWN) {
+          // In EXTENDED mode we convert unknown word into
+          // unigrams:
+          int unigramTokenCount = 0;
+          for(int i=length-1;i>=0;i--) {
+            int charLen = 1;
+            if (i > 0 && Character.isLowSurrogate(fragment[offset+i])) {
+              i--;
+              charLen = 2;
+            }
+            //System.out.println("    extended tok offset="
+            //+ (offset + i));
+            if (!discardPunctuation || !isPunctuation(fragment[offset+i])) {
+              pending.add(new Token(CharacterDefinition.NGRAM,
+                                    fragment,
+                                    offset + i,
+                                    charLen,
+                                    Type.UNKNOWN,
+                                    backPos + i,
+                                    unkDictionary));
+              unigramTokenCount++;
+            }
+          }
+          backCount += unigramTokenCount;
+          
+        } else if (!discardPunctuation || length == 0 || !isPunctuation(fragment[offset])) {
+          pending.add(new Token(backID,
+                                fragment,
+                                offset,
+                                length,
+                                backType,
+                                backPos,
+                                dict));
+          if (VERBOSE) {
+            System.out.println("    add token=" + pending.get(pending.size()-1));
+          }
+          backCount++;
+        } else {
+          if (VERBOSE) {
+            System.out.println("    skip punctuation token=" + new String(fragment, offset, length));
+          }
+        }
+      }
+
+      lastLeftWordID = dict.getLeftId(backID);
+      pos = backPos;
+      bestIDX = nextBestIDX;
+    }
+
+    lastBackTracePos = endPos;
+
+    if (VERBOSE) {
+      System.out.println("  freeBefore pos=" + endPos);
+    }
+    // Notify the circular buffers that we are done with
+    // these positions:
+    buffer.freeBefore(endPos);
+    positions.freeBefore(endPos);
+  }
+
+  Dictionary getDict(Type type) {
+    return dictionaryMap.get(type);
+  }
+
+  private static boolean isPunctuation(char ch) {
+    switch(Character.getType(ch)) {
+      case Character.SPACE_SEPARATOR:
+      case Character.LINE_SEPARATOR:
+      case Character.PARAGRAPH_SEPARATOR:
+      case Character.CONTROL:
+      case Character.FORMAT:
+      case Character.DASH_PUNCTUATION:
+      case Character.START_PUNCTUATION:
+      case Character.END_PUNCTUATION:
+      case Character.CONNECTOR_PUNCTUATION:
+      case Character.OTHER_PUNCTUATION:
+      case Character.MATH_SYMBOL:
+      case Character.CURRENCY_SYMBOL:
+      case Character.MODIFIER_SYMBOL:
+      case Character.OTHER_SYMBOL:
+      case Character.INITIAL_QUOTE_PUNCTUATION:
+      case Character.FINAL_QUOTE_PUNCTUATION:
+        return true;
+      default:
+        return false;
+    }
+  }
 }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Token.java Tue Mar  6 23:17:08 2012
@@ -17,8 +17,8 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Type;
 import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
-import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
 
 public class Token {
   private final Dictionary dictionary;
@@ -30,6 +30,7 @@ public class Token {
   private final int length;
   
   private final int position;
+  private int positionLength;
   
   private final Type type;
   
@@ -40,8 +41,14 @@ public class Token {
     this.length = length;
     this.type = type;
     this.position = position;
+    this.positionLength = positionLength;
     this.dictionary = dictionary;
   }
+
+  @Override
+  public String toString() {
+    return "Token(\"" + new String(surfaceForm, offset, length) + "\" pos=" + position + " type=" + type + " wordId=" + wordId + " leftID=" + dictionary.getLeftId(wordId) + ")";
+  }
   
   /**
    * @return surfaceForm
@@ -144,4 +151,21 @@ public class Token {
   public int getPosition() {
     return position;
   }
+
+  /**
+   * Set the position length (in tokens) of this token.  For normal
+   * tokens this is 1; for compound tokens it's > 1.
+   */
+  public void setPositionLength(int positionLength) {
+    this.positionLength = positionLength;
+  }
+  
+  /**
+   * Get the length (in tokens) of this token.  For normal
+   * tokens this is 1; for compound tokens it's > 1.
+   * @return position length of token
+   */
+  public int getPositionLength() {
+    return positionLength;
+  }
 }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UserDictionary.java Tue Mar  6 23:17:08 2012
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
 import org.apache.lucene.analysis.kuromoji.util.CSVUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.Builder;
@@ -159,6 +160,10 @@ public final class UserDictionary implem
     return found ? toIndexArray(result) : EMPTY_RESULT;
   }
   
+  public TokenInfoFST getFST() {
+    return fst;
+  }
+
   private static final int[][] EMPTY_RESULT = new int[0][];
   
   /**
@@ -181,6 +186,10 @@ public final class UserDictionary implem
     }
     return result.toArray(new int[result.size()][]);
   }
+
+  public int[] lookupSegmentation(int phraseID) {
+    return segmentations[phraseID];
+  }
   
   @Override
   public int getLeftId(int wordId) {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TestExtendedMode.java Tue Mar  6 23:17:08 2012
@@ -25,18 +25,17 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer.Mode;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 
 public class TestExtendedMode extends BaseTokenStreamTestCase {
-  private final Segmenter segmenter = new Segmenter(Mode.EXTENDED);
   private final Analyzer analyzer = new Analyzer() {
     
     @Override
     protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-      Tokenizer tokenizer = new KuromojiTokenizer(segmenter, reader);
+      Tokenizer tokenizer = new KuromojiTokenizer(reader, null, true, Mode.EXTENDED);
       return new TokenStreamComponents(tokenizer, tokenizer);
     }
   };