You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2018/05/20 23:52:52 UTC

[1/2] lucene-solr:master: LUCENE-8273: Rename TermExclusionFilter -> ProtectedTermFilter. Allow ProtectedTermFilterFactory to be used outside of CustomAnalyzer, including in Solr, by allowing wrapped filters and their parameters to be specified on const

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 1e661ed97 -> 97490ac7c
  refs/heads/master f506bc9cb -> d91273ddf


LUCENE-8273: Rename TermExclusionFilter -> ProtectedTermFilter.  Allow ProtectedTermFilterFactory to be used outside of CustomAnalyzer, including in Solr, by allowing wrapped filters and their parameters to be specified on construction.  Add tests for ProtectedTermFilterFactory in lucene/common/analysis/ and in solr/core/.  Add Solr ref guide documentation for ProtectedTermFilterFactory.  Improve javadocs for CustomAnalyzer, ConditionalTokenFilter, and ProtectedTermFilter.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d91273dd
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d91273dd
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d91273dd

Branch: refs/heads/master
Commit: d91273ddf0b524e14ebcc2a90bbedd8d9ae319d4
Parents: f506bc9
Author: Steve Rowe <sa...@apache.org>
Authored: Sun May 20 19:52:07 2018 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Sun May 20 19:52:07 2018 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +-
 .../lucene/analysis/custom/CustomAnalyzer.java  |   2 +-
 .../miscellaneous/ConditionalTokenFilter.java   |  14 +-
 .../miscellaneous/ProtectedTermFilter.java      |  53 ++++++
 .../ProtectedTermFilterFactory.java             | 163 +++++++++++++++++++
 .../miscellaneous/TermExclusionFilter.java      |  52 ------
 .../TermExclusionFilterFactory.java             |  58 -------
 .../analysis/util/AbstractAnalysisFactory.java  |  20 ++-
 ...ache.lucene.analysis.util.TokenFilterFactory |   2 +-
 .../analysis/miscellaneous/protected-1.txt      |  17 ++
 .../analysis/miscellaneous/protected-2.txt      |  17 ++
 .../analysis/custom/TestCustomAnalyzer.java     |   2 +-
 .../TestConditionalTokenFilter.java             |  14 +-
 .../miscellaneous/TestProtectedTermFilter.java  |  48 ++++++
 .../TestProtectedTermFilterFactory.java         | 128 +++++++++++++++
 .../miscellaneous/TestTermExclusionFilter.java  |  48 ------
 .../solr/collection1/conf/protected-1.txt       |  17 ++
 .../solr/collection1/conf/protected-2.txt       |  17 ++
 .../collection1/conf/schema-protected-term.xml  |  86 ++++++++++
 .../ProtectedTermFilterFactoryTest.java         |  84 ++++++++++
 .../solr-ref-guide/src/filter-descriptions.adoc |  47 ++++++
 21 files changed, 712 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 26606de..b13dc7b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -183,8 +183,8 @@ New Features
   filters based on the attributes of the current token. This generalises the keyword
   token logic currently used for stemmers and WDF.  It is integrated into 
   CustomAnalyzer by using the `when` and `whenTerm` builder methods, and a new
-  TermExclusionConditionalFilter is added as an example.  (Alan Woodward,
-  Robert Muir, David Smiley, Steve Rowe, Mike Sokolov)
+  ProtectedTermFilter is added as an example.  (Alan Woodward, Robert Muir,
+  David Smiley, Steve Rowe, Mike Sokolov)
 
 * LUCENE-8310: Ensure IndexFileDeleter accounts for pending deletes. Today we fail
   creating the IndexWriter when the directory has a pending delete. Yet, this

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index 72614ca..19e207f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -87,7 +87,7 @@ import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassI
  *    .whenTerm(t -&gt; t.length() &gt; 10)
  *      .addTokenFilter(&quot;reversestring&quot;)
  *    .endwhen()
- *    .build()
+ *    .build();
  * </pre>
  */
 public final class CustomAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
