You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2018/05/11 10:35:08 UTC

[1/2] lucene-solr:branch_7x: LUCENE-8273: Add ConditionalTokenFilter

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x bd2f4efb1 -> 7f13e5642
  refs/heads/master 8a697ee09 -> 1ce3ebadb


LUCENE-8273: Add ConditionalTokenFilter


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

Branch: refs/heads/branch_7x
Commit: 7f13e5642924aa7f97e21b059553cfd6b5b56c21
Parents: bd2f4ef
Author: Alan Woodward <ro...@apache.org>
Authored: Fri May 11 11:11:21 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Fri May 11 11:34:37 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   7 +
 .../lucene/analysis/custom/CustomAnalyzer.java  | 202 +++++++++++++++-
 .../miscellaneous/ConditionalTokenFilter.java   | 149 ++++++++++++
 .../ConditionalTokenFilterFactory.java          |  85 +++++++
 .../miscellaneous/TermExclusionFilter.java      |  52 +++++
 .../TermExclusionFilterFactory.java             |  58 +++++
 ...ache.lucene.analysis.util.TokenFilterFactory |   1 +
 .../lucene/analysis/core/TestRandomChains.java  |  53 ++++-
 .../analysis/custom/TestCustomAnalyzer.java     |  44 ++++
 .../TestConditionalTokenFilter.java             | 233 +++++++++++++++++++
 .../miscellaneous/TestTermExclusionFilter.java  |  48 ++++
 11 files changed, 920 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8dc3076..e4da240 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -77,6 +77,13 @@ New Features
   provided document. This allows to undelete a soft-deleted document unless it's been claimed
   by a merge. (Simon Willnauer) 
 
+* LUCENE-8273: ConditionalTokenFilter allows analysis chains to skip particular token
+  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)
+
 Bug Fixes
 
 * LUCENE-8266: Detect bogus tiles when creating a standard polygon

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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 a697cce..72614ca 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
@@ -17,8 +17,6 @@
 package org.apache.lucene.analysis.custom;
 
 
-import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassInstance;
-
 import java.io.IOException;
 import java.io.Reader;
 import java.nio.file.Path;
@@ -29,10 +27,15 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilterFactory;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
 import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.ClasspathResourceLoader;
@@ -45,6 +48,8 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.util.SetOnce;
 import org.apache.lucene.util.Version;
 
