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

svn commit: r1324960 [1/2] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/contrib/ lucene/core/ lucene/test-framework/ lucene/test-framework/src/java/org/apache/lucene/analysis/ modules/analysis/ modules/analysis/common/ modules/analysis/common/s...

Author: rmuir
Date: Wed Apr 11 19:54:09 2012
New Revision: 1324960

URL: http://svn.apache.org/viewvc?rev=1324960&view=rev
Log:
LUCENE-3969: Test all ctors in TestRandomChains and fix bugs discovered by the test

Added:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java
      - copied unchanged from r1324948, lucene/dev/branches/lucene3969/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java
Modified:
    lucene/dev/trunk/   (props changed)
    lucene/dev/trunk/dev-tools/   (props changed)
    lucene/dev/trunk/lucene/   (props changed)
    lucene/dev/trunk/lucene/CHANGES.txt   (props changed)
    lucene/dev/trunk/lucene/contrib/   (props changed)
    lucene/dev/trunk/lucene/core/   (props changed)
    lucene/dev/trunk/lucene/test-framework/   (props changed)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockFixedLengthPayloadFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockRandomLookaheadTokenFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java
    lucene/dev/trunk/modules/analysis/CHANGES.txt
    lucene/dev/trunk/modules/analysis/common/   (props changed)
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/TestMappingCharFilter.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTrimFilter.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/ngram/EdgeNGramTokenFilterTest.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerTest.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/ngram/NGramTokenFilterTest.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/ngram/NGramTokenizerTest.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/snowball/TestSnowball.java
    lucene/dev/trunk/solr/   (props changed)
    lucene/dev/trunk/solr/CHANGES.txt   (props changed)
    lucene/dev/trunk/solr/NOTICE.txt   (props changed)
    lucene/dev/trunk/solr/core/   (props changed)
    lucene/dev/trunk/solr/example/   (props changed)
    lucene/dev/trunk/solr/lib/   (props changed)
    lucene/dev/trunk/solr/solrj/   (props changed)

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Apr 11 19:54:09 2012
@@ -100,7 +100,14 @@ public abstract class BaseTokenStreamTes
     }
   }
 
-  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset) throws IOException {
+  // offsetsAreCorrect also validates:
+  //   - graph offsets are correct (all tokens leaving from
+  //     pos X have the same startOffset; all tokens
+  //     arriving to pos Y have the same endOffset)
+  //   - offsets only move forwards (startOffset >=
+  //     lastStartOffset)
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset,
+                                               boolean offsetsAreCorrect) throws IOException {
     assertNotNull(output);
     CheckClearAttributesAttribute checkClearAtt = ts.addAttribute(CheckClearAttributesAttribute.class);
     
@@ -137,6 +144,7 @@ public abstract class BaseTokenStreamTes
 
     ts.reset();
     int pos = -1;
+    int lastStartOffset = 0;
     for (int i = 0; i < output.length; i++) {
       // extra safety to enforce, that the state is not preserved and also assign bogus values
       ts.clearAttributes();
@@ -176,7 +184,12 @@ public abstract class BaseTokenStreamTes
                      endOffset <= finalOffset.intValue());
         }
 
-        if (posLengthAtt != null && posIncrAtt != null) {
+        if (offsetsAreCorrect) {
+          assertTrue("offsets must not go backwards startOffset=" + startOffset + " is < lastStartOffset=" + lastStartOffset, offsetAtt.startOffset() >= lastStartOffset);
+          lastStartOffset = offsetAtt.startOffset();
+        }
+
+        if (offsetsAreCorrect && posLengthAtt != null && posIncrAtt != null) {
           // Validate offset consistency in the graph, ie
           // all tokens leaving from a certain pos have the
           // same startOffset, and all tokens arriving to a
@@ -194,7 +207,7 @@ public abstract class BaseTokenStreamTes
             // We've seen a token leaving from this position
             // before; verify the startOffset is the same:
             //System.out.println("  + vs " + pos + " -> " + startOffset);
-            assertEquals(posToStartOffset.get(pos).intValue(), startOffset);
+            assertEquals("pos=" + pos + " posLen=" + posLength + " token=" + termAtt, posToStartOffset.get(pos).intValue(), startOffset);
           }
 
           final int endPos = pos + posLength;
@@ -207,7 +220,7 @@ public abstract class BaseTokenStreamTes
             // We've seen a token arriving to this position
             // before; verify the endOffset is the same:
             //System.out.println("  + ve " + endPos + " -> " + endOffset);
-            assertEquals(posToEndOffset.get(endPos).intValue(), endOffset);
+            assertEquals("pos=" + pos + " posLen=" + posLength + " token=" + termAtt, posToEndOffset.get(endPos).intValue(), endOffset);
           }
         }
       }
