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

svn commit: r1235028 [3/3] - in /lucene/dev/branches/lucene2858: ./ dev-tools/maven/solr/core/ lucene/ lucene/src/java/org/apache/lucene/codecs/lucene3x/ lucene/src/java/org/apache/lucene/codecs/lucene40/ lucene/src/test-framework/java/org/apache/lucen...

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex Mon Jan 23 22:15:15 2012
@@ -32,6 +32,7 @@ import org.apache.lucene.analysis.util.O
 /**
  * A CharFilter that wraps another Reader and attempts to strip out HTML constructs.
  */
+@SuppressWarnings("fallthrough")
 %%
 
 %unicode 6.0
@@ -151,6 +152,7 @@ InlineElment = ( [aAbBiIqQsSuU]         
   private static final char BR_END_TAG_REPLACEMENT = '\n';
   private static final char SCRIPT_REPLACEMENT = '\n';
   private static final char STYLE_REPLACEMENT = '\n';
+  private static final char REPLACEMENT_CHARACTER = '\uFFFD';
 
   private CharArraySet escapedTags = null;
   private int inputStart;
@@ -350,6 +352,137 @@ InlineElment = ( [aAbBiIqQsSuU]         
     yybegin(CHARACTER_REFERENCE_TAIL);
   }
   "#" { inputSegment.append('#'); yybegin(NUMERIC_CHARACTER); }
+
+//                                             1   1       11              11
+// 0  1   2   3       45              678  9   0   1       23              45
+  "#" [xX][dD][89aAbB][0-9a-fA-F]{2} ";&#" [xX][dD][c-fC-F][0-9a-fA-F]{2} ";" {
+    // Handle paired UTF-16 surrogates.
+    outputSegment = entitySegment;
+    outputSegment.clear();
+    String surrogatePair = yytext();
+    char highSurrogate = '\u0000';
+    try {
+      highSurrogate = (char)Integer.parseInt(surrogatePair.substring(2, 6), 16);
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing high surrogate '"
+                  + surrogatePair.substring(2, 6) + "'";
+    }
+    try {
+      outputSegment.unsafeWrite
+          ((char)Integer.parseInt(surrogatePair.substring(10, 14), 16));
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing low surrogate '"
+                  + surrogatePair.substring(10, 14) + "'";
+    }
+    cumulativeDiff += inputSegment.length() + yylength() - 2;
+    addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
+    inputSegment.clear();
+    yybegin(YYINITIAL);
+    return highSurrogate;
+  }
+
+//                          1   1       11              11
+// 01  2    345    678  9   0   1       23              45
+  "#5" [56] \d{3} ";&#" [xX][dD][c-fC-F][0-9a-fA-F]{2} ";" {
+    // Handle paired UTF-16 surrogates.
+    String surrogatePair = yytext();
+    char highSurrogate = '\u0000';
+    try { // High surrogates are in decimal range [55296, 56319]
+      highSurrogate = (char)Integer.parseInt(surrogatePair.substring(1, 6));
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing high surrogate '"
+                  + surrogatePair.substring(1, 6) + "'";
+    }
+    if (Character.isHighSurrogate(highSurrogate)) {
+      outputSegment = entitySegment;
+      outputSegment.clear();
+      try {
+        outputSegment.unsafeWrite
+            ((char)Integer.parseInt(surrogatePair.substring(10, 14), 16));
+      } catch(Exception e) { // should never happen
+        assert false: "Exception parsing low surrogate '"
+                    + surrogatePair.substring(10, 14) + "'";
+      }
+      cumulativeDiff += inputSegment.length() + yylength() - 2;
+      addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
+      inputSegment.clear();
+      yybegin(YYINITIAL);
+      return highSurrogate;
+    }
+    yypushback(surrogatePair.length() - 1); // Consume only '#'
+    inputSegment.append('#');
+    yybegin(NUMERIC_CHARACTER);
+  }
+
+//                                          1    111     11
+// 0  1   2   3       45              6789  0    123     45
+  "#" [xX][dD][89aAbB][0-9a-fA-F]{2} ";&#5" [67] \d{3}  ";" {
+    // Handle paired UTF-16 surrogates.
+    String surrogatePair = yytext();
+    char highSurrogate = '\u0000';
+    char lowSurrogate = '\u0000';
+    try {
+      highSurrogate = (char)Integer.parseInt(surrogatePair.substring(2, 6), 16);
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing high surrogate '"
+                  + surrogatePair.substring(2, 6) + "'";
+    }
+    try { // Low surrogates are in decimal range [56320, 57343]
+      lowSurrogate = (char)Integer.parseInt(surrogatePair.substring(9, 14));
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing low surrogate '"
+                  + surrogatePair.substring(9, 14) + "'";
+    }
+    if (Character.isLowSurrogate(lowSurrogate)) {
+      outputSegment = entitySegment;
+      outputSegment.clear();
+      outputSegment.unsafeWrite(lowSurrogate);
+      cumulativeDiff += inputSegment.length() + yylength() - 2;
+      addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
+      inputSegment.clear();
+      yybegin(YYINITIAL);
+      return highSurrogate;
+    }
+    yypushback(surrogatePair.length() - 1); // Consume only '#'
+    inputSegment.append('#');
+    yybegin(NUMERIC_CHARACTER);
+  }
+
+//                       1    111     11
+// 01  2    345    6789  0    123     45
+  "#5" [56] \d{3} ";&#5" [67] \d{3}  ";" {
+    // Handle paired UTF-16 surrogates.
+    String surrogatePair = yytext();
+    char highSurrogate = '\u0000';
+    try { // High surrogates are in decimal range [55296, 56319]
+      highSurrogate = (char)Integer.parseInt(surrogatePair.substring(1, 6));
+    } catch(Exception e) { // should never happen
+      assert false: "Exception parsing high surrogate '"
+                  + surrogatePair.substring(1, 6) + "'";
+    }
+    if (Character.isHighSurrogate(highSurrogate)) {
+      char lowSurrogate = '\u0000';
+      try { // Low surrogates are in decimal range [56320, 57343]
+        lowSurrogate = (char)Integer.parseInt(surrogatePair.substring(9, 14));
+      } catch(Exception e) { // should never happen
+        assert false: "Exception parsing low surrogate '"
+                    + surrogatePair.substring(9, 14) + "'";
+      }
+      if (Character.isLowSurrogate(lowSurrogate)) {
+        outputSegment = entitySegment;
+        outputSegment.clear();
+        outputSegment.unsafeWrite(lowSurrogate);
+        cumulativeDiff += inputSegment.length() + yylength() - 2;
+        addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
+        inputSegment.clear();
+        yybegin(YYINITIAL);
+        return highSurrogate;
+      }
+    }
+    yypushback(surrogatePair.length() - 1); // Consume only '#'
+    inputSegment.append('#');
+    yybegin(NUMERIC_CHARACTER);
+  }
 }
 
 <NUMERIC_CHARACTER> {
@@ -359,25 +492,27 @@ InlineElment = ( [aAbBiIqQsSuU]         
     if (matchLength <= 6) { // 10FFFF: max 6 hex chars
       String hexCharRef
           = new String(zzBuffer, zzStartRead + 1, matchLength - 1);
+      int codePoint = 0;
       try {
-        int codePoint = Integer.parseInt(hexCharRef, 16);
-        if (codePoint <= 0x10FFFF) {
-          outputSegment = entitySegment;
-          outputSegment.clear();
+        codePoint = Integer.parseInt(hexCharRef, 16);
+      } catch(Exception e) {
+        assert false: "Exception parsing hex code point '" + hexCharRef + "'";
+      }
+      if (codePoint <= 0x10FFFF) {
+        outputSegment = entitySegment;
+        outputSegment.clear();
+        if (codePoint >= Character.MIN_SURROGATE
+            && codePoint <= Character.MAX_SURROGATE) {
+          outputSegment.unsafeWrite(REPLACEMENT_CHARACTER);
+        } else {
           outputSegment.setLength
               (Character.toChars(codePoint, outputSegment.getArray(), 0));
-          yybegin(CHARACTER_REFERENCE_TAIL);
-        } else {
-          outputSegment = inputSegment;
-          yybegin(YYINITIAL);
-          return outputSegment.nextChar();
         }
-      } catch(NumberFormatException e) {
-        assert false: "NumberFormatException parsing hex code point '"
-                      + hexCharRef + "'";
-      } catch(IllegalArgumentException e) {
-        assert false: "IllegalArgumentException getting chars "
-                      + "for hex code point '" + hexCharRef + "'";
+        yybegin(CHARACTER_REFERENCE_TAIL);
+      } else {
+        outputSegment = inputSegment;
+        yybegin(YYINITIAL);
+        return outputSegment.nextChar();
       }
     } else {
       outputSegment = inputSegment;
@@ -390,25 +525,27 @@ InlineElment = ( [aAbBiIqQsSuU]         
     inputSegment.write(zzBuffer, zzStartRead, matchLength);
     if (matchLength <= 7) { // 0x10FFFF = 1114111: max 7 decimal chars
       String decimalCharRef = yytext();
+      int codePoint = 0;
       try {
-        int codePoint = Integer.parseInt(decimalCharRef);
-        if (codePoint <= 0x10FFFF) {
-          outputSegment = entitySegment;
-          outputSegment.clear();
+        codePoint = Integer.parseInt(decimalCharRef);
+      } catch(Exception e) {
+        assert false: "Exception parsing code point '" + decimalCharRef + "'";
+      }
+      if (codePoint <= 0x10FFFF) {
+        outputSegment = entitySegment;
+        outputSegment.clear();
+        if (codePoint >= Character.MIN_SURROGATE
+            && codePoint <= Character.MAX_SURROGATE) {
+          outputSegment.unsafeWrite(REPLACEMENT_CHARACTER);
+        } else {
           outputSegment.setLength
               (Character.toChars(codePoint, outputSegment.getArray(), 0));
-          yybegin(CHARACTER_REFERENCE_TAIL);
-        } else {
-          outputSegment = inputSegment;
-          yybegin(YYINITIAL);
-          return outputSegment.nextChar();
         }
-      } catch(NumberFormatException e) {
-        assert false: "NumberFormatException parsing code point '"
-                      + decimalCharRef + "'";
-      } catch(IllegalArgumentException e) {
-        assert false: "IllegalArgumentException getting chars for code point '"
-                      + decimalCharRef + "'";
+        yybegin(CHARACTER_REFERENCE_TAIL);
+      } else {
+        outputSegment = inputSegment;
+        yybegin(YYINITIAL);
+        return outputSegment.nextChar();
       }
     } else {
       outputSegment = inputSegment;

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java Mon Jan 23 22:15:15 2012
@@ -154,13 +154,22 @@ public abstract class CompoundWordTokenF
 
     /** Construct the compound token based on a slice of the current {@link CompoundWordTokenFilterBase#termAtt}. */
     public CompoundToken(int offset, int length) {
-      final int newStart = CompoundWordTokenFilterBase.this.offsetAtt.startOffset() + offset;
       this.txt = CompoundWordTokenFilterBase.this.termAtt.subSequence(offset, offset + length);
-      // TODO: This ignores the original endOffset, if a CharFilter/Tokenizer/Filter removed
-      // chars from the term, offsets may not match correctly (other filters producing tokens
-      // may also have this problem):
-      this.startOffset = newStart;
-      this.endOffset = newStart + length;
+      
+      // offsets of the original word
+      int startOff = CompoundWordTokenFilterBase.this.offsetAtt.startOffset();
+      int endOff = CompoundWordTokenFilterBase.this.offsetAtt.endOffset();
+      
+      if (endOff - startOff != CompoundWordTokenFilterBase.this.termAtt.length()) {
+        // if length by start + end offsets doesn't match the term text then assume
+        // this is a synonym and don't adjust the offsets.
+        this.startOffset = startOff;
+        this.endOffset = endOff;
+      } else {
+        final int newStart = startOff + offset;
+        this.startOffset = newStart;
+        this.endOffset = newStart + length;
+      }
     }
 
   }  

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilter.java Mon Jan 23 22:15:15 2012
@@ -60,6 +60,7 @@ public final class HyphenatedWordsFilter
   private final StringBuilder hyphenated = new StringBuilder();
   private State savedState;
   private boolean exhausted = false;
+  private int lastEndOffset = 0;
 
   /**
    * Creates a new HyphenatedWordsFilter
@@ -78,6 +79,7 @@ public final class HyphenatedWordsFilter
     while (!exhausted && input.incrementToken()) {
       char[] term = termAttribute.buffer();
       int termLength = termAttribute.length();
+      lastEndOffset = offsetAttribute.endOffset();
       
       if (termLength > 0 && term[termLength - 1] == '-') {
         // a hyphenated word
@@ -119,6 +121,7 @@ public final class HyphenatedWordsFilter
     hyphenated.setLength(0);
     savedState = null;
     exhausted = false;
+    lastEndOffset = 0;
   }
 
   // ================================================= Helper Methods ================================================
@@ -127,8 +130,6 @@ public final class HyphenatedWordsFilter
    * Writes the joined unhyphenated term
    */
   private void unhyphenate() {
-    int endOffset = offsetAttribute.endOffset();
-    
     restoreState(savedState);
     savedState = null;
     
@@ -140,7 +141,7 @@ public final class HyphenatedWordsFilter
     
     hyphenated.getChars(0, length, term, 0);
     termAttribute.setLength(length);
-    offsetAttribute.setOffset(offsetAttribute.startOffset(), endOffset);
+    offsetAttribute.setOffset(offsetAttribute.startOffset(), lastEndOffset);
     hyphenated.setLength(0);
   }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PatternAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PatternAnalyzer.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PatternAnalyzer.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PatternAnalyzer.java Mon Jan 23 22:15:15 2012
@@ -183,31 +183,33 @@ public final class PatternAnalyzer exten
    * 
    * @param fieldName
    *            the name of the field to tokenize (currently ignored).
+   * @param reader
+   *            reader (e.g. charfilter) of the original text. can be null.
    * @param text
    *            the string to tokenize
    * @return a new token stream
    */
-  public TokenStreamComponents createComponents(String fieldName, String text) {
+  public TokenStreamComponents createComponents(String fieldName, Reader reader, String text) {
     // Ideally the Analyzer superclass should have a method with the same signature, 
     // with a default impl that simply delegates to the StringReader flavour. 
     if (text == null) 
       throw new IllegalArgumentException("text must not be null");
     
     if (pattern == NON_WORD_PATTERN) { // fast path
-      return new TokenStreamComponents(new FastStringTokenizer(text, true, toLowerCase, stopWords));
+      return new TokenStreamComponents(new FastStringTokenizer(reader, text, true, toLowerCase, stopWords));
     } else if (pattern == WHITESPACE_PATTERN) { // fast path
-      return new TokenStreamComponents(new FastStringTokenizer(text, false, toLowerCase, stopWords));
+      return new TokenStreamComponents(new FastStringTokenizer(reader, text, false, toLowerCase, stopWords));
     }
 
-    Tokenizer tokenizer = new PatternTokenizer(text, pattern, toLowerCase);
+    Tokenizer tokenizer = new PatternTokenizer(reader, text, pattern, toLowerCase);
     TokenStream result = (stopWords != null) ? new StopFilter(matchVersion, tokenizer, stopWords) : tokenizer;
     return new TokenStreamComponents(tokenizer, result);
   }
   
   /**
    * Creates a token stream that tokenizes all the text in the given Reader;
-   * This implementation forwards to <code>tokenStream(String, String)</code> and is
-   * less efficient than <code>tokenStream(String, String)</code>.
+   * This implementation forwards to <code>tokenStream(String, Reader, String)</code> and is
+   * less efficient than <code>tokenStream(String, Reader, String)</code>.
    * 
    * @param fieldName
    *            the name of the field to tokenize (currently ignored).
@@ -219,7 +221,7 @@ public final class PatternAnalyzer exten
   public TokenStreamComponents createComponents(String fieldName, Reader reader) {
     try {
       String text = toString(reader);
-      return createComponents(fieldName, text);
+      return createComponents(fieldName, reader, text);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -332,7 +334,8 @@ public final class PatternAnalyzer exten
     private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
     private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
     
-    public PatternTokenizer(String str, Pattern pattern, boolean toLowerCase) {
+    public PatternTokenizer(Reader input, String str, Pattern pattern, boolean toLowerCase) {
+      super(input);
       this.pattern = pattern;
       this.str = str;
       this.matcher = pattern.matcher(str);
@@ -359,7 +362,7 @@ public final class PatternAnalyzer exten
           String text = str.substring(start, end);
           if (toLowerCase) text = text.toLowerCase(locale);
           termAtt.setEmpty().append(text);
-          offsetAtt.setOffset(start, end);
+          offsetAtt.setOffset(correctOffset(start), correctOffset(end));
           return true;
         }
         if (!isMatch) return false;
@@ -369,7 +372,7 @@ public final class PatternAnalyzer exten
     @Override
     public final void end() {
       // set final offset
-      final int finalOffset = str.length();
+      final int finalOffset = correctOffset(str.length());
     	this.offsetAtt.setOffset(finalOffset, finalOffset);
     }
 
@@ -406,7 +409,8 @@ public final class PatternAnalyzer exten
     private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
     private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
     
-    public FastStringTokenizer(String str, boolean isLetter, boolean toLowerCase, Set<?> stopWords) {
+    public FastStringTokenizer(Reader input, String str, boolean isLetter, boolean toLowerCase, Set<?> stopWords) {
+      super(input);
       this.str = str;
       this.isLetter = isLetter;
       this.toLowerCase = toLowerCase;
@@ -458,7 +462,7 @@ public final class PatternAnalyzer exten
         return false;
       }
       termAtt.setEmpty().append(text);
-      offsetAtt.setOffset(start, i);
+      offsetAtt.setOffset(correctOffset(start), correctOffset(i));
       return true;
     }
     
@@ -466,7 +470,7 @@ public final class PatternAnalyzer exten
     public final void end() {
       // set final offset
       final int finalOffset = str.length();
-      this.offsetAtt.setOffset(finalOffset, finalOffset);
+      this.offsetAtt.setOffset(correctOffset(finalOffset), correctOffset(finalOffset));
     }    
     
     private boolean isTokenChar(char c, boolean isLetter) {
@@ -479,6 +483,7 @@ public final class PatternAnalyzer exten
 
     @Override
     public void reset(Reader input) throws IOException {
+      super.reset(input);
       this.str = PatternAnalyzer.toString(input);
     }
 

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilter.java Mon Jan 23 22:15:15 2012
@@ -68,7 +68,7 @@ public final class TrimFilter extends To
       } else {
         termAtt.setEmpty();
       }
-      if (updateOffsets) {
+      if (updateOffsets && len == offsetAtt.endOffset() - offsetAtt.startOffset()) {
         int newStart = offsetAtt.startOffset()+start;
         int newEnd = offsetAtt.endOffset() - (start<end ? endOff:0);
         offsetAtt.setOffset(newStart, newEnd);

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilter.java Mon Jan 23 22:15:15 2012
@@ -405,10 +405,20 @@ public final class WordDelimiterFilter e
     clearAttributes();
     termAttribute.copyBuffer(savedBuffer, iterator.current, iterator.end - iterator.current);
 
-    int startOffSet = (isSingleWord || !hasIllegalOffsets) ? savedStartOffset + iterator.current : savedStartOffset;
-    int endOffSet = (hasIllegalOffsets) ? savedEndOffset : savedStartOffset + iterator.end;
-
-    offsetAttribute.setOffset(startOffSet, endOffSet);
+    int startOffset = savedStartOffset + iterator.current;
+    int endOffset = savedStartOffset + iterator.end;
+    
+    if (hasIllegalOffsets) {
+      // historically this filter did this regardless for 'isSingleWord', 
+      // but we must do a sanity check:
+      if (isSingleWord && startOffset <= savedEndOffset) {
+        offsetAttribute.setOffset(startOffset, savedEndOffset);
+      } else {
+        offsetAttribute.setOffset(savedStartOffset, savedEndOffset);
+      }
+    } else {
+      offsetAttribute.setOffset(startOffset, endOffset);
+    }
     posIncAttribute.setPositionIncrement(position(false));
     typeAttribute.setType(savedType);
   }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java Mon Jan 23 22:15:15 2012
@@ -68,6 +68,7 @@ public final class ThaiWordFilter extend
   private CharTermAttribute clonedTermAtt = null;
   private OffsetAttribute clonedOffsetAtt = null;
   private boolean hasMoreTokensInClone = false;
+  private boolean hasIllegalOffsets = false; // only if the length changed before this filter
 
   /** Creates a new ThaiWordFilter with the specified match version. */
   public ThaiWordFilter(Version matchVersion, TokenStream input) {
@@ -86,7 +87,11 @@ public final class ThaiWordFilter extend
       if (end != BreakIterator.DONE) {
         clonedToken.copyTo(this);
         termAtt.copyBuffer(clonedTermAtt.buffer(), start, end - start);
-        offsetAtt.setOffset(clonedOffsetAtt.startOffset() + start, clonedOffsetAtt.startOffset() + end);
+        if (hasIllegalOffsets) {
+          offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.endOffset());
+        } else {
+          offsetAtt.setOffset(clonedOffsetAtt.startOffset() + start, clonedOffsetAtt.startOffset() + end);
+        }
         if (handlePosIncr) posAtt.setPositionIncrement(1);
         return true;
       }
@@ -102,6 +107,10 @@ public final class ThaiWordFilter extend
     }
     
     hasMoreTokensInClone = true;
+    
+    // if length by start + end offsets doesn't match the term text then assume
+    // this is a synonym and don't adjust the offsets.
+    hasIllegalOffsets = offsetAtt.endOffset() - offsetAtt.startOffset() != termAtt.length();
 
     // we lazy init the cloned token, as in ctor not all attributes may be added
     if (clonedToken == null) {
@@ -118,7 +127,11 @@ public final class ThaiWordFilter extend
     int end = breaker.next();
     if (end != BreakIterator.DONE) {
       termAtt.setLength(end);
-      offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.startOffset() + end);
+      if (hasIllegalOffsets) {
+        offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.endOffset());
+      } else {
+        offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.startOffset() + end);
+      }
       // position increment keeps as it is for first token
       return true;
     }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java Mon Jan 23 22:15:15 2012
@@ -306,13 +306,14 @@ public final class WikipediaTokenizer ex
   @Override
   public void reset() throws IOException {
     super.reset();
-    scanner.yyreset(input);
+    tokens = null;
+    scanner.reset();
   }
 
   @Override
   public void reset(Reader reader) throws IOException {
     super.reset(reader);
-    reset();
+    scanner.yyreset(input);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.java Mon Jan 23 22:15:15 2012
@@ -1,4 +1,4 @@
-/* The following code was generated by JFlex 1.5.0-SNAPSHOT on 9/30/11 12:11 PM */
+/* The following code was generated by JFlex 1.5.0-SNAPSHOT on 1/22/12 10:26 PM */
 
 package org.apache.lucene.analysis.wikipedia;
 
@@ -25,8 +25,8 @@ import org.apache.lucene.analysis.tokena
 /**
  * This class is a scanner generated by 
  * <a href="http://www.jflex.de/">JFlex</a> 1.5.0-SNAPSHOT
- * on 9/30/11 12:11 PM from the specification file
- * <tt>/lucene/jflex/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex</tt>
+ * on 1/22/12 10:26 PM from the specification file
+ * <tt>/home/rmuir/workspace/lucene-clean-trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex</tt>
  */
 class WikipediaTokenizerImpl {
 
@@ -498,6 +498,14 @@ final int setText(StringBuilder buffer){
   return length;
 }
 
+final void reset() {
+  currentTokType = 0;
+  numBalanced = 0;
+  positionInc = 1;
+  numLinkToks = 0;
+  numWikiTokensSeen = 0;
+}
+
 
 
 

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex Mon Jan 23 22:15:15 2012
@@ -91,6 +91,14 @@ final int setText(StringBuilder buffer){
   return length;
 }
 
+final void reset() {
+  currentTokType = 0;
+  numBalanced = 0;
+  positionInc = 1;
+  numLinkToks = 0;
+  numWikiTokensSeen = 0;
+}
+
 
 %}
 

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterTest.java Mon Jan 23 22:15:15 2012
@@ -503,7 +503,7 @@ public class HTMLStripCharFilterTest ext
 
       @Override
       protected Reader initReader(Reader reader) {
-        return new HTMLStripCharFilter(CharReader.get(new BufferedReader(reader)));
+        return new HTMLStripCharFilter(CharReader.get(reader));
       }
     };
     
@@ -817,4 +817,47 @@ public class HTMLStripCharFilterTest ext
         (CharReader.get(new StringReader(text.toString())));
     while (reader.read() != -1);
   }
+
+  public void testUTF16Surrogates() throws Exception {
+    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 HTMLStripCharFilter(CharReader.get(new BufferedReader(reader)));
+      }
+    };
+    // Paired surrogates
+    assertAnalyzesTo(analyzer, " one two &#xD86C;&#XdC01;three",
+        new String[] { "one", "two", "\uD86C\uDC01three" } );
+    assertAnalyzesTo(analyzer, " &#55404;&#XdC01;", new String[] { "\uD86C\uDC01" } );
+    assertAnalyzesTo(analyzer, " &#xD86C;&#56321;", new String[] { "\uD86C\uDC01" } );
+    assertAnalyzesTo(analyzer, " &#55404;&#56321;", new String[] { "\uD86C\uDC01" } );
+
+    // Improperly paired surrogates
+    assertAnalyzesTo(analyzer, " &#55404;&#57999;", new String[] { "\uFFFD\uE28F" } );
+    assertAnalyzesTo(analyzer, " &#xD86C;&#57999;", new String[] { "\uFFFD\uE28F" } );
+    assertAnalyzesTo(analyzer, " &#55002;&#XdC01;", new String[] { "\uD6DA\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#55002;&#56321;", new String[] { "\uD6DA\uFFFD" } );
+
+    // Unpaired high surrogates
+    assertAnalyzesTo(analyzer, " &#Xd921;", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#Xd921", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#Xd921<br>", new String[] { "&#Xd921" } );
+    assertAnalyzesTo(analyzer, " &#55528;", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#55528", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#55528<br>", new String[] { "&#55528" } );
+
+    // Unpaired low surrogates
+    assertAnalyzesTo(analyzer, " &#xdfdb;", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#xdfdb", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#xdfdb<br>", new String[] { "&#xdfdb" } );
+    assertAnalyzesTo(analyzer, " &#57209;", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#57209", new String[] { "\uFFFD" } );
+    assertAnalyzesTo(analyzer, " &#57209<br>", new String[] { "&#57209" } );
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/cn/TestChineseTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/cn/TestChineseTokenizer.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/cn/TestChineseTokenizer.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/cn/TestChineseTokenizer.java Mon Jan 23 22:15:15 2012
@@ -117,5 +117,10 @@ public class TestChineseTokenizer extend
       assertAnalyzesTo(justFilter, "This is a Test. b c d", 
           new String[] { "This", "Test." });
     }
+    
+    /** blast some random strings through the analyzer */
+    public void testRandomStrings() throws Exception {
+      checkRandomData(random, new ChineseAnalyzer(), 10000*RANDOM_MULTIPLIER);
+    }
 
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java Mon Jan 23 22:15:15 2012
@@ -306,4 +306,31 @@ public class CommonGramsFilterTest exten
     TokenFilter nsf = new CommonGramsQueryFilter(cgf);
     assertTokenStreamContents(nsf, new String[] { "the_of" });
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        CommonGramsFilter cgf = new CommonGramsFilter(TEST_VERSION_CURRENT, t, commonWords);
+        return new TokenStreamComponents(t, cgf);
+      }
+    };
+    
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+    
+    Analyzer b = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        CommonGramsFilter cgf = new CommonGramsFilter(TEST_VERSION_CURRENT, t, commonWords);
+        return new TokenStreamComponents(t, new CommonGramsQueryFilter(cgf));
+      }
+    };
+    
+    checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java Mon Jan 23 22:15:15 2012
@@ -18,14 +18,19 @@ package org.apache.lucene.analysis.compo
  */
 
 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.CharReader;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.charfilter.MappingCharFilter;
+import org.apache.lucene.analysis.charfilter.NormalizeCharMap;
 import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
 import org.apache.lucene.analysis.util.CharArraySet;
@@ -299,5 +304,61 @@ public class TestCompoundWordTokenFilter
       }
     }
   }
