You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ja...@apache.org on 2023/08/29 11:04:12 UTC

[solr] branch branch_9x updated: SOLR-15474 Make Circuit breakers pluggable (take 2) (#1725)

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

janhoy pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new d7208131166 SOLR-15474 Make Circuit breakers pluggable (take 2) (#1725)
d7208131166 is described below

commit d72081311660b29bf33fc0508c850d289dce3ba6
Author: Jan Høydahl <ja...@users.noreply.github.com>
AuthorDate: Tue Aug 29 12:47:25 2023 +0200

    SOLR-15474 Make Circuit breakers pluggable (take 2) (#1725)
    
    * SOLR-15474 Make Circuit breakers pluggable
    
    Co-authored-by: Christine Poerschke <cp...@apache.org>
    Co-authored-by: Atri Sharma <at...@gmail.com>
    Signed-off-by: Jan Høydahl <ja...@users.noreply.github.com>
    
    (cherry picked from commit 410004977703489e568c3663c29de53031e7bbd1)
    Signed-off-by: Jan Høydahl <ja...@users.noreply.github.com>
---
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/core/SolrConfig.java  |   4 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  25 +-
 .../org/apache/solr/core/SolrResourceLoader.java   |   1 +
 .../solr/handler/component/SearchHandler.java      |  12 +-
 .../util/circuitbreaker/CPUCircuitBreaker.java     |  19 +-
 .../solr/util/circuitbreaker/CircuitBreaker.java   |  77 ++----
 .../util/circuitbreaker/CircuitBreakerManager.java | 194 +++++----------
 .../circuitbreaker/CircuitBreakerRegistry.java     | 107 ++++++++
 .../util/circuitbreaker/MemoryCircuitBreaker.java  |  21 +-
 ...er.xml => solrconfig-legacy-circuitbreaker.xml} |   5 +-
 ...xml => solrconfig-pluggable-circuitbreaker.xml} |  11 +-
 ...uitBreaker.java => BaseTestCircuitBreaker.java} | 138 ++++++-----
 .../org/apache/solr/util/TestCircuitBreaker.java   | 275 +--------------------
 .../apache/solr/util/TestLegacyCircuitBreaker.java |  38 +++
 .../solr/configsets/_default/conf/solrconfig.xml   |  46 ----
 .../conf/solrconfig.xml                            |  46 ----
 .../deployment-guide/pages/circuit-breakers.adoc   |  53 +---
 18 files changed, 365 insertions(+), 709 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c0bcdd80f64..b6926905aef 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -43,6 +43,8 @@ Improvements
 
 * SOLR-16940: Users can pass Java system properties to the SolrCLI via the SOLR_TOOL_OPTS environment variable. (Houston Putman)
 
+* SOLR-15474: Make Circuit breakers individually pluggable (Atri Sharma, Christine Poerschke, janhoy)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index c976f1a8c7f..8ad87ff98f0 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -90,7 +90,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.DOMConfigNode;
 import org.apache.solr.util.DataConfigNode;
