You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by rm...@apache.org on 2010/02/26 14:10:08 UTC

svn commit: r916666 [3/16] - in /lucene/java/branches/flex_1458: ./ contrib/ contrib/analyzers/common/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/ar/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/bg/ contrib/analyzers/c...

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java Fri Feb 26 13:09:54 2010
@@ -21,11 +21,13 @@
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.LowerCaseFilter;
 import org.apache.lucene.analysis.ReusableAnalyzerBase.TokenStreamComponents; // javadoc @link
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;
 import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.WordlistLoader;
+import org.apache.lucene.analysis.snowball.SnowballFilter;
 import org.apache.lucene.analysis.standard.StandardFilter;
 import org.apache.lucene.analysis.standard.StandardTokenizer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;  // for javadoc
@@ -54,6 +56,9 @@
  * <p>You must specify the required {@link Version}
  * compatibility when creating FrenchAnalyzer:
  * <ul>
+ *   <li> As of 3.1, Snowball stemming is done with SnowballFilter, 
+ *        LowerCaseFilter is used prior to StopFilter, and ElisionFilter and 
+ *        Snowball stopwords are used by default.
  *   <li> As of 2.9, StopFilter preserves position
  *        increments
  * </ul>
@@ -67,7 +72,7 @@
    * Extended list of typical French stopwords.
    * @deprecated use {@link #getDefaultStopSet()} instead
    */
-  // TODO make this private in 3.1
+  // TODO make this private in 3.1, remove in 4.0
   @Deprecated
   public final static String[] FRENCH_STOP_WORDS = {
     "a", "afin", "ai", "ainsi", "après", "attendu", "au", "aujourd", "auquel", "aussi",
@@ -94,6 +99,9 @@
     "été", "être", "ô"
   };
 
+  /** File containing default French stopwords. */
+  public final static String DEFAULT_STOPWORD_FILE = "french_stop.txt";
+  
   /**
    * Contains words that should be indexed but not stemmed.
    */
@@ -109,16 +117,31 @@
   }
   
   private static class DefaultSetHolder {
-    static final Set<?> DEFAULT_STOP_SET = CharArraySet
+    /** @deprecated remove this in Lucene 4.0 */
+    @Deprecated
+    static final Set<?> DEFAULT_STOP_SET_30 = CharArraySet
         .unmodifiableSet(new CharArraySet(Version.LUCENE_CURRENT, Arrays.asList(FRENCH_STOP_WORDS),
             false));
+    static final Set<?> DEFAULT_STOP_SET;
+    static {
+      try {
+        DEFAULT_STOP_SET = 
+          WordlistLoader.getSnowballWordSet(SnowballFilter.class, DEFAULT_STOPWORD_FILE);
+      } catch (IOException ex) {
+        // default set should always be present as it is part of the
+        // distribution (JAR)
+        throw new RuntimeException("Unable to load default stopword set");
+      }
+    }
   }
 
   /**
-   * Builds an analyzer with the default stop words ({@link #FRENCH_STOP_WORDS}).
+   * Builds an analyzer with the default stop words ({@link #getDefaultStopSet}).
    */
   public FrenchAnalyzer(Version matchVersion) {
-    this(matchVersion, DefaultSetHolder.DEFAULT_STOP_SET);
+    this(matchVersion,
+        matchVersion.onOrAfter(Version.LUCENE_31) ? DefaultSetHolder.DEFAULT_STOP_SET
+            : DefaultSetHolder.DEFAULT_STOP_SET_30);
   }
   
   /**
@@ -206,18 +229,34 @@
    * {@link Reader}.
    *
    * @return {@link TokenStreamComponents} built from a {@link StandardTokenizer} 
-   *         filtered with {@link StandardFilter}, {@link StopFilter}, 
-   *         {@link FrenchStemFilter} and {@link LowerCaseFilter}
+   *         filtered with {@link StandardFilter}, {@link ElisionFilter}, 
+   *         {@link LowerCaseFilter}, {@link StopFilter},
+   *         {@link KeywordMarkerTokenFilter} if a stem exclusion set is provided, 
+   *         and {@link SnowballFilter}
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName,
       Reader reader) {
-    final Tokenizer source = new StandardTokenizer(matchVersion, reader);
-    TokenStream result = new StandardFilter(source);
-    result = new StopFilter(matchVersion, result, stopwords);
-    result = new FrenchStemFilter(result, excltable);
-    // Convert to lowercase after stemming!
-    return new TokenStreamComponents(source, new LowerCaseFilter(matchVersion, result));
+    if (matchVersion.onOrAfter(Version.LUCENE_31)) {
+      final Tokenizer source = new StandardTokenizer(matchVersion, reader);
+      TokenStream result = new StandardFilter(source);
+      result = new ElisionFilter(matchVersion, result);
+      result = new LowerCaseFilter(matchVersion, result);
+      result = new StopFilter(matchVersion, result, stopwords);
+      if(!excltable.isEmpty())
+        result = new KeywordMarkerTokenFilter(result, excltable);
+      result = new SnowballFilter(result, new org.tartarus.snowball.ext.FrenchStemmer());
+      return new TokenStreamComponents(source, result);
+    } else {
+      final Tokenizer source = new StandardTokenizer(matchVersion, reader);
+      TokenStream result = new StandardFilter(source);
+      result = new StopFilter(matchVersion, result, stopwords);
+      if(!excltable.isEmpty())
+        result = new KeywordMarkerTokenFilter(result, excltable);
+      result = new FrenchStemFilter(result);
+      // Convert to lowercase after stemming!
+      return new TokenStreamComponents(source, new LowerCaseFilter(matchVersion, result));
+    }
   }
 }
 

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemFilter.java Fri Feb 26 13:09:54 2010
@@ -17,8 +17,11 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;// for javadoc
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.snowball.SnowballFilter;
+import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 
 import java.io.IOException;
@@ -29,29 +32,46 @@
 /**
  * A {@link TokenFilter} that stems french words. 
  * <p>
- * It supports a table of words that should
- * not be stemmed at all. The used stemmer can be changed at runtime after the
+ * The used stemmer can be changed at runtime after the
  * filter object is created (as long as it is a {@link FrenchStemmer}).
  * </p>
+ * <p>
+ * To prevent terms from being stemmed use an instance of
+ * {@link KeywordMarkerTokenFilter} or a custom {@link TokenFilter} that sets
+ * the {@link KeywordAttribute} before this {@link TokenStream}.
+ * </p>
+ * @see KeywordMarkerTokenFilter
+ * @deprecated Use {@link SnowballFilter} with 
+ * {@link org.tartarus.snowball.ext.FrenchStemmer} instead, which has the
+ * same functionality. This filter will be removed in Lucene 4.0
  */