+  
+  // SOLR-2891
+  // *CompoundWordTokenFilter blindly adds term length to offset, but this can take things out of bounds
+  // wrt original text if a previous filter increases the length of the word (in this case ü -> ue)
+  // so in this case we behave like WDF, and preserve any modified offsets
+  public void testInvalidOffsets() throws Exception {
+    final CharArraySet dict = makeDictionary("fall");
+    final NormalizeCharMap normMap = new NormalizeCharMap();
+    normMap.add("ü", "ue");
+    
+    Analyzer analyzer = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        TokenFilter filter = new DictionaryCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, dict);
+        return new TokenStreamComponents(tokenizer, filter);
+      }
+
+      @Override
+      protected Reader initReader(Reader reader) {
+        return new MappingCharFilter(normMap, CharReader.get(reader));
+      }
+    };
 
+    assertAnalyzesTo(analyzer, "banküberfall", 
+        new String[] { "bankueberfall", "fall" },
+        new int[] { 0,  0 },
+        new int[] { 12, 12 });
+  }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    final CharArraySet dict = makeDictionary("a", "e", "i", "o", "u", "y", "bc", "def");
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new DictionaryCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, dict));
+      }
+    };
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+    
+    InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
+    final HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter.getHyphenationTree(is);
+    Analyzer b = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        TokenFilter filter = new HyphenationCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, hyphenator);
+        return new TokenStreamComponents(tokenizer, filter);
+      }
+    };
+    checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java Mon Jan 23 22:15:15 2012
@@ -18,12 +18,15 @@ package org.apache.lucene.analysis.hunsp
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.Reader;
 import java.io.StringReader;
 import java.text.ParseException;
 import java.util.Arrays;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.junit.BeforeClass;