index c8b91dc..6f9ea24 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
@@ -29,10 +29,10 @@ import org.apache.lucene.util.AttributeSource;
 /**
  * Allows skipping TokenFilters based on the current set of attributes.
  *
- * To use, implement the {@link #shouldFilter()} method.  If it returns {@code false},
- * then calling {@link #incrementToken()} will use the wrapped TokenFilter to
- * make changes to the tokenstream.  If it returns {@code true}, then the wrapped
- * filter will be skipped
+ * To use, implement the {@link #shouldFilter()} method.  If it returns {@code true},
+ * then calling {@link #incrementToken()} will use the wrapped TokenFilter(s) to
+ * make changes to the tokenstream.  If it returns {@code false}, then the wrapped
+ * filter(s) will be skipped.
  */
 public abstract class ConditionalTokenFilter extends TokenFilter {
 
@@ -102,9 +102,9 @@ public abstract class ConditionalTokenFilter extends TokenFilter {
   private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
 
   /**
-   * Create a new BypassingTokenFilter
+   * Create a new ConditionalTokenFilter
    * @param input         the input TokenStream
-   * @param inputFactory  a factory function to create a new instance of the TokenFilter to wrap
+   * @param inputFactory  a factory function to create the wrapped filter(s)
    */
   protected ConditionalTokenFilter(TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
     super(input);
@@ -112,7 +112,7 @@ public abstract class ConditionalTokenFilter extends TokenFilter {
   }
 
   /**
-   * Whether or not to execute the wrapped TokenFilter for the current token
+   * Whether or not to execute the wrapped TokenFilter(s) for the current token
    */
   protected abstract boolean shouldFilter() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
new file mode 100644
index 0000000..e4e21be
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
@@ -0,0 +1,53 @@
+/*
+ * 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.miscellaneous;
+
+import java.util.function.Function;
+
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+/**
+ * A ConditionalTokenFilter that only applies its wrapped filters to tokens that
+ * are not contained in a protected set.
+ */
+public class ProtectedTermFilter extends ConditionalTokenFilter {
+
+  private final CharArraySet protectedTerms;
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  /**
+   * Creates a new ProtectedTermFilter
+   * @param protectedTerms  the set of terms to skip the wrapped filters for
+   * @param input         the input TokenStream
+   * @param inputFactory  a factory function to create the wrapped filter(s)
+   */
+  public ProtectedTermFilter(final CharArraySet protectedTerms, TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
+    super(input, inputFactory);
+    this.protectedTerms = protectedTerms;
+  }
+
+  @Override
+  protected boolean shouldFilter() {
+    boolean b = protectedTerms.contains(termAtt.buffer(), 0, termAtt.length());
+    return b == false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
new file mode 100644
index 0000000..1cde6c3
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
@@ -0,0 +1,163 @@
+/*
+ * 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.miscellaneous;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate; // javadocs
+
+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;
+
+/**
+ * Factory for a {@link ProtectedTermFilter}
+ *
+ * <p>CustomAnalyzer example:
+ * <pre class="prettyprint">
+ * Analyzer ana = CustomAnalyzer.builder()
+ *   .withTokenizer("standard")
+ *   .when("protectedterm", "ignoreCase", "true", "protected", "protectedTerms.txt")
+ *     .addTokenFilter("truncate", "prefixLength", "4")
+ *     .addTokenFilter("lowercase")
+ *   .endwhen()
+ *   .build();
+ * </pre>
+ *
+ * <p>Solr example, in which conditional filters are specified via the <code>wrappedFilters</code>
+ * parameter - a comma-separated list of case-insensitive TokenFilter SPI names - and conditional
+ * filter args are specified via <code>filterName.argName</code> parameters:
+ * <pre class="prettyprint">
+ * &lt;fieldType name="reverse_lower_with_exceptions" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.ProtectedTermFilterFactory" ignoreCase="true" protected="protectedTerms.txt"
+ *             wrappedFilters="truncate,lowercase" truncate.prefixLength="4" /&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ *
+ * <p>When using the <code>wrappedFilters</code> parameter, each filter name must be unique, so if you
+ * need to specify the same filter more than once, you must add case-insensitive unique '-id' suffixes
+ * (note that the '-id' suffix is stripped prior to SPI lookup), e.g.:
+ * <pre class="prettyprint">
+ * &lt;fieldType name="double_synonym_with_exceptions" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.ProtectedTermFilterFactory" ignoreCase="true" protected="protectedTerms.txt"
+ *             wrappedFilters="synonymgraph-A,synonymgraph-B"
+ *             synonymgraph-A.synonyms="synonyms-1.txt"
+ *             synonymgraph-B.synonyms="synonyms-2.txt"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ *
+ * <p>See related {@link org.apache.lucene.analysis.custom.CustomAnalyzer.Builder#whenTerm(Predicate)}
+ */
+public class ProtectedTermFilterFactory extends ConditionalTokenFilterFactory implements ResourceLoaderAware {
+
+  public static final String PROTECTED_TERMS = "protected";
+  public static final char FILTER_ARG_SEPARATOR = '.';
+  public static final char FILTER_NAME_ID_SEPARATOR = '-';
+
+  private final String termFiles;
+  private final boolean ignoreCase;
+  private final String wrappedFilters;
+
+  private CharArraySet protectedTerms;
+
+  public ProtectedTermFilterFactory(Map<String, String> args) {
+    super(args);
+    termFiles = require(args, PROTECTED_TERMS);
+    ignoreCase = getBoolean(args, "ignoreCase", false);
+    wrappedFilters = get(args, "wrappedFilters");
+    if (wrappedFilters != null) {
+      handleWrappedFilterArgs(args);
+    }
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  private void handleWrappedFilterArgs(Map<String, String> args) {
+    LinkedHashMap<String, Map<String, String>> wrappedFilterArgs = new LinkedHashMap<>();
+    splitAt(',', wrappedFilters).forEach(filterName -> {          // Format: SPIname[-id]
+      filterName = filterName.trim().toLowerCase(Locale.ROOT);             // Treat case-insensitively
+      if (wrappedFilterArgs.containsKey(filterName)) {
+        throw new IllegalArgumentException("wrappedFilters contains duplicate '"
+            + filterName + "'. Add unique '-id' suffixes (stripped prior to SPI lookup).");
+      }
+      wrappedFilterArgs.put(filterName, new HashMap<>());
+    });
+    for (Iterator<Map.Entry<String, String>> iterator = args.entrySet().iterator(); iterator.hasNext(); ) {
+      Map.Entry<String, String> entry = iterator.next();
+      String filterArgKey = entry.getKey();
+      String argValue = entry.getValue();
+      List<String> splitKey = splitAt(FILTER_ARG_SEPARATOR, filterArgKey); // Format: filterName.argKey
+      if (splitKey.size() == 2) {                                          // Skip if no slash
+        String filterName = splitKey.get(0).toLowerCase(Locale.ROOT);
+        if (wrappedFilterArgs.containsKey(filterName)) {                   // Skip if not in "wrappedFilter" arg
+          Map<String, String> filterArgs = wrappedFilterArgs.computeIfAbsent(filterName, k -> new HashMap<>());
+          String argKey = splitKey.get(1);
+          filterArgs.put(argKey, argValue); // argKey is guaranteed unique, don't need to check for duplicates
+          iterator.remove();
+        }
+      }
+    }
+    if (args.isEmpty()) {
+      populateInnerFilters(wrappedFilterArgs);
+    }
+  }
+
+  private void populateInnerFilters(LinkedHashMap<String, Map<String, String>> wrappedFilterArgs) {
+    List<TokenFilterFactory> innerFilters = new ArrayList<>();
+    wrappedFilterArgs.forEach((filterName, filterArgs) -> {
+      int idSuffixPos = filterName.indexOf(FILTER_NAME_ID_SEPARATOR); // Format: SPIname[-id]
+      if (idSuffixPos != -1) {                                        // Strip '-id' suffix, if any, prior to SPI lookup
+        filterName = filterName.substring(0, idSuffixPos);
+      }
+      innerFilters.add(TokenFilterFactory.forName(filterName, filterArgs));
+    });
+    setInnerFilters(innerFilters);
+  }
+
+  public boolean isIgnoreCase() {
+    return ignoreCase;
+  }
+
+  public CharArraySet getProtectedTerms() {
+    return protectedTerms;
+  }
+
+  @Override
+  protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
+    return new ProtectedTermFilter(protectedTerms, input, inner);
+  }
+
+  @Override
+  public void doInform(ResourceLoader loader) throws IOException {
+    protectedTerms = getWordSet(loader, termFiles, ignoreCase);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
deleted file mode 100644
index 95bf55f..0000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.util.function.Function;
-
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-
-/**
- * A ConditionalTokenFilter that only applies its wrapped filters to tokens that
- * are not contained in an exclusion set.
- */
-public class TermExclusionFilter extends ConditionalTokenFilter {
-
-  private final CharArraySet excludeTerms;
-
-  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-
-  /**
-   * Creates a new TermExclusionFilter
-   * @param excludeTerms  the set of terms to skip the wrapped filters for
-   * @param input         the input TokenStream
-   * @param inputFactory  a factory function to create the wrapped filters
-   */
-  public TermExclusionFilter(final CharArraySet excludeTerms, TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
-    super(input, inputFactory);
-    this.excludeTerms = excludeTerms;
-  }
-
-  @Override
-  protected boolean shouldFilter() {
-    return excludeTerms.contains(termAtt.buffer(), 0, termAtt.length()) == false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
deleted file mode 100644
index f860b72..0000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.function.Function;
-
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.util.ResourceLoader;
-
-/**
- * Factory for a {@link TermExclusionFilter}
- */
-public class TermExclusionFilterFactory extends ConditionalTokenFilterFactory {
-
-  public static final String EXCLUDED_TOKENS = "protected";
-
-  private final String wordFiles;
-  private final boolean ignoreCase;
-
-  private CharArraySet excludeTerms;
-
-  public TermExclusionFilterFactory(Map<String, String> args) {
-    super(args);
-    wordFiles = get(args, EXCLUDED_TOKENS);
-    ignoreCase = getBoolean(args, "ignoreCase", false);
-    if (!args.isEmpty()) {
-      throw new IllegalArgumentException("Unknown parameters: " + args);
-    }
-  }
-
-  @Override
-  protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
-    return new TermExclusionFilter(excludeTerms, input, inner);
-  }
-
-  @Override
-  public void doInform(ResourceLoader loader) throws IOException {
-    excludeTerms = getWordSet(loader, wordFiles, ignoreCase);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
index f8de8a7..954f300 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
@@ -290,12 +290,24 @@ public abstract class AbstractAnalysisFactory {
    * @return a list of file names with the escaping backslashed removed
    */
   protected final List<String> splitFileNames(String fileNames) {
-    if (fileNames == null)
-      return Collections.<String>emptyList();
+    return splitAt(',', fileNames);
+  }
+
+  /**
+   * Splits a list separated by zero or more given separator characters.
+   * List items can contain comma characters escaped by backslash '\'.
+   * Whitespace is NOT trimmed from the returned list items.
+   *
+   * @param list the string containing the split list items
+   * @return a list of items with the escaping backslashes removed
+   */
+  protected final List<String> splitAt(char separator, String list) {
+    if (list == null)
+      return Collections.emptyList();
 
     List<String> result = new ArrayList<>();
-    for (String file : fileNames.split("(?<!\\\\),")) {
-      result.add(file.replaceAll("\\\\(?=,)", ""));
+    for (String item : list.split("(?<!\\\\)[" + separator + "]")) {
+      result.add(item.replaceAll("\\\\(?=[" + separator + "])", ""));
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index ddd3c91..1811920 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -78,7 +78,7 @@ org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilterFactory
 org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilterFactory
 org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilterFactory
-org.apache.lucene.analysis.miscellaneous.TermExclusionFilterFactory
+org.apache.lucene.analysis.miscellaneous.ProtectedTermFilterFactory
 org.apache.lucene.analysis.miscellaneous.TrimFilterFactory
 org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.TypeAsSynonymFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt
new file mode 100644
index 0000000..3fe6d02
--- /dev/null
+++ b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt
new file mode 100644
index 0000000..d9a5c3c
--- /dev/null
+++ b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-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.
+
+golden
+compote

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index fb86c07..01ad75c 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -520,7 +520,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     CustomAnalyzer analyzer = CustomAnalyzer.builder()
         .withTokenizer("whitespace")
         .addTokenFilter("lowercase")
-        .when("termexclusion", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
+        .when("protectedterm", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
           .addTokenFilter("reversestring")
         .endwhen()
         .build();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
index 02d8a78..fed7f68 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
@@ -248,14 +248,14 @@ public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
 
   public void testReadaheadWithFiltering() throws IOException {
 
-    CharArraySet exclusions = new CharArraySet(2, true);
-    exclusions.add("three");
+    CharArraySet protectedTerms = new CharArraySet(2, true);
+    protectedTerms.add("three");
 
     Analyzer analyzer = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
         Tokenizer source = new ClassicTokenizer();
-        TokenStream sink = new TermExclusionFilter(exclusions, source, in -> new ShingleFilter(in, 2));
+        TokenStream sink = new ProtectedTermFilter(protectedTerms, source, in -> new ShingleFilter(in, 2));
         return new TokenStreamComponents(source, sink);
       }
     };
@@ -287,15 +287,15 @@ public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
 
   public void testFilteredTokenFilters() throws IOException {
 
-    CharArraySet exclusions = new CharArraySet(2, true);
-    exclusions.add("foobar");
+    CharArraySet protectedTerms = new CharArraySet(2, true);
+    protectedTerms.add("foobar");
 
     TokenStream ts = whitespaceMockTokenizer("wuthering foobar abc");
-    ts = new TermExclusionFilter(exclusions, ts, in -> new LengthFilter(in, 1, 4));
+    ts = new ProtectedTermFilter(protectedTerms, ts, in -> new LengthFilter(in, 1, 4));
     assertTokenStreamContents(ts, new String[]{ "foobar", "abc" });
 
     ts = whitespaceMockTokenizer("foobar abc");
-    ts = new TermExclusionFilter(exclusions, ts, in -> new LengthFilter(in, 1, 4));
+    ts = new ProtectedTermFilter(protectedTerms, ts, in -> new LengthFilter(in, 1, 4));
     assertTokenStreamContents(ts, new String[]{ "foobar", "abc" });
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
new file mode 100644
index 0000000..4c64f56
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.miscellaneous;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.LowerCaseFilter;
+
+public class TestProtectedTermFilter extends BaseTokenStreamTestCase {
+
+  public void testBasic() throws IOException {
+
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("Alice", 1, 0, 5),
+        new Token("Bob", 1, 6, 9),
+        new Token("Clara", 1, 10, 15),
+        new Token("David", 1, 16, 21)
+    );
+
+    CharArraySet protectedTerms = new CharArraySet(5, true);
+    protectedTerms.add("bob");
+
+    TokenStream ts = new ProtectedTermFilter(protectedTerms, cts, LowerCaseFilter::new);
+    assertTokenStreamContents(ts, new String[]{ "alice", "Bob", "clara", "david" });
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
new file mode 100644
index 0000000..a1aac68
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.miscellaneous;
+
+
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
+
+/**
+ * Simple tests to ensure the simple truncation filter factory is working.
+ */
+public class TestProtectedTermFilterFactory extends BaseTokenStreamFactoryTestCase {
+  public void testInform() throws Exception {
+    ProtectedTermFilterFactory factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt", "ignoreCase", "true", "wrappedFilters", "lowercase");
+    CharArraySet protectedTerms = factory.getProtectedTerms();
+    assertTrue("protectedTerms is null and it shouldn't be", protectedTerms != null);
+    assertTrue("protectedTerms Size: " + protectedTerms.size() + " is not: " + 2, protectedTerms.size() == 2);
+    assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory.isIgnoreCase() == true);
+
+    factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt, protected-2.txt", "ignoreCase", "true", "wrappedFilters", "lowercase");
+    protectedTerms = factory.getProtectedTerms();
+    assertTrue("protectedTerms is null and it shouldn't be", protectedTerms != null);
+    assertTrue("protectedTerms Size: " + protectedTerms.size() + " is not: " + 4, protectedTerms.size() == 4);
+    assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory.isIgnoreCase() == true);
+
+    // defaults
+    factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt");
+    assertEquals(false, factory.isIgnoreCase());
+  }
+
+  public void testBasic() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "lowercase").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "clara", "Bar", "david"});
+  }
+
+  public void testMultipleWrappedFiltersWithParams() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "lowercase, truncate",
+        "truncate.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "cl", "Bar", "da"});
+  }
+
+  public void testMultipleSameNamedFiltersWithParams() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "truncate-A, reversestring, truncate-B",
+        "truncate-A.prefixLength", "3", "truncate-B.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+
+    // same-named wrapped filters, one with an ID and another without
+    stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "truncate, reversestring, truncate-A",
+        "truncate.prefixLength", "3", "truncate-A.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+
+    // Case-insensitive wrapped "filter-id"
+    stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "TRUNCATE-a, reversestring, truncate-b",
+        "truncate-A.prefixLength", "3", "TRUNCATE-B.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+  }
+
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () ->
+      tokenFilterFactory("ProtectedTerm", "protected", "protected-1.txt", "bogusArg", "bogusValue"));
+    assertTrue(exception.getMessage().contains("Unknown parameters"));
+
+    // same-named wrapped filters
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate, truncate"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // case-insensitive same-named wrapped filters
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "TRUNCATE, truncate"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // case-insensitive same-named wrapped filter IDs
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate-ABC, truncate-abc"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // mismatched wrapped filter and associated args
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate-A, reversestring, truncate-B",
+            "truncate.prefixLength", "3", "truncate-A.prefixLength", "2"));
+    assertTrue(exception.getMessage().contains("Unknown parameters: {truncate.prefixLength=3}"));
+
+    // missing required arg(s) for wrapped filter
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    exception = expectThrows(IllegalArgumentException.class, () ->
+      tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "length").create(stream));
+    assertTrue(exception.getMessage().contains("Configuration Error: missing parameter"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
deleted file mode 100644
index 2e6aecf..0000000
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.io.IOException;
-
-import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.CannedTokenStream;
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.core.LowerCaseFilter;
-
-public class TestTermExclusionFilter extends BaseTokenStreamTestCase {
-
-  public void testExcludeTerms() throws IOException {
-
-    CannedTokenStream cts = new CannedTokenStream(
-        new Token("Alice", 1, 0, 5),
-        new Token("Bob", 1, 6, 9),
-        new Token("Clara", 1, 10, 15),
-        new Token("David", 1, 16, 21)
-    );
-
-    CharArraySet exclusions = new CharArraySet(5, true);
-    exclusions.add("bob");
-
-    TokenStream ts = new TermExclusionFilter(exclusions, cts, LowerCaseFilter::new);
-    assertTokenStreamContents(ts, new String[]{ "alice", "Bob", "clara", "david" });
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/solr/core/src/test-files/solr/collection1/conf/protected-1.txt
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/protected-1.txt b/solr/core/src/test-files/solr/collection1/conf/protected-1.txt
new file mode 100644
index 0000000..61c0583
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/protected-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.
+
+foobar
+jaxfopbuz

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/solr/core/src/test-files/solr/collection1/conf/protected-2.txt
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/protected-2.txt b/solr/core/src/test-files/solr/collection1/conf/protected-2.txt
new file mode 100644
index 0000000..d9a5c3c
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/protected-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.
+
+golden
+compote

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml b/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
new file mode 100644
index 0000000..2294394
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<!-- The Solr schema file, version 1.6  -->
+
+<schema name="schema-protected-term" version="1.6">
+  <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
+  <field name="signatureField" type="string" indexed="true" stored="false"/>
+  <dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
+
+  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
+  <field name="_version_" type="long" indexed="false" stored="false" docValues="true"/>
+
+  <fieldType name="prefix4_lower_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate,lowercase" truncate.prefixLength="4"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix4_lower" type="prefix4_lower_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate-A, reversestring, truncate-B"
+              truncate-A.prefixLength="3" truncate-B.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2" type="prefix3_rev_prefix2_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_mixed_IDs_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate, reversestring, truncate-A"
+              truncate.prefixLength="3" truncate-A.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2_mixed_IDs" type="prefix3_rev_prefix2_mixed_IDs_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_mixed_case_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="TRUNCATE-a, reversestring, truncate-b"
+              truncate-A.prefixLength="3" TRUNCATE-B.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2_mixed_case" type="prefix3_rev_prefix2_mixed_case_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <uniqueKey>id</uniqueKey>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java b/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
new file mode 100644
index 0000000..5c776dd
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.solr.analysis;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.miscellaneous.ProtectedTermFilterFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrResourceLoader;
+import org.junit.BeforeClass;
+
+public class ProtectedTermFilterFactoryTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml","schema-protected-term.xml");
+  }
+
+  public void testBasic() throws Exception {
+    String text = "Wuthering FooBar distant goldeN ABC compote";
+    Map<String,String> args = new HashMap<>();
+    args.put("ignoreCase", "true");
+    args.put("protected", "protected-1.txt,protected-2.txt");  // Protected: foobar, jaxfopbuz, golden, compote
+    args.put("wrappedFilters", "lowercase");
+
+    ResourceLoader loader = new SolrResourceLoader(TEST_PATH().resolve("collection1"));
+    ProtectedTermFilterFactory factory = new ProtectedTermFilterFactory(args);
+    factory.inform(loader);
+
+    TokenStream ts = factory.create(whitespaceMockTokenizer(text));
+    BaseTokenStreamTestCase.assertTokenStreamContents(ts,
+        new String[] { "wuthering", "FooBar", "distant", "goldeN", "abc", "compote" });
+  }
+
+  public void testTwoWrappedFilters() {
+    // Index-time: Filters: truncate:4 & lowercase.  Protected (ignoreCase:true): foobar, jaxfopbuz, golden, compote
+    // Query-time: No filters
+    assertU(adoc("id", "1", "prefix4_lower", "Wuthering FooBar distant goldeN ABC compote"));
+    assertU(commit());
+
+    assertQ(req("prefix4_lower:(+wuth +FooBar +dist +goldeN +abc +compote)")
+        , "//result[@numFound=1]"
+    );
+  }
+
+  public void testDuplicateFilters() {
+    // Index-time: Filters: truncate:3 & reversestring & truncate:2.  Protected (ignoreCase:true): foobar, jaxfopbuz, golden, compote
+    // Query-time: No filters
+    assertU(adoc("id", "1",
+        "prefix3_rev_prefix2",            "Wuthering FooBar distant goldeN ABC compote",
+        "prefix3_rev_prefix2_mixed_IDs",  "Wuthering FooBar distant goldeN ABC compote",
+        "prefix3_rev_prefix2_mixed_case", "Wuthering FooBar distant goldeN ABC compote"));
+    assertU(commit());
+
+    assertQ(req("prefix3_rev_prefix2:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+    assertQ(req("prefix3_rev_prefix2_mixed_IDs:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+    assertQ(req("prefix3_rev_prefix2_mixed_case:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d91273dd/solr/solr-ref-guide/src/filter-descriptions.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/filter-descriptions.adoc b/solr/solr-ref-guide/src/filter-descriptions.adoc
index 5b10cd0..6201902 100644
--- a/solr/solr-ref-guide/src/filter-descriptions.adoc
+++ b/solr/solr-ref-guide/src/filter-descriptions.adoc
@@ -1112,6 +1112,53 @@ This filter applies the Porter Stemming Algorithm for English. The results are s
 
 *Out:* "jump", "jump", "jump"
 
+== Protected Term Filter
+
+This filter enables a form of conditional filtering: it only applies its wrapped filters to terms that are *not contained* in a protected set.
+
+*Factory class:* `solr.ProtectedTermFilterFactory`
+
+*Arguments:*
+
+`protected`:: (required) Comma-separated list of files containing protected terms, one per line.
+
+`wrappedFilters`:: (required) Case-insensitive comma-separated list of `TokenFilterFactory` SPI names (strip trailing `(Token)FilterFactory` from the factory name - see the https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html[java.util.ServiceLoader interface]).  Each filter name must be unique, so if you need to specify the same filter more than once, you must add case-insensitive unique `-id` suffixes to each same-SPI-named filter (note that the `-id` suffix is stripped prior to SPI lookup).
+
+`ignoreCase`:: (true/false, default false) Ignore case when testing for protected words. If true, the protected list should contain lowercase words.
+
+*Example:*
+
+All terms except those in `protectedTerms.txt` are truncated at 4 characters and lowercased:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+  <filter class="solr.ProtectedTermFilterFactory"
+          ignoreCase="true" protected="protectedTerms.txt"
+          wrappedFilters="truncate,lowercase"
+          truncate.prefixLength="4"/>
+</analyzer>
+----
+
+*Example:*
+
+This example includes multiple same-named wrapped filters with unique `-id` suffixes.  Note that both the filter SPI names and `-id` suffixes are treated case-insensitively.
+
+For all terms except those in `protectedTerms.txt`, synonyms are added, terms are reversed, and then synonyms are added for the reversed terms:
+
+[source,xml]
+----
+<analyzer type="query">
+  <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+  <filter class="solr.ProtectedTermFilterFactory"
+          ignoreCase="true" protected="protectedTerms.txt"
+          wrappedFilters="SynonymGraph-fwd,ReverseString,SynonymGraph-rev"
+          synonymgraph-FWD.synonyms="fwd-syns.txt"
+          synonymgraph-FWD.synonyms="rev-syns.txt"/>
+</analyzer>
+----
+
 == Remove Duplicates Token Filter
 
 The filter removes duplicate tokens in the stream. Tokens are considered to be duplicates ONLY if they have the same text and position values.


[2/2] lucene-solr:branch_7x: LUCENE-8273: Rename TermExclusionFilter -> ProtectedTermFilter. Allow ProtectedTermFilterFactory to be used outside of CustomAnalyzer, including in Solr, by allowing wrapped filters and their parameters to be specified on co

Posted by sa...@apache.org.
LUCENE-8273: Rename TermExclusionFilter -> ProtectedTermFilter.  Allow ProtectedTermFilterFactory to be used outside of CustomAnalyzer, including in Solr, by allowing wrapped filters and their parameters to be specified on construction.  Add tests for ProtectedTermFilterFactory in lucene/common/analysis/ and in solr/core/.  Add Solr ref guide documentation for ProtectedTermFilterFactory.  Improve javadocs for CustomAnalyzer, ConditionalTokenFilter, and ProtectedTermFilter.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/97490ac7
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/97490ac7
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/97490ac7

Branch: refs/heads/branch_7x
Commit: 97490ac7c66313625c9526a466ab96981e249746
Parents: 1e661ed
Author: Steve Rowe <sa...@apache.org>
Authored: Sun May 20 19:52:07 2018 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Sun May 20 19:52:39 2018 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +-
 .../lucene/analysis/custom/CustomAnalyzer.java  |   2 +-
 .../miscellaneous/ConditionalTokenFilter.java   |  14 +-
 .../miscellaneous/ProtectedTermFilter.java      |  53 ++++++
 .../ProtectedTermFilterFactory.java             | 163 +++++++++++++++++++
 .../miscellaneous/TermExclusionFilter.java      |  52 ------
 .../TermExclusionFilterFactory.java             |  58 -------
 .../analysis/util/AbstractAnalysisFactory.java  |  20 ++-
 ...ache.lucene.analysis.util.TokenFilterFactory |   2 +-
 .../analysis/miscellaneous/protected-1.txt      |  17 ++
 .../analysis/miscellaneous/protected-2.txt      |  17 ++
 .../analysis/custom/TestCustomAnalyzer.java     |   2 +-
 .../TestConditionalTokenFilter.java             |  14 +-
 .../miscellaneous/TestProtectedTermFilter.java  |  48 ++++++
 .../TestProtectedTermFilterFactory.java         | 128 +++++++++++++++
 .../miscellaneous/TestTermExclusionFilter.java  |  48 ------
 .../solr/collection1/conf/protected-1.txt       |  17 ++
 .../solr/collection1/conf/protected-2.txt       |  17 ++
 .../collection1/conf/schema-protected-term.xml  |  86 ++++++++++
 .../ProtectedTermFilterFactoryTest.java         |  84 ++++++++++
 .../solr-ref-guide/src/filter-descriptions.adoc |  47 ++++++
 21 files changed, 712 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a2d7740..f9fe92a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -85,8 +85,8 @@ New Features
   filters based on the attributes of the current token. This generalises the keyword
   token logic currently used for stemmers and WDF.  It is integrated into 
   CustomAnalyzer by using the `when` and `whenTerm` builder methods, and a new
-  TermExclusionConditionalFilter is added as an example.  (Alan Woodward,
-  Robert Muir, David Smiley, Steve Rowe, Mike Sokolov)
+  ProtectedTermFilter is added as an example.  (Alan Woodward, Robert Muir,
+  David Smiley, Steve Rowe, Mike Sokolov)
 
 * LUCENE-8310: Ensure IndexFileDeleter accounts for pending deletes. Today we fail
   creating the IndexWriter when the directory has a pending delete. Yet, this

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index 72614ca..19e207f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -87,7 +87,7 @@ import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassI
  *    .whenTerm(t -&gt; t.length() &gt; 10)
  *      .addTokenFilter(&quot;reversestring&quot;)
  *    .endwhen()
- *    .build()
+ *    .build();
  * </pre>
  */
 public final class CustomAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
index c8b91dc..6f9ea24 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
@@ -29,10 +29,10 @@ import org.apache.lucene.util.AttributeSource;
 /**
  * Allows skipping TokenFilters based on the current set of attributes.
  *
- * To use, implement the {@link #shouldFilter()} method.  If it returns {@code false},
- * then calling {@link #incrementToken()} will use the wrapped TokenFilter to
- * make changes to the tokenstream.  If it returns {@code true}, then the wrapped
- * filter will be skipped
+ * To use, implement the {@link #shouldFilter()} method.  If it returns {@code true},
+ * then calling {@link #incrementToken()} will use the wrapped TokenFilter(s) to
+ * make changes to the tokenstream.  If it returns {@code false}, then the wrapped
+ * filter(s) will be skipped.
  */
 public abstract class ConditionalTokenFilter extends TokenFilter {
 
@@ -102,9 +102,9 @@ public abstract class ConditionalTokenFilter extends TokenFilter {
   private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
 
   /**
-   * Create a new BypassingTokenFilter
+   * Create a new ConditionalTokenFilter
    * @param input         the input TokenStream
-   * @param inputFactory  a factory function to create a new instance of the TokenFilter to wrap
+   * @param inputFactory  a factory function to create the wrapped filter(s)
    */
   protected ConditionalTokenFilter(TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
     super(input);
@@ -112,7 +112,7 @@ public abstract class ConditionalTokenFilter extends TokenFilter {
   }
 
   /**
-   * Whether or not to execute the wrapped TokenFilter for the current token
+   * Whether or not to execute the wrapped TokenFilter(s) for the current token
    */
   protected abstract boolean shouldFilter() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
new file mode 100644
index 0000000..e4e21be
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilter.java
@@ -0,0 +1,53 @@
+/*
+ * 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.miscellaneous;
+
+import java.util.function.Function;
+
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+/**
+ * A ConditionalTokenFilter that only applies its wrapped filters to tokens that
+ * are not contained in a protected set.
+ */
+public class ProtectedTermFilter extends ConditionalTokenFilter {
+
+  private final CharArraySet protectedTerms;
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  /**
+   * Creates a new ProtectedTermFilter
+   * @param protectedTerms  the set of terms to skip the wrapped filters for
+   * @param input         the input TokenStream
+   * @param inputFactory  a factory function to create the wrapped filter(s)
+   */
+  public ProtectedTermFilter(final CharArraySet protectedTerms, TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
+    super(input, inputFactory);
+    this.protectedTerms = protectedTerms;
+  }
+
+  @Override
+  protected boolean shouldFilter() {
+    boolean b = protectedTerms.contains(termAtt.buffer(), 0, termAtt.length());
+    return b == false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
new file mode 100644
index 0000000..1cde6c3
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
@@ -0,0 +1,163 @@
+/*
+ * 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.miscellaneous;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate; // javadocs
+
+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;
+
+/**
+ * Factory for a {@link ProtectedTermFilter}
+ *
+ * <p>CustomAnalyzer example:
+ * <pre class="prettyprint">
+ * Analyzer ana = CustomAnalyzer.builder()
+ *   .withTokenizer("standard")
+ *   .when("protectedterm", "ignoreCase", "true", "protected", "protectedTerms.txt")
+ *     .addTokenFilter("truncate", "prefixLength", "4")
+ *     .addTokenFilter("lowercase")
+ *   .endwhen()
+ *   .build();
+ * </pre>
+ *
+ * <p>Solr example, in which conditional filters are specified via the <code>wrappedFilters</code>
+ * parameter - a comma-separated list of case-insensitive TokenFilter SPI names - and conditional
+ * filter args are specified via <code>filterName.argName</code> parameters:
+ * <pre class="prettyprint">
+ * &lt;fieldType name="reverse_lower_with_exceptions" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.ProtectedTermFilterFactory" ignoreCase="true" protected="protectedTerms.txt"
+ *             wrappedFilters="truncate,lowercase" truncate.prefixLength="4" /&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ *
+ * <p>When using the <code>wrappedFilters</code> parameter, each filter name must be unique, so if you
+ * need to specify the same filter more than once, you must add case-insensitive unique '-id' suffixes
+ * (note that the '-id' suffix is stripped prior to SPI lookup), e.g.:
+ * <pre class="prettyprint">
+ * &lt;fieldType name="double_synonym_with_exceptions" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.ProtectedTermFilterFactory" ignoreCase="true" protected="protectedTerms.txt"
+ *             wrappedFilters="synonymgraph-A,synonymgraph-B"
+ *             synonymgraph-A.synonyms="synonyms-1.txt"
+ *             synonymgraph-B.synonyms="synonyms-2.txt"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ *
+ * <p>See related {@link org.apache.lucene.analysis.custom.CustomAnalyzer.Builder#whenTerm(Predicate)}
+ */
+public class ProtectedTermFilterFactory extends ConditionalTokenFilterFactory implements ResourceLoaderAware {
+
+  public static final String PROTECTED_TERMS = "protected";
+  public static final char FILTER_ARG_SEPARATOR = '.';
+  public static final char FILTER_NAME_ID_SEPARATOR = '-';
+
+  private final String termFiles;
+  private final boolean ignoreCase;
+  private final String wrappedFilters;
+
+  private CharArraySet protectedTerms;
+
+  public ProtectedTermFilterFactory(Map<String, String> args) {
+    super(args);
+    termFiles = require(args, PROTECTED_TERMS);
+    ignoreCase = getBoolean(args, "ignoreCase", false);
+    wrappedFilters = get(args, "wrappedFilters");
+    if (wrappedFilters != null) {
+      handleWrappedFilterArgs(args);
+    }
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  private void handleWrappedFilterArgs(Map<String, String> args) {
+    LinkedHashMap<String, Map<String, String>> wrappedFilterArgs = new LinkedHashMap<>();
+    splitAt(',', wrappedFilters).forEach(filterName -> {          // Format: SPIname[-id]
+      filterName = filterName.trim().toLowerCase(Locale.ROOT);             // Treat case-insensitively
+      if (wrappedFilterArgs.containsKey(filterName)) {
+        throw new IllegalArgumentException("wrappedFilters contains duplicate '"
+            + filterName + "'. Add unique '-id' suffixes (stripped prior to SPI lookup).");
+      }
+      wrappedFilterArgs.put(filterName, new HashMap<>());
+    });
+    for (Iterator<Map.Entry<String, String>> iterator = args.entrySet().iterator(); iterator.hasNext(); ) {
+      Map.Entry<String, String> entry = iterator.next();
+      String filterArgKey = entry.getKey();
+      String argValue = entry.getValue();
+      List<String> splitKey = splitAt(FILTER_ARG_SEPARATOR, filterArgKey); // Format: filterName.argKey
+      if (splitKey.size() == 2) {                                          // Skip if no slash
+        String filterName = splitKey.get(0).toLowerCase(Locale.ROOT);
+        if (wrappedFilterArgs.containsKey(filterName)) {                   // Skip if not in "wrappedFilter" arg
+          Map<String, String> filterArgs = wrappedFilterArgs.computeIfAbsent(filterName, k -> new HashMap<>());
+          String argKey = splitKey.get(1);
+          filterArgs.put(argKey, argValue); // argKey is guaranteed unique, don't need to check for duplicates
+          iterator.remove();
+        }
+      }
+    }
+    if (args.isEmpty()) {
+      populateInnerFilters(wrappedFilterArgs);
+    }
+  }
+
+  private void populateInnerFilters(LinkedHashMap<String, Map<String, String>> wrappedFilterArgs) {
+    List<TokenFilterFactory> innerFilters = new ArrayList<>();
+    wrappedFilterArgs.forEach((filterName, filterArgs) -> {
+      int idSuffixPos = filterName.indexOf(FILTER_NAME_ID_SEPARATOR); // Format: SPIname[-id]
+      if (idSuffixPos != -1) {                                        // Strip '-id' suffix, if any, prior to SPI lookup
+        filterName = filterName.substring(0, idSuffixPos);
+      }
+      innerFilters.add(TokenFilterFactory.forName(filterName, filterArgs));
+    });
+    setInnerFilters(innerFilters);
+  }
+
+  public boolean isIgnoreCase() {
+    return ignoreCase;
+  }
+
+  public CharArraySet getProtectedTerms() {
+    return protectedTerms;
+  }
+
+  @Override
+  protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
+    return new ProtectedTermFilter(protectedTerms, input, inner);
+  }
+
+  @Override
+  public void doInform(ResourceLoader loader) throws IOException {
+    protectedTerms = getWordSet(loader, termFiles, ignoreCase);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
deleted file mode 100644
index 95bf55f..0000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.util.function.Function;
-
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-
-/**
- * A ConditionalTokenFilter that only applies its wrapped filters to tokens that
- * are not contained in an exclusion set.
- */
-public class TermExclusionFilter extends ConditionalTokenFilter {
-
-  private final CharArraySet excludeTerms;
-
-  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-
-  /**
-   * Creates a new TermExclusionFilter
-   * @param excludeTerms  the set of terms to skip the wrapped filters for
-   * @param input         the input TokenStream
-   * @param inputFactory  a factory function to create the wrapped filters
-   */
-  public TermExclusionFilter(final CharArraySet excludeTerms, TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
-    super(input, inputFactory);
-    this.excludeTerms = excludeTerms;
-  }
-
-  @Override
-  protected boolean shouldFilter() {
-    return excludeTerms.contains(termAtt.buffer(), 0, termAtt.length()) == false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
deleted file mode 100644
index f860b72..0000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.function.Function;
-
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.util.ResourceLoader;
-
-/**
- * Factory for a {@link TermExclusionFilter}
- */
-public class TermExclusionFilterFactory extends ConditionalTokenFilterFactory {
-
-  public static final String EXCLUDED_TOKENS = "protected";
-
-  private final String wordFiles;
-  private final boolean ignoreCase;
-
-  private CharArraySet excludeTerms;
-
-  public TermExclusionFilterFactory(Map<String, String> args) {
-    super(args);
-    wordFiles = get(args, EXCLUDED_TOKENS);
-    ignoreCase = getBoolean(args, "ignoreCase", false);
-    if (!args.isEmpty()) {
-      throw new IllegalArgumentException("Unknown parameters: " + args);
-    }
-  }
-
-  @Override
-  protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
-    return new TermExclusionFilter(excludeTerms, input, inner);
-  }
-
-  @Override
-  public void doInform(ResourceLoader loader) throws IOException {
-    excludeTerms = getWordSet(loader, wordFiles, ignoreCase);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
index f8de8a7..954f300 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
@@ -290,12 +290,24 @@ public abstract class AbstractAnalysisFactory {
    * @return a list of file names with the escaping backslashed removed
    */
   protected final List<String> splitFileNames(String fileNames) {
-    if (fileNames == null)
-      return Collections.<String>emptyList();
+    return splitAt(',', fileNames);
+  }
+
+  /**
+   * Splits a list separated by zero or more given separator characters.
+   * List items can contain comma characters escaped by backslash '\'.
+   * Whitespace is NOT trimmed from the returned list items.
+   *
+   * @param list the string containing the split list items
+   * @return a list of items with the escaping backslashes removed
+   */
+  protected final List<String> splitAt(char separator, String list) {
+    if (list == null)
+      return Collections.emptyList();
 
     List<String> result = new ArrayList<>();
-    for (String file : fileNames.split("(?<!\\\\),")) {
-      result.add(file.replaceAll("\\\\(?=,)", ""));
+    for (String item : list.split("(?<!\\\\)[" + separator + "]")) {
+      result.add(item.replaceAll("\\\\(?=[" + separator + "])", ""));
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index ddd3c91..1811920 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -78,7 +78,7 @@ org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilterFactory
 org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilterFactory
 org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilterFactory
-org.apache.lucene.analysis.miscellaneous.TermExclusionFilterFactory
+org.apache.lucene.analysis.miscellaneous.ProtectedTermFilterFactory
 org.apache.lucene.analysis.miscellaneous.TrimFilterFactory
 org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.TypeAsSynonymFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-1.txt
new file mode 100644
index 0000000..3fe6d02
--- /dev/null
+++ b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-2.txt
new file mode 100644
index 0000000..d9a5c3c
--- /dev/null
+++ b/lucene/analysis/common/src/resources/org/apache/lucene/analysis/miscellaneous/protected-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.
+
+golden
+compote

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index b963812..7583884 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -520,7 +520,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     CustomAnalyzer analyzer = CustomAnalyzer.builder()
         .withTokenizer("whitespace")
         .addTokenFilter("lowercase")
-        .when("termexclusion", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
+        .when("protectedterm", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
           .addTokenFilter("reversestring")
         .endwhen()
         .build();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
index 02d8a78..fed7f68 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
@@ -248,14 +248,14 @@ public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
 
   public void testReadaheadWithFiltering() throws IOException {
 
-    CharArraySet exclusions = new CharArraySet(2, true);
-    exclusions.add("three");
+    CharArraySet protectedTerms = new CharArraySet(2, true);
+    protectedTerms.add("three");
 
     Analyzer analyzer = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
         Tokenizer source = new ClassicTokenizer();
-        TokenStream sink = new TermExclusionFilter(exclusions, source, in -> new ShingleFilter(in, 2));
+        TokenStream sink = new ProtectedTermFilter(protectedTerms, source, in -> new ShingleFilter(in, 2));
         return new TokenStreamComponents(source, sink);
       }
     };
@@ -287,15 +287,15 @@ public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
 
   public void testFilteredTokenFilters() throws IOException {
 
-    CharArraySet exclusions = new CharArraySet(2, true);
-    exclusions.add("foobar");
+    CharArraySet protectedTerms = new CharArraySet(2, true);
+    protectedTerms.add("foobar");
 
     TokenStream ts = whitespaceMockTokenizer("wuthering foobar abc");
-    ts = new TermExclusionFilter(exclusions, ts, in -> new LengthFilter(in, 1, 4));
+    ts = new ProtectedTermFilter(protectedTerms, ts, in -> new LengthFilter(in, 1, 4));
     assertTokenStreamContents(ts, new String[]{ "foobar", "abc" });
 
     ts = whitespaceMockTokenizer("foobar abc");
-    ts = new TermExclusionFilter(exclusions, ts, in -> new LengthFilter(in, 1, 4));
+    ts = new ProtectedTermFilter(protectedTerms, ts, in -> new LengthFilter(in, 1, 4));
     assertTokenStreamContents(ts, new String[]{ "foobar", "abc" });
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
new file mode 100644
index 0000000..4c64f56
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.miscellaneous;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.LowerCaseFilter;
+
+public class TestProtectedTermFilter extends BaseTokenStreamTestCase {
+
+  public void testBasic() throws IOException {
+
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("Alice", 1, 0, 5),
+        new Token("Bob", 1, 6, 9),
+        new Token("Clara", 1, 10, 15),
+        new Token("David", 1, 16, 21)
+    );
+
+    CharArraySet protectedTerms = new CharArraySet(5, true);
+    protectedTerms.add("bob");
+
+    TokenStream ts = new ProtectedTermFilter(protectedTerms, cts, LowerCaseFilter::new);
+    assertTokenStreamContents(ts, new String[]{ "alice", "Bob", "clara", "david" });
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
new file mode 100644
index 0000000..a1aac68
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestProtectedTermFilterFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.miscellaneous;
+
+
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
+
+/**
+ * Simple tests to ensure the simple truncation filter factory is working.
+ */
+public class TestProtectedTermFilterFactory extends BaseTokenStreamFactoryTestCase {
+  public void testInform() throws Exception {
+    ProtectedTermFilterFactory factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt", "ignoreCase", "true", "wrappedFilters", "lowercase");
+    CharArraySet protectedTerms = factory.getProtectedTerms();
+    assertTrue("protectedTerms is null and it shouldn't be", protectedTerms != null);
+    assertTrue("protectedTerms Size: " + protectedTerms.size() + " is not: " + 2, protectedTerms.size() == 2);
+    assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory.isIgnoreCase() == true);
+
+    factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt, protected-2.txt", "ignoreCase", "true", "wrappedFilters", "lowercase");
+    protectedTerms = factory.getProtectedTerms();
+    assertTrue("protectedTerms is null and it shouldn't be", protectedTerms != null);
+    assertTrue("protectedTerms Size: " + protectedTerms.size() + " is not: " + 4, protectedTerms.size() == 4);
+    assertTrue(factory.isIgnoreCase() + " does not equal: " + true, factory.isIgnoreCase() == true);
+
+    // defaults
+    factory = (ProtectedTermFilterFactory)tokenFilterFactory("ProtectedTerm",
+        "protected", "protected-1.txt");
+    assertEquals(false, factory.isIgnoreCase());
+  }
+
+  public void testBasic() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "lowercase").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "clara", "Bar", "david"});
+  }
+
+  public void testMultipleWrappedFiltersWithParams() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "lowercase, truncate",
+        "truncate.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "cl", "Bar", "da"});
+  }
+
+  public void testMultipleSameNamedFiltersWithParams() throws Exception {
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "truncate-A, reversestring, truncate-B",
+        "truncate-A.prefixLength", "3", "truncate-B.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+
+    // same-named wrapped filters, one with an ID and another without
+    stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "truncate, reversestring, truncate-A",
+        "truncate.prefixLength", "3", "truncate-A.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+
+    // Case-insensitive wrapped "filter-id"
+    stream = whitespaceMockTokenizer(str);
+    stream = tokenFilterFactory("ProtectedTerm", "ignoreCase", "true",
+        "protected", "protected-1.txt", "wrappedFilters", "TRUNCATE-a, reversestring, truncate-b",
+        "truncate-A.prefixLength", "3", "TRUNCATE-B.prefixLength", "2").create(stream);
+    assertTokenStreamContents(stream, new String[]{"Foo", "al", "Bar", "va"});
+  }
+
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () ->
+      tokenFilterFactory("ProtectedTerm", "protected", "protected-1.txt", "bogusArg", "bogusValue"));
+    assertTrue(exception.getMessage().contains("Unknown parameters"));
+
+    // same-named wrapped filters
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate, truncate"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // case-insensitive same-named wrapped filters
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "TRUNCATE, truncate"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // case-insensitive same-named wrapped filter IDs
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate-ABC, truncate-abc"));
+    assertTrue(exception.getMessage().contains("wrappedFilters contains duplicate"));
+
+    // mismatched wrapped filter and associated args
+    exception = expectThrows(IllegalArgumentException.class, () ->
+        tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "truncate-A, reversestring, truncate-B",
+            "truncate.prefixLength", "3", "truncate-A.prefixLength", "2"));
+    assertTrue(exception.getMessage().contains("Unknown parameters: {truncate.prefixLength=3}"));
+
+    // missing required arg(s) for wrapped filter
+    String str = "Foo Clara Bar David";
+    TokenStream stream = whitespaceMockTokenizer(str);
+    exception = expectThrows(IllegalArgumentException.class, () ->
+      tokenFilterFactory("ProtectedTerm",
+            "protected", "protected-1.txt", "wrappedFilters", "length").create(stream));
+    assertTrue(exception.getMessage().contains("Configuration Error: missing parameter"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
deleted file mode 100644
index 2e6aecf..0000000
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.miscellaneous;
-
-import java.io.IOException;
-
-import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.CannedTokenStream;
-import org.apache.lucene.analysis.CharArraySet;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.core.LowerCaseFilter;
-
-public class TestTermExclusionFilter extends BaseTokenStreamTestCase {
-
-  public void testExcludeTerms() throws IOException {
-
-    CannedTokenStream cts = new CannedTokenStream(
-        new Token("Alice", 1, 0, 5),
-        new Token("Bob", 1, 6, 9),
-        new Token("Clara", 1, 10, 15),
-        new Token("David", 1, 16, 21)
-    );
-
-    CharArraySet exclusions = new CharArraySet(5, true);
-    exclusions.add("bob");
-
-    TokenStream ts = new TermExclusionFilter(exclusions, cts, LowerCaseFilter::new);
-    assertTokenStreamContents(ts, new String[]{ "alice", "Bob", "clara", "david" });
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/solr/core/src/test-files/solr/collection1/conf/protected-1.txt
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/protected-1.txt b/solr/core/src/test-files/solr/collection1/conf/protected-1.txt
new file mode 100644
index 0000000..61c0583
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/protected-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.
+
+foobar
+jaxfopbuz

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/solr/core/src/test-files/solr/collection1/conf/protected-2.txt
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/protected-2.txt b/solr/core/src/test-files/solr/collection1/conf/protected-2.txt
new file mode 100644
index 0000000..d9a5c3c
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/protected-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.
+
+golden
+compote

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml b/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
new file mode 100644
index 0000000..2294394
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-protected-term.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<!-- The Solr schema file, version 1.6  -->
+
+<schema name="schema-protected-term" version="1.6">
+  <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
+  <field name="signatureField" type="string" indexed="true" stored="false"/>
+  <dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
+
+  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
+  <field name="_version_" type="long" indexed="false" stored="false" docValues="true"/>
+
+  <fieldType name="prefix4_lower_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate,lowercase" truncate.prefixLength="4"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix4_lower" type="prefix4_lower_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate-A, reversestring, truncate-B"
+              truncate-A.prefixLength="3" truncate-B.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2" type="prefix3_rev_prefix2_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_mixed_IDs_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="truncate, reversestring, truncate-A"
+              truncate.prefixLength="3" truncate-A.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2_mixed_IDs" type="prefix3_rev_prefix2_mixed_IDs_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <fieldType name="prefix3_rev_prefix2_mixed_case_with_exceptions" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.ProtectedTermFilterFactory" ignoreCase="true"
+              protected="protected-1.txt,protected-2.txt"
+              wrappedFilters="TRUNCATE-a, reversestring, truncate-b"
+              truncate-A.prefixLength="3" TRUNCATE-B.prefixLength="2"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.MockTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+  <field name="prefix3_rev_prefix2_mixed_case" type="prefix3_rev_prefix2_mixed_case_with_exceptions" indexed="true" stored="true" multiValued="true"/>
+
+  <uniqueKey>id</uniqueKey>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java b/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
new file mode 100644
index 0000000..5c776dd
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/analysis/ProtectedTermFilterFactoryTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.solr.analysis;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.miscellaneous.ProtectedTermFilterFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrResourceLoader;
+import org.junit.BeforeClass;
+
+public class ProtectedTermFilterFactoryTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml","schema-protected-term.xml");
+  }
+
+  public void testBasic() throws Exception {
+    String text = "Wuthering FooBar distant goldeN ABC compote";
+    Map<String,String> args = new HashMap<>();
+    args.put("ignoreCase", "true");
+    args.put("protected", "protected-1.txt,protected-2.txt");  // Protected: foobar, jaxfopbuz, golden, compote
+    args.put("wrappedFilters", "lowercase");
+
+    ResourceLoader loader = new SolrResourceLoader(TEST_PATH().resolve("collection1"));
+    ProtectedTermFilterFactory factory = new ProtectedTermFilterFactory(args);
+    factory.inform(loader);
+
+    TokenStream ts = factory.create(whitespaceMockTokenizer(text));
+    BaseTokenStreamTestCase.assertTokenStreamContents(ts,
+        new String[] { "wuthering", "FooBar", "distant", "goldeN", "abc", "compote" });
+  }
+
+  public void testTwoWrappedFilters() {
+    // Index-time: Filters: truncate:4 & lowercase.  Protected (ignoreCase:true): foobar, jaxfopbuz, golden, compote
+    // Query-time: No filters
+    assertU(adoc("id", "1", "prefix4_lower", "Wuthering FooBar distant goldeN ABC compote"));
+    assertU(commit());
+
+    assertQ(req("prefix4_lower:(+wuth +FooBar +dist +goldeN +abc +compote)")
+        , "//result[@numFound=1]"
+    );
+  }
+
+  public void testDuplicateFilters() {
+    // Index-time: Filters: truncate:3 & reversestring & truncate:2.  Protected (ignoreCase:true): foobar, jaxfopbuz, golden, compote
+    // Query-time: No filters
+    assertU(adoc("id", "1",
+        "prefix3_rev_prefix2",            "Wuthering FooBar distant goldeN ABC compote",
+        "prefix3_rev_prefix2_mixed_IDs",  "Wuthering FooBar distant goldeN ABC compote",
+        "prefix3_rev_prefix2_mixed_case", "Wuthering FooBar distant goldeN ABC compote"));
+    assertU(commit());
+
+    assertQ(req("prefix3_rev_prefix2:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+    assertQ(req("prefix3_rev_prefix2_mixed_IDs:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+    assertQ(req("prefix3_rev_prefix2_mixed_case:(+tu +FooBar +si +goldeN +CB +compote)")
+        , "//result[@numFound=1]"
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97490ac7/solr/solr-ref-guide/src/filter-descriptions.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/filter-descriptions.adoc b/solr/solr-ref-guide/src/filter-descriptions.adoc
index 5b10cd0..6201902 100644
--- a/solr/solr-ref-guide/src/filter-descriptions.adoc
+++ b/solr/solr-ref-guide/src/filter-descriptions.adoc
@@ -1112,6 +1112,53 @@ This filter applies the Porter Stemming Algorithm for English. The results are s
 
 *Out:* "jump", "jump", "jump"
 
+== Protected Term Filter
+
+This filter enables a form of conditional filtering: it only applies its wrapped filters to terms that are *not contained* in a protected set.
+
+*Factory class:* `solr.ProtectedTermFilterFactory`
+
+*Arguments:*
+
+`protected`:: (required) Comma-separated list of files containing protected terms, one per line.
+
+`wrappedFilters`:: (required) Case-insensitive comma-separated list of `TokenFilterFactory` SPI names (strip trailing `(Token)FilterFactory` from the factory name - see the https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html[java.util.ServiceLoader interface]).  Each filter name must be unique, so if you need to specify the same filter more than once, you must add case-insensitive unique `-id` suffixes to each same-SPI-named filter (note that the `-id` suffix is stripped prior to SPI lookup).
+
+`ignoreCase`:: (true/false, default false) Ignore case when testing for protected words. If true, the protected list should contain lowercase words.
+
+*Example:*
+
+All terms except those in `protectedTerms.txt` are truncated at 4 characters and lowercased:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+  <filter class="solr.ProtectedTermFilterFactory"
+          ignoreCase="true" protected="protectedTerms.txt"
+          wrappedFilters="truncate,lowercase"
+          truncate.prefixLength="4"/>
+</analyzer>
+----
+
+*Example:*
+
+This example includes multiple same-named wrapped filters with unique `-id` suffixes.  Note that both the filter SPI names and `-id` suffixes are treated case-insensitively.
+
+For all terms except those in `protectedTerms.txt`, synonyms are added, terms are reversed, and then synonyms are added for the reversed terms:
+
+[source,xml]
+----
+<analyzer type="query">
+  <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+  <filter class="solr.ProtectedTermFilterFactory"
+          ignoreCase="true" protected="protectedTerms.txt"
+          wrappedFilters="SynonymGraph-fwd,ReverseString,SynonymGraph-rev"
+          synonymgraph-FWD.synonyms="fwd-syns.txt"
+          synonymgraph-FWD.synonyms="rev-syns.txt"/>
+</analyzer>
+----
+
 == Remove Duplicates Token Filter
 
 The filter removes duplicate tokens in the stream. Tokens are considered to be duplicates ONLY if they have the same text and position values.