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/04/08 01:06:13 UTC

svn commit: r1310910 - in /lucene/dev/trunk: lucene/core/src/java/org/apache/lucene/analysis/ lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/ lucene/core/src/test/org/apache/lucene/analysis/ lucene/core/src/test/org/apache/lucene/searc...

Author: mikemccand
Date: Sat Apr  7 23:06:12 2012
New Revision: 1310910

URL: http://svn.apache.org/viewvc?rev=1310910&view=rev
Log:
LUCENE-3873: add MockGraphTokenFilter, inserting random graph tokens

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingBuffer.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java   (with props)
Modified:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/OffsetAttributeImpl.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedTokenStream.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestSynonymMapFilter.java
    lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java Sat Apr  7 23:06:12 2012
@@ -22,6 +22,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.index.Payload;
 import org.apache.lucene.index.DocsAndPositionsEnum; // for javadoc
@@ -121,13 +122,14 @@ import org.apache.lucene.util.AttributeR
 */
 public class Token extends CharTermAttributeImpl 
                    implements TypeAttribute, PositionIncrementAttribute,
-                              FlagsAttribute, OffsetAttribute, PayloadAttribute {
+                              FlagsAttribute, OffsetAttribute, PayloadAttribute, PositionLengthAttribute {
 
   private int startOffset,endOffset;
   private String type = DEFAULT_TYPE;
   private int flags;
   private Payload payload;
   private int positionIncrement = 1;
+  private int positionLength = 1;
 
   /** Constructs a Token will null text. */
   public Token() {
@@ -270,6 +272,20 @@ public class Token extends CharTermAttri
     return positionIncrement;
   }
 
+  /** Set the position length.
+   * @see PositionLengthAttribute */
+  @Override
+  public void setPositionLength(int positionLength) {
+    this.positionLength = positionLength;
+  }
+
+  /** Get the position length.
+   * @see PositionLengthAttribute */
+  @Override
+  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.
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/OffsetAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/OffsetAttributeImpl.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/OffsetAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/OffsetAttributeImpl.java Sat Apr  7 23:06:12 2012
@@ -40,6 +40,10 @@ public class OffsetAttributeImpl extends
   /** Set the starting and ending offset.
     @see #startOffset() and #endOffset()*/
   public void setOffset(int startOffset, int endOffset) {
+    // TODO: check that these are valid!  IE, each should be
+    // >= 0, and endOffset should be >= startOffset.
+    // Problem is this could "break" existing
+    // tokenizers/filters.
     this.startOffset = startOffset;
     this.endOffset = endOffset;
   }

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1310910&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Sat Apr  7 23:06:12 2012
@@ -0,0 +1,382 @@
+package org.apache.lucene.analysis;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+
+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;
+
+public class TestGraphTokenizers extends BaseTokenStreamTestCase {
+
+  // Makes a graph TokenStream from the string; separate
+  // positions with single space, multiple tokens at the same
+  // position with /, and add optional position length with
+  // :.  EG "a b c" is a simple chain, "a/x b c" adds 'x'
+  // over 'a' at position 0 with posLen=1, "a/x:3 b c" adds
+  // 'x' over a with posLen=3.  Tokens are in normal-form!
+  // So, offsets are computed based on the first token at a
+  // given position.  NOTE: each token must be a single
+  // character!  We assume this when computing offsets...
+  
+  // NOTE: all input tokens must be length 1!!!  This means
+  // you cannot turn on MockCharFilter when random
+  // testing...
+
+  private static class GraphTokenizer extends Tokenizer {
+    private List<Token> tokens;
+    private int upto;
+    private int inputLength;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+    private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+
+    public GraphTokenizer(Reader input) {
+      super(input);
+    }
+
+    @Override
+    public void reset() {
+      tokens = null;
+      upto = 0;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (tokens == null) {
+        fillTokens();
+      }
+      //System.out.println("graphTokenizer: incr upto=" + upto + " vs " + tokens.size());
+      if (upto == tokens.size()) {
+        //System.out.println("  END @ " + tokens.size());
+        return false;
+      } 
+      final Token t = tokens.get(upto++);
+      //System.out.println("  return token=" + t);
+      clearAttributes();
+      termAtt.append(t.toString());
+      offsetAtt.setOffset(t.startOffset(), t.endOffset());
+      posIncrAtt.setPositionIncrement(t.getPositionIncrement());
+      posLengthAtt.setPositionLength(t.getPositionLength());
+      return true;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      // NOTE: somewhat... hackish, but we need this to
+      // satisfy BTSTC:
+      final int lastOffset;
+      if (tokens != null && !tokens.isEmpty()) {
+        lastOffset = tokens.get(tokens.size()-1).endOffset();
+      } else {
+        lastOffset = 0;
+      }
+      offsetAtt.setOffset(correctOffset(lastOffset),
+                          correctOffset(inputLength));
+    }
+
+    private void fillTokens() throws IOException {
+      final StringBuilder sb = new StringBuilder();
+      final char[] buffer = new char[256];
+      while (true) {
+        final int count = input.read(buffer);
+        if (count == -1) {
+          break;
+        }
+        sb.append(buffer, 0, count);
+        //System.out.println("got count=" + count);
+      }
+      //System.out.println("fillTokens: " + sb);
+
+      inputLength = sb.length();
+
+      final String[] parts = sb.toString().split(" ");
+
+      tokens = new ArrayList<Token>();
+      int pos = 0;
+      int maxPos = -1;
+      int offset = 0;
+      //System.out.println("again");
+      for(String part : parts) {
+        final String[] overlapped = part.split("/");
+        boolean firstAtPos = true;
+        int minPosLength = Integer.MAX_VALUE;
+        for(String part2 : overlapped) {
+          final int colonIndex = part2.indexOf(':');
+          final String token;
+          final int posLength;
+          if (colonIndex != -1) {
+            token = part2.substring(0, colonIndex);
+            posLength = Integer.parseInt(part2.substring(1+colonIndex));
+          } else {
+            token = part2;
+            posLength = 1;
+          }
+          maxPos = Math.max(maxPos, pos + posLength);
+          minPosLength = Math.min(minPosLength, posLength);
+          final Token t = new Token(token, offset, offset + 2*posLength - 1);
+          t.setPositionLength(posLength);
+          t.setPositionIncrement(firstAtPos ? 1:0);
+          firstAtPos = false;
+          //System.out.println("  add token=" + t + " startOff=" + t.startOffset() + " endOff=" + t.endOffset());
+          tokens.add(t);
+        }
+        pos += minPosLength;
+        offset = 2 * pos;
+      }
+      assert maxPos <= pos: "input string mal-formed: posLength>1 tokens hang over the end";
+    }
+  }
+
+  public void testMockGraphTokenFilterBasic() throws Exception {
+
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+    }
+  }
+
+  public void testMockGraphTokenFilterOnGraphInput() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new GraphTokenizer(reader);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random, a, false, "a/x:3 c/y:2 d e f/z:4 g h i j k");
+    }
+  }
+
+  // Just deletes (leaving hole) token 'a':
+  private final static class RemoveATokens extends TokenFilter {
+    private int pendingPosInc;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+    public RemoveATokens(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      pendingPosInc = 0;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      while (true) {
+        final boolean gotOne = input.incrementToken();
+        if (!gotOne) {
+          return false;
+        } else if (termAtt.toString().equals("a")) {
+          pendingPosInc += posIncAtt.getPositionIncrement();
+        } else {
+          posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+          pendingPosInc = 0;
+          return true;
+        }
+      }
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t);
+            final TokenStream t3 = new RemoveATokens(t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new RemoveATokens(t);
+            final TokenStream t3 = new MockGraphTokenFilter(random, t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkRandomData(random, a, 5, atLeast(1000));
+    }
+  }
+
+  // Two MockGraphTokenFilters
+  public void testDoubleMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random, t);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkRandomData(random, a, 5, atLeast(1000));
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHolesRandom() throws Exception {
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random, t);
+            final TokenStream t2 = new MockHoleInjectingTokenFilter(random, t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkRandomData(random, a, 5, atLeast(1000));
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHolesRandom() throws Exception {
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+            final Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockHoleInjectingTokenFilter(random, t);
+            final TokenStream t2 = new MockGraphTokenFilter(random, t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkRandomData(random, a, 5, atLeast(1000));
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java Sat Apr  7 23:06:12 2012
@@ -59,6 +59,7 @@ public class TestPositionIncrement exten
       @Override
       public TokenStreamComponents createComponents(String fieldName, Reader reader) {
         return new TokenStreamComponents(new Tokenizer(reader) {
+          // TODO: use CannedTokenStream
           private final String[] TOKENS = {"1", "2", "3", "4", "5"};
           private final int[] INCREMENTS = {0, 2, 1, 0, 1};
           private int i = 0;

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingBuffer.java?rev=1310910&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingBuffer.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRollingBuffer.java Sat Apr  7 23:06:12 2012
@@ -0,0 +1,78 @@
+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.
+ */
+
+public class TestRollingBuffer extends LuceneTestCase {
+
+  private static class Position implements RollingBuffer.Resettable {
+    public int pos;
+
+    @Override
+    public void reset() {
+      pos = -1;
+    }
+  }
+
+  public void test() {
+    
+    final RollingBuffer<Position> buffer = new RollingBuffer<Position>() {
+      @Override
+      protected Position newInstance() {
+        final Position pos = new Position();
+        pos.pos = -1;
+        return pos;
+      }
+    };
+
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      int freeBeforePos = 0;
+      final int maxPos = atLeast(10000);
+      final FixedBitSet posSet = new FixedBitSet(maxPos + 1000);
+      int posUpto = 0;
+      while (freeBeforePos < maxPos) {
+        if (random.nextInt(4) == 1) {
+          final int limit = rarely() ? 1000 : 20;
+          final int inc = random.nextInt(limit);
+          final int pos = freeBeforePos + inc;
+          posUpto = Math.max(posUpto, pos);
+          if (VERBOSE) {
+            System.out.println("  check pos=" + pos + " posUpto=" + posUpto);
+          }
+          final Position posData = buffer.get(pos);
+          if (!posSet.getAndSet(pos)) {
+            assertEquals(-1, posData.pos);
+            posData.pos = pos;
+          } else {
+            assertEquals(pos, posData.pos);
+          }
+        } else {
+          if (posUpto > freeBeforePos) {
+            freeBeforePos += random.nextInt(posUpto - freeBeforePos);
+          }
+          if (VERBOSE) {
+            System.out.println("  freeBeforePos=" + freeBeforePos);
+          }
+          buffer.freeBefore(freeBeforePos);
+        }          
+      }
+
+      buffer.reset();
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedTokenStream.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedTokenStream.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedTokenStream.java Sat Apr  7 23:06:12 2012
@@ -23,6 +23,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 
 /**
  * TokenStream from a canned list of Tokens.
@@ -32,10 +33,11 @@ public final class CannedTokenStream ext
   private int upto = 0;
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   private final PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
   
-  public CannedTokenStream(Token[] tokens) {
+  public CannedTokenStream(Token... tokens) {
     this.tokens = tokens;
   }
   
@@ -49,6 +51,7 @@ public final class CannedTokenStream ext
       termAtt.setEmpty();
       termAtt.append(token.toString());
       posIncrAtt.setPositionIncrement(token.getPositionIncrement());
+      posLengthAtt.setPositionLength(token.getPositionLength());
       offsetAtt.setOffset(token.startOffset(), token.endOffset());
       payloadAtt.setPayload(token.getPayload());
       return true;

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java?rev=1310910&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java Sat Apr  7 23:06:12 2012
@@ -0,0 +1,318 @@
+package org.apache.lucene.analysis;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+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.AttributeSource;
+import org.apache.lucene.util.RollingBuffer;
+import org.apache.lucene.util._TestUtil;
+
+// TODO: sometimes remove tokens too...?
+
+/** Randomly inserts overlapped (posInc=0) tokens with
+ *  posLength sometimes > 1.  The chain must have
+ *  an OffsetAttribute.  */
+
+public final class MockGraphTokenFilter extends TokenFilter {
+
+  private static boolean DEBUG = false;
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  private final long seed;
+  private Random random;
+
+  // Don't init to -1 (caller must first call reset):
+  private int inputPos;
+  private int outputPos;
+  // Don't init to -1 (caller must first call reset):
+  private int lastOutputPos;
+  private boolean end;
+
+  private final class Position implements RollingBuffer.Resettable {
+    final List<AttributeSource.State> states = new ArrayList<AttributeSource.State>();
+    int nextRead;
+
+    // Any token leaving from this position should have this startOffset:
+    int startOffset = -1;
+
+    // Any token arriving to this positoin should have this endOffset:
+    int endOffset = -1;
+
+    @Override
+    public void reset() {
+      states.clear();
+      nextRead = 0;
+      startOffset = -1;
+      endOffset = -1;
+    }
+
+    public void captureState() throws IOException {
+      assert startOffset == offsetAtt.startOffset();
+      states.add(MockGraphTokenFilter.this.captureState());
+    }
+  }
+
+  private final RollingBuffer<Position> positions = new RollingBuffer<Position>() {
+    @Override
+    protected Position newInstance() {
+      return new Position();
+    }
+  };
+
+  public MockGraphTokenFilter(Random random, TokenStream input) {
+    super(input);
+    seed = random.nextLong();
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    end = false;
+    positions.reset();
+    // NOTE: must be "deterministically random" because
+    // BaseTokenStreamTestCase pulls tokens twice on the
+    // same input and asserts they are the same:
+    this.random = new Random(seed);
+    inputPos = -1;
+    outputPos = 0;
+    lastOutputPos = -1;
+  }
+
+  private enum TOKEN_POS {SAME, NEXT, END};
+
+  private TOKEN_POS nextInputToken() throws IOException {
+    assert !end;
+    if (DEBUG) {
+      System.out.println("  call input.incr");
+    }
+    final boolean result = input.incrementToken();
+    if (result) {
+      final int posInc = posIncAtt.getPositionIncrement();
+      final int posLength = posLengthAtt.getPositionLength();
+
+      // NOTE: when posLength > 1, we have a hole... we
+      // don't allow injected tokens to start or end
+      // "inside" a hole, so we don't need to make up
+      // offsets inside it
+
+      assert inputPos != -1 || posInc > 0;
+      inputPos += posInc;
+      if (DEBUG) {
+        System.out.println("    got token term=" + termAtt + " posLength=" + posLength + " posInc=" + posInc + " inputPos=" + inputPos);
+      }
+      final Position posData = positions.get(inputPos);
+      if (posInc == 0) {
+        assert posData.startOffset == offsetAtt.startOffset();
+      } else {
+        assert posData.startOffset == -1;
+        posData.startOffset = offsetAtt.startOffset();
+        if (DEBUG) {
+          System.out.println("    record startOffset[" + inputPos + "]=" + posData.startOffset);
+        }
+      }
+
+      final Position posEndData = positions.get(inputPos + posLength);
+      if (posEndData.endOffset == -1) {
+        // First time we are seeing a token that
+        // arrives to this position: record the
+        // endOffset
+        posEndData.endOffset = offsetAtt.endOffset();
+        if (DEBUG) {
+          System.out.println("    record endOffset[" + (inputPos+posLength) + "]=" + posEndData.endOffset);
+        }
+      } else {
+        // We've already seen a token arriving there;
+        // make sure its endOffset is the same (NOTE:
+        // some tokenizers, eg WDF, will fail
+        // this...):
+        assert posEndData.endOffset == offsetAtt.endOffset(): "posEndData.endOffset=" + posEndData.endOffset + " vs offsetAtt.endOffset()=" + offsetAtt.endOffset();
+      }
+      if (posInc == 0) {
+        return TOKEN_POS.SAME;
+      } else {
+        return TOKEN_POS.NEXT;
+      }
+    } else {
+      if (DEBUG) {
+        System.out.println("    got END");
+      }
+      return TOKEN_POS.END;
+    }
+  }
+
+  private void pushOutputPos() {
+    posIncAtt.setPositionIncrement(outputPos - lastOutputPos);
+    if (DEBUG) {
+      System.out.println("  pushOutputPos: set posInc=" + posIncAtt.getPositionIncrement());
+    }
+    lastOutputPos = outputPos;
+    positions.freeBefore(outputPos);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+
+    if (DEBUG) {
+      System.out.println("MockGraphTF.incr inputPos=" + inputPos + " outputPos=" + outputPos);
+    }
+
+    while (true) {
+      final Position posData = positions.get(outputPos);
+      if (posData.nextRead < posData.states.size()) {
+        // Serve up all buffered tokens from this position:
+        if (DEBUG) {
+          System.out.println("  restore buffered nextRead=" + posData.nextRead + " vs " + posData.states.size());
+        }
+        restoreState(posData.states.get(posData.nextRead++));
+        if (DEBUG) {
+          System.out.println("    term=" + termAtt + " outputPos=" + outputPos);
+        }
+        pushOutputPos();
+        return true;
+      }
+
+      boolean tokenPending = false;
+
+      final int prevInputPos = inputPos;
+
+      if (inputPos == -1 || inputPos == outputPos) {
+        // We've used up the buffered tokens; pull the next
+        // input token:
+        if (end) {
+          return false;
+        }
+        final TOKEN_POS result = nextInputToken();
+        if (result == TOKEN_POS.SAME) {
+          return true;
+        } else if (result == TOKEN_POS.NEXT) {
+          tokenPending = true;
+        } else {
+          // NOTE: we don't set end=true here... because we
+          // are immediately passing through "the end" to
+          // caller (return false), and caller must not call
+          // us again:
+          return false;
+        }
+      } else {
+        assert inputPos > outputPos;
+        if (DEBUG) {
+          System.out.println("  done @ outputPos=" + outputPos);
+        }
+      }
+
+      // We're done (above) serving up all tokens leaving
+      // from the same position; now maybe insert a token.
+      // Note that we may insert more than one token leaving
+      // from this position.  We only inject tokens at
+      // positions where we've seen at least one input token
+      // (ie, we cannot inject inside holes):
+
+      if (prevInputPos != -1  && positions.get(outputPos).startOffset != -1 && random.nextInt(7) == 5) {
+        if (DEBUG) {
+          System.out.println("  inject @ outputPos=" + outputPos);
+        }
+
+        if (tokenPending) {
+          positions.get(inputPos).captureState();
+        }
+        final int posLength = _TestUtil.nextInt(random, 1, 5);
+        final Position posEndData = positions.get(outputPos + posLength);
+
+        // Pull enough tokens until we discover what our
+        // endOffset should be:
+        while (!end && posEndData.endOffset == -1 && inputPos <= (outputPos + posLength)) {
+          if (DEBUG) {
+            System.out.println("  lookahead [endPos=" + (outputPos + posLength) + "]...");
+          }
+          final TOKEN_POS result = nextInputToken();
+          if (result != TOKEN_POS.END) {
+            positions.get(inputPos).captureState();
+          } else {
+            end = true;
+            if (DEBUG) {
+              System.out.println("    force end lookahead");
+            }
+            break;
+          }
+        }
+
+        // TODO: really, here, on hitting end-of-tokens,
+        // we'd like to know the ending "posInc", and allow
+        // our token to extend up until that.  But: a
+        // TokenFilter is not allowed to call end() from
+        // within its incrementToken, so we can't do that.
+        // It may have been better if the ending
+        // posInc/offsets were set when incrementToken
+        // returned false (ie, without having to call the
+        // special end method):
+
+        if (posEndData.endOffset != -1) {
+          assert posEndData.endOffset != -1;
+          clearAttributes();
+          posLengthAtt.setPositionLength(posLength);
+          termAtt.append(_TestUtil.randomUnicodeString(random));
+          pushOutputPos();
+          offsetAtt.setOffset(positions.get(outputPos).startOffset,
+                              positions.get(outputPos + posLength).endOffset);
+          if (DEBUG) {
+            System.out.println("  inject: outputPos=" + outputPos + " startOffset=" + offsetAtt.startOffset() +
+                               " endOffset=" + offsetAtt.endOffset() +
+                               " posLength=" + posLengthAtt.getPositionLength());
+          }
+          // TODO: set TypeAtt too?
+          return true;
+
+        } else {
+          // Either, we hit the end of the tokens (ie, our
+          // attempted posLength is too long because it
+          // hangs out over the end), or, our attempted
+          // posLength ended in the middle of a hole; just
+          // skip injecting in these cases.  We will still
+          // test these cases by having a StopFilter after
+          // MockGraphTokenFilter...
+        }
+
+      } else if (tokenPending) {
+        outputPos = inputPos;
+        if (DEBUG) {
+          System.out.println("  pass-through");
+        }
+        pushOutputPos();
+        return true;
+      } else {
+        // We are skipping over a hole (posInc > 1) from our input:
+        outputPos++;
+        if (DEBUG) {
+          System.out.println("  incr outputPos=" + outputPos);
+        }
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java?rev=1310910&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java Sat Apr  7 23:06:12 2012
@@ -0,0 +1,62 @@
+package org.apache.lucene.analysis;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util._TestUtil;
+
+// Randomly injects holes:
+public final class MockHoleInjectingTokenFilter extends TokenFilter {
+
+  private final long randomSeed;
+  private Random random;
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+  public MockHoleInjectingTokenFilter(Random random, TokenStream in) {
+    super(in);
+    randomSeed = random.nextLong();
+  }
+  
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    random = new Random(randomSeed);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (input.incrementToken()) {
+      final int posInc = posIncAtt.getPositionIncrement();
+      if (posInc > 0 && random.nextInt(5) == 3) {
+        posIncAtt.setPositionIncrement(posInc + _TestUtil.nextInt(random, 1, 5));
+        // TODO: should we tweak offsets...?
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  // TODO: end?
+}
+
+

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java?rev=1310910&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RollingBuffer.java Sat Apr  7 23:06:12 2012
@@ -0,0 +1,130 @@
+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.
+ */
+
+// TODO: probably move this to core at some point (eg,
+// cutover kuromoji, synfilter, LookaheadTokenFilter)
+
+/** Acts like forever growing T[], but internally uses a
+ *  circular buffer to reuse instances of T.
+ * 
+ *  @lucene.internal */
+public abstract class RollingBuffer<T extends RollingBuffer.Resettable> {
+
+  public static interface Resettable {
+    public void reset();
+  }
+
+  @SuppressWarnings("unchecked") private T[] buffer = (T[]) new RollingBuffer.Resettable[8];
+
+  // Next array index to write to:
+  private int nextWrite;
+
+  // Next position to write:
+  private int nextPos;
+
+  // How many valid Position are held in the
+  // array:
+  private int count;
+
+  public RollingBuffer() {
+    for(int idx=0;idx<buffer.length;idx++) {
+      buffer[idx] = newInstance();
+    }
+  }
+
+  protected abstract T newInstance();
+
+  public void reset() {
+    nextWrite--;
+    while (count > 0) {
+      if (nextWrite == -1) {
+        nextWrite = buffer.length - 1;
+      }
+      buffer[nextWrite--].reset();
+      count--;
+    }
+    nextWrite = 0;
+    nextPos = 0;
+    count = 0;
+  }
+
+  // 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 += buffer.length;
+    }
+    return index;
+  }
+
+  /** Get T instance for this absolute position;
+   *  this is allowed to be arbitrarily far "in the
+   *  future" but cannot be before the last freeBefore. */
+  public T get(int pos) {
+    //System.out.println("RA.get pos=" + pos + " nextPos=" + nextPos + " nextWrite=" + nextWrite + " count=" + count);
+    while (pos >= nextPos) {
+      if (count == buffer.length) {
+        @SuppressWarnings("unchecked") T[] newBuffer = (T[]) new Resettable[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        //System.out.println("  grow length=" + newBuffer.length);
+        System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length-nextWrite);
+        System.arraycopy(buffer, 0, newBuffer, buffer.length-nextWrite, nextWrite);
+        for(int i=buffer.length;i<newBuffer.length;i++) {
+          newBuffer[i] = newInstance();
+        }
+        nextWrite = buffer.length;
+        buffer = newBuffer;
+      }
+      if (nextWrite == buffer.length) {
+        nextWrite = 0;
+      }
+      // Should have already been reset:
+      nextWrite++;
+      nextPos++;
+      count++;
+    }
+    assert inBounds(pos);
+    final int index = getIndex(pos);
+    //System.out.println("  pos=" + pos + " nextPos=" + nextPos + " -> index=" + index);
+    //assert buffer[index].pos == pos;
+    return buffer[index];
+  }
+
+  public void freeBefore(int pos) {
+    final int toFree = count - (nextPos - pos);
+    assert toFree >= 0;
+    assert toFree <= count: "toFree=" + toFree + " count=" + count;
+    int index = nextWrite - count;
+    if (index < 0) {
+      index += buffer.length;
+    }
+    for(int i=0;i<toFree;i++) {
+      if (index == buffer.length) {
+        index = 0;
+      }
+      //System.out.println("  fb idx=" + index);
+      buffer[index].reset();
+      index++;
+    }
+    count -= toFree;
+  }
+}

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestStandardAnalyzer.java Sat Apr  7 23:06:12 2012
@@ -1,17 +1,5 @@
 package org.apache.lucene.analysis.core;
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.analysis.standard.StandardTokenizer;
-import org.apache.lucene.util.Version;
-
-import java.io.IOException;
-import java.io.Reader;
-import java.io.StringReader;
-import java.util.Arrays;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -29,6 +17,20 @@ import java.util.Arrays;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockGraphTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.standard.StandardTokenizer;
+import org.apache.lucene.util.Version;
+
 public class TestStandardAnalyzer extends BaseTokenStreamTestCase {
   
   public void testHugeDoc() throws IOException {
@@ -247,4 +249,18 @@ public class TestStandardAnalyzer extend
   public void testRandomHugeStrings() throws Exception {
     checkRandomData(random, new StandardAnalyzer(TEST_VERSION_CURRENT), 200*RANDOM_MULTIPLIER, 8192);
   }
+
+  // Adds random graph after:
+  public void testRandomHugeStringsGraphAfter() throws Exception {
+    checkRandomData(random,
+                    new Analyzer() {
+                      @Override
+                      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+                        Tokenizer tokenizer = new StandardTokenizer(TEST_VERSION_CURRENT, reader);
+                        TokenStream tokenStream = new MockGraphTokenFilter(random, tokenizer);
+                        return new TokenStreamComponents(tokenizer, tokenStream);
+                      }
+                    },
+                    200*RANDOM_MULTIPLIER, 8192);
+  }
 }

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestSynonymMapFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestSynonymMapFilter.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestSynonymMapFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestSynonymMapFilter.java Sat Apr  7 23:06:12 2012
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.BaseTo
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.MockGraphTokenFilter;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
 import org.apache.lucene.analysis.tokenattributes.*;
 import org.apache.lucene.util.CharsRef;
@@ -430,6 +432,57 @@ public class TestSynonymMapFilter extend
     }
   }
   
+  // Adds MockGraphTokenFilter before SynFilter:
+  public void testRandom2GraphBefore() throws Exception {
+    final int numIters = atLeast(10);
+    for (int i = 0; i < numIters; i++) {
+      b = new SynonymMap.Builder(random.nextBoolean());
+      final int numEntries = atLeast(10);
+      for (int j = 0; j < numEntries; j++) {
+        add(randomNonEmptyString(), randomNonEmptyString(), random.nextBoolean());
+      }
+      final SynonymMap map = b.build();
+      final boolean ignoreCase = random.nextBoolean();
+      
+      final Analyzer analyzer = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
+          TokenStream graph = new MockGraphTokenFilter(random, tokenizer);
+          return new TokenStreamComponents(tokenizer, new SynonymFilter(graph, map, ignoreCase));
+        }
+      };
+
+      checkRandomData(random, analyzer, 1000*RANDOM_MULTIPLIER);
+    }
+  }
+
+  // Adds MockGraphTokenFilter after SynFilter:
+  public void testRandom2GraphAfter() throws Exception {
+    final int numIters = atLeast(10);
+    for (int i = 0; i < numIters; i++) {
+      b = new SynonymMap.Builder(random.nextBoolean());
+      final int numEntries = atLeast(10);
+      for (int j = 0; j < numEntries; j++) {
+        add(randomNonEmptyString(), randomNonEmptyString(), random.nextBoolean());
+      }
+      final SynonymMap map = b.build();
+      final boolean ignoreCase = random.nextBoolean();
+      
+      final Analyzer analyzer = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
+          TokenStream syns = new SynonymFilter(tokenizer, map, ignoreCase);
+          TokenStream graph = new MockGraphTokenFilter(random, syns);
+          return new TokenStreamComponents(tokenizer, graph);
+        }
+      };
+
+      checkRandomData(random, analyzer, 1000*RANDOM_MULTIPLIER);
+    }
+  }
+  
   public void testEmptyTerm() throws IOException {
     final int numIters = atLeast(10);
     for (int i = 0; i < numIters; i++) {
@@ -662,7 +715,6 @@ public class TestSynonymMapFilter extend
     final boolean keepOrig = false;
     // b hangs off the end (no input token under it):
     add("a", "a b", keepOrig);
-    final SynonymMap map = b.build();
     tokensIn = new MockTokenizer(new StringReader("a"),
                                  MockTokenizer.WHITESPACE,
                                  true);
@@ -673,8 +725,8 @@ public class TestSynonymMapFilter extend
     tokensIn.close();
 
     tokensOut = new SynonymFilter(tokensIn,
-                                     b.build(),
-                                     true);
+                                  b.build(),
+                                  true);
     termAtt = tokensOut.addAttribute(CharTermAttribute.class);
     posIncrAtt = tokensOut.addAttribute(PositionIncrementAttribute.class);
     offsetAtt = tokensOut.addAttribute(OffsetAttribute.class);

Modified: lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java?rev=1310910&r1=1310909&r2=1310910&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java Sat Apr  7 23:06:12 2012
@@ -26,6 +26,7 @@ import java.io.StringReader;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockGraphTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.ja.JapaneseTokenizer.Mode;
@@ -190,7 +191,21 @@ public class TestJapaneseTokenizer exten
     checkRandomData(random, analyzer, 200*RANDOM_MULTIPLIER, 8192);
     checkRandomData(random, analyzerNoPunct, 200*RANDOM_MULTIPLIER, 8192);
   }
-  
+
+  public void testRandomHugeStringsMockGraphAfter() throws Exception {
+    // Randomly inject graph tokens after JapaneseTokenizer:
+    checkRandomData(random,
+                    new Analyzer() {
+                      @Override
+                      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+                        Tokenizer tokenizer = new JapaneseTokenizer(reader, readDict(), false, Mode.SEARCH);
+                        TokenStream graph = new MockGraphTokenFilter(random, tokenizer);
+                        return new TokenStreamComponents(tokenizer, graph);
+                      }
+                    },
+                    200*RANDOM_MULTIPLIER, 8192);
+  }
+
   public void testLargeDocReliability() throws Exception {
     for (int i = 0; i < 100; i++) {
       String s = _TestUtil.randomUnicodeString(random, 10000);