@@ -57,4 +60,17 @@ public class HunspellStemFilterTest  ext
     filter = new HunspellStemFilter(new KeywordMarkerFilter(tokenizer, set), DICTIONARY);
     assertTokenStreamContents(filter, new String[]{"lucene", "is", "awesome"}, new int[] {1, 1, 1});
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer analyzer = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, DICTIONARY));
+      }  
+    };
+    checkRandomData(random, analyzer, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/PatternAnalyzerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/PatternAnalyzerTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/PatternAnalyzerTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/PatternAnalyzerTest.java Mon Jan 23 22:15:15 2012
@@ -22,6 +22,7 @@ import java.io.StringReader;
 import java.util.Arrays;
 import java.util.regex.Pattern;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.core.StopAnalyzer;
@@ -132,4 +133,10 @@ public class PatternAnalyzerTest extends
     TokenStream ts2 = analyzer.tokenStream("dummy", new StringReader(document));
     assertTokenStreamContents(ts2, expected);
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer a = new PatternAnalyzer(TEST_VERSION_CURRENT, Pattern.compile(","), true, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestASCIIFoldingFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestASCIIFoldingFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestASCIIFoldingFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestASCIIFoldingFilter.java Mon Jan 23 22:15:15 2012
@@ -17,11 +17,14 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.List;
 import java.util.ArrayList;
@@ -1907,4 +1910,17 @@ public class TestASCIIFoldingFilter exte
     assertTrue(stream.incrementToken());
     assertEquals(expected, termAtt.toString());
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new ASCIIFoldingFilter(tokenizer));
+      } 
+    };
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestCapitalizationFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestCapitalizationFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestCapitalizationFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestCapitalizationFilter.java Mon Jan 23 22:15:15 2012
@@ -18,12 +18,14 @@
 package org.apache.lucene.analysis.miscellaneous;
 
 import java.io.IOException;
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Tokenizer;
@@ -117,4 +119,18 @@ public class TestCapitalizationFilter ex
         new String[] { expected }, onlyFirstWord, keep, forceFirstLetter, okPrefix,
         minWordLength, maxWordCount, maxTokenLength);    
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomString() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new CapitalizationFilter(tokenizer));
+      }
+    };
+    
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestHyphenatedWordsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestHyphenatedWordsFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestHyphenatedWordsFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestHyphenatedWordsFilter.java Mon Jan 23 22:15:15 2012
@@ -17,11 +17,14 @@
 
 package org.apache.lucene.analysis.miscellaneous;
 