+import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassInstance;
+
 /**
  * A general-purpose Analyzer that can be created with a builder-style API.
  * Under the hood it uses the factory classes {@link TokenizerFactory},
@@ -73,6 +78,17 @@ import org.apache.lucene.util.Version;
  * <p>The list of names to be used for components can be looked up through:
  * {@link TokenizerFactory#availableTokenizers()}, {@link TokenFilterFactory#availableTokenFilters()},
  * and {@link CharFilterFactory#availableCharFilters()}.
+ * <p>You can create conditional branches in the analyzer by using {@link Builder#when(String, String...)} and
+ * {@link Builder#whenTerm(Predicate)}:
+ * <pre class="prettyprint">
+ * Analyzer ana = CustomAnalyzer.builder()
+ *    .withTokenizer(&quot;standard&quot;)
+ *    .addTokenFilter(&quot;lowercase&quot;)
+ *    .whenTerm(t -&gt; t.length() &gt; 10)
+ *      .addTokenFilter(&quot;reversestring&quot;)
+ *    .endwhen()
+ *    .build()
+ * </pre>
  */
 public final class CustomAnalyzer extends Analyzer {
   
@@ -335,6 +351,13 @@ public final class CustomAnalyzer extends Analyzer {
       componentsAdded = true;
       return this;
     }
+
+    private Builder addTokenFilter(TokenFilterFactory factory) {
+      Objects.requireNonNull(factory, "TokenFilterFactory may not be null");
+      tokenFilters.add(factory);
+      componentsAdded = true;
+      return this;
+    }
     
     /** Adds the given char filter.
      * @param factory class that is used to create the char filter.
@@ -377,6 +400,117 @@ public final class CustomAnalyzer extends Analyzer {
       componentsAdded = true;
       return this;
     }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param name    is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}
+     * @param params  the parameters to be passed to the factory
+     */
+    public ConditionBuilder when(String name, String... params) throws IOException {
+      return when(name, paramsToMap(params));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param name    is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}
+     * @param params  the parameters to be passed to the factory.  The map must be modifiable
+     */
+    @SuppressWarnings("unchecked")
+    public ConditionBuilder when(String name, Map<String, String> params) throws IOException {
+      Class<? extends TokenFilterFactory> clazz = TokenFilterFactory.lookupClass(name);
+      if (ConditionalTokenFilterFactory.class.isAssignableFrom(clazz) == false) {
+        throw new IllegalArgumentException("TokenFilterFactory " + name + " is not a ConditionalTokenFilterFactory");
+      }
+      return when((Class<? extends ConditionalTokenFilterFactory>) clazz, params);
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param factory class that is used to create the ConditionalTokenFilter
+     * @param params  the parameters to be passed to the factory
+     */
+    public ConditionBuilder when(Class<? extends ConditionalTokenFilterFactory> factory, String... params) throws IOException {
+      return when(factory, paramsToMap(params));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param factory class that is used to create the ConditionalTokenFilter
+     * @param params  the parameters to be passed to the factory.  The map must be modifiable
+     */
+    public ConditionBuilder when(Class<? extends ConditionalTokenFilterFactory> factory, Map<String, String> params) throws IOException {
+      return when(newFactoryClassInstance(factory, applyDefaultParams(params)));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     */
+    public ConditionBuilder when(ConditionalTokenFilterFactory factory) {
+      return new ConditionBuilder(factory, this);
+    }
+
+    /**
+     * Apply subsequent token filters if the current token's term matches a predicate
+     *
+     * This is the equivalent of:
+     * <pre>
+     *   when(new ConditionalTokenFilterFactory(Collections.emptyMap()) {
+     *      {@code @}Override
+     *      protected ConditionalTokenFilter create(TokenStream input, Function&lt;TokenStream, TokenStream&gt; inner) {
+     *        return new ConditionalTokenFilter(input, inner) {
+     *          CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+     *          {@code @}Override
+     *          protected boolean shouldFilter() {
+     *            return predicate.test(termAtt);
+     *          }
+     *        };
+     *      }
+     *   });
+     * </pre>
+     */
+    public ConditionBuilder whenTerm(Predicate<CharSequence> predicate) {
+      return new ConditionBuilder(new ConditionalTokenFilterFactory(Collections.emptyMap()) {
+        @Override
+        protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
+          return new ConditionalTokenFilter(input, inner) {
+            CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+            @Override
+            protected boolean shouldFilter() {
+              return predicate.test(termAtt);
+            }
+          };
+        }
+      }, this);
+    }
     
     /** Builds the analyzer. */
     public CustomAnalyzer build() {
@@ -412,11 +546,73 @@ public final class CustomAnalyzer extends Analyzer {
       return map;
     }
     
-    private <T> T applyResourceLoader(T factory) throws IOException {
+    <T> T applyResourceLoader(T factory) throws IOException {
       if (factory instanceof ResourceLoaderAware) {
         ((ResourceLoaderAware) factory).inform(loader);
       }
       return factory;
     }
   }
+
+  /**
+   * Factory class for a {@link ConditionalTokenFilter}
+   */
+  public static class ConditionBuilder {
+
+    private final List<TokenFilterFactory> innerFilters = new ArrayList<>();
+    private final ConditionalTokenFilterFactory factory;
+    private final Builder parent;
+
+    private ConditionBuilder(ConditionalTokenFilterFactory factory, Builder parent) {
+      this.factory = factory;
+      this.parent = parent;
+    }
+
+    /** Adds the given token filter.
+     * @param name is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}.
+     *  The list of possible names can be looked up with {@link TokenFilterFactory#availableTokenFilters()}.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(String name, Map<String, String> params) throws IOException {
+      innerFilters.add(TokenFilterFactory.forName(name, parent.applyDefaultParams(params)));
+      return this;
+    }
+
+    /** Adds the given token filter.
+     * @param name is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}.
+     *  The list of possible names can be looked up with {@link TokenFilterFactory#availableTokenFilters()}.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(String name, String... params) throws IOException {
+      return addTokenFilter(name, parent.paramsToMap(params));
+    }
+
+    /** Adds the given token filter.
+     * @param factory class that is used to create the token filter.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(Class<? extends TokenFilterFactory> factory, Map<String, String> params) throws IOException {
+      innerFilters.add(newFactoryClassInstance(factory, parent.applyDefaultParams(params)));
+      return this;
+    }
+
+    /** Adds the given token filter.
+     * @param factory class that is used to create the token filter.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(Class<? extends TokenFilterFactory> factory, String... params) throws IOException {
+      return addTokenFilter(factory, parent.paramsToMap(params));
+    }
+
+    /**
+     * Close the branch and return to the main analysis chain
+     */
+    public Builder endwhen() throws IOException {
+      factory.setInnerFilters(innerFilters);
+      parent.applyResourceLoader(factory);
+      parent.addTokenFilter(factory);
+      return parent;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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
new file mode 100644
index 0000000..f6f2a54
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
@@ -0,0 +1,149 @@
+/*
+ * 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.function.Function;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+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
+ */
+public abstract class ConditionalTokenFilter extends TokenFilter {
+
+  private enum TokenState {
+    READING, PREBUFFERING, BUFFERING, DELEGATING
+  }
+
+  private final class OneTimeWrapper extends TokenStream {
+
+    public OneTimeWrapper(AttributeSource attributeSource) {
+      super(attributeSource);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (state == TokenState.PREBUFFERING) {
+        state = TokenState.BUFFERING;
+        return true;
+      }
+      if (state == TokenState.DELEGATING) {
+        return false;
+      }
+      return ConditionalTokenFilter.this.incrementToken();
+    }
+
+    @Override
+    public void reset() throws IOException {
+      // clearing attributes etc is done by the parent stream,
+      // so must be avoided here
+    }
+
+    @Override
+    public void end() throws IOException {
+      // clearing attributes etc is done by the parent stream,
+      // so must be avoided here
+    }
+  }
+
+  private final TokenStream delegate;
+  private TokenState state = TokenState.READING;
+  private boolean lastTokenFiltered;
+
+  /**
+   * Create a new BypassingTokenFilter
+   * @param input         the input TokenStream
+   * @param inputFactory  a factory function to create a new instance of the TokenFilter to wrap
+   */
+  protected ConditionalTokenFilter(TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
+    super(input);
+    this.delegate = inputFactory.apply(new OneTimeWrapper(this));
+  }
+
+  /**
+   * Whether or not to execute the wrapped TokenFilter for the current token
+   */
+  protected abstract boolean shouldFilter() throws IOException;
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    this.delegate.reset();
+    this.state = TokenState.READING;
+    this.lastTokenFiltered = false;
+  }
+
+  @Override
+  public void end() throws IOException {
+    super.end();
+    if (lastTokenFiltered) {
+      this.delegate.end();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    this.delegate.close();
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    while (true) {
+      if (state == TokenState.READING) {
+        if (input.incrementToken() == false) {
+          return false;
+        }
+        if (shouldFilter()) {
+          lastTokenFiltered = true;
+          state = TokenState.PREBUFFERING;
+          // we determine that the delegate has emitted all the tokens it can at the current
+          // position when OneTimeWrapper.incrementToken() is called in DELEGATING state.  To
+          // signal this back to the delegate, we return false, so we now need to reset it
+          // to ensure that it can continue to emit more tokens
+          delegate.reset();
+          boolean more = delegate.incrementToken();
+          state = TokenState.DELEGATING;
+          return more;
+        }
+        lastTokenFiltered = false;
+        return true;
+      }
+      if (state == TokenState.BUFFERING) {
+        return input.incrementToken();
+      }
+      if (state == TokenState.DELEGATING) {
+        clearAttributes();
+        if (delegate.incrementToken()) {
+          return true;
+        }
+        // no more cached tokens
+        state = TokenState.READING;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
new file mode 100644
index 0000000..9fe27ac
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.lucene.analysis.TokenFilter;
+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;
+
+/**
+ * Abstract parent class for analysis factories that create {@link ConditionalTokenFilter} instances
+ */
+public abstract class ConditionalTokenFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+
+  private List<TokenFilterFactory> innerFilters;
+
+  protected ConditionalTokenFilterFactory(Map<String, String> args) {
+    super(args);
+  }
+
+  /**
+   * Set the inner filter factories to produce the {@link TokenFilter}s that will be
+   * wrapped by the {@link ConditionalTokenFilter}
+   */
+  public void setInnerFilters(List<TokenFilterFactory> innerFilters) {
+    this.innerFilters = innerFilters;
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    if (innerFilters == null || innerFilters.size() == 0) {
+      return input;
+    }
+    Function<TokenStream, TokenStream> innerStream = ts -> {
+      for (TokenFilterFactory factory : innerFilters) {
+        ts = factory.create(ts);
+      }
+      return ts;
+    };
+    return create(input, innerStream);
+  }
+
+  @Override
+  public final void inform(ResourceLoader loader) throws IOException {
+    if (innerFilters == null)
+      return;
+    for (TokenFilterFactory factory : innerFilters) {
+      if (factory instanceof ResourceLoaderAware) {
+        ((ResourceLoaderAware)factory).inform(loader);
+      }
+    }
+    doInform(loader);
+  }
+
+  /**
+   * Initialises this component with the corresponding {@link ResourceLoader}
+   */
+  protected void doInform(ResourceLoader loader) throws IOException { }
+
+  /**
+   * Modify the incoming {@link TokenStream} with a {@link ConditionalTokenFilter}
+   */
+  protected abstract ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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
new file mode 100644
index 0000000..95bf55f
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
@@ -0,0 +1,52 @@
+/*
+ * 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/7f13e564/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
new file mode 100644
index 0000000..f860b72
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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/7f13e564/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 b46de18..ddd3c91 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,6 +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.TrimFilterFactory
 org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.TypeAsSynonymFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
index 7839203..bef95f8 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
@@ -71,13 +71,14 @@ import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
 import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
 import org.apache.lucene.analysis.hunspell.Dictionary;
 import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilter;
-import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
 import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter;
+import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
 import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
 import org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter;
 import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
@@ -207,6 +208,10 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
         if (ctor.isSynthetic() || ctor.isAnnotationPresent(Deprecated.class) || brokenConstructors.get(ctor) == ALWAYS) {
           continue;
         }
+        // conditional filters are tested elsewhere
+        if (ConditionalTokenFilter.class.isAssignableFrom(c)) {
+          continue;
+        }
         if (Tokenizer.class.isAssignableFrom(c)) {
           assertTrue(ctor.toGenericString() + " has unsupported parameter types",
             allowedTokenizerArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
@@ -696,16 +701,46 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
 
         while (true) {
           final Constructor<? extends TokenFilter> ctor = tokenfilters.get(random.nextInt(tokenfilters.size()));
-          
-          final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
-          if (broken(ctor, args)) {
-            continue;
-          }
-          final TokenFilter flt = createComponent(ctor, args, descr);
-          if (flt != null) {
-            spec.stream = flt;
+          if (random.nextBoolean()) {
+            long seed = random.nextLong();
+            spec.stream = new ConditionalTokenFilter(spec.stream, in -> {
+              final Object args[] = newFilterArgs(random, in, ctor.getParameterTypes());
+              if (broken(ctor, args)) {
+                return in;
+              }
+              descr.append("ConditionalTokenFilter: ");
+              TokenStream ts = createComponent(ctor, args, descr);
+              if (ts == null) {
+                return in;
+              }
+              return ts;
+            }) {
+              Random random = new Random(seed);
+
+              @Override
+              public void reset() throws IOException {
+                super.reset();
+                random = new Random(seed);
+              }
+
+              @Override
+              protected boolean shouldFilter() throws IOException {
+                return random.nextBoolean();
+              }
+            };
             break;
           }
+          else {
+            final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
+            if (broken(ctor, args)) {
+              continue;
+            }
+            final TokenFilter flt = createComponent(ctor, args, descr);
+            if (flt != null) {
+              spec.stream = flt;
+              break;
+            }
+          }
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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 d929bfd..b963812 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
@@ -37,6 +37,7 @@ import org.apache.lucene.analysis.core.LowerCaseTokenizer;
 import org.apache.lucene.analysis.core.StopFilterFactory;
 import org.apache.lucene.analysis.core.WhitespaceTokenizerFactory;
 import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory;
+import org.apache.lucene.analysis.reverse.ReverseStringFilterFactory;
 import org.apache.lucene.analysis.standard.ClassicTokenizerFactory;
 import org.apache.lucene.analysis.standard.StandardTokenizerFactory;
 import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
@@ -500,4 +501,47 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertEquals(new BytesRef("e f c"), analyzer.normalize("dummy", "a b c"));
   }
 
+  public void testConditions() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .whenTerm(t -> t.toString().contains("o"))
+          .addTokenFilter("uppercase")
+          .addTokenFilter(ReverseStringFilterFactory.class)
+        .endwhen()
+        .addTokenFilter("asciifolding")
+        .build();
+
+    assertAnalyzesTo(analyzer, "Héllo world whaT's hãppening",
+        new String[]{ "OLLEH", "DLROW", "what's", "happening" });
+  }
+
+  public void testConditionsWithResourceLoader() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .when("termexclusion", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
+          .addTokenFilter("reversestring")
+        .endwhen()
+        .build();
+
+    assertAnalyzesTo(analyzer, "FOO BAR BAZ",
+        new String[]{ "foo", "bar", "zab" });
+  }
+
+  public void testConditionsWithWrappedResourceLoader() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .whenTerm(t -> t.toString().contains("o") == false)
+          .addTokenFilter("stop",
+            "ignoreCase", "true",
+            "words", "org/apache/lucene/analysis/custom/teststop.txt",
+            "format", "wordset")
+        .endwhen()
+        .build();
+
+    assertAnalyzesTo(analyzer, "foo bar baz", new String[]{ "foo", "baz" });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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
new file mode 100644
index 0000000..e804676
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
@@ -0,0 +1,233 @@
+/*
+ * 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.io.StringReader;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.CharacterUtils;
+import org.apache.lucene.analysis.FilteringTokenFilter;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.synonym.SolrSynonymParser;
+import org.apache.lucene.analysis.synonym.SynonymGraphFilter;
+import org.apache.lucene.analysis.synonym.SynonymMap;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+
+public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
+
+  boolean closed = false;
+  boolean ended = false;
+  boolean reset = false;
+
+  private final class AssertingLowerCaseFilter extends TokenFilter {
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+    public AssertingLowerCaseFilter(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    public final boolean incrementToken() throws IOException {
+      if (input.incrementToken()) {
+        CharacterUtils.toLowerCase(termAtt.buffer(), 0, termAtt.length());
+        return true;
+      } else
+        return false;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      ended = true;
+    }
+
+    @Override
+    public void close() throws IOException {
+      super.close();
+      closed = true;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      reset = true;
+    }
+  }
+
+  public void testSimple() 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)
+    );
+
+    TokenStream t = new ConditionalTokenFilter(cts, AssertingLowerCaseFilter::new) {
+      CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return termAtt.toString().contains("o") == false;
+      }
+    };
+
+    assertTokenStreamContents(t, new String[]{ "alice", "Bob", "clara", "david" });
+    assertTrue(closed);
+    assertTrue(reset);
+    assertTrue(ended);
+  }
+
+  private final class TokenSplitter extends TokenFilter {
+
+    final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    String half;
+
+    protected TokenSplitter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (half == null) {
+        if (input.incrementToken() == false) {
+          return false;
+        }
+        half = termAtt.toString().substring(4);
+        termAtt.setLength(4);
+        return true;
+      }
+      termAtt.setEmpty().append(half);
+      half = null;
+      return true;
+    }
+  }
+
+  public void testMultitokenWrapping() throws IOException {
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("tokenpos1", 0, 9),
+        new Token("tokenpos2", 10, 19),
+        new Token("tokenpos3", 20, 29),
+        new Token("tokenpos4", 30, 39)
+    );
+
+    TokenStream ts = new ConditionalTokenFilter(cts, TokenSplitter::new) {
+      final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return termAtt.toString().contains("2") == false;
+      }
+    };
+
+    assertTokenStreamContents(ts, new String[]{
+        "toke", "npos1", "tokenpos2", "toke", "npos3", "toke", "npos4"
+    });
+  }
+
+  private final class EndTrimmingFilter extends FilteringTokenFilter {
+
+    final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+    public EndTrimmingFilter(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    protected boolean accept() throws IOException {
+      return true;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      offsetAtt.setOffset(0, offsetAtt.endOffset() - 2);
+    }
+  }
+
+  public void testEndPropagation() throws IOException {
+    CannedTokenStream cts1 = new CannedTokenStream(0, 20,
+        new Token("alice", 0, 5), new Token("bob", 6, 8)
+    );
+    TokenStream ts1 = new ConditionalTokenFilter(cts1, EndTrimmingFilter::new) {
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return false;
+      }
+    };
+    assertTokenStreamContents(ts1, new String[]{ "alice", "bob" },
+        null, null, null, null, null, 20);
+
+    CannedTokenStream cts2 = new CannedTokenStream(0, 20,
+        new Token("alice", 0, 5), new Token("bob", 6, 8)
+    );
+    TokenStream ts2 = new ConditionalTokenFilter(cts2, EndTrimmingFilter::new) {
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return true;
+      }
+    };
+    assertTokenStreamContents(ts2, new String[]{ "alice", "bob" },
+        null, null, null, null, null, 18);
+  }
+
+  public void testWrapGraphs() throws Exception {
+
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("a", 0, 1),
+        new Token("b", 2, 3),
+        new Token("c", 4, 5),
+        new Token("d", 6, 7),
+        new Token("e", 8, 9)
+    );
+
+    SynonymMap sm;
+    try (Analyzer analyzer = new MockAnalyzer(random())) {
+      SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
+      parser.parse(new StringReader("a b, f\nc d, g"));
+      sm = parser.build();
+    }
+
+    TokenStream ts = new ConditionalTokenFilter(cts, in -> new SynonymGraphFilter(in, sm, true)) {
+      CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return "c".equals(termAtt.toString()) == false;
+      }
+    };
+
+    assertTokenStreamContents(ts, new String[]{
+        "f", "a", "b", "c", "d", "e"
+        },
+        null, null, null,
+        new int[]{
+        1, 0, 1, 1, 1, 1
+        },
+        new int[]{
+        2, 1, 1, 1, 1, 1
+        });
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f13e564/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
new file mode 100644
index 0000000..2e6aecf
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.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 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" });
+
+  }
+
+}


[2/2] lucene-solr:master: LUCENE-8273: Add ConditionalTokenFilter

Posted by ro...@apache.org.
LUCENE-8273: Add ConditionalTokenFilter


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

Branch: refs/heads/master
Commit: 1ce3ebadbd60d4485145c71b48330d0aabde77ad
Parents: 8a697ee
Author: Alan Woodward <ro...@apache.org>
Authored: Fri May 11 11:11:21 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Fri May 11 11:34:59 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   7 +
 .../lucene/analysis/custom/CustomAnalyzer.java  | 202 +++++++++++++++-
 .../miscellaneous/ConditionalTokenFilter.java   | 149 ++++++++++++
 .../ConditionalTokenFilterFactory.java          |  85 +++++++
 .../miscellaneous/TermExclusionFilter.java      |  52 +++++
 .../TermExclusionFilterFactory.java             |  58 +++++
 ...ache.lucene.analysis.util.TokenFilterFactory |   1 +
 .../lucene/analysis/core/TestRandomChains.java  |  53 ++++-
 .../analysis/custom/TestCustomAnalyzer.java     |  44 ++++
 .../TestConditionalTokenFilter.java             | 233 +++++++++++++++++++
 .../miscellaneous/TestTermExclusionFilter.java  |  48 ++++
 11 files changed, 920 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 50e2845..de790e1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -175,6 +175,13 @@ New Features
   provided document. This allows to undelete a soft-deleted document unless it's been claimed
   by a merge. (Simon Willnauer) 
 
+* LUCENE-8273: ConditionalTokenFilter allows analysis chains to skip particular token
+  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)
+
 Bug Fixes
 
 * LUCENE-8266: Detect bogus tiles when creating a standard polygon and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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 a697cce..72614ca 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
@@ -17,8 +17,6 @@
 package org.apache.lucene.analysis.custom;
 
 
-import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassInstance;
-
 import java.io.IOException;
 import java.io.Reader;
 import java.nio.file.Path;
@@ -29,10 +27,15 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilterFactory;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
 import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.ClasspathResourceLoader;
@@ -45,6 +48,8 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.util.SetOnce;
 import org.apache.lucene.util.Version;
 
+import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassInstance;
+
 /**
  * A general-purpose Analyzer that can be created with a builder-style API.
  * Under the hood it uses the factory classes {@link TokenizerFactory},
@@ -73,6 +78,17 @@ import org.apache.lucene.util.Version;
  * <p>The list of names to be used for components can be looked up through:
  * {@link TokenizerFactory#availableTokenizers()}, {@link TokenFilterFactory#availableTokenFilters()},
  * and {@link CharFilterFactory#availableCharFilters()}.
+ * <p>You can create conditional branches in the analyzer by using {@link Builder#when(String, String...)} and
+ * {@link Builder#whenTerm(Predicate)}:
+ * <pre class="prettyprint">
+ * Analyzer ana = CustomAnalyzer.builder()
+ *    .withTokenizer(&quot;standard&quot;)
+ *    .addTokenFilter(&quot;lowercase&quot;)
+ *    .whenTerm(t -&gt; t.length() &gt; 10)
+ *      .addTokenFilter(&quot;reversestring&quot;)
+ *    .endwhen()
+ *    .build()
+ * </pre>
  */
 public final class CustomAnalyzer extends Analyzer {
   
@@ -335,6 +351,13 @@ public final class CustomAnalyzer extends Analyzer {
       componentsAdded = true;
       return this;
     }
+
+    private Builder addTokenFilter(TokenFilterFactory factory) {
+      Objects.requireNonNull(factory, "TokenFilterFactory may not be null");
+      tokenFilters.add(factory);
+      componentsAdded = true;
+      return this;
+    }
     
     /** Adds the given char filter.
      * @param factory class that is used to create the char filter.
@@ -377,6 +400,117 @@ public final class CustomAnalyzer extends Analyzer {
       componentsAdded = true;
       return this;
     }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param name    is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}
+     * @param params  the parameters to be passed to the factory
+     */
+    public ConditionBuilder when(String name, String... params) throws IOException {
+      return when(name, paramsToMap(params));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param name    is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}
+     * @param params  the parameters to be passed to the factory.  The map must be modifiable
+     */
+    @SuppressWarnings("unchecked")
+    public ConditionBuilder when(String name, Map<String, String> params) throws IOException {
+      Class<? extends TokenFilterFactory> clazz = TokenFilterFactory.lookupClass(name);
+      if (ConditionalTokenFilterFactory.class.isAssignableFrom(clazz) == false) {
+        throw new IllegalArgumentException("TokenFilterFactory " + name + " is not a ConditionalTokenFilterFactory");
+      }
+      return when((Class<? extends ConditionalTokenFilterFactory>) clazz, params);
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param factory class that is used to create the ConditionalTokenFilter
+     * @param params  the parameters to be passed to the factory
+     */
+    public ConditionBuilder when(Class<? extends ConditionalTokenFilterFactory> factory, String... params) throws IOException {
+      return when(factory, paramsToMap(params));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     *
+     * @param factory class that is used to create the ConditionalTokenFilter
+     * @param params  the parameters to be passed to the factory.  The map must be modifiable
+     */
+    public ConditionBuilder when(Class<? extends ConditionalTokenFilterFactory> factory, Map<String, String> params) throws IOException {
+      return when(newFactoryClassInstance(factory, applyDefaultParams(params)));
+    }
+
+    /**
+     * Add a {@link ConditionalTokenFilterFactory} to the analysis chain
+     *
+     * TokenFilters added by subsequent calls to {@link ConditionBuilder#addTokenFilter(String, String...)}
+     * and related functions will only be used if the current token matches the condition.  Consumers
+     * must call {@link ConditionBuilder#endwhen()} to return to the normal tokenfilter
+     * chain once conditional filters have been added
+     */
+    public ConditionBuilder when(ConditionalTokenFilterFactory factory) {
+      return new ConditionBuilder(factory, this);
+    }
+
+    /**
+     * Apply subsequent token filters if the current token's term matches a predicate
+     *
+     * This is the equivalent of:
+     * <pre>
+     *   when(new ConditionalTokenFilterFactory(Collections.emptyMap()) {
+     *      {@code @}Override
+     *      protected ConditionalTokenFilter create(TokenStream input, Function&lt;TokenStream, TokenStream&gt; inner) {
+     *        return new ConditionalTokenFilter(input, inner) {
+     *          CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+     *          {@code @}Override
+     *          protected boolean shouldFilter() {
+     *            return predicate.test(termAtt);
+     *          }
+     *        };
+     *      }
+     *   });
+     * </pre>
+     */
+    public ConditionBuilder whenTerm(Predicate<CharSequence> predicate) {
+      return new ConditionBuilder(new ConditionalTokenFilterFactory(Collections.emptyMap()) {
+        @Override
+        protected ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner) {
+          return new ConditionalTokenFilter(input, inner) {
+            CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+            @Override
+            protected boolean shouldFilter() {
+              return predicate.test(termAtt);
+            }
+          };
+        }
+      }, this);
+    }
     
     /** Builds the analyzer. */
     public CustomAnalyzer build() {
@@ -412,11 +546,73 @@ public final class CustomAnalyzer extends Analyzer {
       return map;
     }
     
-    private <T> T applyResourceLoader(T factory) throws IOException {
+    <T> T applyResourceLoader(T factory) throws IOException {
       if (factory instanceof ResourceLoaderAware) {
         ((ResourceLoaderAware) factory).inform(loader);
       }
       return factory;
     }
   }
+
+  /**
+   * Factory class for a {@link ConditionalTokenFilter}
+   */
+  public static class ConditionBuilder {
+
+    private final List<TokenFilterFactory> innerFilters = new ArrayList<>();
+    private final ConditionalTokenFilterFactory factory;
+    private final Builder parent;
+
+    private ConditionBuilder(ConditionalTokenFilterFactory factory, Builder parent) {
+      this.factory = factory;
+      this.parent = parent;
+    }
+
+    /** Adds the given token filter.
+     * @param name is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}.
+     *  The list of possible names can be looked up with {@link TokenFilterFactory#availableTokenFilters()}.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(String name, Map<String, String> params) throws IOException {
+      innerFilters.add(TokenFilterFactory.forName(name, parent.applyDefaultParams(params)));
+      return this;
+    }
+
+    /** Adds the given token filter.
+     * @param name is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}.
+     *  The list of possible names can be looked up with {@link TokenFilterFactory#availableTokenFilters()}.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(String name, String... params) throws IOException {
+      return addTokenFilter(name, parent.paramsToMap(params));
+    }
+
+    /** Adds the given token filter.
+     * @param factory class that is used to create the token filter.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(Class<? extends TokenFilterFactory> factory, Map<String, String> params) throws IOException {
+      innerFilters.add(newFactoryClassInstance(factory, parent.applyDefaultParams(params)));
+      return this;
+    }
+
+    /** Adds the given token filter.
+     * @param factory class that is used to create the token filter.
+     * @param params the map of parameters to be passed to factory. The map must be modifiable.
+     */
+    public ConditionBuilder addTokenFilter(Class<? extends TokenFilterFactory> factory, String... params) throws IOException {
+      return addTokenFilter(factory, parent.paramsToMap(params));
+    }
+
+    /**
+     * Close the branch and return to the main analysis chain
+     */
+    public Builder endwhen() throws IOException {
+      factory.setInnerFilters(innerFilters);
+      parent.applyResourceLoader(factory);
+      parent.addTokenFilter(factory);
+      return parent;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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
new file mode 100644
index 0000000..f6f2a54
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilter.java
@@ -0,0 +1,149 @@
+/*
+ * 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.function.Function;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+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
+ */
+public abstract class ConditionalTokenFilter extends TokenFilter {
+
+  private enum TokenState {
+    READING, PREBUFFERING, BUFFERING, DELEGATING
+  }
+
+  private final class OneTimeWrapper extends TokenStream {
+
+    public OneTimeWrapper(AttributeSource attributeSource) {
+      super(attributeSource);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (state == TokenState.PREBUFFERING) {
+        state = TokenState.BUFFERING;
+        return true;
+      }
+      if (state == TokenState.DELEGATING) {
+        return false;
+      }
+      return ConditionalTokenFilter.this.incrementToken();
+    }
+
+    @Override
+    public void reset() throws IOException {
+      // clearing attributes etc is done by the parent stream,
+      // so must be avoided here
+    }
+
+    @Override
+    public void end() throws IOException {
+      // clearing attributes etc is done by the parent stream,
+      // so must be avoided here
+    }
+  }
+
+  private final TokenStream delegate;
+  private TokenState state = TokenState.READING;
+  private boolean lastTokenFiltered;
+
+  /**
+   * Create a new BypassingTokenFilter
+   * @param input         the input TokenStream
+   * @param inputFactory  a factory function to create a new instance of the TokenFilter to wrap
+   */
+  protected ConditionalTokenFilter(TokenStream input, Function<TokenStream, TokenStream> inputFactory) {
+    super(input);
+    this.delegate = inputFactory.apply(new OneTimeWrapper(this));
+  }
+
+  /**
+   * Whether or not to execute the wrapped TokenFilter for the current token
+   */
+  protected abstract boolean shouldFilter() throws IOException;
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    this.delegate.reset();
+    this.state = TokenState.READING;
+    this.lastTokenFiltered = false;
+  }
+
+  @Override
+  public void end() throws IOException {
+    super.end();
+    if (lastTokenFiltered) {
+      this.delegate.end();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    this.delegate.close();
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    while (true) {
+      if (state == TokenState.READING) {
+        if (input.incrementToken() == false) {
+          return false;
+        }
+        if (shouldFilter()) {
+          lastTokenFiltered = true;
+          state = TokenState.PREBUFFERING;
+          // we determine that the delegate has emitted all the tokens it can at the current
+          // position when OneTimeWrapper.incrementToken() is called in DELEGATING state.  To
+          // signal this back to the delegate, we return false, so we now need to reset it
+          // to ensure that it can continue to emit more tokens
+          delegate.reset();
+          boolean more = delegate.incrementToken();
+          state = TokenState.DELEGATING;
+          return more;
+        }
+        lastTokenFiltered = false;
+        return true;
+      }
+      if (state == TokenState.BUFFERING) {
+        return input.incrementToken();
+      }
+      if (state == TokenState.DELEGATING) {
+        clearAttributes();
+        if (delegate.incrementToken()) {
+          return true;
+        }
+        // no more cached tokens
+        state = TokenState.READING;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
new file mode 100644
index 0000000..9fe27ac
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.lucene.analysis.TokenFilter;
+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;
+
+/**
+ * Abstract parent class for analysis factories that create {@link ConditionalTokenFilter} instances
+ */
+public abstract class ConditionalTokenFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+
+  private List<TokenFilterFactory> innerFilters;
+
+  protected ConditionalTokenFilterFactory(Map<String, String> args) {
+    super(args);
+  }
+
+  /**
+   * Set the inner filter factories to produce the {@link TokenFilter}s that will be
+   * wrapped by the {@link ConditionalTokenFilter}
+   */
+  public void setInnerFilters(List<TokenFilterFactory> innerFilters) {
+    this.innerFilters = innerFilters;
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    if (innerFilters == null || innerFilters.size() == 0) {
+      return input;
+    }
+    Function<TokenStream, TokenStream> innerStream = ts -> {
+      for (TokenFilterFactory factory : innerFilters) {
+        ts = factory.create(ts);
+      }
+      return ts;
+    };
+    return create(input, innerStream);
+  }
+
+  @Override
+  public final void inform(ResourceLoader loader) throws IOException {
+    if (innerFilters == null)
+      return;
+    for (TokenFilterFactory factory : innerFilters) {
+      if (factory instanceof ResourceLoaderAware) {
+        ((ResourceLoaderAware)factory).inform(loader);
+      }
+    }
+    doInform(loader);
+  }
+
+  /**
+   * Initialises this component with the corresponding {@link ResourceLoader}
+   */
+  protected void doInform(ResourceLoader loader) throws IOException { }
+
+  /**
+   * Modify the incoming {@link TokenStream} with a {@link ConditionalTokenFilter}
+   */
+  protected abstract ConditionalTokenFilter create(TokenStream input, Function<TokenStream, TokenStream> inner);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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
new file mode 100644
index 0000000..95bf55f
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilter.java
@@ -0,0 +1,52 @@
+/*
+ * 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/1ce3ebad/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
new file mode 100644
index 0000000..f860b72
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TermExclusionFilterFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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/1ce3ebad/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 b46de18..ddd3c91 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,6 +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.TrimFilterFactory
 org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.TypeAsSynonymFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
index 7839203..bef95f8 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
@@ -71,13 +71,14 @@ import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
 import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
 import org.apache.lucene.analysis.hunspell.Dictionary;
 import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
+import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilter;
 import org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilter;
-import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
 import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter;
+import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
 import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
 import org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter;
 import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
@@ -207,6 +208,10 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
         if (ctor.isSynthetic() || ctor.isAnnotationPresent(Deprecated.class) || brokenConstructors.get(ctor) == ALWAYS) {
           continue;
         }
+        // conditional filters are tested elsewhere
+        if (ConditionalTokenFilter.class.isAssignableFrom(c)) {
+          continue;
+        }
         if (Tokenizer.class.isAssignableFrom(c)) {
           assertTrue(ctor.toGenericString() + " has unsupported parameter types",
             allowedTokenizerArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
@@ -696,16 +701,46 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
 
         while (true) {
           final Constructor<? extends TokenFilter> ctor = tokenfilters.get(random.nextInt(tokenfilters.size()));
-          
-          final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
-          if (broken(ctor, args)) {
-            continue;
-          }
-          final TokenFilter flt = createComponent(ctor, args, descr);
-          if (flt != null) {
-            spec.stream = flt;
+          if (random.nextBoolean()) {
+            long seed = random.nextLong();
+            spec.stream = new ConditionalTokenFilter(spec.stream, in -> {
+              final Object args[] = newFilterArgs(random, in, ctor.getParameterTypes());
+              if (broken(ctor, args)) {
+                return in;
+              }
+              descr.append("ConditionalTokenFilter: ");
+              TokenStream ts = createComponent(ctor, args, descr);
+              if (ts == null) {
+                return in;
+              }
+              return ts;
+            }) {
+              Random random = new Random(seed);
+
+              @Override
+              public void reset() throws IOException {
+                super.reset();
+                random = new Random(seed);
+              }
+
+              @Override
+              protected boolean shouldFilter() throws IOException {
+                return random.nextBoolean();
+              }
+            };
             break;
           }
+          else {
+            final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
+            if (broken(ctor, args)) {
+              continue;
+            }
+            final TokenFilter flt = createComponent(ctor, args, descr);
+            if (flt != null) {
+              spec.stream = flt;
+              break;
+            }
+          }
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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 d9ea43c..fb86c07 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
@@ -37,6 +37,7 @@ import org.apache.lucene.analysis.core.LowerCaseTokenizer;
 import org.apache.lucene.analysis.core.StopFilterFactory;
 import org.apache.lucene.analysis.core.WhitespaceTokenizerFactory;
 import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory;
+import org.apache.lucene.analysis.reverse.ReverseStringFilterFactory;
 import org.apache.lucene.analysis.standard.ClassicTokenizerFactory;
 import org.apache.lucene.analysis.standard.StandardTokenizerFactory;
 import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
@@ -500,4 +501,47 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertEquals(new BytesRef("e f c"), analyzer.normalize("dummy", "a b c"));
   }
 
+  public void testConditions() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .whenTerm(t -> t.toString().contains("o"))
+          .addTokenFilter("uppercase")
+          .addTokenFilter(ReverseStringFilterFactory.class)
+        .endwhen()
+        .addTokenFilter("asciifolding")
+        .build();
+
+    assertAnalyzesTo(analyzer, "Héllo world whaT's hãppening",
+        new String[]{ "OLLEH", "DLROW", "what's", "happening" });
+  }
+
+  public void testConditionsWithResourceLoader() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .when("termexclusion", "protected", "org/apache/lucene/analysis/custom/teststop.txt")
+          .addTokenFilter("reversestring")
+        .endwhen()
+        .build();
+
+    assertAnalyzesTo(analyzer, "FOO BAR BAZ",
+        new String[]{ "foo", "bar", "zab" });
+  }
+
+  public void testConditionsWithWrappedResourceLoader() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        .withTokenizer("whitespace")
+        .addTokenFilter("lowercase")
+        .whenTerm(t -> t.toString().contains("o") == false)
+          .addTokenFilter("stop",
+            "ignoreCase", "true",
+            "words", "org/apache/lucene/analysis/custom/teststop.txt",
+            "format", "wordset")
+        .endwhen()
+        .build();
+
+    assertAnalyzesTo(analyzer, "foo bar baz", new String[]{ "foo", "baz" });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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
new file mode 100644
index 0000000..e804676
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConditionalTokenFilter.java
@@ -0,0 +1,233 @@
+/*
+ * 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.io.StringReader;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.CharacterUtils;
+import org.apache.lucene.analysis.FilteringTokenFilter;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.synonym.SolrSynonymParser;
+import org.apache.lucene.analysis.synonym.SynonymGraphFilter;
+import org.apache.lucene.analysis.synonym.SynonymMap;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+
+public class TestConditionalTokenFilter extends BaseTokenStreamTestCase {
+
+  boolean closed = false;
+  boolean ended = false;
+  boolean reset = false;
+
+  private final class AssertingLowerCaseFilter extends TokenFilter {
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+    public AssertingLowerCaseFilter(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    public final boolean incrementToken() throws IOException {
+      if (input.incrementToken()) {
+        CharacterUtils.toLowerCase(termAtt.buffer(), 0, termAtt.length());
+        return true;
+      } else
+        return false;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      ended = true;
+    }
+
+    @Override
+    public void close() throws IOException {
+      super.close();
+      closed = true;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      reset = true;
+    }
+  }
+
+  public void testSimple() 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)
+    );
+
+    TokenStream t = new ConditionalTokenFilter(cts, AssertingLowerCaseFilter::new) {
+      CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return termAtt.toString().contains("o") == false;
+      }
+    };
+
+    assertTokenStreamContents(t, new String[]{ "alice", "Bob", "clara", "david" });
+    assertTrue(closed);
+    assertTrue(reset);
+    assertTrue(ended);
+  }
+
+  private final class TokenSplitter extends TokenFilter {
+
+    final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    String half;
+
+    protected TokenSplitter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (half == null) {
+        if (input.incrementToken() == false) {
+          return false;
+        }
+        half = termAtt.toString().substring(4);
+        termAtt.setLength(4);
+        return true;
+      }
+      termAtt.setEmpty().append(half);
+      half = null;
+      return true;
+    }
+  }
+
+  public void testMultitokenWrapping() throws IOException {
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("tokenpos1", 0, 9),
+        new Token("tokenpos2", 10, 19),
+        new Token("tokenpos3", 20, 29),
+        new Token("tokenpos4", 30, 39)
+    );
+
+    TokenStream ts = new ConditionalTokenFilter(cts, TokenSplitter::new) {
+      final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return termAtt.toString().contains("2") == false;
+      }
+    };
+
+    assertTokenStreamContents(ts, new String[]{
+        "toke", "npos1", "tokenpos2", "toke", "npos3", "toke", "npos4"
+    });
+  }
+
+  private final class EndTrimmingFilter extends FilteringTokenFilter {
+
+    final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+    public EndTrimmingFilter(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    protected boolean accept() throws IOException {
+      return true;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      offsetAtt.setOffset(0, offsetAtt.endOffset() - 2);
+    }
+  }
+
+  public void testEndPropagation() throws IOException {
+    CannedTokenStream cts1 = new CannedTokenStream(0, 20,
+        new Token("alice", 0, 5), new Token("bob", 6, 8)
+    );
+    TokenStream ts1 = new ConditionalTokenFilter(cts1, EndTrimmingFilter::new) {
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return false;
+      }
+    };
+    assertTokenStreamContents(ts1, new String[]{ "alice", "bob" },
+        null, null, null, null, null, 20);
+
+    CannedTokenStream cts2 = new CannedTokenStream(0, 20,
+        new Token("alice", 0, 5), new Token("bob", 6, 8)
+    );
+    TokenStream ts2 = new ConditionalTokenFilter(cts2, EndTrimmingFilter::new) {
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return true;
+      }
+    };
+    assertTokenStreamContents(ts2, new String[]{ "alice", "bob" },
+        null, null, null, null, null, 18);
+  }
+
+  public void testWrapGraphs() throws Exception {
+
+    CannedTokenStream cts = new CannedTokenStream(
+        new Token("a", 0, 1),
+        new Token("b", 2, 3),
+        new Token("c", 4, 5),
+        new Token("d", 6, 7),
+        new Token("e", 8, 9)
+    );
+
+    SynonymMap sm;
+    try (Analyzer analyzer = new MockAnalyzer(random())) {
+      SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
+      parser.parse(new StringReader("a b, f\nc d, g"));
+      sm = parser.build();
+    }
+
+    TokenStream ts = new ConditionalTokenFilter(cts, in -> new SynonymGraphFilter(in, sm, true)) {
+      CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      @Override
+      protected boolean shouldFilter() throws IOException {
+        return "c".equals(termAtt.toString()) == false;
+      }
+    };
+
+    assertTokenStreamContents(ts, new String[]{
+        "f", "a", "b", "c", "d", "e"
+        },
+        null, null, null,
+        new int[]{
+        1, 0, 1, 1, 1, 1
+        },
+        new int[]{
+        2, 1, 1, 1, 1, 1
+        });
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ce3ebad/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
new file mode 100644
index 0000000..2e6aecf
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTermExclusionFilter.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 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" });
+
+  }
+
+}