-import org.apache.solr.util.circuitbreaker.CircuitBreakerManager;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -503,7 +503,7 @@ public class SolrConfig implements MapSerializable {
           new SolrPluginInfo(IndexSchemaFactory.class, "schemaFactory", REQUIRE_CLASS),
           new SolrPluginInfo(RestManager.class, "restManager"),
           new SolrPluginInfo(StatsCache.class, "statsCache", REQUIRE_CLASS),
-          new SolrPluginInfo(CircuitBreakerManager.class, "circuitBreaker"));
+          new SolrPluginInfo(CircuitBreaker.class, "circuitBreaker", REQUIRE_CLASS, MULTI_OK));
   public static final Map<String, SolrPluginInfo> classVsSolrPluginInfo;
 
   static {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 661e9f1e29f..b0b1dd2e784 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -173,7 +173,8 @@ import org.apache.solr.util.PropertiesInputStream;
 import org.apache.solr.util.PropertiesOutputStream;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.TestInjection;
-import org.apache.solr.util.circuitbreaker.CircuitBreakerManager;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerRegistry;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
@@ -245,7 +246,7 @@ public class SolrCore implements SolrInfoBean, Closeable {
   private final ConfigSet configSet;
   // singleton listener for all packages used in schema
 
-  private final CircuitBreakerManager circuitBreakerManager;
+  private final CircuitBreakerRegistry circuitBreakerRegistry = new CircuitBreakerRegistry();
 
   private final List<Runnable> confListeners = new CopyOnWriteArrayList<>();
 
@@ -1084,10 +1085,12 @@ public class SolrCore implements SolrInfoBean, Closeable {
       this.configSetProperties = configSet.getProperties();
       // Initialize the metrics manager
       this.coreMetricManager = initCoreMetricManager(solrConfig);
-      this.circuitBreakerManager = initCircuitBreakerManager();
       solrMetricsContext = coreMetricManager.getSolrMetricsContext();
       this.coreMetricManager.loadReporters();
 
+      // init pluggable circuit breakers
+      initPlugins(null, CircuitBreaker.class);
+
       if (updateHandler == null) {
         directoryFactory = initDirectoryFactory();
         recoveryStrategyBuilder = initRecoveryStrategyBuilder();
@@ -1324,13 +1327,6 @@ public class SolrCore implements SolrInfoBean, Closeable {
     return coreMetricManager;
   }
 
-  private CircuitBreakerManager initCircuitBreakerManager() {
-    final PluginInfo info = solrConfig.getPluginInfo(CircuitBreakerManager.class.getName());
-    CircuitBreakerManager circuitBreakerManager = CircuitBreakerManager.build(info);
-
-    return circuitBreakerManager;
-  }
-
   @Override
   public void initializeMetrics(SolrMetricsContext parentContext, String scope) {
     newSearcherCounter = parentContext.counter("new", Category.SEARCHER.toString());
@@ -1710,8 +1706,8 @@ public class SolrCore implements SolrInfoBean, Closeable {
     return updateProcessors;
   }
 
-  public CircuitBreakerManager getCircuitBreakerManager() {
-    return circuitBreakerManager;
+  public CircuitBreakerRegistry getCircuitBreakerRegistry() {
+    return circuitBreakerRegistry;
   }
 
   // this core current usage count
@@ -3164,11 +3160,14 @@ public class SolrCore implements SolrInfoBean, Closeable {
     T def = null;
     for (PluginInfo info : pluginInfos) {
       T o = createInitInstance(info, type, type.getSimpleName(), defClassName);
-      registry.put(info.name, o);
+      if (registry != null) registry.put(info.name, o);
       if (o instanceof SolrMetricProducer) {
         coreMetricManager.registerMetricProducer(
             type.getSimpleName() + "." + info.name, (SolrMetricProducer) o);
       }
+      if (o instanceof CircuitBreaker) {
+        circuitBreakerRegistry.register((CircuitBreaker) o);
+      }
       if (info.isDefault()) {
         def = o;
       }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index e82a05d66b2..ffe7da03481 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -97,6 +97,7 @@ public class SolrResourceLoader
     "request.",
     "update.processor.",
     "util.",
+    "util.circuitbreaker.",
     "spelling.",
     "handler.component.",
     "spelling.suggest.",
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index 73cb880a7fc..5b5672faf8d 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -67,7 +67,7 @@ import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.circuitbreaker.CircuitBreaker;
-import org.apache.solr.util.circuitbreaker.CircuitBreakerManager;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerRegistry;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
@@ -354,23 +354,23 @@ public class SearchHandler extends RequestHandlerBase
 
     final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
 
-    final CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
-    if (circuitBreakerManager.isEnabled()) {
+    final CircuitBreakerRegistry circuitBreakerRegistry = req.getCore().getCircuitBreakerRegistry();
+    if (circuitBreakerRegistry.isEnabled()) {
       List<CircuitBreaker> trippedCircuitBreakers;
 
       if (timer != null) {
         RTimerTree subt = timer.sub("circuitbreaker");
         rb.setTimer(subt);
 
-        trippedCircuitBreakers = circuitBreakerManager.checkTripped();
+        trippedCircuitBreakers = circuitBreakerRegistry.checkTripped();
 
         rb.getTimer().stop();
       } else {
-        trippedCircuitBreakers = circuitBreakerManager.checkTripped();
+        trippedCircuitBreakers = circuitBreakerRegistry.checkTripped();
       }
 
       if (trippedCircuitBreakers != null) {
-        String errorMessage = CircuitBreakerManager.toErrorMessage(trippedCircuitBreakers);
+        String errorMessage = CircuitBreakerRegistry.toErrorMessage(trippedCircuitBreakers);
         rsp.add(STATUS, FAILURE);
         rsp.setException(
             new SolrException(
diff --git a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CPUCircuitBreaker.java b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CPUCircuitBreaker.java
index 70c2fbda507..3e9cf47a2e5 100644
--- a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CPUCircuitBreaker.java
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CPUCircuitBreaker.java
@@ -38,8 +38,7 @@ public class CPUCircuitBreaker extends CircuitBreaker {
   private static final OperatingSystemMXBean operatingSystemMXBean =
       ManagementFactory.getOperatingSystemMXBean();
 
-  private final boolean enabled;
-  private final double cpuUsageThreshold;
+  private double cpuUsageThreshold;
 
   // Assumption -- the value of these parameters will be set correctly before invoking
   // getDebugInfo()
@@ -47,22 +46,16 @@ public class CPUCircuitBreaker extends CircuitBreaker {
 
   private static final ThreadLocal<Double> allowedCPUUsage = ThreadLocal.withInitial(() -> 0.0);
 
-  public CPUCircuitBreaker(CircuitBreakerConfig config) {
-    super(config);
+  public CPUCircuitBreaker() {
+    super();
+  }
 
-    this.enabled = config.getCpuCBEnabled();
-    this.cpuUsageThreshold = config.getCpuCBThreshold();
+  public void setThreshold(double threshold) {
+    this.cpuUsageThreshold = threshold;
   }
 
   @Override
   public boolean isTripped() {
-    if (!isEnabled()) {
-      return false;
-    }
-
-    if (!enabled) {
-      return false;
-    }
 
     double localAllowedCPUUsage = getCpuUsageThreshold();
     double localSeenCPUUsage = calculateLiveCPUUsage();
diff --git a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java
index 79c3bcf6abf..8959c35e7a8 100644
--- a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java
@@ -17,32 +17,32 @@
 
 package org.apache.solr.util.circuitbreaker;
 
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
 /**
- * Default class to define circuit breakers for Solr.
+ * Default base class to define circuit breaker plugins for Solr. <b>Still experimental, may
+ * change</b>
  *
- * <p>There are two (typical) ways to use circuit breakers: 1. Have them checked at admission
- * control by default (use CircuitBreakerManager for the same). 2. Use the circuit breaker in a
- * specific code path(s).
+ * <p>There are two (typical) ways to use circuit breakers:
  *
- * <p>TODO: This class should be grown as the scope of circuit breakers grow.
+ * <ol>
+ *   <li>Have them checked at admission control by default (use CircuitBreakerRegistry for the
+ *       same).
+ *   <li>Use the circuit breaker in a specific code path(s).
+ * </ol>
  *
- * <p>The class and its derivatives raise a standard exception when a circuit breaker is triggered.
- * We should make it into a dedicated exception (https://issues.apache.org/jira/browse/SOLR-14755)
+ * @lucene.experimental
  */
-public abstract class CircuitBreaker {
-  public static final String NAME = "circuitbreaker";
-
-  protected final CircuitBreakerConfig config;
+public abstract class CircuitBreaker implements NamedListInitializedPlugin {
 
-  public CircuitBreaker(CircuitBreakerConfig config) {
-    this.config = config;
+  @Override
+  public void init(NamedList<?> args) {
+    SolrPluginUtils.invokeSetters(this, args);
   }
 
-  // Global config for all circuit breakers. For specific circuit breaker configs, define
-  // your own config.
-  protected boolean isEnabled() {
-    return config.isEnabled();
-  }
+  public CircuitBreaker() {}
 
   /** Check if circuit breaker is tripped. */
   public abstract boolean isTripped();
@@ -52,45 +52,4 @@ public abstract class CircuitBreaker {
 
   /** Get error message when the circuit breaker triggers */
   public abstract String getErrorMessage();
-
-  public static class CircuitBreakerConfig {
-    private final boolean enabled;
-    private final boolean memCBEnabled;
-    private final int memCBThreshold;
-    private final boolean cpuCBEnabled;
-    private final int cpuCBThreshold;
-
-    public CircuitBreakerConfig(
-        final boolean enabled,
-        final boolean memCBEnabled,
-        final int memCBThreshold,
-        final boolean cpuCBEnabled,
-        final int cpuCBThreshold) {
-      this.enabled = enabled;
-      this.memCBEnabled = memCBEnabled;
-      this.memCBThreshold = memCBThreshold;
-      this.cpuCBEnabled = cpuCBEnabled;
-      this.cpuCBThreshold = cpuCBThreshold;
-    }
-
-    public boolean isEnabled() {
-      return enabled;
-    }
-
-    public boolean getMemCBEnabled() {
-      return memCBEnabled;
-    }
-
-    public int getMemCBThreshold() {
-      return memCBThreshold;
-    }
-
-    public boolean getCpuCBEnabled() {
-      return cpuCBEnabled;
-    }
-
-    public int getCpuCBThreshold() {
-      return cpuCBThreshold;
-    }
-  }
 }
diff --git a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
index 3b81a0542e0..52a839ce7a6 100644
--- a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
@@ -17,163 +17,99 @@
 
 package org.apache.solr.util.circuitbreaker;
 
-import com.google.common.annotations.VisibleForTesting;
-import java.util.ArrayList;
-import java.util.List;
+import java.lang.invoke.MethodHandles;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Manages all registered circuit breaker instances. Responsible for a holistic view of whether a
- * circuit breaker has tripped or not.
+ * Single CircuitBreaker that registers both a Memory and a CPU CircuitBreaker. This is only for
+ * backward compatibility with the 9.x versions prior to 9.4.
  *
- * <p>There are two typical ways of using this class's instance: 1. Check if any circuit breaker has
- * triggered -- and know which circuit breaker has triggered. 2. Get an instance of a specific
- * circuit breaker and perform checks.
- *
- * <p>It is a good practice to register new circuit breakers here if you want them checked for every
- * request.
- *
- * <p>NOTE: The current way of registering new default circuit breakers is minimal and not a long
- * term solution. There will be a follow up with a SIP for a schema API design.
+ * @deprecated Use individual Circuit Breakers instead
  */
-public class CircuitBreakerManager implements PluginInfoInitialized {
-  // Class private to potentially allow "family" of circuit breakers to be enabled or disabled
-  private final boolean enableCircuitBreakerManager;
-
-  private final List<CircuitBreaker> circuitBreakerList = new ArrayList<>();
-
-  public CircuitBreakerManager(final boolean enableCircuitBreakerManager) {
-    this.enableCircuitBreakerManager = enableCircuitBreakerManager;
+@Deprecated(since = "9.4")
+public class CircuitBreakerManager extends CircuitBreaker {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private boolean cpuEnabled;
+  private boolean memEnabled;
+  private int memThreshold = 100;
+  private int cpuThreshold = 100;
+  private MemoryCircuitBreaker memCB;
+  private CPUCircuitBreaker cpuCB;
+
+  public CircuitBreakerManager() {
+    super();
   }
 
   @Override
-  public void init(PluginInfo pluginInfo) {
-    CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig = buildCBConfig(pluginInfo);
-
-    // Install the default circuit breakers
-    CircuitBreaker memoryCircuitBreaker = new MemoryCircuitBreaker(circuitBreakerConfig);
-    CircuitBreaker cpuCircuitBreaker = new CPUCircuitBreaker(circuitBreakerConfig);
-
-    register(memoryCircuitBreaker);
-    register(cpuCircuitBreaker);
-  }
-
-  public void register(CircuitBreaker circuitBreaker) {
-    circuitBreakerList.add(circuitBreaker);
+  public boolean isTripped() {
+    return (memEnabled && memCB.isTripped()) || (cpuEnabled && cpuCB.isTripped());
   }
 
-  public void deregisterAll() {
-    circuitBreakerList.clear();
-  }
-  /**
-   * Check and return circuit breakers that have triggered
-   *
-   * @return CircuitBreakers which have triggered, null otherwise.
-   */
-  public List<CircuitBreaker> checkTripped() {
-    List<CircuitBreaker> triggeredCircuitBreakers = null;
-
-    if (enableCircuitBreakerManager) {
-      for (CircuitBreaker circuitBreaker : circuitBreakerList) {
-        if (circuitBreaker.isEnabled() && circuitBreaker.isTripped()) {
-          if (triggeredCircuitBreakers == null) {
-            triggeredCircuitBreakers = new ArrayList<>();
-          }
-
-          triggeredCircuitBreakers.add(circuitBreaker);
-        }
-      }
+  @Override
+  public String getDebugInfo() {
+    StringBuilder sb = new StringBuilder();
+    if (memEnabled) {
+      sb.append(memCB.getDebugInfo()).append("\n");
     }
-
-    return triggeredCircuitBreakers;
-  }
-
-  /**
-   * Returns true if *any* circuit breaker has triggered, false if none have triggered.
-   *
-   * <p>NOTE: This method short circuits the checking of circuit breakers -- the method will return
-   * as soon as it finds a circuit breaker that is enabled and has triggered.
-   */
-  public boolean checkAnyTripped() {
-    if (enableCircuitBreakerManager) {
-      for (CircuitBreaker circuitBreaker : circuitBreakerList) {
-        if (circuitBreaker.isEnabled() && circuitBreaker.isTripped()) {
-          return true;
-        }
-      }
+    if (cpuEnabled) {
+      sb.append(cpuCB.getDebugInfo());
     }
-
-    return false;
+    return sb.toString();
   }
 
-  /**
-   * Construct the final error message to be printed when circuit breakers trip.
-   *
-   * @param circuitBreakerList Input list for circuit breakers.
-   * @return Constructed error message.
-   */
-  public static String toErrorMessage(List<CircuitBreaker> circuitBreakerList) {
+  @Override
+  public String getErrorMessage() {
     StringBuilder sb = new StringBuilder();
-
-    for (CircuitBreaker circuitBreaker : circuitBreakerList) {
-      sb.append(circuitBreaker.getErrorMessage());
+    if (memEnabled) {
+      sb.append(memCB.getErrorMessage());
+    }
+    if (memEnabled && cpuEnabled) {
       sb.append("\n");
     }
-
+    if (cpuEnabled) {
+      sb.append(cpuCB.getErrorMessage());
+    }
     return sb.toString();
   }
 
-  /**
-   * Register default circuit breakers and return a constructed CircuitBreakerManager instance which
-   * serves the given circuit breakers.
-   *
-   * <p>Any default circuit breakers should be registered here.
-   */
-  public static CircuitBreakerManager build(PluginInfo pluginInfo) {
-    boolean enabled =
-        pluginInfo == null
-            ? false
-            : Boolean.parseBoolean(pluginInfo.attributes.getOrDefault("enabled", "false"));
-    CircuitBreakerManager circuitBreakerManager = new CircuitBreakerManager(enabled);
-
-    circuitBreakerManager.init(pluginInfo);
-
-    return circuitBreakerManager;
+  @Override
+  public void init(NamedList<?> args) {
+    super.init(args);
+    log.warn("CircuitBreakerManager is deprecated. Use individual Circuit Breakers instead");
+    if (memEnabled) {
+      memCB = new MemoryCircuitBreaker();
+      memCB.setThreshold(memThreshold);
+    }
+    if (cpuEnabled) {
+      cpuCB = new CPUCircuitBreaker();
+      cpuCB.setThreshold(cpuThreshold);
+    }
   }
 
-  @VisibleForTesting
-  public static CircuitBreaker.CircuitBreakerConfig buildCBConfig(PluginInfo pluginInfo) {
-    boolean enabled = false;
-    boolean cpuCBEnabled = false;
-    boolean memCBEnabled = false;
-    int memCBThreshold = 100;
-    int cpuCBThreshold = 100;
-
-    if (pluginInfo != null) {
-      NamedList<?> args = pluginInfo.initArgs;
+  // The methods below will be called by super class during init
+  public void setMemEnabled(String enabled) {
+    this.memEnabled = Boolean.getBoolean(enabled);
+  }
 
-      enabled = Boolean.parseBoolean(pluginInfo.attributes.getOrDefault("enabled", "false"));
+  public void setMemThreshold(int threshold) {
+    this.memThreshold = threshold;
+  }
 
-      if (args != null) {
-        cpuCBEnabled = Boolean.parseBoolean(args._getStr("cpuEnabled", "false"));
-        memCBEnabled = Boolean.parseBoolean(args._getStr("memEnabled", "false"));
-        memCBThreshold = Integer.parseInt(args._getStr("memThreshold", "100"));
-        cpuCBThreshold = Integer.parseInt(args._getStr("cpuThreshold", "100"));
-      }
-    }
+  public void setMemThreshold(String threshold) {
+    this.memThreshold = Integer.parseInt(threshold);
+  }
 
-    return new CircuitBreaker.CircuitBreakerConfig(
-        enabled, memCBEnabled, memCBThreshold, cpuCBEnabled, cpuCBThreshold);
+  public void setCpuEnabled(String enabled) {
+    this.cpuEnabled = Boolean.getBoolean(enabled);
   }
 
-  public boolean isEnabled() {
-    return enableCircuitBreakerManager;
+  public void setCpuThreshold(int threshold) {
+    this.cpuThreshold = threshold;
   }
 
-  @VisibleForTesting
-  public List<CircuitBreaker> getRegisteredCircuitBreakers() {
-    return circuitBreakerList;
+  public void setCpuThreshold(String threshold) {
+    this.cpuThreshold = Integer.parseInt(threshold);
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerRegistry.java b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerRegistry.java
new file mode 100644
index 00000000000..283203e0f9b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerRegistry.java
@@ -0,0 +1,107 @@
+/*
+ * 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.circuitbreaker;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Keeps track of all registered circuit breaker instances for various request types. Responsible
+ * for a holistic view of whether a circuit breaker has tripped or not.
+ *
+ * @lucene.experimental
+ * @since 9.4
+ */
+public class CircuitBreakerRegistry {
+
+  private final List<CircuitBreaker> circuitBreakerList = new ArrayList<>();
+
+  public CircuitBreakerRegistry() {}
+
+  public void register(CircuitBreaker circuitBreaker) {
+    circuitBreakerList.add(circuitBreaker);
+  }
+
+  @VisibleForTesting
+  public void deregisterAll() {
+    circuitBreakerList.clear();
+  }
+  /**
+   * Check and return circuit breakers that have triggered
+   *
+   * @return CircuitBreakers which have triggered, null otherwise.
+   */
+  public List<CircuitBreaker> checkTripped() {
+    List<CircuitBreaker> triggeredCircuitBreakers = null;
+
+    for (CircuitBreaker circuitBreaker : circuitBreakerList) {
+      if (circuitBreaker.isTripped()) {
+        if (triggeredCircuitBreakers == null) {
+          triggeredCircuitBreakers = new ArrayList<>();
+        }
+
+        triggeredCircuitBreakers.add(circuitBreaker);
+      }
+    }
+
+    return triggeredCircuitBreakers;
+  }
+
+  /**
+   * Returns true if *any* circuit breaker has triggered, false if none have triggered.
+   *
+   * <p>NOTE: This method short circuits the checking of circuit breakers -- the method will return
+   * as soon as it finds a circuit breaker that has triggered.
+   */
+  public boolean checkAnyTripped() {
+    for (CircuitBreaker circuitBreaker : circuitBreakerList) {
+      if (circuitBreaker.isTripped()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Construct the final error message to be printed when circuit breakers trip.
+   *
+   * @param circuitBreakerList Input list for circuit breakers.
+   * @return Constructed error message.
+   */
+  public static String toErrorMessage(List<CircuitBreaker> circuitBreakerList) {
+    StringBuilder sb = new StringBuilder();
+
+    for (CircuitBreaker circuitBreaker : circuitBreakerList) {
+      sb.append(circuitBreaker.getErrorMessage());
+      sb.append("\n");
+    }
+
+    return sb.toString();
+  }
+
+  public boolean isEnabled() {
+    return !circuitBreakerList.isEmpty();
+  }
+
+  @VisibleForTesting
+  public List<CircuitBreaker> getRegisteredCircuitBreakers() {
+    return circuitBreakerList;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java b/solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
index 129ea94579f..49311ddc934 100644
--- a/solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
@@ -26,7 +26,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Tracks the current JVM heap usage and triggers if it exceeds the defined percentage of the
  * maximum heap size allocated to the JVM. This circuit breaker is a part of the default
- * CircuitBreakerManager so is checked for every request -- hence it is realtime. Once the memory
+ * CircuitBreakerRegistry so is checked for every request -- hence it is realtime. Once the memory
  * usage goes below the threshold, it will start allowing queries again.
  *
  * <p>The memory threshold is defined as a percentage of the maximum memory allocated -- see
@@ -36,26 +36,24 @@ public class MemoryCircuitBreaker extends CircuitBreaker {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
 
-  private boolean enabled;
-  private final long heapMemoryThreshold;
+  private long heapMemoryThreshold;
 
   // Assumption -- the value of these parameters will be set correctly before invoking
   // getDebugInfo()
   private static final ThreadLocal<Long> seenMemory = ThreadLocal.withInitial(() -> 0L);
   private static final ThreadLocal<Long> allowedMemory = ThreadLocal.withInitial(() -> 0L);
 
-  public MemoryCircuitBreaker(CircuitBreakerConfig config) {
-    super(config);
-
-    this.enabled = config.getMemCBEnabled();
+  public MemoryCircuitBreaker() {
+    super();
+  }
 
+  public void setThreshold(double thresholdValueInPercentage) {
     long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
 
     if (currentMaxHeap <= 0) {
       throw new IllegalArgumentException("Invalid JVM state for the max heap usage");
     }
 
-    int thresholdValueInPercentage = config.getMemCBThreshold();
     double thresholdInFraction = thresholdValueInPercentage / (double) 100;
     heapMemoryThreshold = (long) (currentMaxHeap * thresholdInFraction);
 
@@ -69,13 +67,6 @@ public class MemoryCircuitBreaker extends CircuitBreaker {
   // overhead of calculating the condition parameters but can result in false positives.
   @Override
   public boolean isTripped() {
-    if (!isEnabled()) {
-      return false;
-    }
-
-    if (!enabled) {
-      return false;
-    }
 
     long localAllowedMemory = getCurrentMemoryThreshold();
     long localSeenMemory = calculateLiveMemoryUsage();
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-legacy-circuitbreaker.xml
similarity index 96%
copy from solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
copy to solr/core/src/test-files/solr/collection1/conf/solrconfig-legacy-circuitbreaker.xml
index 6ab9f893022..fb00756b86c 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-legacy-circuitbreaker.xml
@@ -78,7 +78,10 @@
 
   </query>
 
-  <circuitBreaker class="solr.CircuitBreakerManager" enabled="true">
+  <!-- Legacy config used in 9.x.
+   TODO: Remove this file in 10.0
+   -->
+  <circuitBreaker class="solr.CircuitBreakerManager">
     <str name="memEnabled">true</str>
     <str name="memThreshold">75</str>
     <str name="cpuEnabled">true</str>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml
similarity index 93%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml
index 6ab9f893022..8b27d4fd573 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml
@@ -78,11 +78,12 @@
 
   </query>
 
-  <circuitBreaker class="solr.CircuitBreakerManager" enabled="true">
-    <str name="memEnabled">true</str>
-    <str name="memThreshold">75</str>
-    <str name="cpuEnabled">true</str>
-    <str name="cpuThreshold">75</str>
+  <circuitBreaker class="solr.MemoryCircuitBreaker">
+    <int  name="threshold">75</int>
+  </circuitBreaker>
+
+  <circuitBreaker class="solr.CPUCircuitBreaker">
+    <double  name="threshold">75</double>
   </circuitBreaker>
 
   <initParams path="/select">
diff --git a/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java b/solr/core/src/test/org/apache/solr/util/BaseTestCircuitBreaker.java
similarity index 68%
copy from solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
copy to solr/core/src/test/org/apache/solr/util/BaseTestCircuitBreaker.java
index 47c8d417c07..8e3231f0341 100644
--- a/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
+++ b/solr/core/src/test/org/apache/solr/util/BaseTestCircuitBreaker.java
@@ -27,31 +27,22 @@ import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.core.PluginInfo;
 import org.apache.solr.util.circuitbreaker.CPUCircuitBreaker;
 import org.apache.solr.util.circuitbreaker.CircuitBreaker;
-import org.apache.solr.util.circuitbreaker.CircuitBreakerManager;
 import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
 import org.hamcrest.MatcherAssert;
 import org.junit.After;
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class TestCircuitBreaker extends SolrTestCaseJ4 {
+public abstract class BaseTestCircuitBreaker extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private static final int NUM_DOCS = 20;
 
-  @BeforeClass
-  public static void setUpClass() throws Exception {
-    System.setProperty("filterCache.enabled", "false");
-    System.setProperty("queryResultCache.enabled", "false");
-    System.setProperty("documentCache.enabled", "true");
-
-    initCore("solrconfig-memory-circuitbreaker.xml", "schema.xml");
-    for (int i = 0; i < NUM_DOCS; i++) {
+  protected static void indexDocs() {
+    for (int i = 0; i < 20; i++) {
       assertU(adoc("name", "john smith", "id", "1"));
       assertU(adoc("name", "johathon smith", "id", "2"));
       assertU(adoc("name", "john percival smith", "id", "3"));
@@ -71,21 +62,15 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
 
   @After
   public void after() {
-    h.getCore().getCircuitBreakerManager().deregisterAll();
+    h.getCore().getCircuitBreakerRegistry().deregisterAll();
   }
 
   public void testCBAlwaysTrips() {
     removeAllExistingCircuitBreakers();
 
-    PluginInfo pluginInfo =
-        h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
-
-    CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-        CircuitBreakerManager.buildCBConfig(pluginInfo);
-
-    CircuitBreaker circuitBreaker = new MockCircuitBreaker(circuitBreakerConfig);
+    CircuitBreaker circuitBreaker = new MockCircuitBreaker(true);
 
-    h.getCore().getCircuitBreakerManager().register(circuitBreaker);
+    h.getCore().getCircuitBreakerRegistry().register(circuitBreaker);
 
     expectThrows(
         SolrException.class,
@@ -97,14 +82,12 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
   public void testCBFakeMemoryPressure() {
     removeAllExistingCircuitBreakers();
 
-    PluginInfo pluginInfo =
-        h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
+    CircuitBreaker circuitBreaker = new FakeMemoryPressureCircuitBreaker();
+    MemoryCircuitBreaker memoryCircuitBreaker = (MemoryCircuitBreaker) circuitBreaker;
 
-    CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-        CircuitBreakerManager.buildCBConfig(pluginInfo);
-    CircuitBreaker circuitBreaker = new FakeMemoryPressureCircuitBreaker(circuitBreakerConfig);
+    memoryCircuitBreaker.setThreshold(75);
 
-    h.getCore().getCircuitBreakerManager().register(circuitBreaker);
+    h.getCore().getCircuitBreakerRegistry().register(circuitBreaker);
 
     expectThrows(
         SolrException.class,
@@ -122,15 +105,12 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     try {
       removeAllExistingCircuitBreakers();
 
-      PluginInfo pluginInfo =
-          h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
+      CircuitBreaker circuitBreaker = new BuildingUpMemoryPressureCircuitBreaker();
+      MemoryCircuitBreaker memoryCircuitBreaker = (MemoryCircuitBreaker) circuitBreaker;
 
-      CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-          CircuitBreakerManager.buildCBConfig(pluginInfo);
-      CircuitBreaker circuitBreaker =
-          new BuildingUpMemoryPressureCircuitBreaker(circuitBreakerConfig);
+      memoryCircuitBreaker.setThreshold(75);
 
-      h.getCore().getCircuitBreakerManager().register(circuitBreaker);
+      h.getCore().getCircuitBreakerRegistry().register(circuitBreaker);
 
       List<Future<?>> futures = new ArrayList<>();
 
@@ -166,22 +146,20 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
   }
 
   public void testFakeCPUCircuitBreaker() {
-    AtomicInteger failureCount = new AtomicInteger();
+    removeAllExistingCircuitBreakers();
 
-    ExecutorService executor =
-        ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TestCircuitBreaker"));
-    try {
-      removeAllExistingCircuitBreakers();
+    CircuitBreaker circuitBreaker = new FakeCPUCircuitBreaker();
+    CPUCircuitBreaker cpuCircuitBreaker = (CPUCircuitBreaker) circuitBreaker;
 
-      PluginInfo pluginInfo =
-          h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
+    cpuCircuitBreaker.setThreshold(75);
 
-      CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-          CircuitBreakerManager.buildCBConfig(pluginInfo);
-      CircuitBreaker circuitBreaker = new FakeCPUCircuitBreaker(circuitBreakerConfig);
+    h.getCore().getCircuitBreakerRegistry().register(circuitBreaker);
 
-      h.getCore().getCircuitBreakerManager().register(circuitBreaker);
+    AtomicInteger failureCount = new AtomicInteger();
 
+    ExecutorService executor =
+        ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TestCircuitBreaker"));
+    try {
       List<Future<?>> futures = new ArrayList<>();
 
       for (int i = 0; i < 5; i++) {
@@ -215,23 +193,69 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     }
   }
 
+  public void testResponseWithCBTiming() {
+    removeAllExistingCircuitBreakers();
+
+    assertQ(
+        req("q", "*:*", CommonParams.DEBUG_QUERY, "true"),
+        "//str[@name='rawquerystring']='*:*'",
+        "//str[@name='querystring']='*:*'",
+        "//str[@name='parsedquery']='MatchAllDocsQuery(*:*)'",
+        "//str[@name='parsedquery_toString']='*:*'",
+        "count(//lst[@name='explain']/*)=3",
+        "//lst[@name='explain']/str[@name='1']",
+        "//lst[@name='explain']/str[@name='2']",
+        "//lst[@name='explain']/str[@name='3']",
+        "//str[@name='QParser']",
+        "count(//lst[@name='timing']/*)=3",
+        "//lst[@name='timing']/double[@name='time']",
+        "count(//lst[@name='prepare']/*)>0",
+        "//lst[@name='prepare']/double[@name='time']",
+        "count(//lst[@name='process']/*)>0",
+        "//lst[@name='process']/double[@name='time']");
+
+    CircuitBreaker circuitBreaker = new MockCircuitBreaker(false);
+    h.getCore().getCircuitBreakerRegistry().register(circuitBreaker);
+
+    assertQ(
+        req("q", "*:*", CommonParams.DEBUG_QUERY, "true"),
+        "//str[@name='rawquerystring']='*:*'",
+        "//str[@name='querystring']='*:*'",
+        "//str[@name='parsedquery']='MatchAllDocsQuery(*:*)'",
+        "//str[@name='parsedquery_toString']='*:*'",
+        "count(//lst[@name='explain']/*)=3",
+        "//lst[@name='explain']/str[@name='1']",
+        "//lst[@name='explain']/str[@name='2']",
+        "//lst[@name='explain']/str[@name='3']",
+        "//str[@name='QParser']",
+        "count(//lst[@name='timing']/*)=4",
+        "//lst[@name='timing']/double[@name='time']",
+        "count(//lst[@name='circuitbreaker']/*)>0",
+        "//lst[@name='circuitbreaker']/double[@name='time']",
+        "count(//lst[@name='prepare']/*)>0",
+        "//lst[@name='prepare']/double[@name='time']",
+        "count(//lst[@name='process']/*)>0",
+        "//lst[@name='process']/double[@name='time']");
+  }
+
   private void removeAllExistingCircuitBreakers() {
     List<CircuitBreaker> registeredCircuitBreakers =
-        h.getCore().getCircuitBreakerManager().getRegisteredCircuitBreakers();
+        h.getCore().getCircuitBreakerRegistry().getRegisteredCircuitBreakers();
 
     registeredCircuitBreakers.clear();
   }
 
   private static class MockCircuitBreaker extends MemoryCircuitBreaker {
 
-    public MockCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
+    private final boolean tripped;
+
+    public MockCircuitBreaker(boolean tripped) {
+      this.tripped = tripped;
     }
 
     @Override
     public boolean isTripped() {
-      // Always return true
-      return true;
+      return this.tripped;
     }
 
     @Override
@@ -242,10 +266,6 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
 
   private static class FakeMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
 
-    public FakeMemoryPressureCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-    }
-
     @Override
     protected long calculateLiveMemoryUsage() {
       // Return a number large enough to trigger a pushback from the circuit breaker
@@ -256,9 +276,7 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
   private static class BuildingUpMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
     private AtomicInteger count;
 
-    public BuildingUpMemoryPressureCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-
+    public BuildingUpMemoryPressureCircuitBreaker() {
       this.count = new AtomicInteger(0);
     }
 
@@ -288,10 +306,6 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
   }
 
   private static class FakeCPUCircuitBreaker extends CPUCircuitBreaker {
-    public FakeCPUCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-    }
-
     @Override
     protected double calculateLiveCPUUsage() {
       return 92; // Return a value large enough to trigger the circuit breaker
diff --git a/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java b/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
index 47c8d417c07..37500b4126f 100644
--- a/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
+++ b/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
@@ -17,284 +17,17 @@
 
 package org.apache.solr.util;
 
-import static org.hamcrest.CoreMatchers.containsString;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.util.circuitbreaker.CPUCircuitBreaker;
-import org.apache.solr.util.circuitbreaker.CircuitBreaker;
-import org.apache.solr.util.circuitbreaker.CircuitBreakerManager;
-import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
-import org.hamcrest.MatcherAssert;
-import org.junit.After;
 import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestCircuitBreaker extends SolrTestCaseJ4 {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private static final int NUM_DOCS = 20;
 
+/** Tests the pluggable circuit breaker implementation. The actual tests are in base class. */
+public class TestCircuitBreaker extends BaseTestCircuitBreaker {
   @BeforeClass
   public static void setUpClass() throws Exception {
     System.setProperty("filterCache.enabled", "false");
     System.setProperty("queryResultCache.enabled", "false");
     System.setProperty("documentCache.enabled", "true");
 
-    initCore("solrconfig-memory-circuitbreaker.xml", "schema.xml");
-    for (int i = 0; i < NUM_DOCS; i++) {
-      assertU(adoc("name", "john smith", "id", "1"));
-      assertU(adoc("name", "johathon smith", "id", "2"));
-      assertU(adoc("name", "john percival smith", "id", "3"));
-      assertU(adoc("id", "1", "title", "this is a title.", "inStock_b1", "true"));
-      assertU(adoc("id", "2", "title", "this is another title.", "inStock_b1", "true"));
-      assertU(adoc("id", "3", "title", "Mary had a little lamb.", "inStock_b1", "false"));
-
-      // commit inside the loop to get multiple segments to make search as realistic as possible
-      assertU(commit());
-    }
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  @After
-  public void after() {
-    h.getCore().getCircuitBreakerManager().deregisterAll();
-  }
-
-  public void testCBAlwaysTrips() {
-    removeAllExistingCircuitBreakers();
-
-    PluginInfo pluginInfo =
-        h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
-
-    CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-        CircuitBreakerManager.buildCBConfig(pluginInfo);
-
-    CircuitBreaker circuitBreaker = new MockCircuitBreaker(circuitBreakerConfig);
-
-    h.getCore().getCircuitBreakerManager().register(circuitBreaker);
-
-    expectThrows(
-        SolrException.class,
-        () -> {
-          h.query(req("name:\"john smith\""));
-        });
-  }
-
-  public void testCBFakeMemoryPressure() {
-    removeAllExistingCircuitBreakers();
-
-    PluginInfo pluginInfo =
-        h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
-
-    CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-        CircuitBreakerManager.buildCBConfig(pluginInfo);
-    CircuitBreaker circuitBreaker = new FakeMemoryPressureCircuitBreaker(circuitBreakerConfig);
-
-    h.getCore().getCircuitBreakerManager().register(circuitBreaker);
-
-    expectThrows(
-        SolrException.class,
-        () -> {
-          h.query(req("name:\"john smith\""));
-        });
-  }
-
-  public void testBuildingMemoryPressure() {
-    ExecutorService executor =
-        ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TestCircuitBreaker"));
-
-    AtomicInteger failureCount = new AtomicInteger();
-
-    try {
-      removeAllExistingCircuitBreakers();
-
-      PluginInfo pluginInfo =
-          h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
-
-      CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-          CircuitBreakerManager.buildCBConfig(pluginInfo);
-      CircuitBreaker circuitBreaker =
-          new BuildingUpMemoryPressureCircuitBreaker(circuitBreakerConfig);
-
-      h.getCore().getCircuitBreakerManager().register(circuitBreaker);
-
-      List<Future<?>> futures = new ArrayList<>();
-
-      for (int i = 0; i < 5; i++) {
-        Future<?> future =
-            executor.submit(
-                () -> {
-                  try {
-                    h.query(req("name:\"john smith\""));
-                  } catch (SolrException e) {
-                    MatcherAssert.assertThat(
-                        e.getMessage(), containsString("Circuit Breakers tripped"));
-                    failureCount.incrementAndGet();
-                  } catch (Exception e) {
-                    throw new RuntimeException(e.getMessage());
-                  }
-                });
-
-        futures.add(future);
-      }
-
-      for (Future<?> future : futures) {
-        try {
-          future.get();
-        } catch (Exception e) {
-          throw new RuntimeException(e.getMessage());
-        }
-      }
-    } finally {
-      ExecutorUtil.shutdownAndAwaitTermination(executor);
-      assertEquals("Number of failed queries is not correct", 1, failureCount.get());
-    }
-  }
-
-  public void testFakeCPUCircuitBreaker() {
-    AtomicInteger failureCount = new AtomicInteger();
-
-    ExecutorService executor =
-        ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TestCircuitBreaker"));
-    try {
-      removeAllExistingCircuitBreakers();
-
-      PluginInfo pluginInfo =
-          h.getCore().getSolrConfig().getPluginInfo(CircuitBreakerManager.class.getName());
-
-      CircuitBreaker.CircuitBreakerConfig circuitBreakerConfig =
-          CircuitBreakerManager.buildCBConfig(pluginInfo);
-      CircuitBreaker circuitBreaker = new FakeCPUCircuitBreaker(circuitBreakerConfig);
-
-      h.getCore().getCircuitBreakerManager().register(circuitBreaker);
-
-      List<Future<?>> futures = new ArrayList<>();
-
-      for (int i = 0; i < 5; i++) {
-        Future<?> future =
-            executor.submit(
-                () -> {
-                  try {
-                    h.query(req("name:\"john smith\""));
-                  } catch (SolrException e) {
-                    MatcherAssert.assertThat(
-                        e.getMessage(), containsString("Circuit Breakers tripped"));
-                    failureCount.incrementAndGet();
-                  } catch (Exception e) {
-                    throw new RuntimeException(e.getMessage());
-                  }
-                });
-
-        futures.add(future);
-      }
-
-      for (Future<?> future : futures) {
-        try {
-          future.get();
-        } catch (Exception e) {
-          throw new RuntimeException(e.getMessage());
-        }
-      }
-    } finally {
-      ExecutorUtil.shutdownAndAwaitTermination(executor);
-      assertEquals("Number of failed queries is not correct", 5, failureCount.get());
-    }
-  }
-
-  private void removeAllExistingCircuitBreakers() {
-    List<CircuitBreaker> registeredCircuitBreakers =
-        h.getCore().getCircuitBreakerManager().getRegisteredCircuitBreakers();
-
-    registeredCircuitBreakers.clear();
-  }
-
-  private static class MockCircuitBreaker extends MemoryCircuitBreaker {
-
-    public MockCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-    }
-
-    @Override
-    public boolean isTripped() {
-      // Always return true
-      return true;
-    }
-
-    @Override
-    public String getDebugInfo() {
-      return "MockCircuitBreaker";
-    }
-  }
-
-  private static class FakeMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
-
-    public FakeMemoryPressureCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-    }
-
-    @Override
-    protected long calculateLiveMemoryUsage() {
-      // Return a number large enough to trigger a pushback from the circuit breaker
-      return Long.MAX_VALUE;
-    }
-  }
-
-  private static class BuildingUpMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
-    private AtomicInteger count;
-
-    public BuildingUpMemoryPressureCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-
-      this.count = new AtomicInteger(0);
-    }
-
-    @Override
-    protected long calculateLiveMemoryUsage() {
-      int localCount = count.getAndIncrement();
-
-      if (localCount >= 4) {
-        // TODO: To be removed
-        if (log.isInfoEnabled()) {
-          String logMessage =
-              "Blocking query from BuildingUpMemoryPressureCircuitBreaker for count " + localCount;
-          log.info(logMessage);
-        }
-        return Long.MAX_VALUE;
-      }
-
-      // TODO: To be removed
-      if (log.isInfoEnabled()) {
-        String logMessage =
-            "BuildingUpMemoryPressureCircuitBreaker: Returning unblocking value for count "
-                + localCount;
-        log.info(logMessage);
-      }
-      return Long.MIN_VALUE; // Random number guaranteed to not trip the circuit breaker
-    }
-  }
-
-  private static class FakeCPUCircuitBreaker extends CPUCircuitBreaker {
-    public FakeCPUCircuitBreaker(CircuitBreakerConfig config) {
-      super(config);
-    }
-
-    @Override
-    protected double calculateLiveCPUUsage() {
-      return 92; // Return a value large enough to trigger the circuit breaker
-    }
+    initCore("solrconfig-pluggable-circuitbreaker.xml", "schema.xml");
+    BaseTestCircuitBreaker.indexDocs();
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/util/TestLegacyCircuitBreaker.java b/solr/core/src/test/org/apache/solr/util/TestLegacyCircuitBreaker.java
new file mode 100644
index 00000000000..cd4cc499a36
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/util/TestLegacyCircuitBreaker.java
@@ -0,0 +1,38 @@
+/*
+ * 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 org.junit.BeforeClass;
+
+/**
+ * Tests the original circuit breaker configuration format, which was not configurable.
+ *
+ * @deprecated Remove in 10.0
+ */
+@Deprecated(since = "9.4")
+public class TestLegacyCircuitBreaker extends BaseTestCircuitBreaker {
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    initCore("solrconfig-legacy-circuitbreaker.xml", "schema.xml");
+    BaseTestCircuitBreaker.indexDocs();
+  }
+}
diff --git a/solr/server/solr/configsets/_default/conf/solrconfig.xml b/solr/server/solr/configsets/_default/conf/solrconfig.xml
index 8023d7bb98f..ecd3693f642 100644
--- a/solr/server/solr/configsets/_default/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/_default/conf/solrconfig.xml
@@ -559,52 +559,6 @@
 
   </query>
 
-  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-     Circuit Breaker Section - This section consists of configurations for
-     circuit breakers
-     ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
-  <!-- Circuit breakers are designed to allow stability and predictable query
-     execution. They prevent operations that can take down the node and cause
-     noisy neighbour issues.
-
-     The CircuitBreakerManager is the default manager for all circuit breakers.
-     The enabled flag here controls the activation/deactivation of all circuit
-     breakers specified within.
-  -->
-  <circuitBreaker class="solr.CircuitBreakerManager" enabled="true">
-    <!-- Memory Circuit Breaker
-
-     Specific configuration for max JVM heap usage circuit breaker. This configuration defines
-     whether the circuit breaker is enabled and the threshold percentage of maximum heap allocated
-     beyond which queries will be rejected until the current JVM usage goes below the threshold.
-     The valid value for this range is 50-95.
-
-     Consider a scenario where the max heap allocated is 4 GB and memThreshold is defined as 75.
-     Threshold JVM usage will be 4 * 0.75 = 3 GB. Its generally a good idea to keep this value
-     between 75 - 80% of maximum heap allocated.
-
-     If, at any point, the current JVM heap usage goes above 3 GB, queries will be rejected until
-     the heap usage goes below 3 GB again. If you see queries getting rejected with 503 error code,
-     check for "Circuit Breakers tripped" in logs and the corresponding error message should tell
-     you what transpired (if the failure was caused by tripped circuit breakers).
-    -->
-    <!--
-    <str name="memEnabled">true</str>
-    <str name="memThreshold">75</str>
-    -->
-
-    <!-- CPU Circuit Breaker Configuration
-
-     Specific configuration for CPU utilization based circuit breaker. This configuration defines
-     whether the circuit breaker is enabled and the average load over the last minute at which the
-     circuit breaker should start rejecting queries.
-    -->
-    <!--
-    <str name="cpuEnabled">true</str>
-    <str name="cpuThreshold">75</str>
-    -->
-  </circuitBreaker>
-
   <!-- Request Dispatcher
 
        This section contains instructions for how the SolrDispatchFilter
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index bd61f96dc82..2fd3d0ae857 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -579,52 +579,6 @@
 
   </query>
 
-  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-     Circuit Breaker Section - This section consists of configurations for
-     circuit breakers
-     ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
-  <!-- Circuit breakers are designed to allow stability and predictable query
-     execution. They prevent operations that can take down the node and cause
-     noisy neighbour issues.
-
-     The CircuitBreakerManager is the default manager for all circuit breakers.
-     The enabled flag here controls the activation/deactivation of all circuit
-     breakers specified within.
-  -->
-  <circuitBreaker class="solr.CircuitBreakerManager" enabled="true">
-    <!-- Memory Circuit Breaker
-
-     Specific configuration for max JVM heap usage circuit breaker. This configuration defines
-     whether the circuit breaker is enabled and the threshold percentage of maximum heap allocated
-     beyond which queries will be rejected until the current JVM usage goes below the threshold.
-     The valid value for this range is 50-95.
-
-     Consider a scenario where the max heap allocated is 4 GB and memThreshold is defined as 75.
-     Threshold JVM usage will be 4 * 0.75 = 3 GB. Its generally a good idea to keep this value
-     between 75 - 80% of maximum heap allocated.
-
-     If, at any point, the current JVM heap usage goes above 3 GB, queries will be rejected until
-     the heap usage goes below 3 GB again. If you see queries getting rejected with 503 error code,
-     check for "Circuit Breakers tripped" in logs and the corresponding error message should tell
-     you what transpired (if the failure was caused by tripped circuit breakers).
-    -->
-    <!--
-    <str name="memEnabled">true</str>
-    <str name="memThreshold">75</str>
-    -->
-
-    <!-- CPU Circuit Breaker Configuration
-
-     Specific configuration for CPU utilization based circuit breaker. This configuration defines
-     whether the circuit breaker is enabled and the average load over the last minute at which the
-     circuit breaker should start rejecting queries.
-    -->
-    <!--
-    <str name="cpuEnabled">true</str>
-    <str name="cpuThreshold">75</str>
-    -->
-  </circuitBreaker>
-
   <!-- Request Dispatcher
 
        This section contains instructions for how the SolrDispatchFilter
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/circuit-breakers.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/circuit-breakers.adoc
index 5b29b4acd9c..a96658e61bf 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/circuit-breakers.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/circuit-breakers.adoc
@@ -27,23 +27,7 @@ If circuit breakers are enabled, requests may be rejected under the condition of
 It is up to the client to handle this error and potentially build a retrial logic as this should ideally be a transient situation.
 
 == Circuit Breaker Configurations
-All circuit breaker configurations are listed in the `<circuitBreaker>` tags in `solrconfig.xml` as shown below:
-
-[source,xml]
-----
-<circuitBreaker class="solr.CircuitBreakerManager" enabled="true">
-  <!-- All specific configs in this section -->
-</circuitBreaker>
-----
-
-The `enabled` attribute controls the global activation/deactivation of circuit breakers.
-If this flag is disabled, all circuit breakers will be disabled globally.
-Per circuit breaker configurations are specified in their respective sections later.
-
-This attribute acts as the highest authority and global controller of circuit breakers.
-For using specific circuit breakers, each one needs to be individually enabled in addition to this flag being enabled.
-
-`CircuitBreakerManager` is the default manager for all circuit breakers that should be defined in the tag unless the user wishes to use a custom implementation.
+All circuit breaker configurations are listed as independent `<circuitBreaker>` entries in `solrconfig.xml` as shown below.
 
 == Currently Supported Circuit Breakers
 
@@ -52,54 +36,41 @@ For using specific circuit breakers, each one needs to be individually enabled i
 This circuit breaker tracks JVM heap memory usage and rejects incoming search requests with a 503 error code if the heap usage exceeds a configured percentage of maximum heap allocated to the JVM (-Xmx).
 The main configuration for this circuit breaker is controlling the threshold percentage at which the breaker will trip.
 
-Configuration for JVM heap usage based circuit breaker:
+To enable and configure the JVM heap usage based circuit breaker, add the following:
 
 [source,xml]
 ----
-<str name="memEnabled">true</str>
+<circuitBreaker class="org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker">
+ <double name="threshold">75</double>
+</circuitBreaker>
 ----
 
-Note that this configuration will be overridden by the global circuit breaker flag -- if circuit breakers are disabled, this flag will not help you.
-
-The triggering threshold is defined as a percentage of the max heap allocated to the JVM.
-It is controlled by the below configuration:
-
-[source,xml]
-----
-<str name="memThreshold">75</str>
-----
+The `threshold` is defined as a percentage of the max heap allocated to the JVM.
 
 It does not logically make sense to have a threshold below 50% and above 95% of the max heap allocated to the JVM.
 Hence, the range of valid values for this parameter is [50, 95], both inclusive.
 
 Consider the following example:
 
-JVM has been allocated a maximum heap of 5GB (-Xmx) and `memThreshold` is set to `75`.
+JVM has been allocated a maximum heap of 5GB (-Xmx) and `threshold` is set to `75`.
 In this scenario, the heap usage at which the circuit breaker will trip is 3.75GB.
 
-
 === CPU Utilization
 
 This circuit breaker tracks CPU utilization and triggers if the average CPU utilization over the last one minute exceeds a configurable threshold.
 Note that the value used in computation is over the last one minute -- so a sudden spike in traffic that goes down might still cause the circuit breaker to trigger for a short while before it resolves and updates the value.
 For more details of the calculation, please see https://en.wikipedia.org/wiki/Load_(computing)
 
-Configuration for CPU utilization based circuit breaker:
+To enable and configure the CPU utilization based circuit breaker:
 
 [source,xml]
 ----
-<str name="cpuEnabled">true</str>
+<circuitBreaker class="org.apache.solr.util.circuitbreaker.CPUCircuitBreaker">
+ <double  name="threshold">75</double>
+</circuitBreaker>
 ----
 
-Note that this configuration will be overridden by the global circuit breaker flag -- if circuit breakers are disabled, this flag will not help you.
-
-The triggering threshold is defined in units of CPU utilization.
-The configuration to control this is as below:
-
-[source,xml]
-----
-<str name="cpuThreshold">75</str>
-----
+The `threshold` is defined in units of CPU utilization.
 
 == Performance Considerations