+import java.io.Reader;
 import java.io.StringReader;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
 
 /**
  * HyphenatedWordsFilter test
@@ -46,5 +49,29 @@ public class TestHyphenatedWordsFilter e
 	    ts = new HyphenatedWordsFilter(ts);
 	    assertTokenStreamContents(ts, 
 	        new String[] { "ecological", "develop", "comprehensive-hands-on", "and", "ecology-" });
-	  }
+	}
+	
+	public void testOffsets() throws Exception {
+	  String input = "abc- def geh 1234- 5678-";
+    TokenStream ts = new MockTokenizer(new StringReader(input), MockTokenizer.WHITESPACE, false);
+    ts = new HyphenatedWordsFilter(ts);
+    assertTokenStreamContents(ts, 
+        new String[] { "abcdef", "geh", "12345678-" },
+        new int[] { 0, 9, 13 },
+        new int[] { 8, 12, 24 });
+	}
+	
+  /** blast some random strings through the analyzer */
+  public void testRandomString() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new HyphenatedWordsFilter(tokenizer));
+      }
+    };
+    
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepWordFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepWordFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepWordFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepWordFilter.java Mon Jan 23 22:15:15 2012
@@ -17,13 +17,16 @@
 
 package org.apache.lucene.analysis.miscellaneous;
 
