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 2013/10/21 20:58:44 UTC

svn commit: r1534320 [3/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/ ...

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java Mon Oct 21 18:58:24 2013
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.miscellaneous.LengthFilter;
+import org.apache.lucene.analysis.miscellaneous.CodepointCountFilter;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
@@ -81,7 +81,7 @@ public final class NGramTokenFilter exte
    * @param maxGram the largest n-gram to generate
    */
   public NGramTokenFilter(Version version, TokenStream input, int minGram, int maxGram) {
-    super(new LengthFilter(version, input, minGram, Integer.MAX_VALUE));
+    super(new CodepointCountFilter(version, input, minGram, Integer.MAX_VALUE));
     this.version = version;
     this.charUtils = version.onOrAfter(Version.LUCENE_44)
         ? CharacterUtils.getInstance(version)

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java Mon Oct 21 18:58:24 2013
@@ -217,13 +217,15 @@ public class NGramTokenizer extends Toke
   }
 
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     assert bufferStart <= bufferEnd;
     int endOffset = offset;
     for (int i = bufferStart; i < bufferEnd; ++i) {
       endOffset += Character.charCount(buffer[i]);
     }
     endOffset = correctOffset(endOffset);
+    // set final offset
     offsetAtt.setOffset(endOffset, endOffset);
   }
 

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java Mon Oct 21 18:58:24 2013
@@ -191,7 +191,8 @@ public class PathHierarchyTokenizer exte
   }
 
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     int finalOffset = correctOffset(charsRead);
     offsetAtt.setOffset(finalOffset, finalOffset);

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/ReversePathHierarchyTokenizer.java Mon Oct 21 18:58:24 2013
@@ -176,7 +176,8 @@ public class ReversePathHierarchyTokeniz
   }
 
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     offsetAtt.setOffset(finalOffset, finalOffset);
   }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternTokenizer.java Mon Oct 21 18:58:24 2013
@@ -130,13 +130,15 @@ public final class PatternTokenizer exte
   }
 
   @Override
