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

[lucene-solr] branch branch_8x updated: SOLR-15697: Add ErrorLogMuter to test-framework, deprecating SolrException.ignorePatterns

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 75bba93  SOLR-15697: Add ErrorLogMuter to test-framework, deprecating SolrException.ignorePatterns
75bba93 is described below

commit 75bba93404e84879256d1053f23843cfe0aaf3f1
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Mon Oct 25 09:16:49 2021 -0700

    SOLR-15697: Add ErrorLogMuter to test-framework, deprecating SolrException.ignorePatterns
    
    (cherry picked from commit a32eaf8bb439d6dfd81f0d91f35905c0a7993020)
---
 .../tools/src/groovy/check-source-patterns.groovy  |   2 +-
 solr/CHANGES.txt                                   |   2 +
 .../test/org/apache/solr/ConvertedLegacyTest.java  |  41 ++--
 .../test/org/apache/solr/schema/RankFieldTest.java |  34 +--
 .../java/org/apache/solr/common/SolrException.java |  52 ++++-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |  43 ++--
 .../java/org/apache/solr/util/ErrorLogMuter.java   | 256 +++++++++++++++++++++
 .../org/apache/solr/util/TestErrorLogMuter.java    | 188 +++++++++++++++
 8 files changed, 565 insertions(+), 53 deletions(-)

