You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2021/10/21 14:34:24 UTC

[lucene-solr] branch branch_8x updated: LUCENE-10008: Respect ignoreCase flag in CommonGramsFilterFactory (#2573)

This is an automated email from the ASF dual-hosted git repository.

mikemccand pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 641ac0b  LUCENE-10008: Respect ignoreCase flag in CommonGramsFilterFactory (#2573)
641ac0b is described below

commit 641ac0b36a9257db3a6d2f9d12f422cfe5fddbc3
Author: Vigya Sharma <vi...@gmail.com>
AuthorDate: Thu Oct 21 07:34:04 2021 -0700

    LUCENE-10008: Respect ignoreCase flag in CommonGramsFilterFactory (#2573)
---
 lucene/CHANGES.txt                                 |   2 +
 .../commongrams/CommonGramsFilterFactory.java      |  48 ++-------
 .../lucene/analysis/core/StopFilterFactory.java    |  52 ++-------
 .../en/AbstractWordsFileFilterFactory.java         | 118 +++++++++++++++++++++
 .../miscellaneous/KeepWordFilterFactory.java       |  36 ++-----
 .../commongrams/TestCommonGramsFilterFactory.java  |  72 +++++++++----
 .../lucene/analysis/commongrams/common-1.txt       |  17 +++
 .../lucene/analysis/commongrams/common-2.txt       |  17 +++
 .../analysis/commongrams/common-snowball.txt       |  10 ++
 .../miscellaneous/TestKeepFilterFactory.java       |  27 +++++
 .../analysis/miscellaneous/keep-snowball.txt       |  10 ++
 11 files changed, 280 insertions(+), 129 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 291d080..be50560 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -39,6 +39,8 @@ Bug Fixes
 * LUCENE-10134: ConcurrentSortedSetDocValuesFacetCounts shouldn't share liveDocs Bits across threads.
   (Ankur Goel)
 
+* LUCENE-10008: Respect ignoreCase in CommonGramsFilterFactory (Vigya Sharma)
+
 Build
 ---------------------
 
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
index 523601b..c42d88f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
@@ -16,17 +16,13 @@
  */
 package org.apache.lucene.analysis.commongrams;
 
-
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.en.AbstractWordsFileFilterFactory;
 import org.apache.lucene.analysis.en.EnglishAnalyzer;
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.lucene.analysis.util.ResourceLoaderAware;
-import org.apache.lucene.analysis.util.TokenFilterFactory;
 
 /**
  * Constructs a {@link CommonGramsFilter}.
@@ -41,55 +37,29 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * @since 3.1
  * @lucene.spi {@value #NAME}
  */
-public class CommonGramsFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+public class CommonGramsFilterFactory extends AbstractWordsFileFilterFactory {
 
   /** SPI name */
   public static final String NAME = "commonGrams";
 
-  // TODO: shared base class for Stop/Keep/CommonGrams?
-  private CharArraySet commonWords;
-  private final String commonWordFiles;
-  private final String format;
-  private final boolean ignoreCase;
-  
   /** Creates a new CommonGramsFilterFactory */
   public CommonGramsFilterFactory(Map<String,String> args) {
     super(args);
-    commonWordFiles = get(args, "words");
-    format = get(args, "format");
-    ignoreCase = getBoolean(args, "ignoreCase", false);
-    if (!args.isEmpty()) {
-      throw new IllegalArgumentException("Unknown parameters: " + args);
-    }
-  }
-
-  @Override
-  public void inform(ResourceLoader loader) throws IOException {
-    if (commonWordFiles != null) {
-      if ("snowball".equalsIgnoreCase(format)) {
-        commonWords = getSnowballWordSet(loader, commonWordFiles, ignoreCase);
-      } else {
-        commonWords = getWordSet(loader, commonWordFiles, ignoreCase);
-      }
-    } else {
-      commonWords = EnglishAnalyzer.ENGLISH_STOP_WORDS_SET;
-    }
   }
 
-  public boolean isIgnoreCase() {
-    return ignoreCase;
+  public CharArraySet getCommonWords() {
+    return getWords();
   }
 
-  public CharArraySet getCommonWords() {
-    return commonWords;
+  @Override
+  protected CharArraySet createDefaultWords() {
+    return new CharArraySet(EnglishAnalyzer.ENGLISH_STOP_WORDS_SET, isIgnoreCase());
   }
 
   @Override
   public TokenFilter create(TokenStream input) {
-    CommonGramsFilter commonGrams = new CommonGramsFilter(input, commonWords);
+    CommonGramsFilter commonGrams = new CommonGramsFilter(input, getWords());
     return commonGrams;
   }
 }
- 
-  
-  
+
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
index a2b5b80..b6550f3 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
@@ -16,17 +16,13 @@
  */
 package org.apache.lucene.analysis.core;
 
-
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.WordlistLoader;
+import org.apache.lucene.analysis.en.AbstractWordsFileFilterFactory;
 import org.apache.lucene.analysis.en.EnglishAnalyzer;
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.lucene.analysis.util.ResourceLoaderAware;
-import org.apache.lucene.analysis.util.TokenFilterFactory;
 
 /**
  * Factory for {@link StopFilter}.
@@ -73,59 +69,29 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * @since 3.1
  * @lucene.spi {@value #NAME}
  */
-public class StopFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+public class StopFilterFactory extends AbstractWordsFileFilterFactory {
 
   /** SPI name */
   public static final String NAME = "stop";
 
-  public static final String FORMAT_WORDSET = "wordset";
-  public static final String FORMAT_SNOWBALL = "snowball";
-  
-  private CharArraySet stopWords;
-  private final String stopWordFiles;
-  private final String format;
-  private final boolean ignoreCase;
-  
   /** Creates a new StopFilterFactory */
   public StopFilterFactory(Map<String,String> args) {
     super(args);
-    stopWordFiles = get(args, "words");
-    format = get(args, "format", (null == stopWordFiles ? null : FORMAT_WORDSET));
-    ignoreCase = getBoolean(args, "ignoreCase", false);
-    if (!args.isEmpty()) {
-      throw new IllegalArgumentException("Unknown parameters: " + args);
-    }
-  }
-
-  @Override
-  public void inform(ResourceLoader loader) throws IOException {
-    if (stopWordFiles != null) {
-      if (FORMAT_WORDSET.equalsIgnoreCase(format)) {
-        stopWords = getWordSet(loader, stopWordFiles, ignoreCase);
-      } else if (FORMAT_SNOWBALL.equalsIgnoreCase(format)) {
-        stopWords = getSnowballWordSet(loader, stopWordFiles, ignoreCase);
-      } else {
-        throw new IllegalArgumentException("Unknown 'format' specified for 'words' file: " + format);
-      }
-    } else {
-      if (null != format) {
-        throw new IllegalArgumentException("'format' can not be specified w/o an explicit 'words' file: " + format);
-      }
-      stopWords = new CharArraySet(EnglishAnalyzer.ENGLISH_STOP_WORDS_SET, ignoreCase);
-    }
   }
 
-  public boolean isIgnoreCase() {
-    return ignoreCase;
+  public CharArraySet getStopWords() {
+    return getWords();
   }
 
-  public CharArraySet getStopWords() {
-    return stopWords;
+  @Override
+  protected CharArraySet createDefaultWords() {
+    return new CharArraySet(EnglishAnalyzer.ENGLISH_STOP_WORDS_SET, isIgnoreCase());
   }
 
   @Override
   public TokenStream create(TokenStream input) {
-    StopFilter stopFilter = new StopFilter(input,stopWords);
+    StopFilter stopFilter = new StopFilter(input, getWords());
     return stopFilter;
   }
 }
+
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/AbstractWordsFileFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/AbstractWordsFileFilterFactory.java
new file mode 100644
index 0000000..b74b70f
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/AbstractWordsFileFilterFactory.java
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.analysis.en;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.analysis.WordlistLoader;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+
+/**
+ * Abstract parent class for analysis factories that accept a stopwords file as input.
+ *
+ * <p>Concrete implementations can leverage the following input attributes. All attributes are
+ * optional:
+ *
+ * <ul>
+ *   <li><code>ignoreCase</code> defaults to <code>false</code>
+ *   <li><code>words</code> should be the name of a stopwords file to parse, if not specified the
+ *       factory will use the value provided by {@link #createDefaultWords()} implementation in
+ *       concrete subclass.
+ *   <li><code>format</code> defines how the <code>words</code> file will be parsed, and defaults to
+ *       <code>wordset</code>. If <code>words</code> is not specified, then <code>format</code> must
+ *       not be specified.
+ * </ul>
+ *
+ * <p>The valid values for the <code>format</code> option are:
+ *
+ * <ul>
+ *   <li><code>wordset</code> - This is the default format, which supports one word per line
+ *       (including any intra-word whitespace) and allows whole line comments beginning with the "#"
+ *       character. Blank lines are ignored. See {@link WordlistLoader#getLines
+ *       WordlistLoader.getLines} for details.
+ *   <li><code>snowball</code> - This format allows for multiple words specified on each line, and
+ *       trailing comments may be specified using the vertical line ("&#124;"). Blank lines are
+ *       ignored. See {@link WordlistLoader#getSnowballWordSet WordlistLoader.getSnowballWordSet}
+ *       for details.
+ * </ul>
+ */
+public abstract class AbstractWordsFileFilterFactory extends TokenFilterFactory
+    implements ResourceLoaderAware {
+
+  public static final String FORMAT_WORDSET = "wordset";
+  public static final String FORMAT_SNOWBALL = "snowball";
+
+  private CharArraySet words;
+  private final String wordFiles;
+  private final String format;
+  private final boolean ignoreCase;
+
+  /** Initialize this factory via a set of key-value pairs. */
+  public AbstractWordsFileFilterFactory(Map<String, String> args) {
+    super(args);
+    wordFiles = get(args, "words");
+    format = get(args, "format", (null == wordFiles ? null : FORMAT_WORDSET));
+    ignoreCase = getBoolean(args, "ignoreCase", false);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  /** Initialize the set of stopwords provided via ResourceLoader, or using defaults. */
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    if (wordFiles != null) {
+      if (FORMAT_WORDSET.equalsIgnoreCase(format)) {
+        words = getWordSet(loader, wordFiles, ignoreCase);
+      } else if (FORMAT_SNOWBALL.equalsIgnoreCase(format)) {
+        words = getSnowballWordSet(loader, wordFiles, ignoreCase);
+      } else {
+        throw new IllegalArgumentException(
+            "Unknown 'format' specified for 'words' file: " + format);
+      }
+    } else {
+      if (null != format) {
+        throw new IllegalArgumentException(
+            "'format' can not be specified w/o an explicit 'words' file: " + format);
+      }
+      words = createDefaultWords();
+    }
+  }
+
+  /** Default word set implementation. */
+  protected abstract CharArraySet createDefaultWords();
+
+  public CharArraySet getWords() {
+    return words;
+  }
+
+  public String getWordFiles() {
+    return wordFiles;
+  }
+
+  public String getFormat() {
+    return format;
+  }
+
+  public boolean isIgnoreCase() {
+    return ignoreCase;
+  }
+}
+
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
index 64a83ca..69200c6 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
@@ -16,15 +16,11 @@
  */
 package org.apache.lucene.analysis.miscellaneous;
 
-
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.lucene.analysis.util.ResourceLoaderAware;
-import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.analysis.en.AbstractWordsFileFilterFactory;
 
 /**
  * Factory for {@link KeepWordFilter}. 
@@ -39,48 +35,30 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * @since 3.1
  * @lucene.spi {@value #NAME}
  */
-public class KeepWordFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+public class KeepWordFilterFactory extends AbstractWordsFileFilterFactory {
 
   /** SPI name */
   public static final String NAME = "keepWord";
 
-  private final boolean ignoreCase;
-  private final String wordFiles;
-  private CharArraySet words;
-  
   /** Creates a new KeepWordFilterFactory */
   public KeepWordFilterFactory(Map<String,String> args) {
     super(args);
-    wordFiles = get(args, "words");
-    ignoreCase = getBoolean(args, "ignoreCase", false);
-    if (!args.isEmpty()) {
-      throw new IllegalArgumentException("Unknown parameters: " + args);
-    }
   }
 
   @Override
-  public void inform(ResourceLoader loader) throws IOException {
-    if (wordFiles != null) {
-      words = getWordSet(loader, wordFiles, ignoreCase);
-    }
-  }
-
-  public boolean isIgnoreCase() {
-    return ignoreCase;
-  }
-
-  public CharArraySet getWords() {
-    return words;
+  protected CharArraySet createDefaultWords() {
+    return null;
   }
 
   @Override
   public TokenStream create(TokenStream input) {
     // if the set is null, it means it was empty
-    if (words == null) {
+    if (getWords() == null) {
       return input;
     } else {
-      final TokenStream filter = new KeepWordFilter(input, words);
+      final TokenStream filter = new KeepWordFilter(input, getWords());
       return filter;
     }
   }
 }
+
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/TestCommonGramsFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/TestCommonGramsFilterFactory.java
index 5bcfb3d..9fa81f4 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/TestCommonGramsFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/TestCommonGramsFilterFactory.java
@@ -23,25 +23,26 @@ import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.core.TestStopFilterFactory;
 import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
 import org.apache.lucene.analysis.util.ClasspathResourceLoader;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.util.Version;
 
-/**
- * Tests pretty much copied from StopFilterFactoryTest We use the test files
- * used by the StopFilterFactoryTest TODO: consider creating separate test files
- * so this won't break if stop filter test files change
- **/
 public class TestCommonGramsFilterFactory extends BaseTokenStreamFactoryTestCase {
 
   public void testInform() throws Exception {
-    ResourceLoader loader = new ClasspathResourceLoader(TestStopFilterFactory.class);
+    ResourceLoader loader = new ClasspathResourceLoader(getClass());
     assertTrue("loader is null and it shouldn't be", loader != null);
-    CommonGramsFilterFactory factory = (CommonGramsFilterFactory) tokenFilterFactory("CommonGrams", Version.LATEST, loader,
-        "words", "stop-1.txt", 
-        "ignoreCase", "true");
+    CommonGramsFilterFactory factory =
+        (CommonGramsFilterFactory)
+            tokenFilterFactory(
+                "CommonGrams",
+                Version.LATEST,
+                loader,
+                "words",
+                "common-1.txt",
+                "ignoreCase",
+                "true");
     CharArraySet words = factory.getCommonWords();
     assertTrue("words is null and it shouldn't be", words != null);
     assertTrue("words Size: " + words.size() + " is not: " + 2,
@@ -49,9 +50,16 @@ public class TestCommonGramsFilterFactory extends BaseTokenStreamFactoryTestCase
     assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory
         .isIgnoreCase() == true);
 
-    factory = (CommonGramsFilterFactory) tokenFilterFactory("CommonGrams", Version.LATEST, loader,
-        "words", "stop-1.txt, stop-2.txt", 
-        "ignoreCase", "true");
+    factory =
+        (CommonGramsFilterFactory)
+            tokenFilterFactory(
+                "CommonGrams",
+                Version.LATEST,
+                loader,
+                "words",
+                "common-1.txt, common-2.txt",
+                "ignoreCase",
+                "true");
     words = factory.getCommonWords();
     assertTrue("words is null and it shouldn't be", words != null);
     assertTrue("words Size: " + words.size() + " is not: " + 4,
@@ -59,10 +67,18 @@ public class TestCommonGramsFilterFactory extends BaseTokenStreamFactoryTestCase
     assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory
         .isIgnoreCase() == true);
 
-    factory = (CommonGramsFilterFactory) tokenFilterFactory("CommonGrams", Version.LATEST, loader,
-        "words", "stop-snowball.txt", 
-        "format", "snowball", 
-        "ignoreCase", "true");
+    factory =
+        (CommonGramsFilterFactory)
+            tokenFilterFactory(
+                "CommonGrams",
+                Version.LATEST,
+                loader,
+                "words",
+                "common-snowball.txt",
+                "format",
+                "snowball",
+                "ignoreCase",
+                "true");
     words = factory.getCommonWords();
     assertEquals(8, words.size());
     assertTrue(words.contains("he"));
@@ -89,7 +105,26 @@ public class TestCommonGramsFilterFactory extends BaseTokenStreamFactoryTestCase
     assertTokenStreamContents(stream, 
         new String[] { "testing", "testing_the", "the", "the_factory", "factory" });
   }
-  
+
+  /**
+   * Test that ignoreCase flag is honored when no words are provided and default stopwords are used.
+   */
+  public void testIgnoreCase() throws Exception {
+    ResourceLoader loader = new ClasspathResourceLoader(getClass());
+    CommonGramsFilterFactory factory =
+        (CommonGramsFilterFactory)
+            tokenFilterFactory("CommonGrams", Version.LATEST, loader, "ignoreCase", "true");
+    CharArraySet words = factory.getCommonWords();
+    assertTrue("words is null and it shouldn't be", words != null);
+    assertTrue(words.contains("the"));
+    assertTrue(words.contains("The"));
+    Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+    tokenizer.setReader(new StringReader("testing The factory"));
+    TokenStream stream = factory.create(tokenizer);
+    assertTokenStreamContents(
+        stream, new String[] {"testing", "testing_The", "The", "The_factory", "factory"});
+  }
+
   /** Test that bogus arguments result in exception */
   public void testBogusArguments() throws Exception {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {      
@@ -98,3 +133,4 @@ public class TestCommonGramsFilterFactory extends BaseTokenStreamFactoryTestCase
     assertTrue(expected.getMessage().contains("Unknown parameters"));
   }
 }
+
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-1.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-1.txt
new file mode 100644
index 0000000..8dfe809
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-1.txt
@@ -0,0 +1,17 @@
+# 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.
+
+foo
+bar
\ No newline at end of file
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-2.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-2.txt
new file mode 100644
index 0000000..646b7ff
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-2.txt
@@ -0,0 +1,17 @@
+# 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.
+
+junk
+more
\ No newline at end of file
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-snowball.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-snowball.txt
new file mode 100644
index 0000000..1c0c6f5
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/commongrams/common-snowball.txt
@@ -0,0 +1,10 @@
+ | This is a file in snowball format, empty lines are ignored, '|' is a comment
+ | Additionally, multiple words can be on the same line, allowing stopwords to be
+ | arranged in tables (useful in some languages where they might inflect)
+
+ | fictitious table below
+
+|third person singular
+|Subject Object Possessive Reflexive
+he       him    his        himself| masculine
+she      her    hers       herself| feminine
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepFilterFactory.java
index dde6f94..ea0de1c 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestKeepFilterFactory.java
@@ -40,6 +40,32 @@ public class TestKeepFilterFactory extends BaseTokenStreamFactoryTestCase {
     words = factory.getWords();
     assertTrue("words is null and it shouldn't be", words != null);
     assertTrue("words Size: " + words.size() + " is not: " + 4, words.size() == 4);
+
+    factory =
+        (KeepWordFilterFactory)
+            tokenFilterFactory(
+                "KeepWord",
+                "words",
+                "keep-snowball.txt",
+                "format",
+                "snowball",
+                "ignoreCase",
+                "true");
+    words = factory.getWords();
+    assertEquals(8, words.size());
+    assertTrue(words.contains("he"));
+    assertTrue(words.contains("him"));
+    assertTrue(words.contains("his"));
+    assertTrue(words.contains("himself"));
+    assertTrue(words.contains("she"));
+    assertTrue(words.contains("her"));
+    assertTrue(words.contains("hers"));
+    assertTrue(words.contains("herself"));
+
+    // defaults
+    factory = (KeepWordFilterFactory) tokenFilterFactory("KeepWord");
+    assertTrue(factory.getWords() == null);
+    assertEquals(false, factory.isIgnoreCase());
   }
   
   /** Test that bogus arguments result in exception */
@@ -50,3 +76,4 @@ public class TestKeepFilterFactory extends BaseTokenStreamFactoryTestCase {
     assertTrue(expected.getMessage().contains("Unknown parameters"));
   }
 }
+
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/keep-snowball.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/keep-snowball.txt
new file mode 100644
index 0000000..1c0c6f5
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/keep-snowball.txt
@@ -0,0 +1,10 @@
+ | This is a file in snowball format, empty lines are ignored, '|' is a comment
+ | Additionally, multiple words can be on the same line, allowing stopwords to be
+ | arranged in tables (useful in some languages where they might inflect)
+
+ | fictitious table below
+
+|third person singular
+|Subject Object Possessive Reflexive
+he       him    his        himself| masculine
+she      her    hers       herself| feminine