@@ -222,7 +235,7 @@ public abstract class BaseTokenStreamTes
         assertTrue("posLength must be >= 1", posLengthAtt.getPositionLength() >= 1);
       }
     }
-    assertFalse("TokenStream has more tokens than expected", ts.incrementToken());
+    assertFalse("TokenStream has more tokens than expected (expected count=" + output.length + ")", ts.incrementToken());
     ts.end();
     if (finalOffset != null) {
       assertEquals("finalOffset ", finalOffset.intValue(), offsetAtt.endOffset());
@@ -233,6 +246,10 @@ public abstract class BaseTokenStreamTes
     ts.close();
   }
   
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, true);
+  }
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], Integer finalOffset) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null, finalOffset);
   }
@@ -280,6 +297,10 @@ public abstract class BaseTokenStreamTes
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
     assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
   }
+
+  public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], boolean offsetsAreCorrect) throws IOException {
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
+  }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, null, null, null);
@@ -342,12 +363,12 @@ public abstract class BaseTokenStreamTes
   
   /** utility method for blasting tokenstreams with data to make sure they don't do anything crazy */
   public static void checkRandomData(Random random, Analyzer a, int iterations) throws IOException {
-    checkRandomData(random, a, iterations, 20, false);
+    checkRandomData(random, a, iterations, 20, false, true);
   }
-  
+
   /** utility method for blasting tokenstreams with data to make sure they don't do anything crazy */
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
-    checkRandomData(random, a, iterations, maxWordLength, false);
+    checkRandomData(random, a, iterations, maxWordLength, false, true);
   }
   
   /** 
@@ -355,43 +376,63 @@ public abstract class BaseTokenStreamTes
    * @param simple true if only ascii strings will be used (try to avoid)
    */
   public static void checkRandomData(Random random, Analyzer a, int iterations, boolean simple) throws IOException {
-    checkRandomData(random, a, iterations, 20, simple);
+    checkRandomData(random, a, iterations, 20, simple, true);
   }
   
   static class AnalysisThread extends Thread {
     final int iterations;
     final int maxWordLength;
-    final Random random;
+    final long seed;
     final Analyzer a;
+    final boolean useCharFilter;
     final boolean simple;
+    final boolean offsetsAreCorrect;
+
+    // NOTE: not volatile because we don't want the tests to
+    // add memory barriers (ie alter how threads
+    // interact)... so this is just "best effort":
+    public boolean failed;
     
-    AnalysisThread(Random random, Analyzer a, int iterations, int maxWordLength, boolean simple) {
-      this.random = random;
+    AnalysisThread(long seed, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter, boolean simple, boolean offsetsAreCorrect) {
+      this.seed = seed;
       this.a = a;
       this.iterations = iterations;
       this.maxWordLength = maxWordLength;
+      this.useCharFilter = useCharFilter;
       this.simple = simple;
+      this.offsetsAreCorrect = offsetsAreCorrect;
     }
     
     @Override
     public void run() {
+      boolean success = false;
       try {
         // see the part in checkRandomData where it replays the same text again
         // to verify reproducability/reuse: hopefully this would catch thread hazards.
-        checkRandomData(random, a, iterations, maxWordLength, random.nextBoolean(), simple);
+        checkRandomData(new Random(seed), a, iterations, maxWordLength, useCharFilter, simple, offsetsAreCorrect);
+        success = true;
       } catch (IOException e) {
         Rethrow.rethrow(e);
+      } finally {
+        failed = !success;
       }
     }
   };
   
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean simple) throws IOException {
-    checkRandomData(random, a, iterations, maxWordLength, random.nextBoolean(), simple);
-    // now test with multiple threads
+    checkRandomData(random, a, iterations, maxWordLength, simple, true);
+  }
+
+  public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean simple, boolean offsetsAreCorrect) throws IOException {
+    long seed = random.nextLong();
+    boolean useCharFilter = random.nextBoolean();
+    checkRandomData(new Random(seed), a, iterations, maxWordLength, useCharFilter, simple, offsetsAreCorrect);
+    // now test with multiple threads: note we do the EXACT same thing we did before in each thread,
+    // so this should only really fail from another thread if its an actual thread problem
     int numThreads = _TestUtil.nextInt(random, 4, 8);
-    Thread threads[] = new Thread[numThreads];
+    AnalysisThread threads[] = new AnalysisThread[numThreads];
     for (int i = 0; i < threads.length; i++) {
-      threads[i] = new AnalysisThread(new Random(random.nextLong()), a, iterations, maxWordLength, simple);
+      threads[i] = new AnalysisThread(seed, a, iterations, maxWordLength, useCharFilter, simple, offsetsAreCorrect);
     }
     for (int i = 0; i < threads.length; i++) {
       threads[i].start();
@@ -403,9 +444,14 @@ public abstract class BaseTokenStreamTes
         throw new RuntimeException(e);
       }
     }