+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;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.util.CharArraySet;
 
 /** Test {@link KeepWordFilter} */
@@ -57,4 +60,23 @@ public class TestKeepWordFilter extends 
     stream = new KeepWordFilter(false, stream, new CharArraySet(TEST_VERSION_CURRENT,words, false));
     assertTokenStreamContents(stream, new String[] { "aaa" }, new int[] { 1 });
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    final Set<String> words = new HashSet<String>();
+    words.add( "a" );
+    words.add( "b" );
+    
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        TokenStream stream = new KeepWordFilter(true, tokenizer, new CharArraySet(TEST_VERSION_CURRENT, words, true));
+        return new TokenStreamComponents(tokenizer, stream);
+      }
+    };
+    
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestRemoveDuplicatesTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestRemoveDuplicatesTokenFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestRemoveDuplicatesTokenFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestRemoveDuplicatesTokenFilter.java Mon Jan 23 22:15:15 2012
@@ -17,13 +17,21 @@
 
 package org.apache.lucene.analysis.miscellaneous;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.synonym.SynonymFilter;
+import org.apache.lucene.analysis.synonym.SynonymMap;
 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.util.CharsRef;
+import org.apache.lucene.util._TestUtil;
 
+import java.io.Reader;
 import java.util.Iterator;
 import java.util.Arrays;
 