-  public void end() {
+  public void end() throws IOException {
+    super.end();
     final int ofs = correctOffset(str.length());
     offsetAtt.setOffset(ofs, ofs);
   }
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     fillBuffer(str, input);
     matcher.reset(str);
     index = 0;

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -148,6 +148,7 @@ public final class QueryAutoStopWordAnal
       IndexReader indexReader,
       Collection<String> fields,
       int maxDocFreq) throws IOException {
+    super(delegate.getReuseStrategy());
     this.matchVersion = matchVersion;
     this.delegate = delegate;
     

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java Mon Oct 21 18:58:24 2013
@@ -30,7 +30,7 @@ import org.apache.lucene.util.Version;
  */
 public final class ShingleAnalyzerWrapper extends AnalyzerWrapper {
 
-  private final Analyzer defaultAnalyzer;
+  private final Analyzer delegate;
   private final int maxShingleSize;
   private final int minShingleSize;
   private final String tokenSeparator;
@@ -52,7 +52,7 @@ public final class ShingleAnalyzerWrappe
   /**
    * Creates a new ShingleAnalyzerWrapper
    *
-   * @param defaultAnalyzer Analyzer whose TokenStream is to be filtered
+   * @param delegate Analyzer whose TokenStream is to be filtered
    * @param minShingleSize Min shingle (token ngram) size
    * @param maxShingleSize Max shingle size
    * @param tokenSeparator Used to separate input stream tokens in output shingles
@@ -65,13 +65,14 @@ public final class ShingleAnalyzerWrappe
    *        regardless of whether any shingles are available.
    */
   public ShingleAnalyzerWrapper(
-      Analyzer defaultAnalyzer,
+      Analyzer delegate,
       int minShingleSize,
       int maxShingleSize,
       String tokenSeparator,
       boolean outputUnigrams,
       boolean outputUnigramsIfNoShingles) {
-    this.defaultAnalyzer = defaultAnalyzer;
+    super(delegate.getReuseStrategy());
+    this.delegate = delegate;
 
     if (maxShingleSize < 2) {
       throw new IllegalArgumentException("Max shingle size must be >= 2");
@@ -138,7 +139,7 @@ public final class ShingleAnalyzerWrappe
 
   @Override
   protected Analyzer getWrappedAnalyzer(String fieldName) {
-    return defaultAnalyzer;
+    return delegate;
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java Mon Oct 21 18:58:24 2013
@@ -147,6 +147,12 @@ public final class ShingleFilter extends
    * true if no shingles have been output yet (for outputUnigramsIfNoShingles).
    */
   boolean noShingleOutput = true;
+
+  /**
+   * Holds the State after input.end() was called, so we can
+   * restore it in our end() impl.
+   */
+  private State endState;
   
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
@@ -279,7 +285,7 @@ public final class ShingleFilter extends
   }
 
   @Override
-  public final boolean incrementToken() throws IOException {
+  public boolean incrementToken() throws IOException {
     boolean tokenAvailable = false;
     int builtGramSize = 0;
     if (gramSize.atMinValue() || inputWindow.size() < gramSize.getValue()) {
@@ -364,39 +370,63 @@ public final class ShingleFilter extends
       }
       isNextInputStreamToken = false;
       newTarget.isFiller = false;
-    } else if (!exhausted && input.incrementToken()) {
-      if (null == target) {
-        newTarget = new InputWindowToken(cloneAttributes());
-      } else {
-        this.copyTo(target.attSource);
-      }
-      if (posIncrAtt.getPositionIncrement() > 1) {
-        // Each output shingle must contain at least one input token, 
-        // so no more than (maxShingleSize - 1) filler tokens will be inserted.
-        numFillerTokensToInsert 
-          = Math.min(posIncrAtt.getPositionIncrement() - 1, maxShingleSize - 1);
-        // Save the current token as the next input stream token
-        if (null == nextInputStreamToken) {
-          nextInputStreamToken = cloneAttributes();
+    } else if (!exhausted) {
+      if (input.incrementToken()) {
+        if (null == target) {
+          newTarget = new InputWindowToken(cloneAttributes());
+        } else {
+          this.copyTo(target.attSource);
+        }
+        if (posIncrAtt.getPositionIncrement() > 1) {
+          // Each output shingle must contain at least one input token, 
+          // so no more than (maxShingleSize - 1) filler tokens will be inserted.
+          numFillerTokensToInsert = Math.min(posIncrAtt.getPositionIncrement() - 1, maxShingleSize - 1);
+          // Save the current token as the next input stream token
+          if (null == nextInputStreamToken) {
+            nextInputStreamToken = cloneAttributes();
+          } else {
+            this.copyTo(nextInputStreamToken);
+          }
+          isNextInputStreamToken = true;
+          // A filler token occupies no space
+          newTarget.offsetAtt.setOffset(offsetAtt.startOffset(), offsetAtt.startOffset());
+          newTarget.termAtt.copyBuffer(FILLER_TOKEN, 0, FILLER_TOKEN.length);
+          newTarget.isFiller = true;
+          --numFillerTokensToInsert;
         } else {
-          this.copyTo(nextInputStreamToken);
+          newTarget.isFiller = false;
         }
-        isNextInputStreamToken = true;
-        // A filler token occupies no space
-        newTarget.offsetAtt.setOffset(offsetAtt.startOffset(), offsetAtt.startOffset());
-        newTarget.termAtt.copyBuffer(FILLER_TOKEN, 0, FILLER_TOKEN.length);
-        newTarget.isFiller = true;
-        --numFillerTokensToInsert;
       } else {
-        newTarget.isFiller = false;
+        exhausted = true;
+        input.end();
+        endState = captureState();
+        numFillerTokensToInsert = Math.min(posIncrAtt.getPositionIncrement(), maxShingleSize - 1);
+        if (numFillerTokensToInsert > 0) {
+          nextInputStreamToken = new AttributeSource(getAttributeFactory());
+          nextInputStreamToken.addAttribute(CharTermAttribute.class);
+          OffsetAttribute newOffsetAtt = nextInputStreamToken.addAttribute(OffsetAttribute.class);
+          newOffsetAtt.setOffset(offsetAtt.endOffset(), offsetAtt.endOffset());
+          // Recurse/loop just once:
+          return getNextToken(target);
+        } else {
+          newTarget = null;
+        }
       }
     } else {
       newTarget = null;
-      exhausted = true;
     }
     return newTarget;
   }
 
+  @Override
+  public void end() throws IOException {
+    if (!exhausted) {
+      super.end();
+    } else {
+      restoreState(endState);
+    }
+  }
+
   /**
    * <p>Fills {@link #inputWindow} with input stream tokens, if available, 
    * shifting to the right if the window was previously full.
@@ -445,6 +475,7 @@ public final class ShingleFilter extends
     isOutputHere = false;
     noShingleOutput = true;
     exhausted = false;
+    endState = null;
     if (outputUnigramsIfNoShingles && ! outputUnigrams) {
       // Fix up gramSize if minValue was reset for outputUnigramsIfNoShingles
       gramSize.minValue = minShingleSize;

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java Mon Oct 21 18:58:24 2013
@@ -76,6 +76,8 @@ public final class ClassicTokenizer exte
     "<CJ>",
     "<ACRONYM_DEP>"
   };
+  
+  private int skippedPositions;
 
   private int maxTokenLength = StandardAnalyzer.DEFAULT_MAX_TOKEN_LENGTH;
 
@@ -112,7 +114,7 @@ public final class ClassicTokenizer exte
   }
 
   private void init(Version matchVersion) {
-    this.scanner = new ClassicTokenizerImpl(null); // best effort NPE if you dont call reset
+    this.scanner = new ClassicTokenizerImpl(input);
   }
 
   // this tokenizer generates three attributes:
@@ -130,7 +132,7 @@ public final class ClassicTokenizer exte
   @Override
   public final boolean incrementToken() throws IOException {
     clearAttributes();
-    int posIncr = 1;
+    skippedPositions = 0;
 
     while(true) {
       int tokenType = scanner.getNextToken();
@@ -140,7 +142,7 @@ public final class ClassicTokenizer exte
       }
 
       if (scanner.yylength() <= maxTokenLength) {
-        posIncrAtt.setPositionIncrement(posIncr);
+        posIncrAtt.setPositionIncrement(skippedPositions+1);
         scanner.getText(termAtt);
         final int start = scanner.yychar();
         offsetAtt.setOffset(correctOffset(start), correctOffset(start+termAtt.length()));
@@ -155,19 +157,30 @@ public final class ClassicTokenizer exte
       } else
         // When we skip a too-long term, we still increment the
         // position increment
-        posIncr++;
+        skippedPositions++;
     }
   }
   
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     int finalOffset = correctOffset(scanner.yychar() + scanner.yylength());
     offsetAtt.setOffset(finalOffset, finalOffset);
+    // adjust any skipped tokens
+    posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement()+skippedPositions);
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    scanner.yyreset(input);
   }
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     scanner.yyreset(input);
+    skippedPositions = 0;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java Mon Oct 21 18:58:24 2013
@@ -90,6 +90,8 @@ public final class StandardTokenizer ext
     "<KATAKANA>",
     "<HANGUL>"
   };
+  
+  private int skippedPositions;
 
   private int maxTokenLength = StandardAnalyzer.DEFAULT_MAX_TOKEN_LENGTH;
 
@@ -126,7 +128,7 @@ public final class StandardTokenizer ext
   }
 
   private final void init(Version matchVersion) {
-    this.scanner = new StandardTokenizerImpl(null); // best effort NPE if you dont call reset
+    this.scanner = new StandardTokenizerImpl(input);
   }
 
   // this tokenizer generates three attributes:
@@ -144,7 +146,7 @@ public final class StandardTokenizer ext
   @Override
   public final boolean incrementToken() throws IOException {
     clearAttributes();
-    int posIncr = 1;
+    skippedPositions = 0;
 
     while(true) {
       int tokenType = scanner.getNextToken();
@@ -154,7 +156,7 @@ public final class StandardTokenizer ext
       }
 
       if (scanner.yylength() <= maxTokenLength) {
-        posIncrAtt.setPositionIncrement(posIncr);
+        posIncrAtt.setPositionIncrement(skippedPositions+1);
         scanner.getText(termAtt);
         final int start = scanner.yychar();
         offsetAtt.setOffset(correctOffset(start), correctOffset(start+termAtt.length()));
@@ -163,19 +165,30 @@ public final class StandardTokenizer ext
       } else
         // When we skip a too-long term, we still increment the
         // position increment
-        posIncr++;
+        skippedPositions++;
     }
   }
   
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     int finalOffset = correctOffset(scanner.yychar() + scanner.yylength());
     offsetAtt.setOffset(finalOffset, finalOffset);
+    // adjust any skipped tokens
+    posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement()+skippedPositions);
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    scanner.yyreset(input);
   }
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     scanner.yyreset(input);
+    skippedPositions = 0;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java Mon Oct 21 18:58:24 2013
@@ -76,6 +76,8 @@ public final class UAX29URLEmailTokenize
     "<URL>",
     "<EMAIL>",
   };
+  
+  private int skippedPositions;
 
   private int maxTokenLength = StandardAnalyzer.DEFAULT_MAX_TOKEN_LENGTH;
 
@@ -109,8 +111,8 @@ public final class UAX29URLEmailTokenize
     this.scanner = getScannerFor(matchVersion);
   }
 
-  private static StandardTokenizerInterface getScannerFor(Version matchVersion) {
-    return new UAX29URLEmailTokenizerImpl(null); // best effort NPE if you dont call reset
+  private StandardTokenizerInterface getScannerFor(Version matchVersion) {
+    return new UAX29URLEmailTokenizerImpl(input);
   }
 
   // this tokenizer generates three attributes:
@@ -123,7 +125,7 @@ public final class UAX29URLEmailTokenize
   @Override
   public final boolean incrementToken() throws IOException {
     clearAttributes();
-    int posIncr = 1;
+    skippedPositions = 0;
 
     while(true) {
       int tokenType = scanner.getNextToken();
@@ -133,7 +135,7 @@ public final class UAX29URLEmailTokenize
       }
 
       if (scanner.yylength() <= maxTokenLength) {
-        posIncrAtt.setPositionIncrement(posIncr);
+        posIncrAtt.setPositionIncrement(skippedPositions+1);
         scanner.getText(termAtt);
         final int start = scanner.yychar();
         offsetAtt.setOffset(correctOffset(start), correctOffset(start+termAtt.length()));
@@ -142,19 +144,30 @@ public final class UAX29URLEmailTokenize
       } else
         // When we skip a too-long term, we still increment the
         // position increment
-        posIncr++;
+        skippedPositions++;
     }
   }
   
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     int finalOffset = correctOffset(scanner.yychar() + scanner.yylength());
     offsetAtt.setOffset(finalOffset, finalOffset);
+    // adjust any skipped tokens
+    posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement()+skippedPositions);
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    scanner.yyreset(input);
   }
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     scanner.yyreset(input);
+    skippedPositions = 0;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SolrSynonymParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SolrSynonymParser.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SolrSynonymParser.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SolrSynonymParser.java Mon Oct 21 18:58:24 2013
@@ -54,17 +54,16 @@ import org.apache.lucene.util.CharsRef;
  *  </ol>
  * @lucene.experimental
  */
-public class SolrSynonymParser extends SynonymMap.Builder {
+public class SolrSynonymParser extends SynonymMap.Parser {
   private final boolean expand;
-  private final Analyzer analyzer;
   
   public SolrSynonymParser(boolean dedup, boolean expand, Analyzer analyzer) {
-    super(dedup);
+    super(dedup, analyzer);
     this.expand = expand;
-    this.analyzer = analyzer;
   }
-  
-  public void add(Reader in) throws IOException, ParseException {
+
+  @Override
+  public void parse(Reader in) throws IOException, ParseException {
     LineNumberReader br = new LineNumberReader(in);
     try {
       addInternal(br);
@@ -96,19 +95,19 @@ public class SolrSynonymParser extends S
         String inputStrings[] = split(sides[0], ",");
         inputs = new CharsRef[inputStrings.length];
         for (int i = 0; i < inputs.length; i++) {
-          inputs[i] = analyze(analyzer, unescape(inputStrings[i]).trim(), new CharsRef());
+          inputs[i] = analyze(unescape(inputStrings[i]).trim(), new CharsRef());
         }
         
         String outputStrings[] = split(sides[1], ",");
         outputs = new CharsRef[outputStrings.length];
         for (int i = 0; i < outputs.length; i++) {
-          outputs[i] = analyze(analyzer, unescape(outputStrings[i]).trim(), new CharsRef());
+          outputs[i] = analyze(unescape(outputStrings[i]).trim(), new CharsRef());
         }
       } else {
         String inputStrings[] = split(line, ",");
         inputs = new CharsRef[inputStrings.length];
         for (int i = 0; i < inputs.length; i++) {
-          inputs[i] = analyze(analyzer, unescape(inputStrings[i]).trim(), new CharsRef());
+          inputs[i] = analyze(unescape(inputStrings[i]).trim(), new CharsRef());
         }
         if (expand) {
           outputs = inputs;

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java Mon Oct 21 18:58:24 2013
@@ -61,6 +61,20 @@ import org.apache.lucene.util.Version;
  * the same name as an init param used by the SynonymFilterFactory, the prefix 
  * is mandatory.
  * </p>
+ * 
+ * <p>
+ * The optional {@code format} parameter controls how the synonyms will be parsed:
+ * It supports the short names of {@code solr} for {@link SolrSynonymParser} 
+ * and {@code wordnet} for and {@link WordnetSynonymParser}, or your own 
+ * {@code SynonymMap.Parser} class name. The default is {@code solr}.
+ * A custom {@link SynonymMap.Parser} is expected to have a constructor taking:
+ * <ul>
+ *   <li><code>boolean dedup</code> - true if duplicates should be ignored, false otherwise</li>
+ *   <li><code>boolean expand</code> - true if conflation groups should be expanded, false if they are one-directional</li>
+ *   <li><code>{@link Analyzer} analyzer</code> - an analyzer used for each raw synonym</li>
+ * </ul>
+ * </p>
+ * @see SolrSynonymParser SolrSynonymParser: default format
  */
 public class SynonymFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   private final boolean ignoreCase;
@@ -127,61 +141,44 @@ public class SynonymFilterFactory extend
     }
 
     try {
+      String formatClass = format;
       if (format == null || format.equals("solr")) {
-        // TODO: expose dedup as a parameter?
-        map = loadSolrSynonyms(loader, true, analyzer);
+        formatClass = SolrSynonymParser.class.getName();
       } else if (format.equals("wordnet")) {
-        map = loadWordnetSynonyms(loader, true, analyzer);
-      } else {
-        // TODO: somehow make this more pluggable
-        throw new IllegalArgumentException("Unrecognized synonyms format: " + format);
+        formatClass = WordnetSynonymParser.class.getName();
       }
+      // TODO: expose dedup as a parameter?
+      map = loadSynonyms(loader, formatClass, true, analyzer);
     } catch (ParseException e) {
       throw new IOException("Error parsing synonyms file:", e);
     }
   }
-  
+
   /**
-   * Load synonyms from the solr format, "format=solr".
+   * Load synonyms with the given {@link SynonymMap.Parser} class.
    */
-  private SynonymMap loadSolrSynonyms(ResourceLoader loader, boolean dedup, Analyzer analyzer) throws IOException, ParseException {    
+  private SynonymMap loadSynonyms(ResourceLoader loader, String cname, boolean dedup, Analyzer analyzer) throws IOException, ParseException {
     CharsetDecoder decoder = Charset.forName("UTF-8").newDecoder()
-      .onMalformedInput(CodingErrorAction.REPORT)
-      .onUnmappableCharacter(CodingErrorAction.REPORT);
-    
-    SolrSynonymParser parser = new SolrSynonymParser(dedup, expand, analyzer);
-    File synonymFile = new File(synonyms);
-    if (synonymFile.exists()) {
-      decoder.reset();
-      parser.add(new InputStreamReader(loader.openResource(synonyms), decoder));
-    } else {
-      List<String> files = splitFileNames(synonyms);
-      for (String file : files) {
-        decoder.reset();
-        parser.add(new InputStreamReader(loader.openResource(file), decoder));
-      }
+        .onMalformedInput(CodingErrorAction.REPORT)
+        .onUnmappableCharacter(CodingErrorAction.REPORT);
+
+    SynonymMap.Parser parser;
+    Class<? extends SynonymMap.Parser> clazz = loader.findClass(cname, SynonymMap.Parser.class);
+    try {
+      parser = clazz.getConstructor(boolean.class, boolean.class, Analyzer.class).newInstance(dedup, expand, analyzer);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
-    return parser.build();
-  }
-  
-  /**
-   * Load synonyms from the wordnet format, "format=wordnet".
-   */
-  private SynonymMap loadWordnetSynonyms(ResourceLoader loader, boolean dedup, Analyzer analyzer) throws IOException, ParseException {
-    CharsetDecoder decoder = Charset.forName("UTF-8").newDecoder()
-      .onMalformedInput(CodingErrorAction.REPORT)
-      .onUnmappableCharacter(CodingErrorAction.REPORT);
-    
-    WordnetSynonymParser parser = new WordnetSynonymParser(dedup, expand, analyzer);
+
     File synonymFile = new File(synonyms);
     if (synonymFile.exists()) {
       decoder.reset();
-      parser.add(new InputStreamReader(loader.openResource(synonyms), decoder));
+      parser.parse(new InputStreamReader(loader.openResource(synonyms), decoder));
     } else {
       List<String> files = splitFileNames(synonyms);
       for (String file : files) {
         decoder.reset();
-        parser.add(new InputStreamReader(loader.openResource(file), decoder));
+        parser.parse(new InputStreamReader(loader.openResource(file), decoder));
       }
     }
     return parser.build();

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java Mon Oct 21 18:58:24 2013
@@ -18,6 +18,8 @@ package org.apache.lucene.analysis.synon
  */
 
 import java.io.IOException;
+import java.io.Reader;
+import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -107,39 +109,7 @@ public class SynonymMap {
       return reuse;
     }
     
-    /** Sugar: analyzes the text with the analyzer and
-     *  separates by {@link SynonymMap#WORD_SEPARATOR}.
-     *  reuse and its chars must not be null. */
-    public static CharsRef analyze(Analyzer analyzer, String text, CharsRef reuse) throws IOException {
-      TokenStream ts = analyzer.tokenStream("", text);
-      CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
-      PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
-      ts.reset();
-      reuse.length = 0;
-      while (ts.incrementToken()) {
-        int length = termAtt.length();
-        if (length == 0) {
-          throw new IllegalArgumentException("term: " + text + " analyzed to a zero-length token");
-        }
-        if (posIncAtt.getPositionIncrement() != 1) {
-          throw new IllegalArgumentException("term: " + text + " analyzed to a token with posinc != 1");
-        }
-        reuse.grow(reuse.length + length + 1); /* current + word + separator */
-        int end = reuse.offset + reuse.length;
-        if (reuse.length > 0) {
-          reuse.chars[end++] = SynonymMap.WORD_SEPARATOR;
-          reuse.length++;
-        }
-        System.arraycopy(termAtt.buffer(), 0, reuse.chars, end, length);
-        reuse.length += length;
-      }
-      ts.end();
-      ts.close();
-      if (reuse.length == 0) {
-        throw new IllegalArgumentException("term: " + text + " was completely eliminated by analyzer");
-      }
-      return reuse;
-    }
+
 
     /** only used for asserting! */
     private boolean hasHoles(CharsRef chars) {
@@ -312,4 +282,60 @@ public class SynonymMap {
       return new SynonymMap(fst, words, maxHorizontalContext);
     }
   }
+
+  /**
+   * Abstraction for parsing synonym files.
+   *
+   * @lucene.experimental
+   */
+  public static abstract class Parser extends Builder {
+
+    private final Analyzer analyzer;
+
+    public Parser(boolean dedup, Analyzer analyzer) {
+      super(dedup);
+      this.analyzer = analyzer;
+    }
+
+    /**
+     * Parse the given input, adding synonyms to the inherited {@link Builder}.
+     * @param in The input to parse
+     */
+    public abstract void parse(Reader in) throws IOException, ParseException;
+
+    /** Sugar: analyzes the text with the analyzer and
+     *  separates by {@link SynonymMap#WORD_SEPARATOR}.
+     *  reuse and its chars must not be null. */
+    public CharsRef analyze(String text, CharsRef reuse) throws IOException {
+      try (TokenStream ts = analyzer.tokenStream("", text)) {
+        CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+        PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+        ts.reset();
+        reuse.length = 0;
+        while (ts.incrementToken()) {
+          int length = termAtt.length();
+          if (length == 0) {
+            throw new IllegalArgumentException("term: " + text + " analyzed to a zero-length token");
+          }
+          if (posIncAtt.getPositionIncrement() != 1) {
+            throw new IllegalArgumentException("term: " + text + " analyzed to a token with posinc != 1");
+          }
+          reuse.grow(reuse.length + length + 1); /* current + word + separator */
+          int end = reuse.offset + reuse.length;
+          if (reuse.length > 0) {
+            reuse.chars[end++] = SynonymMap.WORD_SEPARATOR;
+            reuse.length++;
+          }
+          System.arraycopy(termAtt.buffer(), 0, reuse.chars, end, length);
+          reuse.length += length;
+        }
+        ts.end();
+      }
+      if (reuse.length == 0) {
+        throw new IllegalArgumentException("term: " + text + " was completely eliminated by analyzer");
+      }
+      return reuse;
+    }
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/WordnetSynonymParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/WordnetSynonymParser.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/WordnetSynonymParser.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/WordnetSynonymParser.java Mon Oct 21 18:58:24 2013
@@ -32,17 +32,16 @@ import org.apache.lucene.util.CharsRef;
  * @lucene.experimental
  */
 // TODO: allow you to specify syntactic categories (e.g. just nouns, etc)
-public class WordnetSynonymParser extends SynonymMap.Builder {
+public class WordnetSynonymParser extends SynonymMap.Parser {
   private final boolean expand;
-  private final Analyzer analyzer;
   
   public WordnetSynonymParser(boolean dedup, boolean expand, Analyzer analyzer) {
-    super(dedup);
+    super(dedup, analyzer);
     this.expand = expand;
-    this.analyzer = analyzer;
   }
-  
-  public void add(Reader in) throws IOException, ParseException {
+
+  @Override
+  public void parse(Reader in) throws IOException, ParseException {
     LineNumberReader br = new LineNumberReader(in);
     try {
       String line = null;
@@ -89,7 +88,7 @@ public class WordnetSynonymParser extend
     int end = line.lastIndexOf('\'');
     
     String text = line.substring(start, end).replace("''", "'");
-    return analyze(analyzer, text, reuse);
+    return analyze(text, reuse);
   }
   
   private void addInternal(CharsRef synset[], int size) {

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java Mon Oct 21 18:58:24 2013
@@ -62,8 +62,7 @@ public abstract class CharTokenizer exte
     charUtils = CharacterUtils.getInstance(matchVersion);
   }
   
-  // note: bufferIndex is -1 here to best-effort AIOOBE consumers that don't call reset()
-  private int offset = 0, bufferIndex = -1, dataLen = 0, finalOffset = 0;
+  private int offset = 0, bufferIndex = 0, dataLen = 0, finalOffset = 0;
   private static final int MAX_WORD_LEN = 255;
   private static final int IO_BUFFER_SIZE = 4096;
   
@@ -142,13 +141,15 @@ public abstract class CharTokenizer exte
   }
   
   @Override
-  public final void end() {
+  public final void end() throws IOException {
+    super.end();
     // set final offset
     offsetAtt.setOffset(finalOffset, finalOffset);
   }
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     bufferIndex = 0;
     offset = 0;
     dataLen = 0;

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java Mon Oct 21 18:58:24 2013
@@ -34,6 +34,7 @@ public abstract class FilteringTokenFilt
 
   protected final Version version;
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private int skippedPositions;
 
   /**
    * Create a new {@link FilteringTokenFilter}.
@@ -50,7 +51,7 @@ public abstract class FilteringTokenFilt
 
   @Override
   public final boolean incrementToken() throws IOException {
-    int skippedPositions = 0;
+    skippedPositions = 0;
     while (input.incrementToken()) {
       if (accept()) {
         if (skippedPositions != 0) {
@@ -68,6 +69,12 @@ public abstract class FilteringTokenFilt
   @Override
   public void reset() throws IOException {
     super.reset();
+    skippedPositions = 0;
   }
 
+  @Override
+  public void end() throws IOException {
+    super.end();
+    posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement() + skippedPositions);
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/StemmerUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/StemmerUtil.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/StemmerUtil.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/StemmerUtil.java Mon Oct 21 18:58:24 2013
@@ -17,7 +17,11 @@ package org.apache.lucene.analysis.util;
  * limitations under the License.
  */
 
-/** Some commonly-used stemming functions */
+/**
+ * Some commonly-used stemming functions
+ * 
+ * @lucene.internal
+ */
 public class StemmerUtil {
   /** no instance */
   private StemmerUtil() {}
@@ -87,9 +91,10 @@ public class StemmerUtil {
    * @return length of input buffer after deletion
    */
   public static int delete(char s[], int pos, int len) {
-    if (pos < len) 
+    assert pos < len;
+    if (pos < len - 1) { // don't arraycopy if asked to delete last character
       System.arraycopy(s, pos + 1, s, pos, len - pos - 1);
-    
+    }
     return len - 1;
   }
   
@@ -103,9 +108,10 @@ public class StemmerUtil {
    * @return length of input buffer after deletion
    */
   public static int deleteN(char s[], int pos, int len, int nChars) {
-    // TODO: speed up, this is silly
-    for (int i = 0; i < nChars; i++)
-      len = delete(s, pos, len);
-    return len;
+    assert pos + nChars <= len;
+    if (pos + nChars < len) { // don't arraycopy if asked to delete the last characters
+      System.arraycopy(s, pos + nChars, s, pos, len - pos - nChars);
+    }
+    return len - nChars;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizer.java Mon Oct 21 18:58:24 2013
@@ -143,7 +143,7 @@ public final class WikipediaTokenizer ex
    */
   public WikipediaTokenizer(Reader input, int tokenOutput, Set<String> untokenizedTypes) {
     super(input);
-    this.scanner = new WikipediaTokenizerImpl(null); // best effort NPE if you dont call reset
+    this.scanner = new WikipediaTokenizerImpl(this.input);
     init(tokenOutput, untokenizedTypes);
   }
 
@@ -156,7 +156,7 @@ public final class WikipediaTokenizer ex
    */
   public WikipediaTokenizer(AttributeFactory factory, Reader input, int tokenOutput, Set<String> untokenizedTypes) {
     super(factory, input);
-    this.scanner = new WikipediaTokenizerImpl(null); // best effort NPE if you dont call reset
+    this.scanner = new WikipediaTokenizerImpl(this.input);
     init(tokenOutput, untokenizedTypes);
   }
   
@@ -295,6 +295,12 @@ public final class WikipediaTokenizer ex
     offsetAtt.setOffset(correctOffset(start), correctOffset(start + termAtt.length()));
   }
 
+  @Override
+  public void close() throws IOException {
+    super.close();
+    scanner.yyreset(input);
+  }
+
   /*
   * (non-Javadoc)
   *
@@ -302,6 +308,7 @@ public final class WikipediaTokenizer ex
   */
   @Override
   public void reset() throws IOException {
+    super.reset();
     scanner.yyreset(input);
     tokens = null;
     scanner.reset();
@@ -309,7 +316,8 @@ public final class WikipediaTokenizer ex
   }
 
   @Override
-  public void end() {
+  public void end() throws IOException {
+    super.end();
     // set final offset
     final int finalOffset = correctOffset(scanner.yychar() + scanner.yylength());
     this.offsetAtt.setOffset(finalOffset, finalOffset);

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory Mon Oct 21 18:58:24 2013
@@ -55,6 +55,7 @@ org.apache.lucene.analysis.it.ItalianLig
 org.apache.lucene.analysis.lv.LatvianStemFilterFactory
 org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory
 org.apache.lucene.analysis.miscellaneous.CapitalizationFilterFactory
+org.apache.lucene.analysis.miscellaneous.CodepointCountFilterFactory
 org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilterFactory
 org.apache.lucene.analysis.miscellaneous.KeepWordFilterFactory
 org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilterFactory

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/danish_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/danish_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/danish_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/danish_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Danish stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/dutch_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/dutch_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/dutch_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/dutch_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Dutch stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/english_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/english_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/english_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/english_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
  
  | An English stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/finnish_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/finnish_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/finnish_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/finnish_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
  
 | forms of BE
 

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/french_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/french_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/french_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/french_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A French stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/german_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/german_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/german_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/german_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A German stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/hungarian_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/hungarian_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/hungarian_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/hungarian_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
  
 | Hungarian stop word list
 | prepared by Anna Tordai

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/italian_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/italian_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/italian_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/italian_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | An Italian stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/norwegian_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/norwegian_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/norwegian_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/norwegian_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Norwegian stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/portuguese_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/portuguese_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/portuguese_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/portuguese_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Portuguese stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/russian_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/russian_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/russian_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/russian_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | a russian stop word list. comments begin with vertical bar. each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/spanish_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/spanish_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/spanish_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/spanish_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Spanish stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/swedish_stop.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/swedish_stop.txt?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/swedish_stop.txt (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/resources/org/apache/lucene/analysis/snowball/swedish_stop.txt Mon Oct 21 18:58:24 2013
@@ -4,6 +4,8 @@
  | Also see http://www.opensource.org/licenses/bsd-license.html
  |  - Encoding was converted to UTF-8.
  |  - This notice was added.
+ |
+ | NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
 
  | A Swedish stop word list. Comments begin with vertical bar. Each stop
  | word is at the start of a line.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -60,8 +60,8 @@ public class TestArabicAnalyzer extends 
    */
   public void testReusableTokenStream() throws Exception {
     ArabicAnalyzer a = new ArabicAnalyzer(TEST_VERSION_CURRENT);
-    assertAnalyzesToReuse(a, "كبير", new String[] { "كبير" });
-    assertAnalyzesToReuse(a, "كبيرة", new String[] { "كبير" }); // feminine marker
+    assertAnalyzesTo(a, "كبير", new String[] { "كبير" });
+    assertAnalyzesTo(a, "كبيرة", new String[] { "كبير" }); // feminine marker
   }
 
   /**
@@ -86,12 +86,12 @@ public class TestArabicAnalyzer extends 
     CharArraySet set = new CharArraySet(TEST_VERSION_CURRENT, asSet("ساهدهات"), false);
     ArabicAnalyzer a = new ArabicAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET, set);
     assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهدهات" });
-    assertAnalyzesToReuse(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهدهات" });
+    assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهدهات" });
 
     
     a = new ArabicAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET, CharArraySet.EMPTY_SET);
     assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهد" });
-    assertAnalyzesToReuse(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهد" });
+    assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهد" });
   }
   
   /** blast some random strings through the analyzer */

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java Mon Oct 21 18:58:24 2013
@@ -102,7 +102,7 @@ public class TestArabicNormalizationFilt
         return new TokenStreamComponents(tokenizer, new ArabicNormalizationFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java Mon Oct 21 18:58:24 2013
@@ -142,6 +142,6 @@ public class TestArabicStemFilter extend
         return new TokenStreamComponents(tokenizer, new ArabicStemFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -49,8 +49,8 @@ public class TestBulgarianAnalyzer exten
   
   public void testReusableTokenStream() throws IOException {
     Analyzer a = new BulgarianAnalyzer(TEST_VERSION_CURRENT);
-    assertAnalyzesToReuse(a, "документи", new String[] {"документ"});
-    assertAnalyzesToReuse(a, "документ", new String[] {"документ"});
+    assertAnalyzesTo(a, "документи", new String[] {"документ"});
+    assertAnalyzesTo(a, "документ", new String[] {"документ"});
   }
   
   /**

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java Mon Oct 21 18:58:24 2013
@@ -234,6 +234,6 @@ public class TestBulgarianStemmer extend
         return new TokenStreamComponents(tokenizer, new BulgarianStemFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/br/TestBrazilianStemmer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/br/TestBrazilianStemmer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/br/TestBrazilianStemmer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/br/TestBrazilianStemmer.java Mon Oct 21 18:58:24 2013
@@ -157,7 +157,7 @@ public class TestBrazilianStemmer extend
   }
   
   private void checkReuse(Analyzer a, String input, String expected) throws Exception {
-    checkOneTermReuse(a, input, expected);
+    checkOneTerm(a, input, expected);
   }
 
   /** blast some random strings through the analyzer */
@@ -173,6 +173,6 @@ public class TestBrazilianStemmer extend
         return new TokenStreamComponents(tokenizer, new BrazilianStemFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ca/TestCatalanAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ca/TestCatalanAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ca/TestCatalanAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/ca/TestCatalanAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -34,8 +34,8 @@ public class TestCatalanAnalyzer extends
   public void testBasics() throws IOException {
     Analyzer a = new CatalanAnalyzer(TEST_VERSION_CURRENT);
     // stemming
-    checkOneTermReuse(a, "llengües", "llengu");
-    checkOneTermReuse(a, "llengua", "llengu");
+    checkOneTerm(a, "llengües", "llengu");
+    checkOneTerm(a, "llengua", "llengu");
     // stopword
     assertAnalyzesTo(a, "un", new String[] { });
   }
@@ -52,8 +52,8 @@ public class TestCatalanAnalyzer extends
     CharArraySet exclusionSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("llengües"), false);
     Analyzer a = new CatalanAnalyzer(TEST_VERSION_CURRENT, 
         CatalanAnalyzer.getDefaultStopSet(), exclusionSet);
-    checkOneTermReuse(a, "llengües", "llengües");
-    checkOneTermReuse(a, "llengua", "llengu");
+    checkOneTerm(a, "llengües", "llengües");
+    checkOneTerm(a, "llengua", "llengu");
   }
   
   /** blast some random strings through the analyzer */

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -167,14 +167,14 @@ public class TestCJKAnalyzer extends Bas
   }
   
   public void testReusableTokenStream() throws IOException {
-    assertAnalyzesToReuse(analyzer, "あいうえおabcかきくけこ",
+    assertAnalyzesTo(analyzer, "あいうえおabcかきくけこ",
         new String[] { "あい", "いう", "うえ", "えお", "abc", "かき", "きく", "くけ", "けこ" },
         new int[] { 0, 1, 2, 3, 5,  8,  9, 10, 11 },
         new int[] { 2, 3, 4, 5, 8, 10, 11, 12, 13 },
         new String[] { "<DOUBLE>", "<DOUBLE>", "<DOUBLE>", "<DOUBLE>", "<ALPHANUM>", "<DOUBLE>", "<DOUBLE>", "<DOUBLE>", "<DOUBLE>" },
         new int[] { 1, 1, 1, 1, 1,  1,  1,  1,  1});
     
-    assertAnalyzesToReuse(analyzer, "あいうえおabんcかきくけ こ",
+    assertAnalyzesTo(analyzer, "あいうえおabんcかきくけ こ",
         new String[] { "あい", "いう", "うえ", "えお", "ab", "ん", "c", "かき", "きく", "くけ", "こ" },
         new int[] { 0, 1, 2, 3, 5, 7, 8,  9, 10, 11, 14 },
         new int[] { 2, 3, 4, 5, 7, 8, 9, 11, 12, 13, 15 },
@@ -288,6 +288,6 @@ public class TestCJKAnalyzer extends Bas
         return new TokenStreamComponents(tokenizer, new CJKBigramFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKWidthFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKWidthFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKWidthFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/cjk/TestCJKWidthFilter.java Mon Oct 21 18:58:24 2013
@@ -74,6 +74,6 @@ public class TestCJKWidthFilter extends 
         return new TokenStreamComponents(tokenizer, new CJKWidthFilter(tokenizer));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/CommonGramsFilterTest.java Mon Oct 21 18:58:24 2013
@@ -48,6 +48,7 @@ public class CommonGramsFilterTest exten
     assertEquals("the", term.toString());
     assertTrue(cgf.incrementToken());
     assertEquals("the_s", term.toString());
+    cgf.close();
     
     wt.setReader(new StringReader(input));
     cgf.reset();
@@ -67,6 +68,7 @@ public class CommonGramsFilterTest exten
     assertEquals("How_the", term.toString());
     assertTrue(nsf.incrementToken());
     assertEquals("the_s", term.toString());
+    nsf.close();
     
     wt.setReader(new StringReader(input));
     nsf.reset();

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java Mon Oct 21 18:58:24 2013
@@ -240,6 +240,8 @@ public class TestCompoundWordTokenFilter
     assertEquals("Rindfleischüberwachungsgesetz", termAtt.toString());
     assertTrue(tf.incrementToken());
     assertEquals("Rind", termAtt.toString());
+    tf.end();
+    tf.close();
     wsTokenizer.setReader(new StringReader("Rindfleischüberwachungsgesetz"));
     tf.reset();
     assertTrue(tf.incrementToken());
@@ -377,7 +379,7 @@ public class TestCompoundWordTokenFilter
         return new TokenStreamComponents(tokenizer, new DictionaryCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, dict));
       }
     };
-    checkOneTermReuse(a, "", "");
+    checkOneTerm(a, "", "");
     
     InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
     final HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter.getHyphenationTree(is);
@@ -390,6 +392,6 @@ public class TestCompoundWordTokenFilter
         return new TokenStreamComponents(tokenizer, filter);
       }
     };
-    checkOneTermReuse(b, "", "");
+    checkOneTerm(b, "", "");
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestBugInSomething.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestBugInSomething.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestBugInSomething.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestBugInSomething.java Mon Oct 21 18:58:24 2013
@@ -1,5 +1,6 @@
 package org.apache.lucene.analysis.core;
 
+import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.nio.CharBuffer;
@@ -11,10 +12,14 @@ import org.apache.lucene.analysis.MockCh
 import org.apache.lucene.analysis.MockTokenFilter;
 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.commongrams.CommonGramsFilter;
+import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
+import org.apache.lucene.analysis.ngram.NGramTokenFilter;
+import org.apache.lucene.analysis.shingle.ShingleFilter;
 import org.apache.lucene.analysis.util.CharArraySet;
 
 /*
@@ -195,4 +200,58 @@ public class TestBugInSomething extends 
       assertEquals("read(char[], int, int)", e.getMessage());
     }
   }
+  
+  // todo: test framework?
+  
+  static final class SopTokenFilter extends TokenFilter {
+
+    SopTokenFilter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (input.incrementToken()) {
+        System.out.println(input.getClass().getSimpleName() + "->" + this.reflectAsString(false));
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      System.out.println(input.getClass().getSimpleName() + ".end()");
+    }
+
+    @Override
+    public void close() throws IOException {
+      super.close();
+      System.out.println(input.getClass().getSimpleName() + ".close()");
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      System.out.println(input.getClass().getSimpleName() + ".reset()");
+    }
+  }
+  
+  // LUCENE-5269
+  public void testUnicodeShinglesAndNgrams() throws Exception {
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new EdgeNGramTokenizer(TEST_VERSION_CURRENT, reader, 2, 94);
+        //TokenStream stream = new SopTokenFilter(tokenizer);
+        TokenStream stream = new ShingleFilter(tokenizer, 5);
+        //stream = new SopTokenFilter(stream);
+        stream = new NGramTokenFilter(TEST_VERSION_CURRENT, stream, 55, 83);
+        //stream = new SopTokenFilter(stream);
+        return new TokenStreamComponents(tokenizer, stream);
+      }  
+    };
+    checkRandomData(random(), analyzer, 2000);
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -117,12 +117,15 @@ public class TestKeywordAnalyzer extends
 
   // LUCENE-1441
   public void testOffsets() throws Exception {
-    TokenStream stream = new KeywordAnalyzer().tokenStream("field", new StringReader("abcd"));
-    OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
-    stream.reset();
-    assertTrue(stream.incrementToken());
-    assertEquals(0, offsetAtt.startOffset());
-    assertEquals(4, offsetAtt.endOffset());
+    try (TokenStream stream = new KeywordAnalyzer().tokenStream("field", new StringReader("abcd"))) {
+      OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
+      stream.reset();
+      assertTrue(stream.incrementToken());
+      assertEquals(0, offsetAtt.startOffset());
+      assertEquals(4, offsetAtt.endOffset());
+      assertFalse(stream.incrementToken());
+      stream.end();
+    }
   }
   
   /** blast some random strings through the analyzer */

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -46,27 +46,31 @@ public class TestStopAnalyzer extends Ba
 
   public void testDefaults() throws IOException {
     assertTrue(stop != null);
-    TokenStream stream = stop.tokenStream("test", "This is a test of the english stop analyzer");
-    assertTrue(stream != null);
-    CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
-    stream.reset();
+    try (TokenStream stream = stop.tokenStream("test", "This is a test of the english stop analyzer")) {
+      assertTrue(stream != null);
+      CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
+      stream.reset();
     
-    while (stream.incrementToken()) {
-      assertFalse(inValidTokens.contains(termAtt.toString()));
+      while (stream.incrementToken()) {
+        assertFalse(inValidTokens.contains(termAtt.toString()));
+      }
+      stream.end();
     }
   }
 
   public void testStopList() throws IOException {
     CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false);
     StopAnalyzer newStop = new StopAnalyzer(Version.LUCENE_40, stopWordsSet);
-    TokenStream stream = newStop.tokenStream("test", "This is a good test of the english stop analyzer");
-    assertNotNull(stream);
-    CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
+    try (TokenStream stream = newStop.tokenStream("test", "This is a good test of the english stop analyzer")) {
+      assertNotNull(stream);
+      CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
     
-    stream.reset();
-    while (stream.incrementToken()) {
-      String text = termAtt.toString();
-      assertFalse(stopWordsSet.contains(text));
+      stream.reset();
+      while (stream.incrementToken()) {
+        String text = termAtt.toString();
+        assertFalse(stopWordsSet.contains(text));
+      }
+      stream.end();
     }
   }
 
@@ -75,17 +79,19 @@ public class TestStopAnalyzer extends Ba
     StopAnalyzer newStop = new StopAnalyzer(TEST_VERSION_CURRENT, stopWordsSet);
     String s =             "This is a good test of the english stop analyzer with positions";
     int expectedIncr[] =  { 1,   1, 1,          3, 1,  1,      1,            2,   1};
-    TokenStream stream = newStop.tokenStream("test", s);
-    assertNotNull(stream);
-    int i = 0;
-    CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
-    PositionIncrementAttribute posIncrAtt = stream.addAttribute(PositionIncrementAttribute.class);
+    try (TokenStream stream = newStop.tokenStream("test", s)) {
+      assertNotNull(stream);
+      int i = 0;
+      CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
+      PositionIncrementAttribute posIncrAtt = stream.addAttribute(PositionIncrementAttribute.class);
 
-    stream.reset();
-    while (stream.incrementToken()) {
-      String text = termAtt.toString();
-      assertFalse(stopWordsSet.contains(text));
-      assertEquals(expectedIncr[i++],posIncrAtt.getPositionIncrement());
+      stream.reset();
+      while (stream.incrementToken()) {
+        String text = termAtt.toString();
+        assertFalse(stopWordsSet.contains(text));
+        assertEquals(expectedIncr[i++],posIncrAtt.getPositionIncrement());
+      }
+      stream.end();
     }
   }
 

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilter.java Mon Oct 21 18:58:24 2013
@@ -90,6 +90,22 @@ public class TestStopFilter extends Base
     StopFilter stpf01 = new StopFilter(TEST_VERSION_CURRENT, stpf0, stopSet1); // two stop filters concatenated!
     doTestStopPositons(stpf01);
   }
+
+  // LUCENE-3849: make sure after .end() we see the "ending" posInc
+  public void testEndStopword() throws Exception {
+    CharArraySet stopSet = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "of");
+    StopFilter stpf = new StopFilter(Version.LUCENE_40, new MockTokenizer(new StringReader("test of"), MockTokenizer.WHITESPACE, false), stopSet);
+    assertTokenStreamContents(stpf, new String[] { "test" },
+                              new int[] {0},
+                              new int[] {4},
+                              null,
+                              new int[] {1},
+                              null,
+                              7,
+                              1,
+                              null,
+                              true);    
+  }
   
   private void doTestStopPositons(StopFilter stpf) throws IOException {
     CharTermAttribute termAtt = stpf.getAttribute(CharTermAttribute.class);

Modified: lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilterFactory.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilterFactory.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopFilterFactory.java Mon Oct 21 18:58:24 2013
@@ -57,6 +57,11 @@ public class TestStopFilterFactory exten
     assertTrue(words.contains("her"));
     assertTrue(words.contains("hers"));
     assertTrue(words.contains("herself"));
+
+    // defaults
+    factory = (StopFilterFactory) tokenFilterFactory("Stop");
+    assertEquals(StopAnalyzer.ENGLISH_STOP_WORDS_SET, factory.getStopWords());
+    assertEquals(false, factory.isIgnoreCase());
   }
   
   /** Test that bogus arguments result in exception */
@@ -68,4 +73,30 @@ public class TestStopFilterFactory exten
       assertTrue(expected.getMessage().contains("Unknown parameters"));
     }
   }
+
+  /** Test that bogus arguments result in exception */
+  public void testBogusFormats() throws Exception {
+    try {
+      tokenFilterFactory("Stop", 
+                         "words", "stop-snowball.txt",
+                         "format", "bogus");
+      fail();
+    } catch (IllegalArgumentException expected) {
+      String msg = expected.getMessage();
+      assertTrue(msg, msg.contains("Unknown"));
+      assertTrue(msg, msg.contains("format"));
+      assertTrue(msg, msg.contains("bogus"));
+    }
+    try {
+      tokenFilterFactory("Stop", 
+                         // implicit default words file
+                         "format", "bogus");
+      fail();
+    } catch (IllegalArgumentException expected) {
+      String msg = expected.getMessage();
+      assertTrue(msg, msg.contains("can not be specified"));
+      assertTrue(msg, msg.contains("format"));
+      assertTrue(msg, msg.contains("bogus"));
+    }
+  }
 }