+    for (int i = 0; i < threads.length; i++) {
+      if (threads[i].failed) {
+        throw new RuntimeException("some thread(s) failed");
+      }
+    }
   }
 
-  private static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter, boolean simple) throws IOException {
+  private static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter, boolean simple, boolean offsetsAreCorrect) throws IOException {
 
     final LineFileDocs docs = new LineFileDocs(random);
 
@@ -437,7 +483,7 @@ public abstract class BaseTokenStreamTes
       }
 
       try {
-        checkAnalysisConsistency(random, a, useCharFilter, text);
+        checkAnalysisConsistency(random, a, useCharFilter, text, offsetsAreCorrect);
       } catch (Throwable t) {
         // TODO: really we should pass a random seed to
         // checkAnalysisConsistency then print it here too:
@@ -477,6 +523,10 @@ public abstract class BaseTokenStreamTes
   }
 
   public static void checkAnalysisConsistency(Random random, Analyzer a, boolean useCharFilter, String text) throws IOException {
+    checkAnalysisConsistency(random, a, useCharFilter, text, true);
+  }
+
+  public static void checkAnalysisConsistency(Random random, Analyzer a, boolean useCharFilter, String text, boolean offsetsAreCorrect) throws IOException {
 
     if (VERBOSE) {
       System.out.println(Thread.currentThread().getName() + ": NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
@@ -616,7 +666,8 @@ public abstract class BaseTokenStreamTes
                                 types.toArray(new String[types.size()]),
                                 toIntArray(positions),
                                 toIntArray(positionLengths),
-                                text.length());
+                                text.length(),
+                                offsetsAreCorrect);
     } else if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
       // offset + pos + type
       assertTokenStreamContents(ts, 
@@ -626,7 +677,8 @@ public abstract class BaseTokenStreamTes
                                 types.toArray(new String[types.size()]),
                                 toIntArray(positions),
                                 null,
-                                text.length());
+                                text.length(),
+                                offsetsAreCorrect);
     } else if (posIncAtt != null && posLengthAtt != null && offsetAtt != null) {
       // offset + pos + posLength
       assertTokenStreamContents(ts, 
@@ -636,7 +688,8 @@ public abstract class BaseTokenStreamTes
                                 null,
                                 toIntArray(positions),
                                 toIntArray(positionLengths),
-                                text.length());
+                                text.length(),
+                                offsetsAreCorrect);
     } else if (posIncAtt != null && offsetAtt != null) {
       // offset + pos
       assertTokenStreamContents(ts, 
@@ -646,7 +699,8 @@ public abstract class BaseTokenStreamTes
                                 null,
                                 toIntArray(positions),
                                 null,
-                                text.length());
+                                text.length(),
+                                offsetsAreCorrect);
     } else if (offsetAtt != null) {
       // offset
       assertTokenStreamContents(ts, 
@@ -656,7 +710,8 @@ public abstract class BaseTokenStreamTes
                                 null,
                                 null,
                                 null,
-                                text.length());
+                                text.length(),
+                                offsetsAreCorrect);
     } else {
       // terms only
       assertTokenStreamContents(ts, 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java Wed Apr 11 19:54:09 2012
@@ -151,7 +151,7 @@ public abstract class LookaheadTokenFilt
         startPosData.startOffset = startOffset;
       } else {
         // Make sure our input isn't messing up offsets:
-        assert startPosData.startOffset == startOffset;
+        assert startPosData.startOffset == startOffset: "prev startOffset=" + startPosData.startOffset + " vs new startOffset=" + startOffset + " inputPos=" + inputPos;
       }
 
       final int endOffset = offsetAtt.endOffset();
@@ -159,7 +159,7 @@ public abstract class LookaheadTokenFilt
         endPosData.endOffset = endOffset;
       } else {
         // Make sure our input isn't messing up offsets:
-        assert endPosData.endOffset == endOffset;
+        assert endPosData.endOffset == endOffset: "prev endOffset=" + endPosData.endOffset + " vs new endOffset=" + endOffset + " inputPos=" + inputPos;
       }
 
       tokenPending = true;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java Wed Apr 11 19:54:09 2012