+@Deprecated
 public final class FrenchStemFilter extends TokenFilter {
 
 	/**
 	 * The actual token in the input stream.
 	 */
 	private FrenchStemmer stemmer = null;
-	private Set exclusions = null;
+	private Set<?> exclusions = null;
 	
-	private TermAttribute termAtt;
+	private final TermAttribute termAtt;
+  private final KeywordAttribute keywordAttr;
 
 	public FrenchStemFilter( TokenStream in ) {
           super(in);
 		stemmer = new FrenchStemmer();
 		termAtt = addAttribute(TermAttribute.class);
+    keywordAttr = addAttribute(KeywordAttribute.class);
 	}
 
-
-	public FrenchStemFilter( TokenStream in, Set exclusiontable ) {
+  /**
+   * 
+   * @param in the {@link TokenStream} to filter
+   * @param exclusiontable a set of terms not to be stemmed
+   * @deprecated use {@link KeywordAttribute} with {@link KeywordMarkerTokenFilter} instead.
+   */
+	@Deprecated // TODO remove in 3.2
+	public FrenchStemFilter( TokenStream in, Set<?> exclusiontable ) {
 		this( in );
 		exclusions = exclusiontable;
 	}
@@ -65,7 +85,7 @@
 	    String term = termAtt.term();
 
 	    // Check the exclusion table
-	    if ( exclusions == null || !exclusions.contains( term ) ) {
+	    if ( !keywordAttr.isKeyword() && (exclusions == null || !exclusions.contains( term )) ) {
 	      String s = stemmer.stem( term );
 	      // If not stemmed, don't waste the time  adjusting the token.
 	      if ((s != null) && !s.equals( term ) )
@@ -86,8 +106,10 @@
 	}
 	/**
 	 * Set an alternative exclusion list for this filter.
+   * @deprecated use {@link KeywordAttribute} with {@link KeywordMarkerTokenFilter} instead.
 	 */
-	public void setExclusionTable( Map exclusiontable ) {
+	@Deprecated // TODO remove in 3.2
+	public void setExclusionTable( Map<?,?> exclusiontable ) {
 		exclusions = new HashSet(exclusiontable.keySet());
 	}
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemmer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemmer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemmer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/fr/FrenchStemmer.java Fri Feb 26 13:09:54 2010
@@ -25,8 +25,10 @@
  * refer to http://snowball.sourceforge.net/french/stemmer.html<br>
  * (French stemming algorithm) for details
  * </p>
+ * @deprecated Use {@link org.tartarus.snowball.ext.FrenchStemmer} instead, 
+ * which has the same functionality. This filter will be removed in Lucene 4.0
  */
-
+@Deprecated
 public class FrenchStemmer {
 
     /**

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/EmptyTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/EmptyTokenStream.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/EmptyTokenStream.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/EmptyTokenStream.java Fri Feb 26 13:09:54 2010
@@ -18,7 +18,6 @@
  */
 
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Token;
 
 import java.io.IOException;
 

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/PrefixAwareTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/PrefixAwareTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/PrefixAwareTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/miscellaneous/PrefixAwareTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -114,6 +114,7 @@
   
   private void setCurrentToken(Token token) {
     if (token == null) return;
+    clearAttributes();
     termAtt.setTermBuffer(token.termBuffer(), 0, token.termLength());
     posIncrAtt.setPositionIncrement(token.getPositionIncrement());
     flagsAtt.setFlags(token.getFlags());

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -70,6 +70,7 @@
   private char[] curTermBuffer;
   private int curTermLength;
   private int curGramSize;
+  private int tokStart;
   
   private final TermAttribute termAtt;
   private final OffsetAttribute offsetAtt;
@@ -126,6 +127,7 @@
           curTermBuffer = (char[]) termAtt.termBuffer().clone();
           curTermLength = termAtt.termLength();
           curGramSize = minGram;
+          tokStart = offsetAtt.startOffset();
         }
       }
       if (curGramSize <= maxGram) {
@@ -134,7 +136,8 @@
           // grab gramSize chars from front or back
           int start = side == Side.FRONT ? 0 : curTermLength - curGramSize;
           int end = start + curGramSize;
-          offsetAtt.setOffset(start, end);
+          clearAttributes();
+          offsetAtt.setOffset(tokStart + start, tokStart + end);
           termAtt.setTermBuffer(curTermBuffer, start, curGramSize);
           curGramSize++;
           return true;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
@@ -38,6 +37,7 @@
   private int curTermLength;
   private int curGramSize;
   private int curPos;
+  private int tokStart;
   
   private TermAttribute termAtt;
   private OffsetAttribute offsetAtt;
@@ -83,12 +83,14 @@
           curTermLength = termAtt.termLength();
           curGramSize = minGram;
           curPos = 0;
+          tokStart = offsetAtt.startOffset();
         }
       }
       while (curGramSize <= maxGram) {
         while (curPos+curGramSize <= curTermLength) {     // while there is input
+          clearAttributes();
           termAtt.setTermBuffer(curTermBuffer, curPos, curGramSize);
-          offsetAtt.setOffset(curPos, curPos+curGramSize);
+          offsetAtt.setOffset(tokStart + curPos, tokStart + curPos + curGramSize);
           curPos++;
           return true;
         }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizer.java Fri Feb 26 13:09:54 2010
@@ -17,7 +17,6 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java Fri Feb 26 13:09:54 2010
@@ -19,10 +19,15 @@
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;
+import org.apache.lucene.analysis.LowerCaseFilter;
+import org.apache.lucene.analysis.ReusableAnalyzerBase;
 import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.WordlistLoader;
+import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter;
+import org.apache.lucene.analysis.snowball.SnowballFilter;
 import org.apache.lucene.analysis.standard.StandardFilter;
 import org.apache.lucene.analysis.standard.StandardTokenizer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;  // for javadoc
@@ -31,7 +36,6 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.Reader;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -49,28 +53,31 @@
  * exclusion list is empty by default.
  * </p>
  *
+ * <a name="version"/>
+ * <p>You must specify the required {@link Version}
+ * compatibility when creating DutchAnalyzer:
+ * <ul>
+ *   <li> As of 3.1, Snowball stemming is done with SnowballFilter, 
+ *        LowerCaseFilter is used prior to StopFilter, and Snowball 
+ *        stopwords are used by default.
+ *   <li> As of 2.9, StopFilter preserves position
+ *        increments
+ * </ul>
+ * 
  * <p><b>NOTE</b>: This class uses the same {@link Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
  */
-public final class DutchAnalyzer extends Analyzer {
+public final class DutchAnalyzer extends ReusableAnalyzerBase {
   /**
    * List of typical Dutch stopwords.
    * @deprecated use {@link #getDefaultStopSet()} instead
    */
   @Deprecated
-  public final static String[] DUTCH_STOP_WORDS =
-      {
-        "de", "en", "van", "ik", "te", "dat", "die", "in", "een",
-        "hij", "het", "niet", "zijn", "is", "was", "op", "aan", "met", "als", "voor", "had",
-        "er", "maar", "om", "hem", "dan", "zou", "of", "wat", "mijn", "men", "dit", "zo",
-        "door", "over", "ze", "zich", "bij", "ook", "tot", "je", "mij", "uit", "der", "daar",
-        "haar", "naar", "heb", "hoe", "heeft", "hebben", "deze", "u", "want", "nog", "zal",
-        "me", "zij", "nu", "ge", "geen", "omdat", "iets", "worden", "toch", "al", "waren",
-        "veel", "meer", "doen", "toen", "moet", "ben", "zonder", "kan", "hun", "dus",
-        "alles", "onder", "ja", "eens", "hier", "wie", "werd", "altijd", "doch", "wordt",
-        "wezen", "kunnen", "ons", "zelf", "tegen", "na", "reeds", "wil", "kon", "niets",
-        "uw", "iemand", "geweest", "andere"
-      };
+  public final static String[] DUTCH_STOP_WORDS = getDefaultStopSet().toArray(new String[0]);
+  
+  /** File containing default Dutch stopwords. */
+  public final static String DEFAULT_STOPWORD_FILE = "dutch_stop.txt";
+
   /**
    * Returns an unmodifiable instance of the default stop-words set.
    * @return an unmodifiable instance of the default stop-words set.
@@ -80,9 +87,18 @@
   }
   
   private static class DefaultSetHolder {
-    static final Set<?> DEFAULT_STOP_SET = CharArraySet
-        .unmodifiableSet(new CharArraySet(Version.LUCENE_CURRENT, 
-            Arrays.asList(DUTCH_STOP_WORDS), false));
+    static final Set<?> DEFAULT_STOP_SET;
+
+    static {
+      try {
+        DEFAULT_STOP_SET = WordlistLoader.getSnowballWordSet(SnowballFilter.class, 
+            DEFAULT_STOPWORD_FILE);
+      } catch (IOException ex) {
+        // default set should always be present as it is part of the
+        // distribution (JAR)
+        throw new RuntimeException("Unable to load default stopword set");
+      }
+    }
   }
 
 
@@ -215,50 +231,38 @@
     }
   }
 
-  /**
-   * Creates a {@link TokenStream} which tokenizes all the text in the 
-   * provided {@link Reader}.
-   *
-   * @return A {@link TokenStream} built from a {@link StandardTokenizer}
-   *   filtered with {@link StandardFilter}, {@link StopFilter}, 
-   *   and {@link DutchStemFilter}
-   */
-  @Override
-  public TokenStream tokenStream(String fieldName, Reader reader) {
-    TokenStream result = new StandardTokenizer(matchVersion, reader);
-    result = new StandardFilter(result);
-    result = new StopFilter(matchVersion, result, stoptable);
-    result = new DutchStemFilter(result, excltable, stemdict);
-    return result;
-  }
-  
-  private class SavedStreams {
-    Tokenizer source;
-    TokenStream result;
-  };
-  
+
   /**
    * Returns a (possibly reused) {@link TokenStream} which tokenizes all the 
    * text in the provided {@link Reader}.
    *
    * @return A {@link TokenStream} built from a {@link StandardTokenizer}
-   *   filtered with {@link StandardFilter}, {@link StopFilter}, 
-   *   and {@link DutchStemFilter}
+   *   filtered with {@link StandardFilter}, {@link LowerCaseFilter}, 
+   *   {@link StopFilter}, {@link KeywordMarkerTokenFilter} if a stem exclusion set is provided,
+   *   {@link StemmerOverrideFilter}, and {@link SnowballFilter}
    */
   @Override
-  public TokenStream reusableTokenStream(String fieldName, Reader reader)
-      throws IOException {
-    SavedStreams streams = (SavedStreams) getPreviousTokenStream();
-    if (streams == null) {
-      streams = new SavedStreams();
-      streams.source = new StandardTokenizer(matchVersion, reader);
-      streams.result = new StandardFilter(streams.source);
-      streams.result = new StopFilter(matchVersion, streams.result, stoptable);
-      streams.result = new DutchStemFilter(streams.result, excltable, stemdict);
-      setPreviousTokenStream(streams);
+  protected TokenStreamComponents createComponents(String fieldName,
+      Reader aReader) {
+    if (matchVersion.onOrAfter(Version.LUCENE_31)) {
+      final Tokenizer source = new StandardTokenizer(matchVersion, aReader);
+      TokenStream result = new StandardFilter(source);
+      result = new LowerCaseFilter(matchVersion, result);
+      result = new StopFilter(matchVersion, result, stoptable);
+      if (!excltable.isEmpty())
+        result = new KeywordMarkerTokenFilter(result, excltable);
+      if (!stemdict.isEmpty())
+        result = new StemmerOverrideFilter(matchVersion, result, stemdict);
+      result = new SnowballFilter(result, new org.tartarus.snowball.ext.DutchStemmer());
+      return new TokenStreamComponents(source, result);
     } else {
-      streams.source.reset(reader);
+      final Tokenizer source = new StandardTokenizer(matchVersion, aReader);
+      TokenStream result = new StandardFilter(source);
+      result = new StopFilter(matchVersion, result, stoptable);
+      if (!excltable.isEmpty())
+        result = new KeywordMarkerTokenFilter(result, excltable);
+      result = new DutchStemFilter(result, stemdict);
+      return new TokenStreamComponents(source, result);
     }
-    return streams.result;
   }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemFilter.java Fri Feb 26 13:09:54 2010
@@ -23,8 +23,11 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;// for javadoc
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.snowball.SnowballFilter;
+import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 
 /**
@@ -34,34 +37,58 @@
  * not be stemmed at all. The stemmer used can be changed at runtime after the
  * filter object is created (as long as it is a {@link DutchStemmer}).
  * </p>
+ * <p>
+ * To prevent terms from being stemmed use an instance of
+ * {@link KeywordMarkerTokenFilter} or a custom {@link TokenFilter} that sets
+ * the {@link KeywordAttribute} before this {@link TokenStream}.
+ * </p>
+ * @see KeywordMarkerTokenFilter
+ * @deprecated Use {@link SnowballFilter} with 
+ * {@link org.tartarus.snowball.ext.DutchStemmer} instead, which has the
+ * same functionality. This filter will be removed in Lucene 4.0
  */
+@Deprecated
 public final class DutchStemFilter extends TokenFilter {
   /**
    * The actual token in the input stream.
    */
   private DutchStemmer stemmer = null;
-  private Set exclusions = null;
+  private Set<?> exclusions = null;
   
-  private TermAttribute termAtt;
+  private final TermAttribute termAtt;
+  private final KeywordAttribute keywordAttr;
 
   public DutchStemFilter(TokenStream _in) {
     super(_in);
     stemmer = new DutchStemmer();
     termAtt = addAttribute(TermAttribute.class);
+    keywordAttr = addAttribute(KeywordAttribute.class);
   }
 
   /**
    * Builds a DutchStemFilter that uses an exclusion table.
+   * @deprecated use {@link KeywordAttribute} with {@link KeywordMarkerTokenFilter} instead.
    */
-  public DutchStemFilter(TokenStream _in, Set exclusiontable) {
+  @Deprecated
+  public DutchStemFilter(TokenStream _in, Set<?> exclusiontable) {
     this(_in);
     exclusions = exclusiontable;
   }
+  
+  /**
+   * @param stemdictionary Dictionary of word stem pairs, that overrule the algorithm
+   */
+  public DutchStemFilter(TokenStream _in,  Map<?,?> stemdictionary) {
+    this(_in);
+    stemmer.setStemDictionary(stemdictionary);
+  }
 
   /**
    * @param stemdictionary Dictionary of word stem pairs, that overrule the algorithm
+   * @deprecated use {@link KeywordAttribute} with {@link KeywordMarkerTokenFilter} instead.
    */
-  public DutchStemFilter(TokenStream _in, Set exclusiontable, Map stemdictionary) {
+  @Deprecated
+  public DutchStemFilter(TokenStream _in, Set<?> exclusiontable, Map<?,?> stemdictionary) {
     this(_in, exclusiontable);
     stemmer.setStemDictionary(stemdictionary);
   }
@@ -72,11 +99,11 @@
   @Override
   public boolean incrementToken() throws IOException {
     if (input.incrementToken()) {
-      String term = termAtt.term();
+      final String term = termAtt.term();
 
       // Check the exclusion table.
-      if (exclusions == null || !exclusions.contains(term)) {
-        String s = stemmer.stem(term);
+      if (!keywordAttr.isKeyword() && (exclusions == null || !exclusions.contains(term))) {
+        final String s = stemmer.stem(term);
         // If not stemmed, don't waste the time adjusting the token.
         if ((s != null) && !s.equals(term))
           termAtt.setTermBuffer(s);
@@ -98,8 +125,10 @@
 
   /**
    * Set an alternative exclusion list for this filter.
+   * @deprecated use {@link KeywordAttribute} with {@link KeywordMarkerTokenFilter} instead.
    */
-  public void setExclusionTable(HashSet exclusiontable) {
+  @Deprecated
+  public void setExclusionTable(HashSet<?> exclusiontable) {
     exclusions = exclusiontable;
   }
 
@@ -107,7 +136,7 @@
    * Set dictionary for stemming, this dictionary overrules the algorithm,
    * so you can correct for a particular unwanted word-stem pair.
    */
-  public void setStemDictionary(HashMap dict) {
+  public void setStemDictionary(HashMap<?,?> dict) {
     if (stemmer != null)
       stemmer.setStemDictionary(dict);
   }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemmer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemmer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemmer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/nl/DutchStemmer.java Fri Feb 26 13:09:54 2010
@@ -26,8 +26,10 @@
  * the <a href="http://snowball.tartarus.org/algorithms/dutch/stemmer.html">dutch stemming</a>
  * algorithm in Martin Porter's snowball project.
  * </p>
+ * @deprecated Use {@link org.tartarus.snowball.ext.DutchStemmer} instead, 
+ * which has the same functionality. This filter will be removed in Lucene 4.0
  */
-
+@Deprecated
 public class DutchStemmer {
   /**
    * Buffer for the terms while stemming them.

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -17,7 +17,6 @@
  */
 
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilter.java Fri Feb 26 13:09:54 2010
@@ -19,7 +19,6 @@
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.index.Payload;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/position/PositionFilter.java Fri Feb 26 13:09:54 2010
@@ -21,7 +21,6 @@
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 
 /** Set the positionIncrement of all tokens to the "positionIncrement",

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java Fri Feb 26 13:09:54 2010
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.io.Reader;
 import java.util.Arrays;
 import java.util.Map;
@@ -26,10 +27,15 @@
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.LowerCaseFilter;
 import org.apache.lucene.analysis.ReusableAnalyzerBase.TokenStreamComponents; // javadoc @link
+import org.apache.lucene.analysis.snowball.SnowballFilter;
+import org.apache.lucene.analysis.standard.StandardFilter;
+import org.apache.lucene.analysis.standard.StandardTokenizer;
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;
 import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.WordlistLoader;
 import org.apache.lucene.util.Version;
 
 /**
@@ -39,13 +45,22 @@
  * will not be indexed at all).
  * A default set of stopwords is used unless an alternative list is specified.
  * </p>
+ * <a name="version"/>
+ * <p>You must specify the required {@link Version}
+ * compatibility when creating RussianAnalyzer:
+ * <ul>
+ *   <li> As of 3.1, StandardTokenizer is used, Snowball stemming is done with
+ *        SnowballFilter, and Snowball stopwords are used by default.
+ * </ul>
  */
 public final class RussianAnalyzer extends StopwordAnalyzerBase
 {
     /**
-     * List of typical Russian stopwords.
+     * List of typical Russian stopwords. (for backwards compatibility)
+     * @deprecated Remove this for LUCENE 4.0
      */
-    private static final String[] RUSSIAN_STOP_WORDS = {
+    @Deprecated
+    private static final String[] RUSSIAN_STOP_WORDS_30 = {
       "а", "без", "более", "бы", "был", "была", "были", "было", "быть", "в",
       "вам", "вас", "весь", "во", "вот", "все", "всего", "всех", "вы", "где", 
       "да", "даже", "для", "до", "его", "ее", "ей", "ею", "если", "есть", 
@@ -58,14 +73,44 @@
       "чем", "что", "чтобы", "чье", "чья", "эта", "эти", "это", "я"
     };
     
+    /** File containing default Russian stopwords. */
+    public final static String DEFAULT_STOPWORD_FILE = "russian_stop.txt";
+    
     private static class DefaultSetHolder {
-      static final Set<?> DEFAULT_STOP_SET = CharArraySet
+      /** @deprecated remove this for Lucene 4.0 */
+      @Deprecated
+      static final Set<?> DEFAULT_STOP_SET_30 = CharArraySet
           .unmodifiableSet(new CharArraySet(Version.LUCENE_CURRENT, 
-              Arrays.asList(RUSSIAN_STOP_WORDS), false));
+              Arrays.asList(RUSSIAN_STOP_WORDS_30), false));
+      static final Set<?> DEFAULT_STOP_SET;
+      
+      static {
+        try {
+          DEFAULT_STOP_SET = 
+            WordlistLoader.getSnowballWordSet(SnowballFilter.class, DEFAULT_STOPWORD_FILE);
+        } catch (IOException ex) {
+          // default set should always be present as it is part of the
+          // distribution (JAR)
+          throw new RuntimeException("Unable to load default stopword set");
+        }
+      }
+    }
+    
+    private final Set<?> stemExclusionSet;
+    
+    /**
+     * Returns an unmodifiable instance of the default stop-words set.
+     * 
+     * @return an unmodifiable instance of the default stop-words set.
+     */
+    public static Set<?> getDefaultStopSet() {
+      return DefaultSetHolder.DEFAULT_STOP_SET;
     }
 
     public RussianAnalyzer(Version matchVersion) {
-      this(matchVersion, DefaultSetHolder.DEFAULT_STOP_SET);
+      this(matchVersion,
+        matchVersion.onOrAfter(Version.LUCENE_31) ? DefaultSetHolder.DEFAULT_STOP_SET
+            : DefaultSetHolder.DEFAULT_STOP_SET_30);
     }
   
     /**
@@ -86,9 +131,24 @@
      *          a stopword set
      */
     public RussianAnalyzer(Version matchVersion, Set<?> stopwords){
+      this(matchVersion, stopwords, CharArraySet.EMPTY_SET);
+    }
+    
+    /**
+     * Builds an analyzer with the given stop words
+     * 
+     * @param matchVersion
+     *          lucene compatibility version
+     * @param stopwords
+     *          a stopword set
+     * @param stemExclusionSet a set of words not to be stemmed
+     */
+    public RussianAnalyzer(Version matchVersion, Set<?> stopwords, Set<?> stemExclusionSet){
       super(matchVersion, stopwords);
+      this.stemExclusionSet = CharArraySet.unmodifiableSet(CharArraySet.copy(matchVersion, stemExclusionSet));
     }
    
+   
     /**
      * Builds an analyzer with the given stop words.
      * TODO: create a Set version of this ctor
@@ -105,17 +165,30 @@
      * provided {@link Reader}.
      *
      * @return {@link TokenStreamComponents} built from a 
-     *   {@link RussianLetterTokenizer} filtered with 
+     *   {@link StandardTokenizer} filtered with {@link StandardFilter},
      *   {@link LowerCaseFilter}, {@link StopFilter}, 
-     *   and {@link RussianStemFilter}
+     *   {@link KeywordMarkerTokenFilter} if a stem exclusion set is provided,
+     *   and {@link SnowballFilter}
      */
     @Override
     protected TokenStreamComponents createComponents(String fieldName,
         Reader reader) {
-      final Tokenizer source = new RussianLetterTokenizer(reader);
-      TokenStream result = new LowerCaseFilter(matchVersion, source);
-      result = new StopFilter(matchVersion, result, stopwords);
-      return new TokenStreamComponents(source, new RussianStemFilter(result));
-      
+      if (matchVersion.onOrAfter(Version.LUCENE_31)) {
+        final Tokenizer source = new StandardTokenizer(matchVersion, reader);
+        TokenStream result = new StandardFilter(source);
+        result = new LowerCaseFilter(matchVersion, result);
+        result = new StopFilter(matchVersion, result, stopwords);
+        if (!stemExclusionSet.isEmpty()) result = new KeywordMarkerTokenFilter(
+            result, stemExclusionSet);
+        result = new SnowballFilter(result, new org.tartarus.snowball.ext.RussianStemmer());
+        return new TokenStreamComponents(source, result);
+      } else {
+        final Tokenizer source = new RussianLetterTokenizer(matchVersion, reader);
+        TokenStream result = new LowerCaseFilter(matchVersion, source);
+        result = new StopFilter(matchVersion, result, stopwords);
+        if (!stemExclusionSet.isEmpty()) result = new KeywordMarkerTokenFilter(
+          result, stemExclusionSet);
+        return new TokenStreamComponents(source, new RussianStemFilter(result));
+      }
     }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianLetterTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianLetterTokenizer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianLetterTokenizer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianLetterTokenizer.java Fri Feb 26 13:09:54 2010
@@ -21,40 +21,112 @@
 import org.apache.lucene.analysis.CharTokenizer;
 import org.apache.lucene.analysis.Tokenizer; // for javadocs
 import org.apache.lucene.analysis.LetterTokenizer; // for javadocs
+import org.apache.lucene.analysis.standard.StandardTokenizer; // for javadocs
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.Version;
 
 /**
  * A RussianLetterTokenizer is a {@link Tokenizer} that extends {@link LetterTokenizer}
- * by also allowing the basic latin digits 0-9. 
+ * by also allowing the basic Latin digits 0-9.
+ * <p>
+ * <a name="version"/>
+ * You must specify the required {@link Version} compatibility when creating
+ * {@link RussianLetterTokenizer}:
+ * <ul>
+ * <li>As of 3.1, {@link CharTokenizer} uses an int based API to normalize and
+ * detect token characters. See {@link CharTokenizer#isTokenChar(int)} and
+ * {@link CharTokenizer#normalize(int)} for details.</li>
+ * </ul>
+ * @deprecated Use {@link StandardTokenizer} instead, which has the same functionality.
+ * This filter will be removed in Lucene 4.0 
  */
-
+@Deprecated
 public class RussianLetterTokenizer extends CharTokenizer
 {    
-    public RussianLetterTokenizer(Reader in)
-    {
-    	super(in);
+    private static final int DIGIT_0 = '0';
+    private static final int DIGIT_9 = '9';
+    
+    /**
+     * Construct a new RussianLetterTokenizer. * @param matchVersion Lucene version
+     * to match See {@link <a href="#version">above</a>}
+     * 
+     * @param in
+     *          the input to split up into tokens
+     */
+    public RussianLetterTokenizer(Version matchVersion, Reader in) {
+      super(matchVersion, in);
     }
 
-    public RussianLetterTokenizer(AttributeSource source, Reader in)
-    {
-        super(source, in);
+    /**
+     * Construct a new RussianLetterTokenizer using a given {@link AttributeSource}.
+     * 
+     * @param matchVersion
+     *          Lucene version to match See {@link <a href="#version">above</a>}
+     * @param source
+     *          the attribute source to use for this {@link Tokenizer}
+     * @param in
+     *          the input to split up into tokens
+     */
+    public RussianLetterTokenizer(Version matchVersion, AttributeSource source, Reader in) {
+      super(matchVersion, source, in);
     }
 
-    public RussianLetterTokenizer(AttributeFactory factory, Reader in)
-    {
-        super(factory, in);
+    /**
+     * Construct a new RussianLetterTokenizer using a given
+     * {@link org.apache.lucene.util.AttributeSource.AttributeFactory}. * @param
+     * matchVersion Lucene version to match See
+     * {@link <a href="#version">above</a>}
+     * 
+     * @param factory
+     *          the attribute factory to use for this {@link Tokenizer}
+     * @param in
+     *          the input to split up into tokens
+     */
+    public RussianLetterTokenizer(Version matchVersion, AttributeFactory factory, Reader in) {
+      super(matchVersion, factory, in);
     }
     
     /**
+     * Construct a new RussianLetterTokenizer.
+     * 
+     * @deprecated use {@link #RussianLetterTokenizer(Version, Reader)} instead. This will
+     *             be removed in Lucene 4.0.
+     */
+    @Deprecated
+    public RussianLetterTokenizer(Reader in) {
+      super(in);
+    }
+
+    /**
+     * Construct a new RussianLetterTokenizer using a given {@link AttributeSource}.
+     * 
+     * @deprecated use {@link #RussianLetterTokenizer(Version, AttributeSource, Reader)}
+     *             instead. This will be removed in Lucene 4.0.
+     */
+    @Deprecated
+    public RussianLetterTokenizer(AttributeSource source, Reader in) {
+      super(source, in);
+    }
+
+    /**
+     * Construct a new RussianLetterTokenizer using a given
+     * {@link org.apache.lucene.util.AttributeSource.AttributeFactory}.
+     * 
+     * @deprecated use {@link #RussianLetterTokenizer(Version, AttributeSource.AttributeFactory, Reader)}
+     *             instead. This will be removed in Lucene 4.0.
+     */
+    @Deprecated
+    public RussianLetterTokenizer(AttributeFactory factory, Reader in) {
+      super(factory, in);
+    }
+    
+    
+    /**
      * Collects only characters which satisfy
-     * {@link Character#isLetter(char)}.
+     * {@link Character#isLetter(int)}.
      */
     @Override
-    protected boolean isTokenChar(char c)
-    {
-        if (Character.isLetter(c) || (c >= '0' && c <= '9'))
-            return true;
-        else
-            return false;
+    protected boolean isTokenChar(int c) {
+        return Character.isLetter(c) || (c >= DIGIT_0 && c <= DIGIT_9);
     }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemFilter.java Fri Feb 26 13:09:54 2010
@@ -17,11 +17,14 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;// for javadoc
 import org.apache.lucene.analysis.LowerCaseFilter; // for javadoc
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.analysis.ru.RussianStemmer;//javadoc @link
+import org.apache.lucene.analysis.snowball.SnowballFilter; // javadoc @link
 
 import java.io.IOException;
 
@@ -32,7 +35,17 @@
  * The input should be filtered by {@link LowerCaseFilter} before passing it to RussianStemFilter ,
  * because RussianStemFilter only works with lowercase characters.
  * </p>
+ * <p>
+ * To prevent terms from being stemmed use an instance of
+ * {@link KeywordMarkerTokenFilter} or a custom {@link TokenFilter} that sets
+ * the {@link KeywordAttribute} before this {@link TokenStream}.
+ * </p>
+ * @see KeywordMarkerTokenFilter
+ * @deprecated Use {@link SnowballFilter} with 
+ * {@link org.tartarus.snowball.ext.RussianStemmer} instead, which has the
+ * same functionality. This filter will be removed in Lucene 4.0
  */
+@Deprecated
 public final class RussianStemFilter extends TokenFilter
 {
     /**
@@ -40,13 +53,15 @@
      */
     private RussianStemmer stemmer = null;
 
-    private TermAttribute termAtt;
+    private final TermAttribute termAtt;
+    private final KeywordAttribute keywordAttr;
 
     public RussianStemFilter(TokenStream in)
     {
         super(in);
         stemmer = new RussianStemmer();
         termAtt = addAttribute(TermAttribute.class);
+        keywordAttr = addAttribute(KeywordAttribute.class);
     }
     /**
      * Returns the next token in the stream, or null at EOS
@@ -55,10 +70,12 @@
     public final boolean incrementToken() throws IOException
     {
       if (input.incrementToken()) {
-        String term = termAtt.term();
-        String s = stemmer.stem(term);
-        if (s != null && !s.equals(term))
-          termAtt.setTermBuffer(s);
+        if(!keywordAttr.isKeyword()) {
+          final String term = termAtt.term();
+          final String s = stemmer.stem(term);
+          if (s != null && !s.equals(term))
+            termAtt.setTermBuffer(s);
+        }
         return true;
       } else {
         return false;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemmer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemmer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemmer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/ru/RussianStemmer.java Fri Feb 26 13:09:54 2010
@@ -19,7 +19,10 @@
 
 /**
  * Russian stemming algorithm implementation (see http://snowball.sourceforge.net for detailed description).
+ * @deprecated Use {@link org.tartarus.snowball.ext.RussianStemmer} instead, 
+ * which has the same functionality. This filter will be removed in Lucene 4.0
  */
+@Deprecated
 class RussianStemmer
 {
     // positions of RV, R1 and R2 respectively

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java Fri Feb 26 13:09:54 2010
@@ -34,7 +34,9 @@
 public final class ShingleAnalyzerWrapper extends Analyzer {
 
   private final Analyzer defaultAnalyzer;
-  private int maxShingleSize = 2;
+  private int maxShingleSize = ShingleFilter.DEFAULT_MAX_SHINGLE_SIZE;
+  private int minShingleSize = ShingleFilter.DEFAULT_MIN_SHINGLE_SIZE;
+  private String tokenSeparator = ShingleFilter.TOKEN_SEPARATOR;
   private boolean outputUnigrams = true;
 
   public ShingleAnalyzerWrapper(Analyzer defaultAnalyzer) {
@@ -44,7 +46,13 @@
 
   public ShingleAnalyzerWrapper(Analyzer defaultAnalyzer, int maxShingleSize) {
     this(defaultAnalyzer);
-    this.maxShingleSize = maxShingleSize;
+    setMaxShingleSize(maxShingleSize);
+  }
+
+  public ShingleAnalyzerWrapper(Analyzer defaultAnalyzer, int minShingleSize, int maxShingleSize) {
+    this(defaultAnalyzer);
+    setMaxShingleSize(maxShingleSize);
+    setMinShingleSize(minShingleSize);
   }
 
   /**
@@ -58,29 +66,73 @@
   /**
    * Wraps {@link StandardAnalyzer}. 
    */
-  public ShingleAnalyzerWrapper(Version matchVersion, int nGramSize) {
+  public ShingleAnalyzerWrapper(Version matchVersion, int minShingleSize, int maxShingleSize) {
     this(matchVersion);
-    this.maxShingleSize = nGramSize;
+    setMaxShingleSize(maxShingleSize);
+    setMinShingleSize(minShingleSize);
   }
 
   /**
-   * The max shingle (ngram) size
+   * The max shingle (token ngram) size
    * 
-   * @return The max shingle (ngram) size
+   * @return The max shingle (token ngram) size
    */
   public int getMaxShingleSize() {
     return maxShingleSize;
   }
 
   /**
-   * Set the maximum size of output shingles
-   * 
+   * Set the maximum size of output shingles (default: 2)
+   *
    * @param maxShingleSize max shingle size
    */
   public void setMaxShingleSize(int maxShingleSize) {
+    if (maxShingleSize < 2) {
+      throw new IllegalArgumentException("Max shingle size must be >= 2");
+    }
     this.maxShingleSize = maxShingleSize;
   }
 
+  /**
+   * The min shingle (token ngram) size
+   * 
+   * @return The min shingle (token ngram) size
+   */
+  public int getMinShingleSize() {
+    return minShingleSize;
+  }
+
+  /**
+   * <p>Set the min shingle size (default: 2).
+   * <p>This method requires that the passed in minShingleSize is not greater
+   * than maxShingleSize, so make sure that maxShingleSize is set before
+   * calling this method.
+   *
+   * @param minShingleSize min size of output shingles
+   */
+  public void setMinShingleSize(int minShingleSize) {
+    if (minShingleSize < 2) {
+      throw new IllegalArgumentException("Min shingle size must be >= 2");
+    }
+    if (minShingleSize > maxShingleSize) {
+      throw new IllegalArgumentException
+        ("Min shingle size must be <= max shingle size");
+    }
+    this.minShingleSize = minShingleSize;
+  }
+
+  public String getTokenSeparator() {
+    return tokenSeparator;
+  }
+
+  /**
+   * Sets the string to use when joining adjacent tokens to form a shingle
+   * @param tokenSeparator used to separate input stream tokens in output shingles
+   */
+  public void setTokenSeparator(String tokenSeparator) {
+    this.tokenSeparator = (tokenSeparator == null ? "" : tokenSeparator);
+  }
+  
   public boolean isOutputUnigrams() {
     return outputUnigrams;
   }
@@ -104,8 +156,10 @@
     } catch (IOException e) {
       wrapped = defaultAnalyzer.tokenStream(fieldName, reader);
     }
-    ShingleFilter filter = new ShingleFilter(wrapped);
+    ShingleFilter filter = new ShingleFilter(wrapped, minShingleSize, maxShingleSize);
+    filter.setMinShingleSize(minShingleSize);
     filter.setMaxShingleSize(maxShingleSize);
+    filter.setTokenSeparator(tokenSeparator);
     filter.setOutputUnigrams(outputUnigrams);
     return filter;
   }
@@ -113,7 +167,7 @@
   private class SavedStreams {
     TokenStream wrapped;
     ShingleFilter shingle;
-  };
+  }
   
   @Override
   public TokenStream reusableTokenStream(String fieldName, Reader reader) throws IOException {
@@ -135,6 +189,8 @@
       }
     }
     streams.shingle.setMaxShingleSize(maxShingleSize);
+    streams.shingle.setMinShingleSize(minShingleSize);
+    streams.shingle.setTokenSeparator(tokenSeparator);
     streams.shingle.setOutputUnigrams(outputUnigrams);
     return streams.shingle;
   }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java Fri Feb 26 13:09:54 2010
@@ -18,18 +18,15 @@
  */
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.LinkedList;
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.AttributeSource.State;
+
 
 /**
  * <p>A ShingleFilter constructs shingles (token n-grams) from a token stream.
@@ -44,26 +41,59 @@
  */
 public final class ShingleFilter extends TokenFilter {
 
-  private LinkedList<State> shingleBuf = new LinkedList<State>();
-  private StringBuilder[] shingles;
-  private String tokenType = "shingle";
-
   /**
    * filler token for when positionIncrement is more than 1
    */
   public static final char[] FILLER_TOKEN = { '_' };
 
-
   /**
    * default maximum shingle size is 2.
    */
   public static final int DEFAULT_MAX_SHINGLE_SIZE = 2;
 
   /**
-   * The string to use when joining adjacent tokens to form a shingle
+   * default minimum shingle size is 2.
+   */
+  public static final int DEFAULT_MIN_SHINGLE_SIZE = 2;
+
+  /**
+   * default token type attribute value is "shingle" 
+   */
+  public static final String DEFAULT_TOKEN_TYPE = "shingle";
+  
+  /**
+   * The default string to use when joining adjacent tokens to form a shingle
    */
   public static final String TOKEN_SEPARATOR = " ";
 
+
+  /**
+   * The sequence of input stream tokens (or filler tokens, if necessary)
+   * that will be composed to form output shingles.
+   */
+  private LinkedList<State> inputWindow = new LinkedList<State>();
+  
+  /**
+   * The number of input tokens in the next output token.  This is the "n" in
+   * "token n-grams".
+   */
+  private CircularSequence gramSize;
+
+  /**
+   * Shingle text is composed here.
+   */
+  private StringBuilder shingleBuilder = new StringBuilder();
+
+  /**
+   * The token type attribute value to use - default is "shingle"
+   */
+  private String tokenType = DEFAULT_TOKEN_TYPE;
+
+  /**
+   * The string to use when joining adjacent tokens to form a shingle
+   */
+  private String tokenSeparator = TOKEN_SEPARATOR;
+
   /**
    * By default, we output unigrams (individual tokens) as well as shingles
    * (token n-grams).
@@ -76,15 +106,40 @@
   private int maxShingleSize;
 
   /**
-   * Constructs a ShingleFilter with the specified single size from the
+   * minimum shingle size (number of tokens)
+   */
+  private int minShingleSize;
+
+  /**
+   * The remaining number of filler tokens inserted into the input stream
+   * from which shingles are composed, to handle position increments greater
+   * than one.
+   */
+  private int numFillerTokensToInsert;
+
+  /**
+   * The next input stream token.
+   */
+  private State nextInputStreamToken;
+  
+  private final TermAttribute termAtt;
+  private final OffsetAttribute offsetAtt;
+  private final PositionIncrementAttribute posIncrAtt;
+  private final TypeAttribute typeAtt;
+
+
+  /**
+   * Constructs a ShingleFilter with the specified shingle size from the
    * {@link TokenStream} <code>input</code>
    *
    * @param input input stream
+   * @param minShingleSize minimum shingle size produced by the filter.
    * @param maxShingleSize maximum shingle size produced by the filter.
    */
-  public ShingleFilter(TokenStream input, int maxShingleSize) {
+  public ShingleFilter(TokenStream input, int minShingleSize, int maxShingleSize) {
     super(input);
     setMaxShingleSize(maxShingleSize);
+    setMinShingleSize(minShingleSize);
     this.termAtt = addAttribute(TermAttribute.class);
     this.offsetAtt = addAttribute(OffsetAttribute.class);
     this.posIncrAtt = addAttribute(PositionIncrementAttribute.class);
@@ -92,22 +147,34 @@
   }
 
   /**
-   * Construct a ShingleFilter with default shingle size.
+   * Constructs a ShingleFilter with the specified shingle size from the
+   * {@link TokenStream} <code>input</code>
+   *
+   * @param input input stream
+   * @param maxShingleSize maximum shingle size produced by the filter.
+   */
+  public ShingleFilter(TokenStream input, int maxShingleSize) {
+    this(input, DEFAULT_MIN_SHINGLE_SIZE, maxShingleSize);
+  }
+  
+  /**
+   * Construct a ShingleFilter with default shingle size: 2.
    *
    * @param input input stream
    */
   public ShingleFilter(TokenStream input) {
-    this(input, DEFAULT_MAX_SHINGLE_SIZE);
+    this(input, DEFAULT_MIN_SHINGLE_SIZE, DEFAULT_MAX_SHINGLE_SIZE);
   }
 
   /**
-   * Construct a ShingleFilter with the specified token type for shingle tokens.
+   * Construct a ShingleFilter with the specified token type for shingle tokens
+   * and the default shingle size: 2
    *
    * @param input input stream
    * @param tokenType token type for shingle tokens
    */
   public ShingleFilter(TokenStream input, String tokenType) {
-    this(input, DEFAULT_MAX_SHINGLE_SIZE);
+    this(input, DEFAULT_MIN_SHINGLE_SIZE, DEFAULT_MAX_SHINGLE_SIZE);
     setTokenType(tokenType);
   }
 
@@ -130,6 +197,7 @@
    */
   public void setOutputUnigrams(boolean outputUnigrams) {
     this.outputUnigrams = outputUnigrams;
+    gramSize = new CircularSequence();
   }
 
   /**
@@ -141,203 +209,239 @@
     if (maxShingleSize < 2) {
       throw new IllegalArgumentException("Max shingle size must be >= 2");
     }
-    shingles = new StringBuilder[maxShingleSize];
-    for (int i = 0; i < shingles.length; i++) {
-      shingles[i] = new StringBuilder();
-    }
     this.maxShingleSize = maxShingleSize;
   }
 
   /**
-   * Clear the StringBuilders that are used for storing the output shingles.
+   * <p>Set the min shingle size (default: 2).
+   * <p>This method requires that the passed in minShingleSize is not greater
+   * than maxShingleSize, so make sure that maxShingleSize is set before
+   * calling this method.
+   * <p>The unigram output option is independent of the min shingle size.
+   *
+   * @param minShingleSize min size of output shingles
    */
-  private void clearShingles() {
-    for (int i = 0; i < shingles.length; i++) {
-      shingles[i].setLength(0);
+  public void setMinShingleSize(int minShingleSize) {
+    if (minShingleSize < 2) {
+      throw new IllegalArgumentException("Min shingle size must be >= 2");
     }
+    if (minShingleSize > maxShingleSize) {
+      throw new IllegalArgumentException
+        ("Min shingle size must be <= max shingle size");
+    }
+    this.minShingleSize = minShingleSize;
+    gramSize = new CircularSequence();
+  }
+
+  /**
+   * Sets the string to use when joining adjacent tokens to form a shingle
+   * @param tokenSeparator used to separate input stream tokens in output shingles
+   */
+  public void setTokenSeparator(String tokenSeparator) {
+    this.tokenSeparator = null == tokenSeparator ? "" : tokenSeparator;
   }
-  
-  private AttributeSource.State nextToken;
-  private int shingleBufferPosition;
-  private int[] endOffsets;
 
   /* (non-Javadoc)
    * @see org.apache.lucene.analysis.TokenStream#next()
    */
   @Override
   public final boolean incrementToken() throws IOException {
-    while (true) {
-      if (nextToken == null) {
-        if (!fillShingleBuffer()) {
-          return false;
-        }
-      }
-      
-      nextToken = shingleBuf.getFirst();
-      
-      if (outputUnigrams) {
-        if (shingleBufferPosition == 0) {
-          restoreState(nextToken);
-          posIncrAtt.setPositionIncrement(1);
-          shingleBufferPosition++;
-          return true;
-        }
-      } else if (shingleBufferPosition % this.maxShingleSize == 0){
-        shingleBufferPosition++;
+    boolean tokenAvailable = false; 
+    if (gramSize.atMinValue() || inputWindow.size() < gramSize.getValue()) {
+      shiftInputWindow();
+    }
+    if ( ! inputWindow.isEmpty()) {
+      restoreState(inputWindow.getFirst());
+      if (1 == gramSize.getValue()) {
+        posIncrAtt.setPositionIncrement(1);
+        gramSize.advance();
+        tokenAvailable = true;
+      } else if (inputWindow.size() >= gramSize.getValue()) {
+        getNextShingle();
+        gramSize.advance();
+        tokenAvailable = true;
       }
-  
-      if (shingleBufferPosition < shingleBuf.size()) {
-        restoreState(nextToken);
-        typeAtt.setType(tokenType);
-        offsetAtt.setOffset(offsetAtt.startOffset(), endOffsets[shingleBufferPosition]);
-        StringBuilder buf = shingles[shingleBufferPosition];
-        int termLength = buf.length();
-        char[] termBuffer = termAtt.termBuffer();
-        if (termBuffer.length < termLength)
-          termBuffer = termAtt.resizeTermBuffer(termLength);
-        buf.getChars(0, termLength, termBuffer, 0);
-        termAtt.setTermLength(termLength);
-        if ((! outputUnigrams) && shingleBufferPosition % this.maxShingleSize == 1) {
-          posIncrAtt.setPositionIncrement(1);
-        } else {
-          posIncrAtt.setPositionIncrement(0);
-        }
-        shingleBufferPosition++;
-        if (shingleBufferPosition == shingleBuf.size()) {
-          nextToken = null;
-          shingleBufferPosition = 0;
-        }
-        return true;
-      } else {
-        nextToken = null;
-        shingleBufferPosition = 0;
+    }
+    return tokenAvailable;
+  }
+
+  /**
+   * <p>Makes the next token a shingle of length {@link #gramSize}, 
+   * composed of tokens taken from {@link #inputWindow}.
+   * <p>Callers of this method must first insure that there are at least 
+   * <code>gramSize</code> tokens available in <code>inputWindow</code>.
+   */
+  private void getNextShingle() {
+    int startOffset = offsetAtt.startOffset();
+
+    int minTokNum = gramSize.getValue() - 1; // zero-based inputWindow position
+    if (gramSize.getValue() == minShingleSize) {
+      // Clear the shingle text buffer if this is the first shingle
+      // at the current position in the input stream.
+      shingleBuilder.setLength(0);
+      minTokNum = 0;
+    }
+    for (int tokNum = minTokNum ; tokNum < gramSize.getValue() ; ++tokNum) {
+      if (tokNum > 0) {
+        shingleBuilder.append(tokenSeparator);
       }
+      restoreState(inputWindow.get(tokNum));
+      shingleBuilder.append(termAtt.termBuffer(), 0, termAtt.termLength());
+    }
+    char[] termBuffer = termAtt.termBuffer();
+    int termLength = shingleBuilder.length();
+    if (termBuffer.length < termLength) {
+      termBuffer = termAtt.resizeTermBuffer(termLength);
     }
+    shingleBuilder.getChars(0, termLength, termBuffer, 0);
+    termAtt.setTermLength(termLength);
+    posIncrAtt.setPositionIncrement(gramSize.atMinValue() ? 1 : 0);
+    typeAtt.setType(tokenType);
+    offsetAtt.setOffset(startOffset, offsetAtt.endOffset());
   }
   
-  private int numFillerTokensToInsert;
-  private AttributeSource.State currentToken;
-  private boolean hasCurrentToken;
-   
-  private TermAttribute termAtt;
-  private OffsetAttribute offsetAtt;
-  private PositionIncrementAttribute posIncrAtt;
-  private TypeAttribute typeAtt;
-  
   /**
-   * Get the next token from the input stream and push it on the token buffer.
-   * If we encounter a token with position increment > 1, we put filler tokens
-   * on the token buffer.
-   * <p/>
-   * Returns null when the end of the input stream is reached.
-   * @return the next token, or null if at end of input stream
+   * <p>Get the next token from the input stream.
+   * <p>If the next token has <code>positionIncrement > 1</code>,
+   * <code>positionIncrement - 1</code> {@link #FILLER_TOKEN}s are
+   * inserted first.
+   * @return false for end of stream; true otherwise
    * @throws IOException if the input stream has a problem
    */
   private boolean getNextToken() throws IOException {
-    
-    while (true) {
-  	  if (numFillerTokensToInsert > 0) {
-  	    if (currentToken == null) {
-  	      currentToken = captureState();
-  	    } else {
-  	      restoreState(currentToken);
-  	    }
-  	    numFillerTokensToInsert--;
-        // A filler token occupies no space
-  	    offsetAtt.setOffset(offsetAtt.startOffset(), offsetAtt.startOffset());
-  	    termAtt.setTermBuffer(FILLER_TOKEN, 0, FILLER_TOKEN.length);
-        return true;
-  	  } 
-  	  
-  	  if (hasCurrentToken) {
-  	    if (currentToken != null) {
-  	      restoreState(currentToken);
-  	      currentToken = null;
-  	    }
-  	    hasCurrentToken = false;
-  	    return true;
-  	  }
-  	  
-  	  if (!input.incrementToken()) return false;
-  	  hasCurrentToken = true;
-  	  
-  	  if (posIncrAtt.getPositionIncrement() > 1) {
-  	    numFillerTokensToInsert = posIncrAtt.getPositionIncrement() - 1;
-  	  }
+    boolean success = false;
+    if (numFillerTokensToInsert > 0) {
+      insertFillerToken();
+      success = true;
+    } else if (null != nextInputStreamToken) {
+      restoreState(nextInputStreamToken);
+      nextInputStreamToken = null;
+      success = true;
+    } else if (input.incrementToken()) {
+      if (posIncrAtt.getPositionIncrement() > 1) {
+        numFillerTokensToInsert = posIncrAtt.getPositionIncrement() - 1;
+        insertFillerToken();
+      }
+      success = true;
     }
+    return success;
 	}
 
   /**
-   * Fill the output buffer with new shingles.
+   * Inserts a {@link #FILLER_TOKEN} and decrements
+   * {@link #numFillerTokensToInsert}.
+   */
+  private void insertFillerToken() {
+    if (null == nextInputStreamToken) {
+      nextInputStreamToken = captureState();
+    } else {
+      restoreState(nextInputStreamToken);
+    }
+    --numFillerTokensToInsert;
+    // A filler token occupies no space
+    offsetAtt.setOffset(offsetAtt.startOffset(), offsetAtt.startOffset());
+    termAtt.setTermBuffer(FILLER_TOKEN, 0, FILLER_TOKEN.length);
+  }
+
+  /**
+   * <p>Fills {@link #inputWindow} with input stream tokens, if available, 
+   * shifting to the right if the window was previously full.
+   * <p>Resets {@link #gramSize} to its minimum value.
    *
    * @throws IOException if there's a problem getting the next token
    */
-  private boolean fillShingleBuffer() throws IOException {
-    boolean addedToken = false;
-    /*
-     * Try to fill the shingle buffer.
-     */
-    do {
-      if (getNextToken()) {
-        shingleBuf.add(captureState());
-        if (shingleBuf.size() > maxShingleSize)
-        {
-          shingleBuf.removeFirst();
-        }
-        addedToken = true;
-      } else {
+  private void shiftInputWindow() throws IOException {
+    if (inputWindow.size() > 0) {
+      inputWindow.removeFirst();
+    }
+    while (getNextToken()) {
+      inputWindow.add(captureState());
+      if (inputWindow.size() == maxShingleSize) {
         break;
       }
-    } while (shingleBuf.size() < maxShingleSize);
-
-    if (shingleBuf.isEmpty()) {
-      return false;
     }
+    gramSize.reset();
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    gramSize.reset();
+    inputWindow.clear();
+    numFillerTokensToInsert = 0;
+  }
+
+
+  /**
+   * <p>An instance of this class is used to maintain the number of input
+   * stream tokens that will be used to compose the next unigram or shingle:
+   * {@link #gramSize}.
+   * <p><code>gramSize</code> will take on values from the circular sequence
+   * <b>{ [ 1, ] {@link #minShingleSize} [ , ... , {@link #maxShingleSize} ] }</b>.
+   * <p>1 is included in the circular sequence only if 
+   * {@link #outputUnigrams} = true.
+   */
+  private class CircularSequence {
+    private int value;
+    private int minValue;
     
-    /*
-     * If no new token could be added to the shingle buffer, we have reached
-     * the end of the input stream and have to discard the least recent token.
+    public CircularSequence() {
+      minValue = outputUnigrams ? 1 : minShingleSize;
+      reset();
+    }
+
+    /**
+     * {@see #advance()}
+     * @return the current value.  
      */
-    if (! addedToken) {
-      shingleBuf.removeFirst();
+    public int getValue() {
+      return value;
     }
     
-    if (shingleBuf.isEmpty()) {
-      return false;
+    /**
+     * <p>Increments this circular number's value to the next member in the
+     * circular sequence
+     * <code>gramSize</code> will take on values from the circular sequence
+     * <b>{ [ 1, ] {@link #minShingleSize} [ , ... , {@link #maxShingleSize} ] }</b>.
+     * <p>1 is included in the circular sequence only if 
+     * {@link #outputUnigrams} = true.
+     * 
+     * @return the next member in the circular sequence
+     */
+    public int advance() {
+      if (value == 1) {
+        value = minShingleSize;
+      } else if (value == maxShingleSize) {
+        reset();
+      } else {
+        ++value;
+      }
+      return value;
     }
 
-    clearShingles();
-
-    endOffsets = new int[shingleBuf.size()];
-    for (int i = 0; i < endOffsets.length; i++) {
-      endOffsets[i] = 0;
+    /**
+     * <p>Sets this circular number's value to the first member of the 
+     * circular sequence
+     * <p><code>gramSize</code> will take on values from the circular sequence
+     * <b>{ [ 1, ] {@link #minShingleSize} [ , ... , {@link #maxShingleSize} ] }</b>.
+     * <p>1 is included in the circular sequence only if 
+     * {@link #outputUnigrams} = true.
+     */
+    public void reset() {
+      value = minValue;
     }
 
-    int i = 0;
-    for (Iterator<State> it = shingleBuf.iterator(); it.hasNext(); ) {
-      restoreState(it.next());
-      for (int j = i; j < shingles.length; j++) {
-        if (shingles[j].length() != 0) {
-          shingles[j].append(TOKEN_SEPARATOR);
-        }
-        shingles[j].append(termAtt.termBuffer(), 0, termAtt.termLength());
-      }
-
-      endOffsets[i] = offsetAtt.endOffset();
-      i++;
+    /**
+     * <p>Returns true if the current value is the first member of the circular
+     * sequence.
+     * <p>If {@link #outputUnigrams} = true, the first member of the circular
+     * sequence will be 1; otherwise, it will be {@link #minShingleSize}.
+     * 
+     * @return true if the current value is the first member of the circular
+     *  sequence; false otherwise
+     */
+    public boolean atMinValue() {
+      return value == minValue;
     }
-    
-    return true;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    super.reset();
-    nextToken = null;
-    shingleBufferPosition = 0;
-    shingleBuf.clear();
-    numFillerTokensToInsert = 0;
-    currentToken = null;
-    hasCurrentToken = false;
   }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java Fri Feb 26 13:09:54 2010
@@ -377,6 +377,7 @@
     } while (token == request_next_token);
     if (token == null) return false;
 
+    clearAttributes();
     termAtt.setTermBuffer(token.termBuffer(), 0, token.termLength());
     posIncrAtt.setPositionIncrement(token.getPositionIncrement());
     flagsAtt.setFlags(token.getFlags());

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java Fri Feb 26 13:09:54 2010
@@ -19,7 +19,6 @@
 import java.io.IOException;
 import java.util.Locale;
 import java.lang.Character.UnicodeBlock;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/tr/package.html
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/tr/package.html?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/tr/package.html (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/java/org/apache/lucene/analysis/tr/package.html Fri Feb 26 13:09:54 2010
@@ -17,15 +17,6 @@
 -->
 <html><head></head>
 <body>
-Support for Turkish.
-<p>
-This package contains just the TokenStream for handling turkish casing,
-for a stemmer please see the snowball package. 
-</p>
-<p>
-WARNING: SnowballAnalyzer uses LowerCaseFilter by default, even when the
-language is set to Turkish, so you will need to construct your own
-analyzer that combines TurkishLowerCaseFilter and SnowballFilter.
-</p>
+Analyzer for Turkish.
 </body>
-</html>
\ No newline at end of file
+</html>

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicAnalyzer.java Fri Feb 26 13:09:54 2010
@@ -17,11 +17,13 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.util.Version;
 
 /**
@@ -84,4 +86,17 @@
     assertAnalyzesTo(a, "The quick brown fox.", new String[] { "quick",
         "brown", "fox" });
   }
+  
+  public void testWithStemExclusionSet() throws IOException {
+    Set<String> set = new HashSet<String>();
+    set.add("ساهدهات");
+    ArabicAnalyzer a = new ArabicAnalyzer(Version.LUCENE_CURRENT, CharArraySet.EMPTY_SET, set);
+    assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهدهات" });
+    assertAnalyzesToReuse(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهدهات" });
+
+    
+    a = new ArabicAnalyzer(Version.LUCENE_CURRENT, CharArraySet.EMPTY_SET, CharArraySet.EMPTY_SET);
+    assertAnalyzesTo(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهد" });
+    assertAnalyzesToReuse(a, "كبيرة the quick ساهدهات", new String[] { "كبير","the", "quick", "ساهد" });
+  }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicNormalizationFilter.java Fri Feb 26 13:09:54 2010
@@ -21,7 +21,7 @@
 import java.io.StringReader;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.util.Version;
 
 /**
  * Test the Arabic Normalization Filter
@@ -86,7 +86,7 @@
   }  
   
   private void check(final String input, final String expected) throws IOException {
-    ArabicLetterTokenizer tokenStream = new ArabicLetterTokenizer(new StringReader(input));
+    ArabicLetterTokenizer tokenStream = new ArabicLetterTokenizer(Version.LUCENE_CURRENT, new StringReader(input));
     ArabicNormalizationFilter filter = new ArabicNormalizationFilter(tokenStream);
     assertTokenStreamContents(filter, new String[]{expected});
   }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/ar/TestArabicStemFilter.java Fri Feb 26 13:09:54 2010
@@ -21,7 +21,9 @@
 import java.io.StringReader;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;
+import org.apache.lucene.util.Version;
 
 /**
  * Test the Arabic Normalization Filter
@@ -112,11 +114,19 @@
   public void testNonArabic() throws IOException {
     check("English", "English");
   }
+  
+  public void testWithKeywordAttribute() throws IOException {
+    CharArraySet set = new CharArraySet(Version.LUCENE_CURRENT, 1, true);
+    set.add("ساهدهات");
+    ArabicLetterTokenizer tokenStream  = new ArabicLetterTokenizer(Version.LUCENE_CURRENT, new StringReader("ساهدهات"));
+
+    ArabicStemFilter filter = new ArabicStemFilter(new KeywordMarkerTokenFilter(tokenStream, set));
+    assertTokenStreamContents(filter, new String[]{"ساهدهات"});
+  }
 
   private void check(final String input, final String expected) throws IOException {
-    ArabicLetterTokenizer tokenStream  = new ArabicLetterTokenizer(new StringReader(input));
+    ArabicLetterTokenizer tokenStream  = new ArabicLetterTokenizer(Version.LUCENE_CURRENT, new StringReader(input));
     ArabicStemFilter filter = new ArabicStemFilter(tokenStream);
     assertTokenStreamContents(filter, new String[]{expected});
   }
-
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianAnalyzer.java Fri Feb 26 13:09:54 2010
@@ -22,6 +22,7 @@
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.util.Version;
 
 /**
@@ -67,4 +68,11 @@
     
     assertAnalyzesTo(a, "градове", new String[] {"град"});
   }
+  
+  public void testWithStemExclusionSet() throws IOException {
+    CharArraySet set = new CharArraySet(Version.LUCENE_31, 1, true);
+    set.add("строеве");
+    Analyzer a = new BulgarianAnalyzer(Version.LUCENE_CURRENT, CharArraySet.EMPTY_SET, set);
+    assertAnalyzesTo(a, "строевете строеве", new String[] { "строй", "строеве" });
+  }
 }

Modified: lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java?rev=916666&r1=916665&r2=916666&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java (original)
+++ lucene/java/branches/flex_1458/contrib/analyzers/common/src/test/org/apache/lucene/analysis/bg/TestBulgarianStemmer.java Fri Feb 26 13:09:54 2010
@@ -18,8 +18,12 @@
  */
 
 import java.io.IOException;
+import java.io.StringReader;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.KeywordMarkerTokenFilter;
+import org.apache.lucene.analysis.WhitespaceTokenizer;
 import org.apache.lucene.util.Version;
 
 /**
@@ -207,4 +211,15 @@
     assertAnalyzesTo(a, "строя", new String[] {"стр"});
     assertAnalyzesTo(a, "строят", new String[] {"стр"});
   }
+
+  public void testWithKeywordAttribute() throws IOException {
+    CharArraySet set = new CharArraySet(Version.LUCENE_31, 1, true);
+    set.add("строеве");
+    WhitespaceTokenizer tokenStream = new WhitespaceTokenizer(Version.LUCENE_CURRENT, 
+        new StringReader("строевете строеве"));
+
+    BulgarianStemFilter filter = new BulgarianStemFilter(
+        new KeywordMarkerTokenFilter(tokenStream, set));
+    assertTokenStreamContents(filter, new String[] { "строй", "строеве" });
+  }
 }