@@ -116,6 +124,45 @@ public class TestRemoveDuplicatesTokenFi
              
   }
   
+  // some helper methods for the below test with synonyms
+  private String randomNonEmptyString() {
+    while(true) {
+      final String s = _TestUtil.randomUnicodeString(random).trim();
+      if (s.length() != 0 && s.indexOf('\u0000') == -1) {
+        return s;
+      }
+    }
+  }
+  
+  private void add(SynonymMap.Builder b, String input, String output, boolean keepOrig) {
+    b.add(new CharsRef(input.replaceAll(" +", "\u0000")),
+          new CharsRef(output.replaceAll(" +", "\u0000")),
+          keepOrig);
+  }
   
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    final int numIters = atLeast(10);
+    for (int i = 0; i < numIters; i++) {
+      SynonymMap.Builder b = new SynonymMap.Builder(random.nextBoolean());
+      final int numEntries = atLeast(10);
+      for (int j = 0; j < numEntries; j++) {
+        add(b, 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 stream = new SynonymFilter(tokenizer, map, ignoreCase);
+          return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
+        }
+      };
+
+      checkRandomData(random, analyzer, 1000*RANDOM_MULTIPLIER);
+    }
+  }
 
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTrimFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTrimFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTrimFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTrimFilter.java Mon Jan 23 22:15:15 2012
@@ -18,11 +18,15 @@
 package org.apache.lucene.analysis.miscellaneous;
 
 import java.io.IOException;