@@ -76,7 +76,7 @@ public final class MockAnalyzer extends 
    * MockAnalyzer(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false}).
    */
   public MockAnalyzer(Random random, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    this(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false);
+    this(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, true);
   }
 
   /** 
@@ -93,7 +93,8 @@ public final class MockAnalyzer extends 
   public TokenStreamComponents createComponents(String fieldName, Reader reader) {
     MockTokenizer tokenizer = new MockTokenizer(reader, runAutomaton, lowerCase, maxTokenLength);
     tokenizer.setEnableChecks(enableChecks);
-    TokenFilter filt = new MockTokenFilter(tokenizer, filter, enablePositionIncrements);
+    MockTokenFilter filt = new MockTokenFilter(tokenizer, filter);
+    filt.setEnablePositionIncrements(enablePositionIncrements);
     return new TokenStreamComponents(tokenizer, maybePayload(filt, fieldName));
   }
   

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java Wed Apr 11 19:54:09 2012
@@ -34,7 +34,9 @@ public class MockCharFilter extends Char
     // TODO: instead of fixed remainder... maybe a fixed
     // random seed?
     this.remainder = remainder;
-    assert remainder >= 0 && remainder < 10 : "invalid parameter";
+    if (remainder < 0 || remainder >= 10) {
+      throw new IllegalArgumentException("invalid remainder parameter (must be 0..10): " + remainder);
+    }
   }
   
   // for testing only, uses a remainder of 0

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockFixedLengthPayloadFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockFixedLengthPayloadFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockFixedLengthPayloadFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockFixedLengthPayloadFilter.java Wed Apr 11 19:54:09 2012
@@ -34,6 +34,9 @@ public final class MockFixedLengthPayloa
 
   public MockFixedLengthPayloadFilter(Random random, TokenStream in, int length) {
     super(in);
+    if (length < 0) {
+      throw new IllegalArgumentException("length must be >= 0");
+    }
     this.random = random;
     this.bytes = new byte[length];
     this.payload = new Payload(bytes);

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockRandomLookaheadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockRandomLookaheadTokenFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockRandomLookaheadTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockRandomLookaheadTokenFilter.java Wed Apr 11 19:54:09 2012
@@ -31,10 +31,12 @@ public final class MockRandomLookaheadTo
 
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final Random random;
+  private final long seed;
 
   public MockRandomLookaheadTokenFilter(Random random, TokenStream in) {
     super(in);
-    this.random = random;
+    this.seed = random.nextLong();
+    this.random = new Random(seed);
   }
 
   @Override
@@ -57,9 +59,6 @@ public final class MockRandomLookaheadTo
 
     if (!end) {
       while (true) {
-        // We can use un-re-seeded random, because how far
-        // ahead we peek should never alter the resulting
-        // tokens as seen by the consumer:
         if (random.nextInt(3) == 1) {
           if (!peekToken()) {
             if (DEBUG) {
@@ -91,4 +90,10 @@ public final class MockRandomLookaheadTo
     }
     return result;
   }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    random.setSeed(seed);
+  }
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java Wed Apr 11 19:54:09 2012
@@ -55,7 +55,7 @@ public final class MockTokenFilter exten
       makeString("with"))));
   
   private final CharacterRunAutomaton filter;
-  private boolean enablePositionIncrements = false;
+  private boolean enablePositionIncrements = true;
 
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
@@ -67,14 +67,16 @@ public final class MockTokenFilter exten
    * @param filter DFA representing the terms that should be removed.
    * @param enablePositionIncrements true if the removal should accumulate position increments.
    */