diff --git a/lucene/tools/src/groovy/check-source-patterns.groovy b/lucene/tools/src/groovy/check-source-patterns.groovy
index 2f80815..7af2840 100644
--- a/lucene/tools/src/groovy/check-source-patterns.groovy
+++ b/lucene/tools/src/groovy/check-source-patterns.groovy
@@ -177,7 +177,7 @@ ant.fileScanner{
     }
   }
   if (f.name.endsWith('.java')) {
-    if (text.contains('org.slf4j.LoggerFactory')) {
+    if (text.contains('org.slf4j.LoggerFactory') && ! f.name.equals("ErrorLogMuter.java")) {
       if (!validLoggerPattern.matcher(text).find()) {
         reportViolation(f, 'invalid logging pattern [not private static final, uses static class name]');
       }
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a1e3619..bf037cf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -46,6 +46,8 @@ Other Changes
 
 * SOLR-15678: Allow only known content types in ShowFileRequestHandler (janhoy, Gus Heck, Mal Aware)
 
+* SOLR-15697: Add ErrorLogMuter to test-framework, deprecating SolrException.ignorePatterns (hossman)
+
 ======================= 8.10.1 =======================
 
 Other Changes
diff --git a/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java b/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java
index 603dd03..1ba28ee 100644
--- a/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java
+++ b/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java
@@ -18,6 +18,7 @@ package org.apache.solr;
 
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.request.*;
+import org.apache.solr.util.ErrorLogMuter;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -771,26 +772,30 @@ public class ConvertedLegacyTest extends SolrTestCaseJ4 {
             ,"//doc[3]/arr[@name='id_i' and .='1001']"
             );
 
-    ignoreException("shouldbeunindexed");
-    ignoreException("nullfirst");
-    ignoreException("abcde12345");
-    ignoreException("aaa");
-
     // Sort parsing exception tests.  (SOLR-6, SOLR-99)
-    assertQEx( "can not sort unindexed fields",
-        req( "q","id_i:1000", "sort", "shouldbeunindexed asc" ), 400 );
+    try (ErrorLogMuter errors = ErrorLogMuter.substring("shouldbeunindexed")) {
+      assertQEx( "can not sort unindexed fields",
+                 req( "q","id_i:1000", "sort", "shouldbeunindexed asc" ), 400 );
+      assertEquals(1, errors.getCount());
+    }
     
-    assertQEx( "invalid query format",
-        req( "q","id_i:1000", "sort", "nullfirst" ), 400 );
-
-    assertQEx( "unknown sort field",
-        req( "q","id_i:1000", "sort", "abcde12345 asc" ), 400 ); 
-
-    assertQEx( "unknown sort order",
-        req( "q","id_i:1000", "sort", "nullfirst aaa" ), 400 );
-
-    resetExceptionIgnores();
-
+    try (ErrorLogMuter errors = ErrorLogMuter.substring("nullfirst")) {
+      assertQEx( "invalid query format",
+                 req( "q","id_i:1000", "sort", "nullfirst" ), 400 );
+      assertEquals(1, errors.getCount());
+    }
+
+    try (ErrorLogMuter abc = ErrorLogMuter.substring("abcde12345");
+         ErrorLogMuter aaa = ErrorLogMuter.substring("aaa")) {
+      assertQEx( "unknown sort field",
+                 req( "q","id_i:1000", "sort", "abcde12345 asc" ), 400 ); 
+
+      assertQEx( "unknown sort order",
+                 req( "q","id_i:1000", "sort", "nullfirst aaa" ), 400 );
+      
+      assertEquals(1, abc.getCount());
+      assertEquals(1, aaa.getCount());
+    }
         
     // test prefix query
 
diff --git a/solr/core/src/test/org/apache/solr/schema/RankFieldTest.java b/solr/core/src/test/org/apache/solr/schema/RankFieldTest.java
index 11877f0..3763af1 100644
--- a/solr/core/src/test/org/apache/solr/schema/RankFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/RankFieldTest.java
@@ -25,6 +25,7 @@ import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.util.ErrorLogMuter;
 import org.apache.solr.util.TestHarness;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -57,53 +58,54 @@ public class RankFieldTest extends SolrTestCaseJ4 {
   }
   
   public void testBadFormat() {
-    ignoreException("Expecting float");
-    assertFailedU(adoc(
+    try (ErrorLogMuter errors = ErrorLogMuter.substring("Expecting float")) {
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, "foo"
         ));
 
-    assertFailedU(adoc(
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, "1.2.3"
         ));
+      assertEquals(2, errors.getCount());
+    }
     
-    unIgnoreException("Expecting float");
-    
-    ignoreException("must be finite");
-    assertFailedU(adoc(
+    try (ErrorLogMuter errors = ErrorLogMuter.substring("must be finite")) {
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(Float.POSITIVE_INFINITY)
         ));
 
-    assertFailedU(adoc(
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(Float.NEGATIVE_INFINITY)
         ));
     
-    assertFailedU(adoc(
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(Float.NaN)
         ));
+      assertEquals(3, errors.getCount());
+    }
     
-    unIgnoreException("must be finite");
-    
-    ignoreException("must be a positive");
-    assertFailedU(adoc(
+    try (ErrorLogMuter errors = ErrorLogMuter.substring("must be a positive")) {
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(-0.0f)
         ));
 
-    assertFailedU(adoc(
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(-1f)
         ));
 
-    assertFailedU(adoc(
+      assertFailedU(adoc(
         "id", "1",
         RANK_1, Float.toString(0.0f)
         ));
-    unIgnoreException("must be a positive");
+      assertEquals(3, errors.getCount());
+    }
   }
   
   public void testAddRandom() {
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrException.java b/solr/solrj/src/java/org/apache/solr/common/SolrException.java
index 0a6a9ea..dd3075a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrException.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrException.java
@@ -136,12 +136,24 @@ public class SolrException extends RuntimeException {
     return getMetadata(ROOT_ERROR_CLASS);
   }
 
-  /** @see #ignorePatterns */
+  /** 
+   * This method was initially created to aid in testing situations that were known to cause ERRORs.  It should no longer be used by any new code.
+   *
+   * @see #ignorePatterns 
+   * @deprecated Use the Logger directly
+   */
+  @Deprecated
   public void log(Logger log) {
     log(log,this);
   }
   
-  /** @see #ignorePatterns */
+  /** 
+   * This method was initially created to aid in testing situations that were known to cause ERRORs.  It should no longer be used by any new code.
+   *
+   * @see #ignorePatterns 
+   * @deprecated Use the Logger directly
+   */
+  @Deprecated
   public static void log(Logger log, Throwable e) {
     if (log.isErrorEnabled()) {
       String ignore = doIgnoreToStr(null, e);
@@ -153,7 +165,13 @@ public class SolrException extends RuntimeException {
     }
   }
 
-  /** @see #ignorePatterns */
+  /** 
+   * This method was initially created to aid in testing situations that were known to cause ERRORs.  It should no longer be used by any new code.
+   *
+   * @see #ignorePatterns 
+   * @deprecated Use the Logger directly
+   */
+  @Deprecated
   public static void log(Logger log, String msg, Throwable e) {
     if (log.isErrorEnabled()) {
       String ignore = doIgnoreToStr(msg, e);
@@ -165,7 +183,13 @@ public class SolrException extends RuntimeException {
     }
   }
   
-  /** @see #ignorePatterns */
+  /** 
+   * This method was initially created to aid in testing situations that were known to cause ERRORs.  It should no longer be used by any new code.
+   *
+   * @see #ignorePatterns 
+   * @deprecated Use the Logger directly
+   */
+  @Deprecated
   public static void log(Logger log, String msg) {
     if (log.isErrorEnabled()) {
       String ignore = doIgnoreToStr(msg, null);
@@ -177,6 +201,13 @@ public class SolrException extends RuntimeException {
     }
   }
 
+  /** 
+   * This method was initially created to aid in testing situations that were known to cause ERRORs.  It should no longer be used by any new code.
+   *
+   * @see #ignorePatterns 
+   * @deprecated use {@link Throwable#printStackTrace} directly
+   */
+  @Deprecated
   public static String toStr(Throwable e) {
     CharArrayWriter cw = new CharArrayWriter();
     PrintWriter pw = new PrintWriter(cw);
@@ -189,7 +220,10 @@ public class SolrException extends RuntimeException {
    * For test code: If non-null, prevents calls to {@link #log} from logging any msg or exception (stack trace) that matches an included regular expressions.
    *
    * A {@link java.util.concurrent.CopyOnWriteArraySet is recommended}.
+   *
+   * @deprecated use <code>ErrorLogMuter</code> in Solr test-framework.
    */
+  @Deprecated
   public static Set<String> ignorePatterns;
 
   /** 
@@ -199,7 +233,9 @@ public class SolrException extends RuntimeException {
    * @param stacktrace the stringified stack trace of the exception, used for the acutal regex checking
    * @see #ignorePatterns
    * @see #toStr
+   * @deprecated use <code>ErrorLogMuter</code> in Solr test-framework.
    */
+  @Deprecated
   public static String doIgnore(Throwable t, String stacktrace) { 
     if (t != null && t instanceof AssertionError) return null;
     
@@ -208,7 +244,11 @@ public class SolrException extends RuntimeException {
     return doIgnoreToStr(ignorePatterns, stacktrace, null);
   }
 
-  /** @see #doIgnoreToStr(Set, String, Throwable) */
+  /** 
+   * @see #doIgnoreToStr(Set, String, Throwable) 
+   * @deprecated Not needed once {@link #ignorePatterns} is removed
+   */
+  @Deprecated
   private static String doIgnoreToStr(String msg, Throwable t) {
     if (t != null && t instanceof AssertionError) return null;
     
@@ -225,7 +265,9 @@ public class SolrException extends RuntimeException {
    * @param exceptionToCheck if non-null, will be stringified and concatenated with stringToCheck before testing patterns
    * @see #ignorePatterns
    * @see #toStr
+   * @deprecated Not needed once {@link #ignorePatterns} is removed
    */
+  @Deprecated
   private static String doIgnoreToStr(Set<String> ignorePats, String stringToCheck, Throwable exceptionToCheck) {
     if (null == ignorePats) return null;
     
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 7aa02d6..0c20d12 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -60,7 +60,7 @@ import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
@@ -138,6 +138,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.util.BaseTestHarness;
 import org.apache.solr.util.ExternalPaths;
 import org.apache.solr.util.LogLevel;
+import org.apache.solr.util.ErrorLogMuter;
 import org.apache.solr.util.RandomizeSSL;
 import org.apache.solr.util.RandomizeSSL.SSLRandomizer;
 import org.apache.solr.util.RefCounted;
@@ -160,6 +161,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
+import org.apache.commons.io.IOUtils;
+
 import static java.util.Objects.requireNonNull;
 import static org.apache.solr.cloud.SolrZkServer.ZK_WHITELIST_PROPERTY;
 import static org.apache.solr.common.cloud.ZkStateReader.URL_SCHEME;
@@ -683,27 +686,41 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     }
   }
 
-  /** Causes an exception matching the regex pattern to not be logged. */
+  private final static Map<String,ErrorLogMuter> errorMuters = new ConcurrentHashMap<>();
+  
+  /** 
+   * Causes any ERROR log messages matching with a substring matching the regex pattern to be filtered out by the ROOT logger
+   *
+   * @see #resetExceptionIgnores
+   * @deprecated use a {@link ErrorLogMuter} instead
+   */
+  @Deprecated
   public static void ignoreException(String pattern) {
-    if (SolrException.ignorePatterns == null) // usually initialized already but in case not...
-      resetExceptionIgnores();
-    SolrException.ignorePatterns.add(pattern);
+    errorMuters.computeIfAbsent(pattern, (pat) -> ErrorLogMuter.regex(pat));
   }
 
+  /**
+   * @see #ignoreException
+   * @deprecated use a {@link ErrorLogMuter} instead
+   */
+  @Deprecated
   public static void unIgnoreException(String pattern) {
-    if (SolrException.ignorePatterns != null)
-      SolrException.ignorePatterns.remove(pattern);
+    errorMuters.computeIfPresent(pattern, (pat, muter) -> { IOUtils.closeQuietly(muter); return null; } );
   }
-
+  
   /**
-   * Clears all exception patterns, although keeps {@code "ignore_exception"}.
-   * {@link SolrTestCaseJ4} calls this in {@link AfterClass} so usually tests don't need to call this.
+   * Clears all exception patterns, immediately re-registering {@code "ignore_exception"}.
+   * {@link SolrTestCaseJ4} calls this in both {@link BeforeClass} {@link AfterClass} so usually tests don't need to call this.
+   * 
+   * @see #ignoreException
+   * @deprecated use a {@link ErrorLogMuter} instead
    */
+  @Deprecated
   public static void resetExceptionIgnores() {
-    // CopyOnWrite for safety; see SOLR-11757
-    SolrException.ignorePatterns = new CopyOnWriteArraySet<>(Collections.singleton("ignore_exception"));
+    errorMuters.forEach( (k, muter) -> { IOUtils.closeQuietly(muter); errorMuters.remove(k); } );
+    ignoreException("ignore_exception");
   }
-
+  
   protected static String getClassName() {
     return getTestClass().getName();
   }
diff --git a/solr/test-framework/src/java/org/apache/solr/util/ErrorLogMuter.java b/solr/test-framework/src/java/org/apache/solr/util/ErrorLogMuter.java
new file mode 100644
index 0000000..089acdd
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/util/ErrorLogMuter.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.lang.invoke.MethodHandles;
+import java.io.Closeable;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.function.Predicate;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.solr.common.util.SuppressForbidden;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.Marker;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.Logger;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.filter.AbstractFilter;
+import org.apache.logging.log4j.message.Message;
+
+/**
+ * <p>
+ * Helper code for "Muting" ERROR log messages that you expect as a result of the things you are testing, 
+ * so they aren't a distraction in test logs.  Example usage...
+ * </p>
+ * <code>
+ *  try (ErrorLogMuter errors = ErrorLogMuter.substring("nullfirst")) {
+ *    assertQEx( "invalid query format",
+ *               req( "q","id_i:1000", "sort", "nullfirst" ), 400 );
+ *    assertEquals(1, errors.getCount());
+ *  }
+ * </code>
+ * <p>
+ * ERROR messages are considered a match if their input matches either the message String, or the <code>toString</code> of an included 
+ * {@link Throwable}, or any of the recursive {@link Throwable#getCause}es of an included <code>Throwable</code>.
+ * </p>
+ * <p>
+ * Matching ERROR messages are "muted" by filtering them out of the ROOT logger.  Any Appenders attached to more specific 
+ * Loggers may still include these "muted" ERRROR messages.
+ * </p>
+ */
+@SuppressForbidden(reason="We need to use log4J2 classes directly to check that the ErrorLogMuter is working")
+public final class ErrorLogMuter implements Closeable, AutoCloseable {
+
+  // far easier to use FQN for our (one) slf4j Logger then to use a FQN every time we refe to log4j2 Logger
+  private static final org.slf4j.Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); // nowarn_valid_logger
+
+  private final static LoggerContext CTX = LoggerContext.getContext(false);
+  
+  /** @see #createName */
+  private final static AtomicInteger ID_GEN = new AtomicInteger(0);
+  /** generate a unique name for each muter to use in it's own lifecycle logging */
+  private static String createName(final String type) {
+    return MethodHandles.lookup().lookupClass().getSimpleName() + "-" + type + "-" + ID_GEN.incrementAndGet();
+  }
+
+  /** Mutes ERROR log messages that contain the input as a substring */
+  public static ErrorLogMuter substring(final String substr) { 
+    final String name = createName("substring");
+    log.info("Creating {} for ERROR logs containing the substring: {}", name, substr);
+    return new ErrorLogMuter(name, (str) -> { return str.contains(substr); });
+  }
+  
+  /** 
+   * Mutes ERROR log messages that <em>partially</em> match the specified regex.  
+   * @see Matcher#find
+   */
+  public static ErrorLogMuter regex(final String regex) { 
+    return regex(Pattern.compile(regex));
+  }
+  /** 
+   * Mutes ERROR log messages that <em>partially</em> match the specified regex.  
+   * @see Matcher#find
+   */
+  public static ErrorLogMuter regex(final Pattern pat) { 
+    final String name = createName("regex");
+    log.info("Creating {} for ERROR logs matching regex: {}", name, pat);
+    return new ErrorLogMuter(name, (str) -> { return pat.matcher(str).find(); });
+  }
+
+  private final String name;
+  private final CountingFilter rootFilter;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  /** @see StringPredicateErrorOrThrowableFilter */
+  private ErrorLogMuter(final String name, final Predicate<String> predicate) {
+    assert null != name;
+    assert null != predicate;
+
+    final LoggerConfig rootLoggerConfig = CTX.getConfiguration().getRootLogger();
+
+    this.name = name;
+    this.rootFilter = new StringPredicateErrorOrThrowableFilter(predicate);
+    rootLoggerConfig.addFilter(this.rootFilter);
+    
+    CTX.updateLoggers();
+  }
+
+  /** 
+   * The number of ERROR messages muted (by this instance) so far in it's lifecycle.
+   * This number may be less then the number of ERROR messages expected if multiple ErrorLogMuter 
+   * objects are in use which match the same ERROR log messages
+   */
+  public int getCount() {
+    return rootFilter.getCount();
+  }
+  
+  public void close() {
+    if (! closed.getAndSet(true)) { // Don't muck with log4j if we accidently get a double close
+      CTX.getConfiguration().getRootLogger().removeFilter(rootFilter);
+      CTX.updateLoggers();
+      if (log.isInfoEnabled()) {
+        log.info("Closing {} after mutting {} log messages", this.name, getCount());
+      }
+    }
+  }
+
+  @SuppressForbidden(reason="We need to use log4J2 classes directly to check that the ErrorLogMuter is working")
+  private static interface CountingFilter extends Filter {
+    /** The number of messages that have been filtered */
+    public int getCount();
+  }
+  
+  /**
+   * <p>
+   * Given a String predicate, this filter DENY's log events that are at least as specific as <code>ERROR</code>, and have a message or 
+   * {@link Throwable#toString} that matches the predicate.  When a Throwable is evaluated, the predicate is tested against the 
+   * <code>toString()</code> of all nested causes.
+   * </p>
+   * <p>
+   * Any message that is not at least as specific as <code>ERROR</code>, or that the predicate does not match, are left NEUTRAL.
+   * </p>
+   */
+  @SuppressForbidden(reason="We need to use log4J2 classes directly to check that the ErrorLogMuter is working")
+  private static final class StringPredicateErrorOrThrowableFilter extends AbstractFilter implements CountingFilter {
+    // This could probably be implemented with a combination of "LevelRangeFilter" and "ConjunctionFilter" if "ConjunctionFilter" existed
+    // Since it doesn't, we write our own more specialized impl instead of writing & combining multiple generalized versions
+
+    final Predicate<String> predicate;
+    final AtomicInteger count = new AtomicInteger(0);
+    public StringPredicateErrorOrThrowableFilter(final Predicate<String> predicate) {
+      super(Filter.Result.DENY, Filter.Result.NEUTRAL);
+      assert null != predicate;
+      this.predicate = predicate;
+    }
+
+    public int getCount() {
+      return count.get();
+    }
+
+    // NOTE: This is inspired by log4j's RegexFilter, but with an eye to being more "garbage-free" friendly
+    // Oddly, StringMatchFilter does things differnetly and acts like it needs to (re?) format msgs when params are provided
+    // Since RegexFilter has tests, and StringMatchFilter doesn't, we assume RegexFilter knows what it's doing...
+
+    /** The main logic of our filter: Evaluate predicate against log msg &amp; throwable msg(s) if and only if ERROR; else mismatch */
+    private Filter.Result doFilter(final Level level, final String msg, final Throwable throwable) {
+      if (level.isMoreSpecificThan(Level.ERROR)) {
+        if (null != msg && predicate.test(msg)) {
+          return matchAndCount();
+        }
+        for (Throwable t = throwable; null != t; t = t.getCause()) {
+          if (predicate.test(t.toString())) {
+            return matchAndCount();
+          }
+        }
+      }
+      return getOnMismatch();
+    }
+    
+    /** helper to be called by doFilter anytime it wants to return a "match" */
+    private Filter.Result matchAndCount() {
+      count.incrementAndGet();
+      return getOnMatch();
+    }
+    
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object... params) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4, Object p5) {
+      return doFilter(level,msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4, Object p5, Object p6) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4,
+                         Object p5, Object p6, Object p7) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4, Object p5, Object p6, Object p7, Object p8) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, String msg,
+                         Object p0, Object p1, Object p2, Object p3, Object p4, Object p5, Object p6, Object p7, Object p8, Object p9) {
+      return doFilter(level, msg, null);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, Object msg,
+                         Throwable t) {
+      return doFilter(level, null == msg ? null : msg.toString(), t);
+    }
+    public Result filter(Logger logger, Level level, Marker marker, Message msg,
+                         Throwable t) {
+      return doFilter(level, msg.getFormattedMessage(), t);
+    }
+    public Result filter(LogEvent event) {
+      // NOTE: For our usage, we're not worried about needing to filter LogEvents rom remote JVMs with ThrowableProxy
+      // stand ins for Throwabls that don't exist in our classloader...
+      return doFilter(event.getLevel(),
+                      event.getMessage().getFormattedMessage(),
+                      event.getThrown());
+    }
+  }
+}
diff --git a/solr/test-framework/src/test/org/apache/solr/util/TestErrorLogMuter.java b/solr/test-framework/src/test/org/apache/solr/util/TestErrorLogMuter.java
new file mode 100644
index 0000000..7fc062b
--- /dev/null
+++ b/solr/test-framework/src/test/org/apache/solr/util/TestErrorLogMuter.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.SolrTestCaseJ4;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.AbstractAppender;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.impl.MutableLogEvent;
+
+import static org.hamcrest.core.StringContains.containsString;
+
+@SuppressForbidden(reason="We need to use log4J2 classes directly to check that the ErrorLogMuter is working")
+public class TestErrorLogMuter extends SolrTestCaseJ4 {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @LogLevel("=WARN")
+  public void testErrorMutingRegex() throws Exception {
+
+    final ListAppender rootSanityCheck = new ListAppender("sanity-checker");
+    try {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().addAppender(rootSanityCheck, Level.WARN, null);
+      LoggerContext.getContext(false).updateLoggers();
+
+      try (ErrorLogMuter x = ErrorLogMuter.regex("eRrOr\\s+Log")) {
+        assertEquals(0, x.getCount());
+
+        log.error("This is an {} Log msg that x should be muted", "eRrOr");
+        assertEquals(1, x.getCount());
+        
+        log.error("This is an {} Log msg that x should not mute", "err");
+        log.warn("This is an warn message, mentioning 'eRrOr Log', that should also not be muted");
+        assertEquals(1, x.getCount());
+
+        log.error("This {} because of the {} msg", "error", "thowable",
+                  new Exception("outer", new Exception("inner eRrOr Log throwable")));
+        assertEquals(2, x.getCount());
+
+      }
+    } finally {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().removeAppender(rootSanityCheck.getName());
+      LoggerContext.getContext(false).updateLoggers();
+    }
+    
+    // the root loger should not have seen anything that was muted...
+    assertEquals(2, rootSanityCheck.getEvents().size());
+    assertThat(rootSanityCheck.getEvents().get(0).getMessage().getFormattedMessage(), containsString("should not mute"));
+    assertThat(rootSanityCheck.getEvents().get(1).getMessage().getFormattedMessage(), containsString("also not be muted"));
+  }
+
+  @LogLevel("=WARN") 
+  public void testMultipleMuters() throws Exception {
+
+    // Add a ListAppender to our ROOT logger so we can sanity check what log messages it gets
+    final ListAppender rootSanityCheck = new ListAppender("sanity-checker");
+    try {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().addAppender(rootSanityCheck, Level.WARN, null);
+      LoggerContext.getContext(false).updateLoggers();
+      
+      // sanity check that muters "mute" in the order used...
+      // (If this fails, then it means log4j has changed the precedence order it uses when addFilter is called,
+      // if that happens, we'll need to change our impl to check if an impl of some special "container" Filter subclass we create.
+      // is in the list of ROOT filters -- if not add one, and then "add" the specific muting filter to our "container" Filter)
+      try (ErrorLogMuter x = ErrorLogMuter.substring("xxx");
+           ErrorLogMuter y = ErrorLogMuter.regex(Pattern.compile("YYY", Pattern.CASE_INSENSITIVE));
+           ErrorLogMuter z = ErrorLogMuter.regex("(xxx|yyy)")) {
+           
+        
+        log.error("xx{}  ", "x");
+        log.error("    yyy");
+        log.error("xxx yyy");
+        log.warn("xxx  yyy");
+        log.error("abc", new Exception("yyy"));
+        
+        assertEquals(2, x.getCount()); // x is first, so it swallows up the "x + y" message
+        assertEquals(2, y.getCount()); // doesn't get anything x already got
+        assertEquals(0, z.getCount()); // doesn't get anything x already got
+      }
+    } finally {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().removeAppender(rootSanityCheck.getName());
+      LoggerContext.getContext(false).updateLoggers();
+    }
+
+    assertEquals(1, rootSanityCheck.getEvents().size()); // our warning
+  }
+
+  @LogLevel("=WARN") 
+  public void testDeprecatedBaseClassMethods() throws Exception {
+    
+    // Add a ListAppender to our ROOT logger so we can sanity check what log messages it gets
+    final ListAppender rootSanityCheck = new ListAppender("sanity-checker");
+    try {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().addAppender(rootSanityCheck, Level.WARN, null);
+      LoggerContext.getContext(false).updateLoggers();
+      
+      log.error("this matches the default ignore_exception pattern");
+      log.error("something matching foo that should make it"); // 1
+      assertEquals(1, rootSanityCheck.getEvents().size());
+      ignoreException("foo");
+      log.error("something matching foo that should NOT make it");
+      ignoreException("foo");
+      ignoreException("ba+r");
+      log.error("something matching foo that should still NOT make it");
+      log.error("something matching baaaar that should NOT make it");
+      log.warn("A warning should be fine even if it matches ignore_exception and foo and bar"); // 2
+      assertEquals(2, rootSanityCheck.getEvents().size());
+      unIgnoreException("foo");
+      log.error("another thing matching foo that should make it"); // 3
+      assertEquals(3, rootSanityCheck.getEvents().size());
+      log.error("something matching baaaar that should still NOT make it");
+      resetExceptionIgnores();
+      log.error("this still matches the default ignore_exception pattern");
+      log.error("but something matching baaaar should make it now"); // 4
+      assertEquals(4, rootSanityCheck.getEvents().size());
+
+    } finally {
+      LoggerContext.getContext(false).getConfiguration().getRootLogger().removeAppender(rootSanityCheck.getName());
+      LoggerContext.getContext(false).updateLoggers();
+    }
+    assertEquals(4, rootSanityCheck.getEvents().size());
+  }
+  
+  /**
+   * Maintains an in memory List of log events.
+   * <p>
+   * Inspired by <code>org.apache.logging.log4j.core.test.appender.ListAppender</code>
+   * but we have much simpler needs.
+   */
+  @SuppressForbidden(reason="We need to use log4J2 classes directly to check that the ErrorLogMuter is working")
+  public static final class ListAppender extends AbstractAppender {
+    // Use Collections.synchronizedList rather than CopyOnWriteArrayList because we expect
+    // more frequent writes than reads.
+    private final List<LogEvent> events = Collections.synchronizedList(new ArrayList<>());
+    private final List<LogEvent> publicEvents = Collections.unmodifiableList(events);
+    
+    public ListAppender(final String name) {
+      super(name, null, null, true, Property.EMPTY_ARRAY);
+      assert null != name;
+    }
+    
+    @Override
+    public void append(final LogEvent event) {
+      if (event instanceof MutableLogEvent) {
+        // must take snapshot or subsequent calls to logger.log() will modify this event
+        events.add(((MutableLogEvent) event).createMemento());
+      } else {
+        events.add(event);
+      }
+      if (log.isDebugEnabled()) {
+        log.debug("{} intercepted a log event (#{})", this.getName(), events.size());
+      }
+    }
+
+    /** Returns an immutable view of captured log events, contents can change as events are logged */
+    public List<LogEvent> getEvents() {
+      return publicEvents;
+    }
+  }
+}