+import java.io.Reader;
 import java.util.Collection;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.*;
 
 /**
@@ -103,4 +107,27 @@ public class TestTrimFilter extends Base
       }
     }
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.KEYWORD, false);
+        return new TokenStreamComponents(tokenizer, new TrimFilter(tokenizer, false));
+      } 
+    };
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+    
+    Analyzer b = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.KEYWORD, false);
+        return new TokenStreamComponents(tokenizer, new TrimFilter(tokenizer, true));
+      } 
+    };
+    checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java Mon Jan 23 22:15:15 2012
@@ -298,4 +298,28 @@ public class TestWordDelimiterFilter ext
         new int[] { 10, 15, 15 },
         new int[] { 2, 1, 0 });
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    int numIterations = atLeast(5);
+    for (int i = 0; i < numIterations; i++) {
+      final int flags = random.nextInt(512);
+      final CharArraySet protectedWords;
+      if (random.nextBoolean()) {
+        protectedWords = new CharArraySet(TEST_VERSION_CURRENT, new HashSet<String>(Arrays.asList("a", "b", "cd")), false);
+      } else {
+        protectedWords = null;
+      }
+      
+      Analyzer a = new Analyzer() {
+        
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, new WordDelimiterFilter(tokenizer, flags, protectedWords));
+        }
+      };
+      checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+    }
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java Mon Jan 23 22:15:15 2012
@@ -160,7 +160,7 @@ public class TestSegmentingTokenizerBase
         hasSentence = false;
         clearAttributes();
         termAtt.copyBuffer(buffer, sentenceStart, sentenceEnd-sentenceStart);
-        offsetAtt.setOffset(offset+sentenceStart, offset+sentenceEnd);
+        offsetAtt.setOffset(correctOffset(offset+sentenceStart), correctOffset(offset+sentenceEnd));
         return true;
       } else {
         return false;
@@ -215,7 +215,7 @@ public class TestSegmentingTokenizerBase
       
       clearAttributes();
       termAtt.copyBuffer(buffer, wordStart, wordEnd-wordStart);
-      offsetAtt.setOffset(offset+wordStart, offset+wordEnd);
+      offsetAtt.setOffset(correctOffset(offset+wordStart), correctOffset(offset+wordEnd));
       posIncAtt.setPositionIncrement(posIncAtt.getPositionIncrement() + posBoost);
       posBoost = 0;
       return true;

Modified: lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/common/src/test/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerTest.java Mon Jan 23 22:15:15 2012
@@ -18,12 +18,15 @@
 
 package org.apache.lucene.analysis.wikipedia;
 
+import java.io.Reader;
 import java.io.StringReader;
 import java.io.IOException;
 import java.util.Set;
 import java.util.HashSet;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 
 import static org.apache.lucene.analysis.wikipedia.WikipediaTokenizer.*;
@@ -169,4 +172,17 @@ public class WikipediaTokenizerTest exte
     assertFalse(tf.incrementToken());
     tf.close();
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new WikipediaTokenizer(reader);
+        return new TokenStreamComponents(tokenizer, tokenizer);
+      } 
+    };
+    checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java Mon Jan 23 22:15:15 2012
@@ -111,7 +111,7 @@ public final class ICUTokenizer extends 
   @Override
   public void end() throws IOException {
     final int finalOffset = (length < 0) ? offset : offset + length;
-    offsetAtt.setOffset(finalOffset, finalOffset);
+    offsetAtt.setOffset(correctOffset(finalOffset), correctOffset(finalOffset));
   }  
 
   /*

Modified: lucene/dev/branches/lucene2858/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java Mon Jan 23 22:15:15 2012
@@ -102,4 +102,9 @@ public class TestMorfologikAnalyzer exte
     assertPOSToken(ts, "list",  "subst:sg:loc.voc:m3");
     assertPOSToken(ts, "lista", "subst:sg:dat.loc:f");
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandom() throws Exception {
+    checkRandomData(random, getTestAnalyzer(), 10000 * RANDOM_MULTIPLIER); 
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java Mon Jan 23 22:15:15 2012
@@ -16,11 +16,17 @@
  */
 package org.apache.lucene.analysis.phonetic;
 