-  public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
+  public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter) {
     super(input);
     this.filter = filter;
-    this.enablePositionIncrements = enablePositionIncrements;
   }
   
   @Override
   public boolean incrementToken() throws IOException {
+    // TODO: fix me when posInc=false, to work like FilteringTokenFilter in that case and not return
+    // initial token with posInc=0 ever
+    
     // return the first non-stop word found
     int skippedPositions = 0;
     while (input.incrementToken()) {

Modified: lucene/dev/trunk/modules/analysis/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/CHANGES.txt?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/CHANGES.txt (original)
+++ lucene/dev/trunk/modules/analysis/CHANGES.txt Wed Apr 11 19:54:09 2012
@@ -42,6 +42,16 @@ Bug fixes
  * LUCENE-3820: PatternReplaceCharFilter could return invalid token positions. 
    (Dawid Weiss)
 
+ * LUCENE-3969: Throw IAE on bad arguments that could cause confusing errors in
+   CompoundWordTokenFilterBase, PatternTokenizer, PositionFilter,
+   SnowballFilter, PathHierarchyTokenizer, ReversePathHierarchyTokenizer, 
+   WikipediaTokenizer, and KeywordTokenizer. ShingleFilter and 
+   CommonGramsFilter now populate PositionLengthAttribute. Fixed
+   PathHierarchyTokenizer to reset() all state. Protect against AIOOBE in
+   ReversePathHierarchyTokenizer if skip is large. Fixed wrong final
+   offset calculation in PathHierarchyTokenizer. 
+   (Mike McCandless, Uwe Schindler, Robert Muir)
+
 New Features
 
  * LUCENE-2341: A new analyzer/ filter: Morfologik - a dictionary-driven lemmatizer 
@@ -108,4 +118,4 @@ New Features
    (Chris Male, Robert Muir)
 
  * SOLR-2764: Create a NorwegianLightStemmer and NorwegianMinimalStemmer (janhoy)
-   
\ No newline at end of file
+   

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java Wed Apr 11 19:54:09 2012
@@ -16,6 +16,7 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.util.Version;
@@ -54,6 +55,7 @@ public final class CommonGramsFilter ext
   private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
   private final TypeAttribute typeAttribute = addAttribute(TypeAttribute.class);
   private final PositionIncrementAttribute posIncAttribute = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAttribute = addAttribute(PositionLengthAttribute.class);
 
   private int lastStartOffset;
   private boolean lastWasCommon;
@@ -166,6 +168,7 @@ public final class CommonGramsFilter ext
     buffer.getChars(0, length, termText, 0);
     termAttribute.setLength(length);
     posIncAttribute.setPositionIncrement(0);
+    posLenAttribute.setPositionLength(2); // bigram
     offsetAttribute.setOffset(lastStartOffset, endOffset);
     typeAttribute.setType(GRAM_TYPE);
     buffer.setLength(0);

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java Wed Apr 11 19:54:09 2012
@@ -82,8 +82,17 @@ public abstract class CompoundWordTokenF
     super(input);
     
     this.tokens=new LinkedList<CompoundToken>();
+    if (minWordSize < 0) {
+      throw new IllegalArgumentException("minWordSize cannot be negative");
+    }
     this.minWordSize=minWordSize;
+    if (minSubwordSize < 0) {
+      throw new IllegalArgumentException("minSubwordSize cannot be negative");
+    }
     this.minSubwordSize=minSubwordSize;
+    if (maxSubwordSize < 0) {
+      throw new IllegalArgumentException("maxSubwordSize cannot be negative");
+    }
     this.maxSubwordSize=maxSubwordSize;
     this.onlyLongestMatch=onlyLongestMatch;
     this.dictionary = dictionary;

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java Wed Apr 11 19:54:09 2012
@@ -191,6 +191,8 @@ public class HyphenationCompoundWordToke
         // we only put subwords to the token stream
         // that are longer than minPartSize
         if (partLength < this.minSubwordSize) {
+          // BOGUS/BROKEN/FUNKY/WACKO: somehow we have negative 'parts' according to the 
+          // calculation above, and we rely upon minSubwordSize being >=0 to filter them out...
           continue;
         }
 

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java Wed Apr 11 19:54:09 2012
@@ -43,16 +43,25 @@ public final class KeywordTokenizer exte
 
   public KeywordTokenizer(Reader input, int bufferSize) {
     super(input);
+    if (bufferSize <= 0) {
+      throw new IllegalArgumentException("bufferSize must be > 0");
+    }
     termAtt.resizeBuffer(bufferSize);
   }
 
   public KeywordTokenizer(AttributeSource source, Reader input, int bufferSize) {
     super(source, input);
+    if (bufferSize <= 0) {
+      throw new IllegalArgumentException("bufferSize must be > 0");
+    }
     termAtt.resizeBuffer(bufferSize);
   }
 
   public KeywordTokenizer(AttributeFactory factory, Reader input, int bufferSize) {
     super(factory, input);
+    if (bufferSize <= 0) {
+      throw new IllegalArgumentException("bufferSize must be > 0");
+    }
     termAtt.resizeBuffer(bufferSize);
   }
   

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java Wed Apr 11 19:54:09 2012
@@ -65,6 +65,12 @@ public class PathHierarchyTokenizer exte
 
   public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) {
     super(input);
+    if (bufferSize < 0) {
+      throw new IllegalArgumentException("bufferSize cannot be negative");
+    }
+    if (skip < 0) {
+      throw new IllegalArgumentException("skip cannot be negative");
+    }
     termAtt.resizeBuffer(bufferSize);
 
     this.delimiter = delimiter;
@@ -85,10 +91,11 @@ public class PathHierarchyTokenizer exte
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   private final PositionIncrementAttribute posAtt = addAttribute(PositionIncrementAttribute.class);
   private int startPosition = 0;
-  private int finalOffset = 0;
   private int skipped = 0;
   private boolean endDelimiter = false;
   private StringBuilder resultToken;
+  
+  private int charsRead = 0;
 
 
   @Override
@@ -112,12 +119,13 @@ public class PathHierarchyTokenizer exte
 
     while (true) {
       int c = input.read();
-      if( c < 0 ){
+      if (c >= 0) {
+        charsRead++;
+      } else {
         if( skipped > skip ) {
           length += resultToken.length();
           termAtt.setLength(length);
-          finalOffset = correctOffset(startPosition + length);
-          offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
+           offsetAtt.setOffset(correctOffset(startPosition), correctOffset(startPosition + length));
           if( added ){
             resultToken.setLength(0);
             resultToken.append(termAtt.buffer(), 0, length);
@@ -125,7 +133,6 @@ public class PathHierarchyTokenizer exte
           return added;
         }
         else{
-          finalOffset = correctOffset(startPosition + length);
           return false;
         }
       }
@@ -168,8 +175,7 @@ public class PathHierarchyTokenizer exte
     }
     length += resultToken.length();
     termAtt.setLength(length);
-    finalOffset = correctOffset(startPosition + length);
-    offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
+    offsetAtt.setOffset(correctOffset(startPosition), correctOffset(startPosition+length));
     resultToken.setLength(0);
     resultToken.append(termAtt.buffer(), 0, length);
     return true;
@@ -178,15 +184,17 @@ public class PathHierarchyTokenizer exte
   @Override
   public final void end() {
     // set final offset
+    int finalOffset = correctOffset(charsRead);
     offsetAtt.setOffset(finalOffset, finalOffset);
   }
 
   @Override
-  public void reset(Reader input) throws IOException {
-    super.reset(input);
+  public void reset() throws IOException {
+    super.reset();
     resultToken.setLength(0);
-    finalOffset = 0;
+    charsRead = 0;
     endDelimiter = false;
     skipped = 0;
+    startPosition = 0;
   }
 }

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java Wed Apr 11 19:54:09 2012
@@ -77,6 +77,12 @@ public class ReversePathHierarchyTokeniz
 
   public ReversePathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) {
     super(input);
+    if (bufferSize < 0) {
+      throw new IllegalArgumentException("bufferSize cannot be negative");
+    }
+    if (skip < 0) {
+      throw new IllegalArgumentException("skip cannot be negative");
+    }
     termAtt.resizeBuffer(bufferSize);
     this.delimiter = delimiter;
     this.replacement = replacement;
@@ -137,7 +143,11 @@ public class ReversePathHierarchyTokeniz
       }
       resultToken.getChars(0, resultToken.length(), resultTokenBuffer, 0);
       resultToken.setLength(0);
-      endPosition = delimiterPositions.get(delimitersCount-1 - skip);
+      int idx = delimitersCount-1 - skip;
+      if (idx >= 0) {
+        // otherwise its ok, because we will skip and return false
+        endPosition = delimiterPositions.get(idx);
+      }
       finalOffset = correctOffset(length);
       posAtt.setPositionIncrement(1);
     }
@@ -163,10 +173,11 @@ public class ReversePathHierarchyTokeniz
   }
 
   @Override
-  public void reset(Reader input) throws IOException {
-    super.reset(input);
+  public void reset() throws IOException {
+    super.reset();
     resultToken.setLength(0);
     finalOffset = 0;
+    endPosition = 0;
     skipped = 0;
     delimitersCount = -1;
     delimiterPositions.clear();

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java Wed Apr 11 19:54:09 2012
@@ -69,8 +69,17 @@ public final class PatternTokenizer exte
     super(input);
     this.pattern = pattern;
     this.group = group;
+
+    // Use "" instead of str so don't consume chars
+    // (fillBuffer) from the input on throwing IAE below:
+    matcher = pattern.matcher("");
+
+    // confusingly group count depends ENTIRELY on the pattern but is only accessible via matcher
+    if (group >= 0 && group > matcher.groupCount()) {
+      throw new IllegalArgumentException("invalid group specified: pattern only has: " + matcher.groupCount() + " capturing groups");
+    }
     fillBuffer(str, input);
-    matcher = pattern.matcher(str);
+    matcher.reset(str);
     index = 0;
   }
 

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java Wed Apr 11 19:54:09 2012
@@ -57,6 +57,9 @@ public final class PositionFilter extend
    */
   public PositionFilter(final TokenStream input, final int positionIncrement) {
     super(input);
+    if (positionIncrement < 0) {
+      throw new IllegalArgumentException("positionIncrement may not be negative");
+    }
     this.positionIncrement = positionIncrement;
   }
 

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java Wed Apr 11 19:54:09 2012
@@ -23,9 +23,10 @@ import java.util.LinkedList;
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+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.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.util.AttributeSource;
 
@@ -150,6 +151,7 @@ public final class ShingleFilter extends
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
   private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
 
 
@@ -319,6 +321,7 @@ public final class ShingleFilter extends
           noShingleOutput = false;
         }
         offsetAtt.setOffset(offsetAtt.startOffset(), nextToken.offsetAtt.endOffset());
+        posLenAtt.setPositionLength(builtGramSize);
         isOutputHere = true;
         gramSize.advance();
         tokenAvailable = true;
@@ -436,6 +439,8 @@ public final class ShingleFilter extends
     super.reset();
     gramSize.reset();
     inputWindow.clear();
+    nextInputStreamToken = null;
+    isNextInputStreamToken = false;
     numFillerTokensToInsert = 0;
     isOutputHere = false;
     noShingleOutput = true;

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java Wed Apr 11 19:54:09 2012
@@ -67,7 +67,7 @@ public final class SnowballFilter extend
         Class.forName("org.tartarus.snowball.ext." + name + "Stemmer").asSubclass(SnowballProgram.class);
       stemmer = stemClass.newInstance();
     } catch (Exception e) {
-      throw new RuntimeException(e.toString());
+      throw new IllegalArgumentException("Invalid stemmer class specified: " + name, e);
     }
   }
 

