You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by at...@apache.org on 2020/08/20 07:51:42 UTC

[lucene-solr] branch master updated: SOLR-14615: Implement CPU Utilization Based Circuit Breaker (#1737)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2f37f40  SOLR-14615: Implement CPU Utilization Based Circuit Breaker (#1737)
2f37f40 is described below

commit 2f37f401715b6cefe0425ca591c6764f2072045e
Author: Atri Sharma <at...@gmail.com>
AuthorDate: Thu Aug 20 13:21:26 2020 +0530

    SOLR-14615: Implement CPU Utilization Based Circuit Breaker (#1737)
    
    This commit introduces CPU based circuit breaker. This circuit breaker
    tracks the average CPU load per minute and triggers if the value exceeds
    a configurable value.
    
    This commit also adds a specific control flag for Memory Circuit Breaker
    to allow enabling/disabling the same.
---
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/core/SolrConfig.java  |  27 +++--
 .../util/circuitbreaker/CPUCircuitBreaker.java     | 116 +++++++++++++++++++++
 .../solr/util/circuitbreaker/CircuitBreaker.java   |   8 ++
 .../util/circuitbreaker/CircuitBreakerManager.java |  13 ++-
 .../util/circuitbreaker/MemoryCircuitBreaker.java  |  20 +++-
 .../resources/EditableSolrConfigAttributes.json    |  13 ++-
 .../conf/solrconfig-memory-circuitbreaker.xml      |  11 +-
 .../test/org/apache/solr/core/SolrCoreTest.java    |   3 +-
 .../org/apache/solr/util/TestCircuitBreaker.java   | 112 ++++++++++++++++++--
 .../solr/configsets/_default/conf/solrconfig.xml   |  40 ++++---
 solr/solr-ref-guide/src/circuit-breakers.adoc      |  38 +++++--
 12 files changed, 348 insertions(+), 55 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 911834e..18ae75e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -42,6 +42,8 @@ Improvements
 
 * SOLR-13528 Rate Limiting in Solr (Atri Sharma, Mike Drob)
 
+* SOLR-14615: CPU Utilization Based Circuit Breaker (Atri Sharma)
+
 Other Changes
 ----------------------
 * SOLR-14656: Autoscaling framework removed (Ishan Chattopadhyaya, noble, Ilan Ginzburg)
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 743a5f4..0a86f0c 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -228,10 +228,13 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     queryResultMaxDocsCached = getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
     enableLazyFieldLoading = getBool("query/enableLazyFieldLoading", false);
 
-    useCircuitBreakers = getBool("circuitBreaker/useCircuitBreakers", false);
-    memoryCircuitBreakerThresholdPct = getInt("circuitBreaker/memoryCircuitBreakerThresholdPct", 95);
+    useCircuitBreakers = getBool("circuitBreakers/@enabled", false);
+    cpuCBEnabled = getBool("circuitBreakers/cpuBreaker/@enabled", false);
+    memCBEnabled = getBool("circuitBreakers/memBreaker/@enabled", false);
+    memCBThreshold = getInt("circuitBreakers/memBreaker/@threshold", 95);
+    cpuCBThreshold = getInt("circuitBreakers/cpuBreaker/@threshold", 95);
 
-    validateMemoryBreakerThreshold();
+    validateCircuitBreakerThresholds();
     
     filterCacheConfig = CacheConfig.getConfig(this, "query/filterCache");
     queryResultCacheConfig = CacheConfig.getConfig(this, "query/queryResultCache");
@@ -530,7 +533,10 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
   // Circuit Breaker Configuration
   public final boolean useCircuitBreakers;
-  public final int memoryCircuitBreakerThresholdPct;
+  public final int memCBThreshold;
+  public final boolean memCBEnabled;
+  public final boolean cpuCBEnabled;
+  public final int cpuCBThreshold;
 
   // IndexConfig settings
   public final SolrIndexConfig indexConfig;
@@ -811,10 +817,12 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     loader.reloadLuceneSPI();
   }
 
-  private void validateMemoryBreakerThreshold() {
+  private void validateCircuitBreakerThresholds() {
     if (useCircuitBreakers) {
-      if (memoryCircuitBreakerThresholdPct > 95 || memoryCircuitBreakerThresholdPct < 50) {
-        throw new IllegalArgumentException("Valid value range of memoryCircuitBreakerThresholdPct is 50 -  95");
+      if (memCBEnabled) {
+        if (memCBThreshold > 95 || memCBThreshold < 50) {
+          throw new IllegalArgumentException("Valid value range of memoryCircuitBreakerThresholdPct is 50 -  95");
+        }
       }
     }
   }
@@ -889,7 +897,10 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     m.put("enableLazyFieldLoading", enableLazyFieldLoading);
     m.put("maxBooleanClauses", booleanQueryMaxClauseCount);
     m.put("useCircuitBreakers", useCircuitBreakers);
-    m.put("memoryCircuitBreakerThresholdPct", memoryCircuitBreakerThresholdPct);
+    m.put("cpuCircuitBreakerEnabled", cpuCBEnabled);
+    m.put("memoryCircuitBreakerEnabled", memCBEnabled);
+    m.put("memoryCircuitBreakerThresholdPct", memCBThreshold);
+    m.put("cpuCircuitBreakerThreshold", cpuCBThreshold);
     for (SolrPluginInfo plugin : plugins) {
       List<PluginInfo> infos = getPluginInfos(plugin.clazz.getName());
       if (infos == null || infos.isEmpty()) continue;
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
new file mode 100644
index 0000000..45dc1e8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/circuitbreaker/CPUCircuitBreaker.java
@@ -0,0 +1,116 @@
+/*
+ * 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 java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+
+import org.apache.solr.core.SolrConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * Tracks current CPU usage and triggers if the specified threshold is breached.
+ *
+ * This circuit breaker gets the average CPU load over the last minute and uses
+ * that data to take a decision. We depend on OperatingSystemMXBean which does
+ * not allow a configurable interval of collection of data.
+ * //TODO: Use Codahale Meter to calculate the value locally.
+ * </p>
+ *
+ * <p>
+ * The configuration to define which mode to use and the trigger threshold are defined in
+ * solrconfig.xml
+ * </p>
+ */
+public class CPUCircuitBreaker extends CircuitBreaker {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final OperatingSystemMXBean operatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean();
+
+  private final boolean enabled;
+  private final double cpuUsageThreshold;
+
+  // Assumption -- the value of these parameters will be set correctly before invoking getDebugInfo()
+  private static final ThreadLocal<Double> seenCPUUsage = ThreadLocal.withInitial(() -> 0.0);
+
+  private static final ThreadLocal<Double> allowedCPUUsage = ThreadLocal.withInitial(() -> 0.0);
+
+  public CPUCircuitBreaker(SolrConfig solrConfig) {
+    super(solrConfig);
+
+    this.enabled = solrConfig.cpuCBEnabled;
+    this.cpuUsageThreshold = solrConfig.cpuCBThreshold;
+  }
+
+  @Override
+  public boolean isTripped() {
+    if (!isEnabled()) {
+      return false;
+    }
+
+    if (!enabled) {
+      return false;
+    }
+
+    double localAllowedCPUUsage = getCpuUsageThreshold();
+    double localSeenCPUUsage = calculateLiveCPUUsage();
+
+    if (localSeenCPUUsage < 0) {
+      if (log.isWarnEnabled()) {
+        String msg = "Unable to get CPU usage";
+
+        log.warn(msg);
+      }
+
+      return false;
+    }
+
+    allowedCPUUsage.set(localAllowedCPUUsage);
+
+    seenCPUUsage.set(localSeenCPUUsage);
+
+    return (localSeenCPUUsage >= localAllowedCPUUsage);
+  }
+
+  @Override
+  public String getDebugInfo() {
+
+    if (seenCPUUsage.get() == 0.0 || seenCPUUsage.get() == 0.0) {
+      log.warn("CPUCircuitBreaker's monitored values (seenCPUUSage, allowedCPUUsage) not set");
+    }
+
+    return "seenCPUUSage=" + seenCPUUsage.get() + " allowedCPUUsage=" + allowedCPUUsage.get();
+  }
+
+  @Override
+  public String getErrorMessage() {
+    return "CPU Circuit Breaker triggered as seen CPU usage is above allowed threshold." +
+        "Seen CPU usage " + seenCPUUsage.get() + " and allocated threshold " +
+        allowedCPUUsage.get();
+  }
+
+  public double getCpuUsageThreshold() {
+    return cpuUsageThreshold;
+  }
+
+  protected double calculateLiveCPUUsage() {
+    return operatingSystemMXBean.getSystemLoadAverage();
+  }
+}
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 f56f81e..63ad7fd 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
@@ -27,6 +27,9 @@ import org.apache.solr.core.SolrConfig;
  *  2. Use the circuit breaker in a specific code path(s).
  *
  * TODO: This class should be grown as the scope of circuit breakers grow.
+ *
+ * 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)
  * </p>
  */
 public abstract class CircuitBreaker {
@@ -53,4 +56,9 @@ public abstract class CircuitBreaker {
    * Get debug useful info.
    */
   public abstract String getDebugInfo();
+
+  /**
+   * Get error message when the circuit breaker triggers
+   */
+  public abstract String getErrorMessage();
 }
\ No newline at end of file
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 584b933..ed7f62d 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
@@ -20,6 +20,7 @@ package org.apache.solr.util.circuitbreaker;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.core.SolrConfig;
 
 /**
@@ -107,9 +108,7 @@ public class CircuitBreakerManager {
     StringBuilder sb = new StringBuilder();
 
     for (CircuitBreaker circuitBreaker : circuitBreakerList) {
-      sb.append(circuitBreaker.getClass().getName());
-      sb.append(" ");
-      sb.append(circuitBreaker.getDebugInfo());
+      sb.append(circuitBreaker.getErrorMessage());
       sb.append("\n");
     }
 
@@ -127,8 +126,16 @@ public class CircuitBreakerManager {
 
     // Install the default circuit breakers
     CircuitBreaker memoryCircuitBreaker = new MemoryCircuitBreaker(solrConfig);
+    CircuitBreaker cpuCircuitBreaker = new CPUCircuitBreaker(solrConfig);
+
     circuitBreakerManager.register(memoryCircuitBreaker);
+    circuitBreakerManager.register(cpuCircuitBreaker);
 
     return circuitBreakerManager;
   }
+
+  @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 629d84a..797677d 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
@@ -43,22 +43,25 @@ 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;
 
   // Assumption -- the value of these parameters will be set correctly before invoking getDebugInfo()
-  private final ThreadLocal<Long> seenMemory = new ThreadLocal<>();
-  private final ThreadLocal<Long> allowedMemory = new ThreadLocal<>();
+  private static final ThreadLocal<Long> seenMemory = ThreadLocal.withInitial(() -> 0L);
+  private static final ThreadLocal<Long> allowedMemory = ThreadLocal.withInitial(() -> 0L);
 
   public MemoryCircuitBreaker(SolrConfig solrConfig) {
     super(solrConfig);
 
+    this.enabled = solrConfig.memCBEnabled;
+
     long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
 
     if (currentMaxHeap <= 0) {
       throw new IllegalArgumentException("Invalid JVM state for the max heap usage");
     }
 
-    int thresholdValueInPercentage = solrConfig.memoryCircuitBreakerThresholdPct;
+    int thresholdValueInPercentage = solrConfig.memCBThreshold;
     double thresholdInFraction = thresholdValueInPercentage / (double) 100;
     heapMemoryThreshold = (long) (currentMaxHeap * thresholdInFraction);
 
@@ -76,6 +79,10 @@ public class MemoryCircuitBreaker extends CircuitBreaker {
       return false;
     }
 
+    if (!enabled) {
+      return false;
+    }
+
     long localAllowedMemory = getCurrentMemoryThreshold();
     long localSeenMemory = calculateLiveMemoryUsage();
 
@@ -95,6 +102,13 @@ public class MemoryCircuitBreaker extends CircuitBreaker {
     return "seenMemory=" + seenMemory.get() + " allowedMemory=" + allowedMemory.get();
   }
 
+  @Override
+  public String getErrorMessage() {
+    return "Memory Circuit Breaker triggered as JVM heap usage values are greater than allocated threshold." +
+        "Seen JVM heap memory usage " + seenMemory.get() + " and allocated threshold " +
+        allowedMemory.get();
+  }
+
   private long getCurrentMemoryThreshold() {
     return heapMemoryThreshold;
   }
diff --git a/solr/core/src/resources/EditableSolrConfigAttributes.json b/solr/core/src/resources/EditableSolrConfigAttributes.json
index 0ed8333..1f49982 100644
--- a/solr/core/src/resources/EditableSolrConfigAttributes.json
+++ b/solr/core/src/resources/EditableSolrConfigAttributes.json
@@ -55,8 +55,17 @@
     "queryResultMaxDocsCached":1,
     "enableLazyFieldLoading":1,
     "boolTofilterOptimizer":1,
-    "useCircuitBreakers":10,
-    "memoryCircuitBreakerThresholdPct":20,
+    "circuitBreakers":{
+      "enabled":10,
+      "memBreaker":{
+        "enabled":10,
+        "threshold":20
+      },
+      "cpuBreaker":{
+        "enabled":10,
+        "threshold":20
+      }
+    },
     "maxBooleanClauses":1},
   "jmx":{
     "agentId":0,
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-memory-circuitbreaker.xml
index b6b20ff..d6adf92 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-memory-circuitbreaker.xml
@@ -78,13 +78,10 @@
 
   </query>
 
-  <circuitBreaker>
-
-    <useCircuitBreakers>true</useCircuitBreakers>
-
-    <memoryCircuitBreakerThresholdPct>75</memoryCircuitBreakerThresholdPct>
-
-  </circuitBreaker>
+  <circuitBreakers enabled="true">
+    <cpuBreaker enabled="true" threshold="75"/>
+    <memBreaker enabled="true" threshold="75"/>
+  </circuitBreakers>
 
   <initParams path="/select">
     <lst name="defaults">
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 8b20471..59069db 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -267,7 +267,8 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
     assertEquals("wrong config for enableLazyFieldLoading", true, solrConfig.enableLazyFieldLoading);
     assertEquals("wrong config for queryResultWindowSize", 10, solrConfig.queryResultWindowSize);
     assertEquals("wrong config for useCircuitBreakers", false, solrConfig.useCircuitBreakers);
-    assertEquals("wrong config for memoryCircuitBreakerThresholdPct", 95, solrConfig.memoryCircuitBreakerThresholdPct);
+    assertEquals("wrong config for memoryCircuitBreakerThresholdPct", 95, solrConfig.memCBThreshold);
+    assertEquals("wrong config for cpuCircuitBreakerThreshold", 95, solrConfig.cpuCBThreshold);
   }
 
   /**
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 f41667f..347568c 100644
--- a/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
+++ b/solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
@@ -18,8 +18,11 @@
 package org.apache.solr.util;
 
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -31,6 +34,7 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CPUCircuitBreaker;
 import org.apache.solr.util.circuitbreaker.CircuitBreaker;
 import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
 import org.junit.After;
@@ -41,6 +45,9 @@ import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.hamcrest.CoreMatchers.containsString;
+
+@SuppressWarnings({"rawtypes"})
 public class TestCircuitBreaker extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final static int NUM_DOCS = 20;
@@ -84,6 +91,8 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
     args.put(CommonParams.FL, "id");
 
+    removeAllExistingCircuitBreakers();
+
     CircuitBreaker circuitBreaker = new MockCircuitBreaker(h.getCore().getSolrConfig());
 
     h.getCore().getCircuitBreakerManager().register(circuitBreaker);
@@ -99,6 +108,8 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
     args.put(CommonParams.FL, "id");
 
+    removeAllExistingCircuitBreakers();
+
     CircuitBreaker circuitBreaker = new FakeMemoryPressureCircuitBreaker(h.getCore().getSolrConfig());
 
     h.getCore().getCircuitBreakerManager().register(circuitBreaker);
@@ -119,33 +130,42 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     AtomicInteger failureCount = new AtomicInteger();
 
     try {
+      removeAllExistingCircuitBreakers();
+
       CircuitBreaker circuitBreaker = new BuildingUpMemoryPressureCircuitBreaker(h.getCore().getSolrConfig());
 
       h.getCore().getCircuitBreakerManager().register(circuitBreaker);
 
+      List<Future<?>> futures = new ArrayList<>();
+
       for (int i = 0; i < 5; i++) {
-        executor.submit(() -> {
+        Future<?> future = executor.submit(() -> {
           try {
             h.query(req("name:\"john smith\""));
           } catch (SolrException e) {
-            if (!e.getMessage().startsWith("Circuit Breakers tripped")) {
-              if (log.isInfoEnabled()) {
-                String logMessage = "Expected error message for testBuildingMemoryPressure was not received. Error message " + e.getMessage();
-                log.info(logMessage);
-              }
-              throw new RuntimeException("Expected error message was not received. Error message " + e.getMessage());
-            }
+            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());
+        }
       }
 
       executor.shutdown();
       try {
         executor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
       } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
         throw new RuntimeException(e.getMessage());
       }
 
@@ -157,6 +177,59 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     }
   }
 
+  public void testFakeCPUCircuitBreaker() {
+    AtomicInteger failureCount = new AtomicInteger();
+
+    ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    try {
+      removeAllExistingCircuitBreakers();
+
+      CircuitBreaker circuitBreaker = new FakeCPUCircuitBreaker(h.getCore().getSolrConfig());
+
+      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) {
+            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());
+        }
+      }
+
+      executor.shutdown();
+      try {
+        executor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e.getMessage());
+      }
+
+      assertEquals("Number of failed queries is not correct",5, failureCount.get());
+    } finally {
+      if (!executor.isShutdown()) {
+        executor.shutdown();
+      }
+    }
+  }
+
   public void testResponseWithCBTiming() {
     assertQ(req("q", "*:*", CommonParams.DEBUG_QUERY, "true"),
         "//str[@name='rawquerystring']='*:*'",
@@ -179,7 +252,13 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     );
   }
 
-  private class MockCircuitBreaker extends CircuitBreaker {
+  private void removeAllExistingCircuitBreakers() {
+    List<CircuitBreaker> registeredCircuitBreakers = h.getCore().getCircuitBreakerManager().getRegisteredCircuitBreakers();
+
+    registeredCircuitBreakers.clear();
+  }
+
+  private static class MockCircuitBreaker extends MemoryCircuitBreaker {
 
     public MockCircuitBreaker(SolrConfig solrConfig) {
       super(solrConfig);
@@ -197,7 +276,7 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     }
   }
 
-  private class FakeMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
+  private static class FakeMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
 
     public FakeMemoryPressureCircuitBreaker(SolrConfig solrConfig) {
       super(solrConfig);
@@ -210,7 +289,7 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
     }
   }
 
-  private class BuildingUpMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
+  private static class BuildingUpMemoryPressureCircuitBreaker extends MemoryCircuitBreaker {
     private AtomicInteger count;
 
     public BuildingUpMemoryPressureCircuitBreaker(SolrConfig solrConfig) {
@@ -240,4 +319,15 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
       return Long.MIN_VALUE; // Random number guaranteed to not trip the circuit breaker
     }
   }
+
+  private static class FakeCPUCircuitBreaker extends CPUCircuitBreaker {
+    public FakeCPUCircuitBreaker(SolrConfig solrConfig) {
+      super(solrConfig);
+    }
+
+    @Override
+    protected double calculateLiveCPUUsage() {
+      return 92; // Return a value large enough to trigger the circuit breaker
+    }
+  }
 }
diff --git a/solr/server/solr/configsets/_default/conf/solrconfig.xml b/solr/server/solr/configsets/_default/conf/solrconfig.xml
index 33b6cd5..a69e46c 100644
--- a/solr/server/solr/configsets/_default/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/_default/conf/solrconfig.xml
@@ -582,27 +582,24 @@
      Circuit Breaker Section - This section consists of configurations for
      circuit breakers
      ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
-  <circuitBreaker>
-    <!-- Enable Circuit Breakers
+
+    <!-- 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.
 
      This flag is the uber control switch which controls the activation/deactivation of all circuit
-     breakers. At the moment, the only circuit breaker (max JVM circuit breaker) does not have its
-     own specific configuration. However, if a circuit breaker wishes to be independently configurable,
+     breakers. If a circuit breaker wishes to be independently configurable,
      they are free to add their specific configuration but need to ensure that this flag is always
      respected - this should have veto over all independent configuration flags.
     -->
-    <!--
-   <useCircuitBreakers>true</useCircuitBreakers>
-    -->
+    <circuitBreakers enabled="true">
 
-    <!-- Memory Circuit Breaker Threshold In Percentage
+    <!-- Memory Circuit Breaker Configuration
 
-     Specific configuration for max JVM heap usage circuit breaker. This configuration defines the
-     threshold percentage of maximum heap allocated beyond which queries will be rejected until the
+     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 range for this value is 50-95.
 
      Consider a scenario where the max heap allocated is 4 GB and memoryCircuitBreakerThreshold is
@@ -613,12 +610,31 @@
      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).
+
+     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).
     -->
     <!--
-   <memoryCircuitBreakerThresholdPct>100</memoryCircuitBreakerThresholdPct>
+   <memBreaker enabled="true" threshold="75"/>
     -->
 
-  </circuitBreaker>
+      <!-- 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.
+
+     Consider a scenario where the max heap allocated is 4 GB and memoryCircuitBreakerThreshold 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.
+    -->
+
+      <!--
+       <cpuBreaker enabled="true" threshold="75"/>
+      -->
+
+  </circuitBreakers>
 
 
   <!-- Request Dispatcher
diff --git a/solr/solr-ref-guide/src/circuit-breakers.adoc b/solr/solr-ref-guide/src/circuit-breakers.adoc
index 1629c32..56451b0 100644
--- a/solr/solr-ref-guide/src/circuit-breakers.adoc
+++ b/solr/solr-ref-guide/src/circuit-breakers.adoc
@@ -32,9 +32,14 @@ will be disabled globally. Per circuit breaker configurations are specified in t
 
 [source,xml]
 ----
-<useCircuitBreakers>false</useCircuitBreakers>
+<circuitBreakers enabled="true">
+  <!-- All specific configs in this section -->
+</circuitBreakers>
 ----
 
+This flag 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.
+
 == Currently Supported Circuit Breakers
 
 === JVM Heap Usage Based Circuit Breaker
@@ -42,26 +47,43 @@ This circuit breaker tracks JVM heap memory usage and rejects incoming search re
 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.
 
-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.
+Configuration for JVM heap usage based circuit breaker:
 
 [source,xml]
 ----
-<memoryCircuitBreakerThresholdPct>75</memoryCircuitBreakerThresholdPct>
+<memBreaker enabled="true" threshold="75"/>
 ----
 
+Note that this configuration will be overridden by the global circuit breaker flag -- if circuit breakers are disabled, this flag
+will not help you. Also, the triggering 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 memoryCircuitBreakerThresholdPct is set to 75. In this scenario, the heap usage
 at which the circuit breaker will trip is 3.75GB.
 
-Note that this circuit breaker is checked for each incoming search request and considers the current heap usage of the node, i.e every search
-request will get the live heap usage and compare it against the set memory threshold. The check does not impact performance,
-but any performance regressions that are suspected to be caused by this feature should be reported to the dev list.
 
+=== CPU Utilization Based Circuit Breaker
+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:
+
+[source,xml]
+----
+<cpuBreaker enabled="true" threshold="20"/>
+----
+
+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.
 
 == Performance Considerations
-It is worth noting that while JVM circuit breaker does not add any noticeable overhead per query, having too many
+It is worth noting that while JVM or CPU circuit breakers do not add any noticeable overhead per query, having too many
 circuit breakers checked for a single request can cause a performance overhead.
 
 In addition, it is a good practice to exponentially back off while retrying requests on a busy node.