+import java.io.Reader;
 import java.io.StringReader;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.util._TestUtil;
 
 public class DoubleMetaphoneFilterTest extends BaseTokenStreamTestCase {
 
@@ -65,4 +71,28 @@ public class DoubleMetaphoneFilterTest e
     assertTokenStreamContents(filter, new String[] { "12345", "#$%@#^%&", "HL" });
   }
 
+  public void testRandom() throws Exception {
+    final int codeLen = _TestUtil.nextInt(random, 1, 8);
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, false));
+      }
+      
+    };
+    checkRandomData(random, a, 1000 * RANDOM_MULTIPLIER);
+    
+    Analyzer b = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, true));
+      }
+      
+    };
+    checkRandomData(random, b, 1000 * RANDOM_MULTIPLIER); 
+  }
 }

Modified: lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java (original)
+++ lucene/dev/branches/lucene2858/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java Mon Jan 23 22:15:15 2012
@@ -17,6 +17,8 @@
 
 package org.apache.lucene.analysis.phonetic;
 
+import java.io.IOException;
+import java.io.Reader;
 import java.io.StringReader;
 
 import org.apache.commons.codec.Encoder;
@@ -25,7 +27,9 @@ import org.apache.commons.codec.language
 import org.apache.commons.codec.language.Metaphone;
 import org.apache.commons.codec.language.RefinedSoundex;
 import org.apache.commons.codec.language.Soundex;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
 
@@ -70,4 +74,33 @@ public class TestPhoneticFilter extends 
     PhoneticFilter filter = new PhoneticFilter(tokenizer, encoder, inject);
     assertTokenStreamContents(filter, expected);
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws IOException {
+    Encoder encoders[] = new Encoder[] {
+      new Metaphone(), new DoubleMetaphone(), new Soundex(), new RefinedSoundex(), new Caverphone()
+    };
+    
+    for (final Encoder e : encoders) {
+      Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
+        }   
+      };
+      
+      checkRandomData(random, a, 1000*RANDOM_MULTIPLIER);
+      
+      Analyzer b = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
+        }   
+      };
+      
+      checkRandomData(random, b, 1000*RANDOM_MULTIPLIER);
+    }
+  }
 }

Modified: lucene/dev/branches/lucene2858/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/solr/CHANGES.txt?rev=1235028&r1=1235027&r2=1235028&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene2858/solr/CHANGES.txt Mon Jan 23 22:15:15 2012
@@ -450,6 +450,10 @@ New Features
 * SOLR-1709: Distributed support for Date and Numeric Range Faceting
   (Peter Sturge, David Smiley, hossman, Simon Willnauer)
 
+* SOLR-3054, LUCENE-3671: Add TypeTokenFilterFactory that creates TypeTokenFilter
+  that filters tokens based on their TypeAttribute.  (Tommaso Teofili via
+  Uwe Schindler)
+
 Optimizations
 ----------------------
 * SOLR-1931: Speedup for LukeRequestHandler and admin/schema browser. New parameter
@@ -509,6 +513,11 @@ Bug Fixes
     from Unicode character classes [:ID_Start:] and [:ID_Continue:].
   - Uppercase character entities "&QUOT;", "&COPY;", "&GT;", "&LT;", "&REG;",
     and "&AMP;" are now recognized and handled as if they were in lowercase.
+  - The REPLACEMENT CHARACTER U+FFFD is now used to replace numeric character 
+    entities for unpaired UTF-16 low and high surrogates (in the range
+    [U+D800-U+DFFF]).
+  - Properly paired numeric character entities for UTF-16 surrogates are now
+    converted to the corresponding code units.
   - Opening tags with unbalanced quotation marks are now properly stripped.
   - Literal "<" and ">" characters in opening tags, regardless of whether they
     appear inside quotation marks, now inhibit recognition (and stripping) of
@@ -526,6 +535,10 @@ Bug Fixes
     attributes and their values, are left intact in the output.
   (Steve Rowe)
 
+* LUCENE-3717: Fixed offset bugs in TrimFilter, WordDelimiterFilter, and
+  HyphenatedWordsFilter where they would create invalid offsets in
+  some situations, leading to problems in highlighting.  (Robert Muir)
+
 Other Changes
 ----------------------
 * SOLR-2922: Upgrade commons-io and commons-lang to 2.1 and 2.6, respectively. (koji)