Modified: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java Wed Apr 11 19:54:09 2012
@@ -177,6 +177,12 @@ public final class WikipediaTokenizer ex
   }
   
   private void init(int tokenOutput, Set<String> untokenizedTypes) {
+    // TODO: cutover to enum
+    if (tokenOutput != TOKENS_ONLY &&
+        tokenOutput != UNTOKENIZED_ONLY &&
+        tokenOutput != BOTH) {
+      throw new IllegalArgumentException("tokenOutput must be TOKENS_ONLY, UNTOKENIZED_ONLY or BOTH");
+    }
     this.tokenOutput = tokenOutput;
     this.untokenizedTypes = untokenizedTypes;    
   }

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/TestMappingCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/TestMappingCharFilter.java?rev=1324960&r1=1324959&r2=1324960&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/TestMappingCharFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/TestMappingCharFilter.java Wed Apr 11 19:54:09 2012
@@ -19,6 +19,8 @@ package org.apache.lucene.analysis.charf
 
 import java.io.Reader;
 import java.io.StringReader;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
@@ -27,6 +29,8 @@ 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;
 
 public class TestMappingCharFilter extends BaseTokenStreamTestCase {
 
@@ -190,4 +194,67 @@ public class TestMappingCharFilter exten
     int numRounds = RANDOM_MULTIPLIER * 10000;
     checkRandomData(random, analyzer, numRounds);
   }
