You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2014/08/09 00:42:54 UTC

svn commit: r1616901 [2/11] - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/ lucene/analysis/common/src/java/org/apache/lucene/analysis/br/ luc...

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43CompoundWordTokenFilterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43CompoundWordTokenFilterBase.java?rev=1616901&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43CompoundWordTokenFilterBase.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43CompoundWordTokenFilterBase.java Fri Aug  8 22:42:48 2014
@@ -0,0 +1,162 @@
+package org.apache.lucene.analysis.compound;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * Base class for decomposition token filters using pre-4.4 behavior.
+ * <p>
+ * @deprecated Use {@link CompoundWordTokenFilterBase}
+ */
+@Deprecated
+public abstract class Lucene43CompoundWordTokenFilterBase extends TokenFilter {
+  /**
+   * The default for minimal word length that gets decomposed
+   */
+  public static final int DEFAULT_MIN_WORD_SIZE = 5;
+
+  /**
+   * The default for minimal length of subwords that get propagated to the output of this filter
+   */
+  public static final int DEFAULT_MIN_SUBWORD_SIZE = 2;
+
+  /**
+   * The default for maximal length of subwords that get propagated to the output of this filter
+   */
+  public static final int DEFAULT_MAX_SUBWORD_SIZE = 15;
+
+  protected final CharArraySet dictionary;
+  protected final LinkedList<CompoundToken> tokens;
+  protected final int minWordSize;
+  protected final int minSubwordSize;
+  protected final int maxSubwordSize;
+  protected final boolean onlyLongestMatch;
+  
+  protected final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  protected final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  
+  private AttributeSource.State current;
+
+  protected Lucene43CompoundWordTokenFilterBase(TokenStream input, CharArraySet dictionary, boolean onlyLongestMatch) {
+    this(input,dictionary,DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, onlyLongestMatch);
+  }
+
+  protected Lucene43CompoundWordTokenFilterBase(TokenStream input, CharArraySet dictionary) {
+    this(input,dictionary,DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  protected Lucene43CompoundWordTokenFilterBase(TokenStream input, CharArraySet dictionary, int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input);
+    this.tokens=new LinkedList<>();
+    if (minWordSize < 0) {
+      throw new IllegalArgumentException("minWordSize cannot be negative");
+    }
+    this.minWordSize=minWordSize;
+    if (minSubwordSize < 0) {
+      throw new IllegalArgumentException("minSubwordSize cannot be negative");
+    }
+    this.minSubwordSize=minSubwordSize;
+    if (maxSubwordSize < 0) {
+      throw new IllegalArgumentException("maxSubwordSize cannot be negative");
+    }
+    this.maxSubwordSize=maxSubwordSize;
+    this.onlyLongestMatch=onlyLongestMatch;
+    this.dictionary = dictionary;
+  }
+  
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (!tokens.isEmpty()) {
+      assert current != null;
+      CompoundToken token = tokens.removeFirst();
+      restoreState(current); // keep all other attributes untouched
+      termAtt.setEmpty().append(token.txt);
+      offsetAtt.setOffset(token.startOffset, token.endOffset);
+      posIncAtt.setPositionIncrement(0);
+      return true;
+    }
+
+    current = null; // not really needed, but for safety
+    if (input.incrementToken()) {
+      // Only words longer than minWordSize get processed
+      if (termAtt.length() >= this.minWordSize) {
+        decompose();
+        // only capture the state if we really need it for producing new tokens
+        if (!tokens.isEmpty()) {
+          current = captureState();
+        }
+      }
+      // return original token:
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /** Decomposes the current {@link #termAtt} and places {@link CompoundToken} instances in the {@link #tokens} list.
+   * The original token may not be placed in the list, as it is automatically passed through this filter.
+   */
+  protected abstract void decompose();
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    tokens.clear();
+    current = null;
+  }
+  
+  /**
+   * Helper class to hold decompounded token information
+   */
+  protected class CompoundToken {
+    public final CharSequence txt;
+    public final int startOffset, endOffset;
+
+    /** Construct the compound token based on a slice of the current {@link Lucene43CompoundWordTokenFilterBase#termAtt}. */
+    public CompoundToken(int offset, int length) {
+      this.txt = Lucene43CompoundWordTokenFilterBase.this.termAtt.subSequence(offset, offset + length);
+      
+      // offsets of the original word
+      int startOff = Lucene43CompoundWordTokenFilterBase.this.offsetAtt.startOffset();
+      int endOff = Lucene43CompoundWordTokenFilterBase.this.offsetAtt.endOffset();
+      
+      if (endOff - startOff != Lucene43CompoundWordTokenFilterBase.this.termAtt.length()) {
+        // if length by start + end offsets doesn't match the term text then assume
+        // this is a synonym and don't adjust the offsets.
+        this.startOffset = startOff;
+        this.endOffset = endOff;
+      } else {
+        final int newStart = startOff + offset;
+        this.startOffset = newStart;
+        this.endOffset = newStart + length;
+      }
+    }
+
+  }  
+}

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43DictionaryCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43DictionaryCompoundWordTokenFilter.java?rev=1616901&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43DictionaryCompoundWordTokenFilter.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43DictionaryCompoundWordTokenFilter.java Fri Aug  8 22:42:48 2014
@@ -0,0 +1,100 @@
+package org.apache.lucene.analysis.compound;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.CharArraySet;
+
+/**
+ * A {@link TokenFilter} that decomposes compound words found in many Germanic languages, using
+ * pre-4.4 behavior.
+ * @deprecated Use {@link DictionaryCompoundWordTokenFilter}.
+ */
+@Deprecated
+public class Lucene43DictionaryCompoundWordTokenFilter extends Lucene43CompoundWordTokenFilterBase {
+  
+  /**
+   * Creates a new {@link Lucene43DictionaryCompoundWordTokenFilter}
+   *
+   * @param input
+   *          the {@link TokenStream} to process
+   * @param dictionary
+   *          the word dictionary to match against.
+   */
+  public Lucene43DictionaryCompoundWordTokenFilter(TokenStream input, CharArraySet dictionary) {
+    super(input, dictionary);
+    if (dictionary == null) {
+      throw new IllegalArgumentException("dictionary cannot be null");
+    }
+  }
+  
+  /**
+   * Creates a new {@link Lucene43DictionaryCompoundWordTokenFilter}
+   *
+   * @param input
+   *          the {@link TokenStream} to process
+   * @param dictionary
+   *          the word dictionary to match against.
+   * @param minWordSize
+   *          only words longer than this get processed
+   * @param minSubwordSize
+   *          only subwords longer than this get to the output stream
+   * @param maxSubwordSize
+   *          only subwords shorter than this get to the output stream
+   * @param onlyLongestMatch
+   *          Add only the longest matching subword to the stream
+   */
+  public Lucene43DictionaryCompoundWordTokenFilter(TokenStream input, CharArraySet dictionary,
+                                                   int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input, dictionary, minWordSize, minSubwordSize, maxSubwordSize, onlyLongestMatch);
+    if (dictionary == null) {
+      throw new IllegalArgumentException("dictionary cannot be null");
+    }
+  }
+
+  @Override
+  protected void decompose() {
+    final int len = termAtt.length();
+    for (int i=0;i<=len-this.minSubwordSize;++i) {
+        CompoundToken longestMatchToken=null;
+        for (int j=this.minSubwordSize;j<=this.maxSubwordSize;++j) {
+            if(i+j>len) {
+                break;
+            }
+            if(dictionary.contains(termAtt.buffer(), i, j)) {
+                if (this.onlyLongestMatch) {
+                   if (longestMatchToken!=null) {
+                     if (longestMatchToken.txt.length()<j) {
+                       longestMatchToken=new CompoundToken(i,j);
+                     }
+                   } else {
+                     longestMatchToken=new CompoundToken(i,j);
+                   }
+                } else {
+                   tokens.add(new CompoundToken(i,j));
+                }
+            } 
+        }
+        if (this.onlyLongestMatch && longestMatchToken!=null) {
+          tokens.add(longestMatchToken);
+        }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43HyphenationCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43HyphenationCompoundWordTokenFilter.java?rev=1616901&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43HyphenationCompoundWordTokenFilter.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/Lucene43HyphenationCompoundWordTokenFilter.java Fri Aug  8 22:42:48 2014
@@ -0,0 +1,217 @@
+package org.apache.lucene.analysis.compound;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.compound.hyphenation.Hyphenation;
+import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.xml.sax.InputSource;
+
+/**
+ * A {@link TokenFilter} that decomposes compound words found in many Germanic languages,
+ * using pre-4.4 behavior.
+ *
+ * @deprecated Use {@link HyphenationCompoundWordTokenFilter}.
+ */
+@Deprecated
+public class Lucene43HyphenationCompoundWordTokenFilter extends
+    Lucene43CompoundWordTokenFilterBase {
+  private HyphenationTree hyphenator;
+
+  /**
+   * Creates a new {@link Lucene43HyphenationCompoundWordTokenFilter} instance.
+   *
+   * @param input
+   *          the {@link TokenStream} to process
+   * @param hyphenator
+   *          the hyphenation pattern tree to use for hyphenation
+   * @param dictionary
+   *          the word dictionary to match against.
+   */
+  public Lucene43HyphenationCompoundWordTokenFilter(TokenStream input,
+                                                    HyphenationTree hyphenator, CharArraySet dictionary) {
+    this(input, hyphenator, dictionary, DEFAULT_MIN_WORD_SIZE,
+        DEFAULT_MIN_SUBWORD_SIZE, DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  /**
+   * Creates a new {@link Lucene43HyphenationCompoundWordTokenFilter} instance.
+   *
+   * @param input
+   *          the {@link TokenStream} to process
+   * @param hyphenator
+   *          the hyphenation pattern tree to use for hyphenation
+   * @param dictionary
+   *          the word dictionary to match against.
+   * @param minWordSize
+   *          only words longer than this get processed
+   * @param minSubwordSize
+   *          only subwords longer than this get to the output stream
+   * @param maxSubwordSize
+   *          only subwords shorter than this get to the output stream
+   * @param onlyLongestMatch
+   *          Add only the longest matching subword to the stream
+   */
+  public Lucene43HyphenationCompoundWordTokenFilter(TokenStream input,
+                                                    HyphenationTree hyphenator, CharArraySet dictionary, int minWordSize,
+                                                    int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input, dictionary, minWordSize, minSubwordSize, maxSubwordSize,
+        onlyLongestMatch);
+
+    this.hyphenator = hyphenator;
+  }
+
+  /**
+   * Create a HyphenationCompoundWordTokenFilter with no dictionary.
+   * <p>
+   * Calls {@link #Lucene43HyphenationCompoundWordTokenFilter(TokenStream, HyphenationTree, CharArraySet, int, int, int, boolean)
+   * HyphenationCompoundWordTokenFilter(matchVersion, input, hyphenator,
+   * null, minWordSize, minSubwordSize, maxSubwordSize }
+   */
+  public Lucene43HyphenationCompoundWordTokenFilter(TokenStream input,
+                                                    HyphenationTree hyphenator, int minWordSize, int minSubwordSize,
+                                                    int maxSubwordSize) {
+    this(input, hyphenator, null, minWordSize, minSubwordSize,
+        maxSubwordSize, false);
+  }
+  
+  /**
+   * Create a HyphenationCompoundWordTokenFilter with no dictionary.
+   * <p>
+   * Calls {@link #Lucene43HyphenationCompoundWordTokenFilter(TokenStream, HyphenationTree, int, int, int)
+   * HyphenationCompoundWordTokenFilter(matchVersion, input, hyphenator, 
+   * DEFAULT_MIN_WORD_SIZE, DEFAULT_MIN_SUBWORD_SIZE, DEFAULT_MAX_SUBWORD_SIZE }
+   */
+  public Lucene43HyphenationCompoundWordTokenFilter(TokenStream input,
+                                                    HyphenationTree hyphenator) {
+    this(input, hyphenator, DEFAULT_MIN_WORD_SIZE, DEFAULT_MIN_SUBWORD_SIZE,
+        DEFAULT_MAX_SUBWORD_SIZE);
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationFilename the filename of the XML grammar to load
+   * @return An object representing the hyphenation patterns
+   * @throws IOException If there is a low-level I/O error.
+   */
+  public static HyphenationTree getHyphenationTree(String hyphenationFilename)
+      throws IOException {
+    return getHyphenationTree(new InputSource(hyphenationFilename));
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationFile the file of the XML grammar to load
+   * @return An object representing the hyphenation patterns
+   * @throws IOException If there is a low-level I/O error.
+   */
+  public static HyphenationTree getHyphenationTree(File hyphenationFile)
+      throws IOException {
+    return getHyphenationTree(new InputSource(hyphenationFile.toURI().toASCIIString()));
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationSource the InputSource pointing to the XML grammar
+   * @return An object representing the hyphenation patterns
+   * @throws IOException If there is a low-level I/O error.
+   */
+  public static HyphenationTree getHyphenationTree(InputSource hyphenationSource)
+      throws IOException {
+    HyphenationTree tree = new HyphenationTree();
+    tree.loadPatterns(hyphenationSource);
+    return tree;
+  }
+
+  @Override
+  protected void decompose() {
+    // get the hyphenation points
+    Hyphenation hyphens = hyphenator.hyphenate(termAtt.buffer(), 0, termAtt.length(), 1, 1);
+    // No hyphen points found -> exit
+    if (hyphens == null) {
+      return;
+    }
+
+    final int[] hyp = hyphens.getHyphenationPoints();
+
+    for (int i = 0; i < hyp.length; ++i) {
+      int remaining = hyp.length - i;
+      int start = hyp[i];
+      CompoundToken longestMatchToken = null;
+      for (int j = 1; j < remaining; j++) {
+        int partLength = hyp[i + j] - start;
+
+        // if the part is longer than maxSubwordSize we
+        // are done with this round
+        if (partLength > this.maxSubwordSize) {
+          break;
+        }
+
+        // we only put subwords to the token stream
+        // that are longer than minPartSize
+        if (partLength < this.minSubwordSize) {
+          // BOGUS/BROKEN/FUNKY/WACKO: somehow we have negative 'parts' according to the 
+          // calculation above, and we rely upon minSubwordSize being >=0 to filter them out...
+          continue;
+        }
+
+        // check the dictionary
+        if (dictionary == null || dictionary.contains(termAtt.buffer(), start, partLength)) {
+          if (this.onlyLongestMatch) {
+            if (longestMatchToken != null) {
+              if (longestMatchToken.txt.length() < partLength) {
+                longestMatchToken = new CompoundToken(start, partLength);
+              }
+            } else {
+              longestMatchToken = new CompoundToken(start, partLength);
+            }
+          } else {
+            tokens.add(new CompoundToken(start, partLength));
+          }
+        } else if (dictionary.contains(termAtt.buffer(), start, partLength - 1)) {
+          // check the dictionary again with a word that is one character
+          // shorter
+          // to avoid problems with genitive 's characters and other binding
+          // characters
+          if (this.onlyLongestMatch) {
+            if (longestMatchToken != null) {
+              if (longestMatchToken.txt.length() < partLength - 1) {
+                longestMatchToken = new CompoundToken(start, partLength - 1);
+              }
+            } else {
+              longestMatchToken = new CompoundToken(start, partLength - 1);
+            }
+          } else {
+            tokens.add(new CompoundToken(start, partLength - 1));
+          }
+        }
+      }
+      if (this.onlyLongestMatch && longestMatchToken!=null) {
+        tokens.add(longestMatchToken);
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -17,8 +17,6 @@ package org.apache.lucene.analysis.core;
  * limitations under the License.
  */
 
-import java.io.Reader;
-
 import org.apache.lucene.analysis.Analyzer;
 
 /**

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java Fri Aug  8 22:42:48 2014
@@ -18,13 +18,11 @@ package org.apache.lucene.analysis.core;
  */
 
 import java.io.IOException;
-import java.io.Reader;
 
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.util.AttributeFactory;
-import org.apache.lucene.util.AttributeSource;
 
 /**
  * Emits the entire input as a single token.

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java Fri Aug  8 22:42:48 2014
@@ -20,7 +20,6 @@ package org.apache.lucene.analysis.core;
 import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.util.AttributeFactory;
 
-import java.io.Reader;
 import java.util.Map;
 
 /**

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizer.java Fri Aug  8 22:42:48 2014
@@ -20,7 +20,6 @@ package org.apache.lucene.analysis.core;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.util.CharTokenizer;
 import org.apache.lucene.util.AttributeFactory;
-import org.apache.lucene.util.Version;
 
 /**
  * A LetterTokenizer is a tokenizer that divides text at non-letters. That's to
@@ -30,41 +29,25 @@ import org.apache.lucene.util.Version;
  * Note: this does a decent job for most European languages, but does a terrible
  * job for some Asian languages, where words are not separated by spaces.
  * </p>
- * <p>
- * <a name="version"/>
- * You must specify the required {@link Version} compatibility when creating
- * {@link LetterTokenizer}:
- * <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>
- * </p>
  */
 
 public class LetterTokenizer extends CharTokenizer {
   
   /**
    * Construct a new LetterTokenizer.
-   * 
-   * @param matchVersion
-   *          Lucene version to match See {@link <a href="#version">above</a>}
    */
-  public LetterTokenizer(Version matchVersion) {
-    super(matchVersion);
+  public LetterTokenizer() {
   }
   
   /**
    * Construct a new LetterTokenizer using a given
    * {@link org.apache.lucene.util.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}
    */
-  public LetterTokenizer(Version matchVersion, AttributeFactory factory) {
-    super(matchVersion, factory);
+  public LetterTokenizer(AttributeFactory factory) {
+    super(factory);
   }
   
   /** Collects only characters which satisfy

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java Fri Aug  8 22:42:48 2014
@@ -36,7 +36,6 @@ public class LetterTokenizerFactory exte
   /** Creates a new LetterTokenizerFactory */
   public LetterTokenizerFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -44,6 +43,6 @@ public class LetterTokenizerFactory exte
 
   @Override
   public LetterTokenizer create(AttributeFactory factory) {
-    return new LetterTokenizer(luceneMatchVersion, factory);
+    return new LetterTokenizer(factory);
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilter.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilter.java Fri Aug  8 22:42:48 2014
@@ -23,30 +23,21 @@ import org.apache.lucene.analysis.TokenF
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.CharacterUtils;
-import org.apache.lucene.util.Version;
 
 /**
  * Normalizes token text to lower case.
- * <a name="version"/>
- * <p>You must specify the required {@link Version}
- * compatibility when creating LowerCaseFilter:
- * <ul>
- *   <li> As of 3.1, supplementary characters are properly lowercased.
- * </ul>
  */
 public final class LowerCaseFilter extends TokenFilter {
-  private final CharacterUtils charUtils;
+  private final CharacterUtils charUtils = CharacterUtils.getInstance();
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   
   /**
    * Create a new LowerCaseFilter, that normalizes token text to lower case.
    * 
-   * @param matchVersion See <a href="#version">above</a>
    * @param in TokenStream to filter
    */
-  public LowerCaseFilter(Version matchVersion, TokenStream in) {
+  public LowerCaseFilter(TokenStream in) {
     super(in);
-    charUtils = CharacterUtils.getInstance(matchVersion);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java Fri Aug  8 22:42:48 2014
@@ -40,7 +40,6 @@ public class LowerCaseFilterFactory exte
   /** Creates a new LowerCaseFilterFactory */
   public LowerCaseFilterFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -48,7 +47,7 @@ public class LowerCaseFilterFactory exte
 
   @Override
   public LowerCaseFilter create(TokenStream input) {
-    return new LowerCaseFilter(luceneMatchVersion,input);
+    return new LowerCaseFilter(input);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizer.java Fri Aug  8 22:42:48 2014
@@ -17,13 +17,8 @@ package org.apache.lucene.analysis.core;
  * limitations under the License.
  */
 
-import java.io.Reader;
-
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.util.CharTokenizer;
 import org.apache.lucene.util.AttributeFactory;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.Version;
 
 /**
  * LowerCaseTokenizer performs the function of LetterTokenizer
@@ -35,41 +30,24 @@ import org.apache.lucene.util.Version;
  * Note: this does a decent job for most European languages, but does a terrible
  * job for some Asian languages, where words are not separated by spaces.
  * </p>
- * <p>
- * <a name="version"/>
- * You must specify the required {@link Version} compatibility when creating
- * {@link LowerCaseTokenizer}:
- * <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>
- * </p>
  */
 public final class LowerCaseTokenizer extends LetterTokenizer {
   
   /**
    * Construct a new LowerCaseTokenizer.
-   * 
-   * @param matchVersion
-   *          Lucene version to match See {@link <a href="#version">above</a>}
-   * 
    */
-  public LowerCaseTokenizer(Version matchVersion) {
-    super(matchVersion);
+  public LowerCaseTokenizer() {
   }
 
   /**
    * Construct a new LowerCaseTokenizer using a given
    * {@link org.apache.lucene.util.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}
    */
-  public LowerCaseTokenizer(Version matchVersion, AttributeFactory factory) {
-    super(matchVersion, factory);
+  public LowerCaseTokenizer(AttributeFactory factory) {
+    super(factory);
   }
   
   /** Converts char to lower case

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizerFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizerFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseTokenizerFactory.java Fri Aug  8 22:42:48 2014
@@ -39,7 +39,6 @@ public class LowerCaseTokenizerFactory e
   /** Creates a new LowerCaseTokenizerFactory */
   public LowerCaseTokenizerFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -47,7 +46,7 @@ public class LowerCaseTokenizerFactory e
 
   @Override
   public LowerCaseTokenizer create(AttributeFactory factory) {
-    return new LowerCaseTokenizer(luceneMatchVersion, factory);
+    return new LowerCaseTokenizer(factory);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -17,38 +17,21 @@ package org.apache.lucene.analysis.core;
  * limitations under the License.
  */
 
-import java.io.Reader;
-
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.util.CharTokenizer;
-import org.apache.lucene.util.Version;
 
 /** An {@link Analyzer} that filters {@link LetterTokenizer} 
  *  with {@link LowerCaseFilter} 
- * <p>
- * <a name="version">You must specify the required {@link Version} compatibility
- * when creating {@link CharTokenizer}:
- * <ul>
- * <li>As of 3.1, {@link LowerCaseTokenizer} uses an int based API to normalize and
- * detect token codepoints. See {@link CharTokenizer#isTokenChar(int)} and
- * {@link CharTokenizer#normalize(int)} for details.</li>
- * </ul>
- * <p>
  **/
 public final class SimpleAnalyzer extends Analyzer {
 
-  private final Version matchVersion;
-  
   /**
    * Creates a new {@link SimpleAnalyzer}
-   * @param matchVersion Lucene version to match See {@link <a href="#version">above</a>}
    */
-  public SimpleAnalyzer(Version matchVersion) {
-    this.matchVersion = matchVersion;
+  public SimpleAnalyzer() {
   }
   
   @Override
   protected TokenStreamComponents createComponents(final String fieldName) {
-    return new TokenStreamComponents(new LowerCaseTokenizer(matchVersion));
+    return new TokenStreamComponents(new LowerCaseTokenizer());
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -27,20 +27,10 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.util.WordlistLoader;
-import org.apache.lucene.util.Version;
-
-/** Filters {@link LetterTokenizer} with {@link LowerCaseFilter} and {@link StopFilter}.
- *
- * <a name="version"/>
- * <p>You must specify the required {@link Version}
- * compatibility when creating StopAnalyzer:
- * <ul>
- *    <li> As of 3.1, StopFilter correctly handles Unicode 4.0
- *         supplementary characters in stopwords
- *   <li> As of 2.9, position increments are preserved
- * </ul>
-*/
 
+/** 
+ * Filters {@link LetterTokenizer} with {@link LowerCaseFilter} and {@link StopFilter}.
+ */
 public final class StopAnalyzer extends StopwordAnalyzerBase {
   
   /** An unmodifiable set containing some common English words that are not usually useful
@@ -55,40 +45,35 @@ public final class StopAnalyzer extends 
       "that", "the", "their", "then", "there", "these",
       "they", "this", "to", "was", "will", "with"
     );
-    final CharArraySet stopSet = new CharArraySet(Version.LUCENE_CURRENT, 
-        stopWords, false);
+    final CharArraySet stopSet = new CharArraySet(stopWords, false);
     ENGLISH_STOP_WORDS_SET = CharArraySet.unmodifiableSet(stopSet); 
   }
   
   /** Builds an analyzer which removes words in
    *  {@link #ENGLISH_STOP_WORDS_SET}.
-   * @param matchVersion See <a href="#version">above</a>
    */
-  public StopAnalyzer(Version matchVersion) {
-    this(matchVersion, ENGLISH_STOP_WORDS_SET);
+  public StopAnalyzer() {
+    this(ENGLISH_STOP_WORDS_SET);
   }
 
   /** Builds an analyzer with the stop words from the given set.
-   * @param matchVersion See <a href="#version">above</a>
    * @param stopWords Set of stop words */
-  public StopAnalyzer(Version matchVersion, CharArraySet stopWords) {
-    super(matchVersion, stopWords);
+  public StopAnalyzer(CharArraySet stopWords) {
+    super(stopWords);
   }
 
   /** Builds an analyzer with the stop words from the given file.
-   * @see WordlistLoader#getWordSet(Reader, Version)
-   * @param matchVersion See <a href="#version">above</a>
+   * @see WordlistLoader#getWordSet(Reader)
    * @param stopwordsFile File to load stop words from */
-  public StopAnalyzer(Version matchVersion, File stopwordsFile) throws IOException {
-    this(matchVersion, loadStopwordSet(stopwordsFile, matchVersion));
+  public StopAnalyzer(File stopwordsFile) throws IOException {
+    this(loadStopwordSet(stopwordsFile));
   }
 
   /** Builds an analyzer with the stop words from the given reader.
-   * @see WordlistLoader#getWordSet(Reader, Version)
-   * @param matchVersion See <a href="#version">above</a>
+   * @see WordlistLoader#getWordSet(Reader)
    * @param stopwords Reader to load stop words from */
-  public StopAnalyzer(Version matchVersion, Reader stopwords) throws IOException {
-    this(matchVersion, loadStopwordSet(stopwords, matchVersion));
+  public StopAnalyzer(Reader stopwords) throws IOException {
+    this(loadStopwordSet(stopwords));
   }
 
   /**
@@ -102,9 +87,8 @@ public final class StopAnalyzer extends 
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new LowerCaseTokenizer(matchVersion);
-    return new TokenStreamComponents(source, new StopFilter(matchVersion,
-          source, stopwords));
+    final Tokenizer source = new LowerCaseTokenizer();
+    return new TokenStreamComponents(source, new StopFilter(source, stopwords));
   }
 }
 

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilter.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilter.java Fri Aug  8 22:42:48 2014
@@ -24,19 +24,9 @@ import org.apache.lucene.analysis.util.F
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.CharArraySet;
-import org.apache.lucene.util.Version;
 
 /**
  * Removes stop words from a token stream.
- * 
- * <a name="version"/>
- * <p>You must specify the required {@link Version}
- * compatibility when creating StopFilter:
- * <ul>
- *   <li> As of 3.1, StopFilter correctly handles Unicode 4.0
- *         supplementary characters in stopwords and position
- *         increments are preserved
- * </ul>
  */
 public final class StopFilter extends FilteringTokenFilter {
 
@@ -47,17 +37,14 @@ public final class StopFilter extends Fi
    * Constructs a filter which removes words from the input TokenStream that are
    * named in the Set.
    * 
-   * @param matchVersion
-   *          Lucene version to enable correct Unicode 4.0 behavior in the stop
-   *          set if Version > 3.0.  See <a href="#version">above</a> for details.
    * @param in
    *          Input stream
    * @param stopWords
    *          A {@link CharArraySet} representing the stopwords.
-   * @see #makeStopSet(Version, java.lang.String...)
+   * @see #makeStopSet(java.lang.String...)
    */
-  public StopFilter(Version matchVersion, TokenStream in, CharArraySet stopWords) {
-    super(matchVersion, in);
+  public StopFilter(TokenStream in, CharArraySet stopWords) {
+    super(in);
     this.stopWords = stopWords;
   }
 
@@ -67,12 +54,11 @@ public final class StopFilter extends Fi
    * This permits this stopWords construction to be cached once when
    * an Analyzer is constructed.
    * 
-   * @param matchVersion Lucene version to enable correct Unicode 4.0 behavior in the returned set if Version > 3.0
    * @param stopWords An array of stopwords
-   * @see #makeStopSet(Version, java.lang.String[], boolean) passing false to ignoreCase
+   * @see #makeStopSet(java.lang.String[], boolean) passing false to ignoreCase
    */
-  public static CharArraySet makeStopSet(Version matchVersion, String... stopWords) {
-    return makeStopSet(matchVersion, stopWords, false);
+  public static CharArraySet makeStopSet(String... stopWords) {
+    return makeStopSet(stopWords, false);
   }
   
   /**
@@ -81,38 +67,35 @@ public final class StopFilter extends Fi
    * This permits this stopWords construction to be cached once when
    * an Analyzer is constructed.
    * 
-   * @param matchVersion Lucene version to enable correct Unicode 4.0 behavior in the returned set if Version > 3.0
    * @param stopWords A List of Strings or char[] or any other toString()-able list representing the stopwords
    * @return A Set ({@link CharArraySet}) containing the words
-   * @see #makeStopSet(Version, java.lang.String[], boolean) passing false to ignoreCase
+   * @see #makeStopSet(java.lang.String[], boolean) passing false to ignoreCase
    */
-  public static CharArraySet makeStopSet(Version matchVersion, List<?> stopWords) {
-    return makeStopSet(matchVersion, stopWords, false);
+  public static CharArraySet makeStopSet(List<?> stopWords) {
+    return makeStopSet(stopWords, false);
   }
     
   /**
    * Creates a stopword set from the given stopword array.
    * 
-   * @param matchVersion Lucene version to enable correct Unicode 4.0 behavior in the returned set if Version > 3.0
    * @param stopWords An array of stopwords
    * @param ignoreCase If true, all words are lower cased first.  
    * @return a Set containing the words
    */    
-  public static CharArraySet makeStopSet(Version matchVersion, String[] stopWords, boolean ignoreCase) {
-    CharArraySet stopSet = new CharArraySet(matchVersion, stopWords.length, ignoreCase);
+  public static CharArraySet makeStopSet(String[] stopWords, boolean ignoreCase) {
+    CharArraySet stopSet = new CharArraySet(stopWords.length, ignoreCase);
     stopSet.addAll(Arrays.asList(stopWords));
     return stopSet;
   }
   
   /**
    * Creates a stopword set from the given stopword list.
-   * @param matchVersion Lucene version to enable correct Unicode 4.0 behavior in the returned set if Version > 3.0
    * @param stopWords A List of Strings or char[] or any other toString()-able list representing the stopwords
    * @param ignoreCase if true, all words are lower cased first
    * @return A Set ({@link CharArraySet}) containing the words
    */
-  public static CharArraySet makeStopSet(Version matchVersion, List<?> stopWords, boolean ignoreCase){
-    CharArraySet stopSet = new CharArraySet(matchVersion, stopWords.size(), ignoreCase);
+  public static CharArraySet makeStopSet(List<?> stopWords, boolean ignoreCase){
+    CharArraySet stopSet = new CharArraySet(stopWords.size(), ignoreCase);
     stopSet.addAll(stopWords);
     return stopSet;
   }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java Fri Aug  8 22:42:48 2014
@@ -81,7 +81,6 @@ public class StopFilterFactory extends T
   /** Creates a new StopFilterFactory */
   public StopFilterFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     stopWordFiles = get(args, "words");
     format = get(args, "format", (null == stopWordFiles ? null : FORMAT_WORDSET));
     ignoreCase = getBoolean(args, "ignoreCase", false);
@@ -104,7 +103,7 @@ public class StopFilterFactory extends T
       if (null != format) {
         throw new IllegalArgumentException("'format' can not be specified w/o an explicit 'words' file: " + format);
       }
-      stopWords = new CharArraySet(luceneMatchVersion, StopAnalyzer.ENGLISH_STOP_WORDS_SET, ignoreCase);
+      stopWords = new CharArraySet(StopAnalyzer.ENGLISH_STOP_WORDS_SET, ignoreCase);
     }
   }
 
@@ -118,7 +117,7 @@ public class StopFilterFactory extends T
 
   @Override
   public TokenStream create(TokenStream input) {
-    StopFilter stopFilter = new StopFilter(luceneMatchVersion,input,stopWords);
+    StopFilter stopFilter = new StopFilter(input,stopWords);
     return stopFilter;
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilter.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilter.java Fri Aug  8 22:42:48 2014
@@ -22,7 +22,6 @@ import java.util.Set;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.analysis.util.FilteringTokenFilter;
-import org.apache.lucene.util.Version;
 
 /**
  * Removes tokens whose types appear in a set of blocked types from a token stream.
@@ -35,14 +34,13 @@ public final class TypeTokenFilter exten
 
   /**
    * Create a new {@link TypeTokenFilter}.
-   * @param version      the Lucene match version
    * @param input        the {@link TokenStream} to consume
    * @param stopTypes    the types to filter
    * @param useWhiteList if true, then tokens whose type is in stopTypes will
    *                     be kept, otherwise they will be filtered out
    */
-  public TypeTokenFilter(Version version, TokenStream input, Set<String> stopTypes, boolean useWhiteList) {
-    super(version, input);
+  public TypeTokenFilter(TokenStream input, Set<String> stopTypes, boolean useWhiteList) {
+    super(input);
     this.stopTypes = stopTypes;
     this.useWhiteList = useWhiteList;
   }
@@ -50,10 +48,9 @@ public final class TypeTokenFilter exten
   /**
    * Create a new {@link TypeTokenFilter} that filters tokens out
    * (useWhiteList=false).
-   * @see #TypeTokenFilter(Version, TokenStream, Set, boolean)
    */
-  public TypeTokenFilter(Version version, TokenStream input, Set<String> stopTypes) {
-    this(version, input, stopTypes, false);
+  public TypeTokenFilter(TokenStream input, Set<String> stopTypes) {
+    this(input, stopTypes, false);
   }
 
   /**

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilterFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/TypeTokenFilterFactory.java Fri Aug  8 22:42:48 2014
@@ -72,7 +72,7 @@ public class TypeTokenFilterFactory exte
 
   @Override
   public TokenStream create(TokenStream input) {
-    final TokenStream filter = new TypeTokenFilter(luceneMatchVersion, input, stopTypes, useWhitelist);
+    final TokenStream filter = new TypeTokenFilter(input, stopTypes, useWhitelist);
     return filter;
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilter.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilter.java Fri Aug  8 22:42:48 2014
@@ -23,13 +23,9 @@ import org.apache.lucene.analysis.TokenF
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.CharacterUtils;
-import org.apache.lucene.util.Version;
 
 /**
  * Normalizes token text to UPPER CASE.
- * <a name="version"/>
- * <p>You must specify the required {@link Version}
- * compatibility when creating UpperCaseFilter
  * 
  * <p><b>NOTE:</b> In Unicode, this transformation may lose information when the
  * upper case character represents more than one lower case character. Use this filter
@@ -37,18 +33,16 @@ import org.apache.lucene.util.Version;
  * general search matching
  */
 public final class UpperCaseFilter extends TokenFilter {
-  private final CharacterUtils charUtils;
+  private final CharacterUtils charUtils = CharacterUtils.getInstance();
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   
   /**
    * Create a new UpperCaseFilter, that normalizes token text to upper case.
    * 
-   * @param matchVersion See <a href="#version">above</a>
    * @param in TokenStream to filter
    */
-  public UpperCaseFilter(Version matchVersion, TokenStream in) {
+  public UpperCaseFilter(TokenStream in) {
     super(in);
-    charUtils = CharacterUtils.getInstance(matchVersion);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilterFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UpperCaseFilterFactory.java Fri Aug  8 22:42:48 2014
@@ -45,7 +45,6 @@ public class UpperCaseFilterFactory exte
   /** Creates a new UpperCaseFilterFactory */
   public UpperCaseFilterFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -53,7 +52,7 @@ public class UpperCaseFilterFactory exte
 
   @Override
   public UpperCaseFilter create(TokenStream input) {
-    return new UpperCaseFilter(luceneMatchVersion,input);
+    return new UpperCaseFilter(input);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -17,38 +17,21 @@ package org.apache.lucene.analysis.core;
  * limitations under the License.
  */
 
-import java.io.Reader;
-
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.util.CharTokenizer;
-import org.apache.lucene.util.Version;
 
 /**
  * An Analyzer that uses {@link WhitespaceTokenizer}.
- * <p>
- * <a name="version">You must specify the required {@link Version} compatibility
- * when creating {@link CharTokenizer}:
- * <ul>
- * <li>As of 3.1, {@link WhitespaceTokenizer} uses an int based API to normalize and
- * detect token codepoints. See {@link CharTokenizer#isTokenChar(int)} and
- * {@link CharTokenizer#normalize(int)} for details.</li>
- * </ul>
- * <p>
  **/
 public final class WhitespaceAnalyzer extends Analyzer {
   
-  private final Version matchVersion;
-  
   /**
    * Creates a new {@link WhitespaceAnalyzer}
-   * @param matchVersion Lucene version to match See {@link <a href="#version">above</a>}
    */
-  public WhitespaceAnalyzer(Version matchVersion) {
-    this.matchVersion = matchVersion;
+  public WhitespaceAnalyzer() {
   }
   
   @Override
   protected TokenStreamComponents createComponents(final String fieldName) {
-    return new TokenStreamComponents(new WhitespaceTokenizer(matchVersion));
+    return new TokenStreamComponents(new WhitespaceTokenizer());
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizer.java Fri Aug  8 22:42:48 2014
@@ -17,50 +17,31 @@ package org.apache.lucene.analysis.core;
  * limitations under the License.
  */
 
-import java.io.Reader;
-
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.util.CharTokenizer;
 import org.apache.lucene.util.AttributeFactory;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.Version;
 
 /**
  * A WhitespaceTokenizer is a tokenizer that divides text at whitespace.
- * Adjacent sequences of non-Whitespace characters form tokens. <a
- * name="version"/>
- * <p>
- * You must specify the required {@link Version} compatibility when creating
- * {@link WhitespaceTokenizer}:
- * <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>
+ * Adjacent sequences of non-Whitespace characters form tokens.
  */
 public final class WhitespaceTokenizer extends CharTokenizer {
   
   /**
-   * Construct a new WhitespaceTokenizer. * @param matchVersion Lucene version
-   * to match See {@link <a href="#version">above</a>}
-   *
+   * Construct a new WhitespaceTokenizer.
    */
-  public WhitespaceTokenizer(Version matchVersion) {
-    super(matchVersion);
+  public WhitespaceTokenizer() {
   }
 
   /**
    * Construct a new WhitespaceTokenizer using a given
    * {@link org.apache.lucene.util.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}
    */
-  public WhitespaceTokenizer(Version matchVersion, AttributeFactory factory) {
-    super(matchVersion, factory);
+  public WhitespaceTokenizer(AttributeFactory factory) {
+    super(factory);
   }
   
   /** Collects only characters which do not satisfy

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java Fri Aug  8 22:42:48 2014
@@ -20,7 +20,6 @@ package org.apache.lucene.analysis.core;
 import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.util.AttributeFactory;
 
-import java.io.Reader;
 import java.util.Map;
 
 /**
@@ -37,7 +36,6 @@ public class WhitespaceTokenizerFactory 
   /** Creates a new WhitespaceTokenizerFactory */
   public WhitespaceTokenizerFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -45,6 +43,6 @@ public class WhitespaceTokenizerFactory 
 
   @Override
   public WhitespaceTokenizer create(AttributeFactory factory) {
-    return new WhitespaceTokenizer(luceneMatchVersion, factory);
+    return new WhitespaceTokenizer(factory);
   }
 }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -29,7 +29,6 @@ import org.apache.lucene.analysis.util.C
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.Version;
 
 import java.io.*;
 import java.nio.charset.StandardCharsets;
@@ -61,7 +60,7 @@ public final class CzechAnalyzer extends
     static {
       try {
         DEFAULT_SET = WordlistLoader.getWordSet(IOUtils.getDecodingReader(CzechAnalyzer.class, 
-            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8), "#", Version.LUCENE_CURRENT);
+            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8), "#");
       } catch (IOException ex) {
         // default set should always be present as it is part of the
         // distribution (JAR)
@@ -75,34 +74,30 @@ public final class CzechAnalyzer extends
 
   /**
    * Builds an analyzer with the default stop words ({@link #getDefaultStopSet()}).
-   *
-   * @param matchVersion Lucene version to match
    */
-  public CzechAnalyzer(Version matchVersion) {
-    this(matchVersion, DefaultSetHolder.DEFAULT_SET);
+  public CzechAnalyzer() {
+    this(DefaultSetHolder.DEFAULT_SET);
   }
 
   /**
    * Builds an analyzer with the given stop words.
    *
-   * @param matchVersion Lucene version to match
    * @param stopwords a stopword set
    */
-  public CzechAnalyzer(Version matchVersion, CharArraySet stopwords) {
-    this(matchVersion, stopwords, CharArraySet.EMPTY_SET);
+  public CzechAnalyzer(CharArraySet stopwords) {
+    this(stopwords, CharArraySet.EMPTY_SET);
   }
 
   /**
    * Builds an analyzer with the given stop words and a set of work to be
    * excluded from the {@link CzechStemFilter}.
    * 
-   * @param matchVersion Lucene version to match
    * @param stopwords a stopword set
    * @param stemExclusionTable a stemming exclusion set
    */
-  public CzechAnalyzer(Version matchVersion, CharArraySet stopwords, CharArraySet stemExclusionTable) {
-    super(matchVersion, stopwords);
-    this.stemExclusionTable = CharArraySet.unmodifiableSet(CharArraySet.copy(matchVersion, stemExclusionTable));
+  public CzechAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionTable) {
+    super(stopwords);
+    this.stemExclusionTable = CharArraySet.unmodifiableSet(CharArraySet.copy(stemExclusionTable));
   }
 
   /**
@@ -115,16 +110,16 @@ public final class CzechAnalyzer extends
    *         {@link StandardFilter}, {@link LowerCaseFilter}, {@link StopFilter}
    *         , and {@link CzechStemFilter} (only if version is >= LUCENE_31). If
    *         a stem exclusion set is provided via
-   *         {@link #CzechAnalyzer(Version, CharArraySet, CharArraySet)} a
+   *         {@link #CzechAnalyzer(CharArraySet, CharArraySet)} a
    *         {@link SetKeywordMarkerFilter} is added before
    *         {@link CzechStemFilter}.
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new StandardTokenizer(matchVersion);
-    TokenStream result = new StandardFilter(matchVersion, source);
-    result = new LowerCaseFilter(matchVersion, result);
-    result = new StopFilter( matchVersion, result, stopwords);
+    final Tokenizer source = new StandardTokenizer();
+    TokenStream result = new StandardFilter(source);
+    result = new LowerCaseFilter(result);
+    result = new StopFilter(result, stopwords);
     if(!this.stemExclusionTable.isEmpty())
       result = new SetKeywordMarkerFilter(result, stemExclusionTable);
     result = new CzechStemFilter(result);

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -34,7 +34,6 @@ import org.apache.lucene.analysis.util.C
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.Version;
 import org.tartarus.snowball.ext.DanishStemmer;
 
 /**
@@ -64,7 +63,7 @@ public final class DanishAnalyzer extend
     static {
       try {
         DEFAULT_STOP_SET = WordlistLoader.getSnowballWordSet(IOUtils.getDecodingReader(SnowballFilter.class, 
-            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8), Version.LUCENE_CURRENT);
+            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8));
       } catch (IOException ex) {
         // default set should always be present as it is part of the
         // distribution (JAR)
@@ -76,18 +75,17 @@ public final class DanishAnalyzer extend
   /**
    * Builds an analyzer with the default stop words: {@link #DEFAULT_STOPWORD_FILE}.
    */
-  public DanishAnalyzer(Version matchVersion) {
-    this(matchVersion, DefaultSetHolder.DEFAULT_STOP_SET);
+  public DanishAnalyzer() {
+    this(DefaultSetHolder.DEFAULT_STOP_SET);
   }
   
   /**
    * Builds an analyzer with the given stop words.
    * 
-   * @param matchVersion lucene compatibility version
    * @param stopwords a stopword set
    */
-  public DanishAnalyzer(Version matchVersion, CharArraySet stopwords) {
-    this(matchVersion, stopwords, CharArraySet.EMPTY_SET);
+  public DanishAnalyzer(CharArraySet stopwords) {
+    this(stopwords, CharArraySet.EMPTY_SET);
   }
 
   /**
@@ -95,14 +93,12 @@ public final class DanishAnalyzer extend
    * provided this analyzer will add a {@link SetKeywordMarkerFilter} before
    * stemming.
    * 
-   * @param matchVersion lucene compatibility version
    * @param stopwords a stopword set
    * @param stemExclusionSet a set of terms not to be stemmed
    */
-  public DanishAnalyzer(Version matchVersion, CharArraySet stopwords, CharArraySet stemExclusionSet) {
-    super(matchVersion, stopwords);
-    this.stemExclusionSet = CharArraySet.unmodifiableSet(CharArraySet.copy(
-        matchVersion, stemExclusionSet));
+  public DanishAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionSet) {
+    super(stopwords);
+    this.stemExclusionSet = CharArraySet.unmodifiableSet(CharArraySet.copy(stemExclusionSet));
   }
 
   /**
@@ -119,10 +115,10 @@ public final class DanishAnalyzer extend
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new StandardTokenizer(matchVersion);
-    TokenStream result = new StandardFilter(matchVersion, source);
-    result = new LowerCaseFilter(matchVersion, result);
-    result = new StopFilter(matchVersion, result, stopwords);
+    final Tokenizer source = new StandardTokenizer();
+    TokenStream result = new StandardFilter(source);
+    result = new LowerCaseFilter(result);
+    result = new StopFilter(result, stopwords);
     if(!stemExclusionSet.isEmpty())
       result = new SetKeywordMarkerFilter(result, stemExclusionSet);
     result = new SnowballFilter(result, new DanishStemmer());

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -36,7 +36,6 @@ import org.apache.lucene.analysis.util.C
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.Version;
 
 /**
  * {@link Analyzer} for German language. 
@@ -48,7 +47,7 @@ import org.apache.lucene.util.Version;
  * exclusion list is empty by default.
  * </p>
  * 
- * <p><b>NOTE</b>: This class uses the same {@link Version}
+ * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
  */
 public final class GermanAnalyzer extends StopwordAnalyzerBase {
@@ -69,7 +68,7 @@ public final class GermanAnalyzer extend
     static {
       try {
         DEFAULT_SET = WordlistLoader.getSnowballWordSet(IOUtils.getDecodingReader(SnowballFilter.class, 
-            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8), Version.LUCENE_CURRENT);
+            DEFAULT_STOPWORD_FILE, StandardCharsets.UTF_8));
       } catch (IOException ex) {
         // default set should always be present as it is part of the
         // distribution (JAR)
@@ -91,35 +90,31 @@ public final class GermanAnalyzer extend
    * Builds an analyzer with the default stop words:
    * {@link #getDefaultStopSet()}.
    */
-  public GermanAnalyzer(Version matchVersion) {
-    this(matchVersion, DefaultSetHolder.DEFAULT_SET);
+  public GermanAnalyzer() {
+    this(DefaultSetHolder.DEFAULT_SET);
   }
   
   /**
    * Builds an analyzer with the given stop words 
    * 
-   * @param matchVersion
-   *          lucene compatibility version
    * @param stopwords
    *          a stopword set
    */
-  public GermanAnalyzer(Version matchVersion, CharArraySet stopwords) {
-    this(matchVersion, stopwords, CharArraySet.EMPTY_SET);
+  public GermanAnalyzer(CharArraySet stopwords) {
+    this(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 stemming exclusion set
    */
-  public GermanAnalyzer(Version matchVersion, CharArraySet stopwords, CharArraySet stemExclusionSet) {
-    super(matchVersion, stopwords);
-    exclusionSet = CharArraySet.unmodifiableSet(CharArraySet.copy(matchVersion, stemExclusionSet));
+  public GermanAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionSet) {
+    super(stopwords);
+    exclusionSet = CharArraySet.unmodifiableSet(CharArraySet.copy(stemExclusionSet));
   }
 
   /**
@@ -135,10 +130,10 @@ public final class GermanAnalyzer extend
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new StandardTokenizer(matchVersion);
-    TokenStream result = new StandardFilter(matchVersion, source);
-    result = new LowerCaseFilter(matchVersion, result);
-    result = new StopFilter( matchVersion, result, stopwords);
+    final Tokenizer source = new StandardTokenizer();
+    TokenStream result = new StandardFilter(source);
+    result = new LowerCaseFilter(result);
+    result = new StopFilter(result, stopwords);
     result = new SetKeywordMarkerFilter(result, exclusionSet);
     result = new GermanNormalizationFilter(result);
     result = new GermanLightStemFilter(result);

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java Fri Aug  8 22:42:48 2014
@@ -28,7 +28,6 @@ import org.apache.lucene.analysis.standa
 import org.apache.lucene.analysis.standard.StandardTokenizer;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
-import org.apache.lucene.util.Version;
 
 /**
  * {@link Analyzer} for the Greek language. 
@@ -38,7 +37,7 @@ import org.apache.lucene.util.Version;
  * A default set of stopwords is used unless an alternative list is specified.
  * </p>
  * 
- * <p><b>NOTE</b>: This class uses the same {@link Version}
+ * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
  */
 public final class GreekAnalyzer extends StopwordAnalyzerBase {
@@ -69,10 +68,9 @@ public final class GreekAnalyzer extends
   
   /**
    * Builds an analyzer with the default stop words.
-   * @param matchVersion Lucene compatibility version
    */
-  public GreekAnalyzer(Version matchVersion) {
-    this(matchVersion, DefaultSetHolder.DEFAULT_SET);
+  public GreekAnalyzer() {
+    this(DefaultSetHolder.DEFAULT_SET);
   }
   
   /**
@@ -81,11 +79,10 @@ public final class GreekAnalyzer extends
    * <b>NOTE:</b> The stopwords set should be pre-processed with the logic of 
    * {@link GreekLowerCaseFilter} for best results.
    *  
-   * @param matchVersion Lucene compatibility version
    * @param stopwords a stopword set
    */
-  public GreekAnalyzer(Version matchVersion, CharArraySet stopwords) {
-    super(matchVersion, stopwords);
+  public GreekAnalyzer(CharArraySet stopwords) {
+    super(stopwords);
   }
   
   /**
@@ -100,10 +97,10 @@ public final class GreekAnalyzer extends
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new StandardTokenizer(matchVersion);
-    TokenStream result = new GreekLowerCaseFilter(matchVersion, source);
-    result = new StandardFilter(matchVersion, result);
-    result = new StopFilter(matchVersion, result, stopwords);
+    final Tokenizer source = new StandardTokenizer();
+    TokenStream result = new GreekLowerCaseFilter(source);
+    result = new StandardFilter(result);
+    result = new StopFilter(result, stopwords);
     result = new GreekStemFilter(result);
     return new TokenStreamComponents(source, result);
   }

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilter.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilter.java Fri Aug  8 22:42:48 2014
@@ -22,32 +22,22 @@ import org.apache.lucene.analysis.TokenF
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.CharacterUtils;
-import org.apache.lucene.util.Version;
 
 /**
  * Normalizes token text to lower case, removes some Greek diacritics,
  * and standardizes final sigma to sigma. 
- * <a name="version"/>
- * <p>You must specify the required {@link Version}
- * compatibility when creating GreekLowerCaseFilter:
- * <ul>
- *   <li> As of 3.1, supplementary characters are properly lowercased.
- * </ul>
  */
 public final class GreekLowerCaseFilter extends TokenFilter {
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-  private final CharacterUtils charUtils;
+  private final CharacterUtils charUtils = CharacterUtils.getInstance();
 
   /**
    * Create a GreekLowerCaseFilter that normalizes Greek token text.
    * 
-   * @param matchVersion Lucene compatibility version, 
-   *   See <a href="#version">above</a>
    * @param in TokenStream to filter
    */
-  public GreekLowerCaseFilter(Version matchVersion, TokenStream in) {
+  public GreekLowerCaseFilter(TokenStream in) {
     super(in);
-    this.charUtils = CharacterUtils.getInstance(matchVersion);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java Fri Aug  8 22:42:48 2014
@@ -40,7 +40,6 @@ public class GreekLowerCaseFilterFactory
   /** Creates a new GreekLowerCaseFilterFactory */
   public GreekLowerCaseFilterFactory(Map<String,String> args) {
     super(args);
-    assureMatchVersion();
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -48,7 +47,7 @@ public class GreekLowerCaseFilterFactory
 
   @Override
   public GreekLowerCaseFilter create(TokenStream in) {
-    return new GreekLowerCaseFilter(luceneMatchVersion, in);
+    return new GreekLowerCaseFilter(in);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekStemmer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekStemmer.java?rev=1616901&r1=1616900&r2=1616901&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekStemmer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekStemmer.java Fri Aug  8 22:42:48 2014
@@ -1,7 +1,6 @@
 package org.apache.lucene.analysis.el;
 
 import org.apache.lucene.analysis.util.CharArraySet;
-import org.apache.lucene.util.Version;
 
 import java.util.Arrays;
 
@@ -205,7 +204,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc4 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc4 = new CharArraySet(
       Arrays.asList("θ", "δ", "ελ", "γαλ", "ν", "π", "ιδ", "παρ"),
       false);
   
@@ -231,7 +230,7 @@ public class GreekStemmer {
     return len;
   }
 
-  private static final CharArraySet exc6 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc6 = new CharArraySet(
       Arrays.asList("αλ", "αδ", "ενδ", "αμαν", "αμμοχαλ", "ηθ", "ανηθ",
           "αντιδ", "φυσ", "βρωμ", "γερ", "εξωδ", "καλπ", "καλλιν", "καταδ",
           "μουλ", "μπαν", "μπαγιατ", "μπολ", "μποσ", "νιτ", "ξικ", "συνομηλ",
@@ -256,7 +255,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc7 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc7 = new CharArraySet(
       Arrays.asList("αναπ", "αποθ", "αποκ", "αποστ", "βουβ", "ξεθ", "ουλ",
           "πεθ", "πικρ", "ποτ", "σιχ", "χ"), 
       false);
@@ -283,11 +282,11 @@ public class GreekStemmer {
     return len;
   }
 
-  private static final CharArraySet exc8a = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc8a = new CharArraySet(
       Arrays.asList("τρ", "τσ"),
       false);
 
-  private static final CharArraySet exc8b = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc8b = new CharArraySet(
       Arrays.asList("βετερ", "βουλκ", "βραχμ", "γ", "δραδουμ", "θ", "καλπουζ",
           "καστελ", "κορμορ", "λαοπλ", "μωαμεθ", "μ", "μουσουλμ", "ν", "ουλ",
           "π", "πελεκ", "πλ", "πολισ", "πορτολ", "σαρακατσ", "σουλτ",
@@ -346,7 +345,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc9 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc9 = new CharArraySet(
       Arrays.asList("αβαρ", "βεν", "εναρ", "αβρ", "αδ", "αθ", "αν", "απλ",
           "βαρον", "ντρ", "σκ", "κοπ", "μπορ", "νιφ", "παγ", "παρακαλ", "σερπ",
           "σκελ", "συρφ", "τοκ", "υ", "δ", "εμ", "θαρρ", "θ"), 
@@ -434,11 +433,11 @@ public class GreekStemmer {
     return len;
   }
 
-  private static final CharArraySet exc12a = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc12a = new CharArraySet(
       Arrays.asList("π", "απ", "συμπ", "ασυμπ", "ακαταπ", "αμεταμφ"),
       false);
 
-  private static final CharArraySet exc12b = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc12b = new CharArraySet(
       Arrays.asList("αλ", "αρ", "εκτελ", "ζ", "μ", "ξ", "παρακαλ", "αρ", "προ", "νισ"),
       false);
   
@@ -458,7 +457,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc13 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc13 = new CharArraySet(
       Arrays.asList("διαθ", "θ", "παρακαταθ", "προσθ", "συνθ"),
       false);
   
@@ -492,7 +491,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc14 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc14 = new CharArraySet(
       Arrays.asList("φαρμακ", "χαδ", "αγκ", "αναρρ", "βρομ", "εκλιπ", "λαμπιδ",
           "λεχ", "μ", "πατ", "ρ", "λ", "μεδ", "μεσαζ", "υποτειν", "αμ", "αιθ",
           "ανηκ", "δεσποζ", "ενδιαφερ", "δε", "δευτερευ", "καθαρευ", "πλε",
@@ -530,7 +529,7 @@ public class GreekStemmer {
    return len;
   }
   
-  private static final CharArraySet exc15a = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc15a = new CharArraySet(
       Arrays.asList("αβαστ", "πολυφ", "αδηφ", "παμφ", "ρ", "ασπ", "αφ", "αμαλ",
           "αμαλλι", "ανυστ", "απερ", "ασπαρ", "αχαρ", "δερβεν", "δροσοπ",
           "ξεφ", "νεοπ", "νομοτ", "ολοπ", "ομοτ", "προστ", "προσωποπ", "συμπ",
@@ -539,7 +538,7 @@ public class GreekStemmer {
           "ουλαμ", "ουρ", "π", "τρ", "μ"), 
       false);
   
-  private static final CharArraySet exc15b = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc15b = new CharArraySet(
       Arrays.asList("ψοφ", "ναυλοχ"),
       false);
   
@@ -576,7 +575,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc16 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc16 = new CharArraySet(
       Arrays.asList("ν", "χερσον", "δωδεκαν", "ερημον", "μεγαλον", "επταν"),
       false);
   
@@ -596,7 +595,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc17 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc17 = new CharArraySet(
       Arrays.asList("ασβ", "σβ", "αχρ", "χρ", "απλ", "αειμν", "δυσχρ", "ευχρ", "κοινοχρ", "παλιμψ"),
       false);
   
@@ -610,7 +609,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc18 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc18 = new CharArraySet(
       Arrays.asList("ν", "ρ", "σπι", "στραβομουτσ", "κακομουτσ", "εξων"),
       false);
   
@@ -634,7 +633,7 @@ public class GreekStemmer {
     return len;
   }
   
-  private static final CharArraySet exc19 = new CharArraySet(Version.LUCENE_CURRENT,
+  private static final CharArraySet exc19 = new CharArraySet(
       Arrays.asList("παρασουσ", "φ", "χ", "ωριοπλ", "αζ", "αλλοσουσ", "ασουσ"),
       false);