+  
+  @Ignore("wrong finalOffset: https://issues.apache.org/jira/browse/LUCENE-3971")
+  public void testFinalOffsetSpecialCase() throws Exception {  
+    final NormalizeCharMap map = new NormalizeCharMap();
+    map.add("t", "");
+    // even though this below rule has no effect, the test passes if you remove it!!
+    map.add("tmakdbl", "c");
+    
+    Analyzer analyzer = 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 MappingCharFilter(map, CharReader.get(reader));
+      }
+    };
+    
+    String text = "gzw f quaxot";
+    checkAnalysisConsistency(random, analyzer, false, text);
+  }
+  
+  @Ignore("wrong finalOffset: https://issues.apache.org/jira/browse/LUCENE-3971")
+  public void testRandomMaps() throws Exception {
+    for (int i = 0; i < 100; i++) {
+      final NormalizeCharMap map = randomMap();
+      Analyzer analyzer = 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 MappingCharFilter(map, CharReader.get(reader));
+        }
+      };
+      int numRounds = RANDOM_MULTIPLIER * 100;
+      checkRandomData(random, analyzer, numRounds);
+    }
+  }
+  
+  private NormalizeCharMap randomMap() {
+    NormalizeCharMap map = new NormalizeCharMap();
+    // we can't add duplicate keys, or NormalizeCharMap gets angry
+    Set<String> keys = new HashSet<String>();
+    int num = random.nextInt(5);
+    //System.out.println("NormalizeCharMap=");
+    for (int i = 0; i < num; i++) {
+      String key = _TestUtil.randomSimpleString(random);
+      if (!keys.contains(key)) {
+        String value = _TestUtil.randomSimpleString(random);
+        map.add(key, value);
+        keys.add(key);
+        //System.out.println("mapping: '" + key + "' => '" + value + "'");
+      }
+    }
+    return map;
+  }
 }