You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/07/01 16:35:35 UTC

[GitHub] [lucene-solr] madrob commented on a change in pull request #1626: SOLR-14588: Implement Circuit Breakers

madrob commented on a change in pull request #1626:
URL: https://github.com/apache/lucene-solr/pull/1626#discussion_r448460273



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void register(CircuitBreaker.CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreaker.CircuitBreakerType circuitBreakerType) {
+    assert circuitBreakerType != null;
+
+    return circuitBreakerMap.get(circuitBreakerType);
+  }
+
+  /**
+   * Check and return circuit breakers that have triggered
+   * @return CircuitBreakers which have triggered, null otherwise.
+   */
+  public Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> checkTripped() {
+    Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> triggeredCircuitBreakers = null;
+
+    for (Map.Entry<CircuitBreaker.CircuitBreakerType, CircuitBreaker> entry : circuitBreakerMap.entrySet()) {
+      CircuitBreaker circuitBreaker = entry.getValue();
+
+      if (circuitBreaker.isEnabled() &&
+          circuitBreaker.isTripped()) {
+        if (triggeredCircuitBreakers == null) {
+          triggeredCircuitBreakers = new HashMap<>();
+        }
+
+        triggeredCircuitBreakers.put(entry.getKey(), 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 is enabled and has triggered.
+   * </p>
+   */
+  public boolean checkAnyTripped() {
+    for (Map.Entry<CircuitBreaker.CircuitBreakerType, CircuitBreaker> entry : circuitBreakerMap.entrySet()) {
+      CircuitBreaker circuitBreaker = entry.getValue();
+
+      if (circuitBreaker.isEnabled() &&
+          circuitBreaker.isTripped()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Construct the final error message to be printed when circuit breakers trip.
+   *
+   * @param circuitBreakerMap Input list for circuit breakers.
+   * @return Constructed error message.
+   */
+  public static String toErrorMessage(Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap) {
+    assert circuitBreakerMap != null;
+
+    StringBuilder sb = new StringBuilder();
+
+    for (CircuitBreaker.CircuitBreakerType circuitBreakerType : circuitBreakerMap.keySet()) {
+      sb.append(circuitBreakerType.toString() + " " + circuitBreakerMap.get(circuitBreakerType).getDebugInfo() + "\n");

Review comment:
       do multiple append calls instead of string concat.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.solr.core.SolrConfig;
+
+/**
+ * Default class to define circuit breakers for Solr.
+ * <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).
+ *
+ * TODO: This class should be grown as the scope of circuit breakers grow.
+ * </p>
+ */
+public abstract class CircuitBreaker {
+  public static final String NAME = "circuitbreaker";
+
+  protected final SolrConfig solrConfig;
+
+  public CircuitBreaker(SolrConfig solrConfig) {
+    this.solrConfig = solrConfig;
+  }
+
+  // Global config for all circuit breakers. For specific circuit breaker configs, define
+  // your own config.
+  protected boolean isEnabled() {
+    return solrConfig.useCircuitBreakers;
+  }
+
+  /**
+   * Check if circuit breaker is tripped.
+   */
+  public abstract boolean isTripped();
+
+  /**
+   * Get debug useful info.
+   */
+  public abstract String getDebugInfo();
+
+  /**
+   * Types of circuit breakers.
+   */
+  public enum CircuitBreakerType {
+    MEMORY,
+    CPU

Review comment:
       This is unused? Maybe delete it for now and add it back when somebody comes up with an implementation?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -1164,6 +1168,16 @@ private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
     return coreMetricManager;
   }
 
+  private CircuitBreakerManager initCircuitBreakerManager() {
+    if (solrConfig.useCircuitBreakers) {
+      CircuitBreakerManager circuitBreakerManager = CircuitBreakerManager.build(solrConfig);
+
+      return circuitBreakerManager;
+    }
+
+    return null;

Review comment:
       This is a little scary that we sometimes return null here? I think I would prefer an implementation of CBM that knows it is turned off so we don't accidentally get an NPE somewhere else.

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+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.SolrConfig;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+
+  @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();
+  }
+
+  public void testCBAlwaysTrips() throws IOException {
+    HashMap<String, String> args = new HashMap<String, String>();
+
+    args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
+    args.put(CommonParams.FL, "id");
+
+    CircuitBreaker circuitBreaker = new MockCircuitBreaker(h.getCore().getSolrConfig());
+
+    h.getCore().getCircuitBreakerManager().register(CircuitBreaker.CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+
+    circuitBreaker = new MemoryCircuitBreaker(h.getCore().getSolrConfig());
+
+    h.getCore().getCircuitBreakerManager().register(CircuitBreaker.CircuitBreakerType.MEMORY, circuitBreaker);
+  }
+
+  public void testCBFakeMemoryPressure() throws IOException {
+    HashMap<String, String> args = new HashMap<String, String>();
+
+    args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
+    args.put(CommonParams.FL, "id");
+
+    CircuitBreaker circuitBreaker = new FakeMemoryPressureCircuitBreaker(h.getCore().getSolrConfig());
+
+    h.getCore().getCircuitBreakerManager().register(CircuitBreaker.CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+
+    circuitBreaker = new MemoryCircuitBreaker(h.getCore().getSolrConfig());
+
+    h.getCore().getCircuitBreakerManager().register(CircuitBreaker.CircuitBreakerType.MEMORY, circuitBreaker);

Review comment:
       What is this testing? Is this just resetting the circuitbreakermanager? Should be in a Before/After blocks?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();

Review comment:
       This doesn't seem right... I can only have one circuit breaker of each type? I can imagine having multiple breakers that all work in conjunction - limit QPS to X, and limit QPM to Y, for example.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void register(CircuitBreaker.CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreaker.CircuitBreakerType circuitBreakerType) {
+    assert circuitBreakerType != null;
+
+    return circuitBreakerMap.get(circuitBreakerType);
+  }
+
+  /**
+   * Check and return circuit breakers that have triggered
+   * @return CircuitBreakers which have triggered, null otherwise.
+   */
+  public Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> checkTripped() {
+    Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> triggeredCircuitBreakers = null;
+
+    for (Map.Entry<CircuitBreaker.CircuitBreakerType, CircuitBreaker> entry : circuitBreakerMap.entrySet()) {
+      CircuitBreaker circuitBreaker = entry.getValue();
+
+      if (circuitBreaker.isEnabled() &&
+          circuitBreaker.isTripped()) {
+        if (triggeredCircuitBreakers == null) {
+          triggeredCircuitBreakers = new HashMap<>();
+        }
+
+        triggeredCircuitBreakers.put(entry.getKey(), 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 is enabled and has triggered.
+   * </p>
+   */
+  public boolean checkAnyTripped() {

Review comment:
       This is unused?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.MemoryMXBean;
+
+import org.apache.solr.core.SolrConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * 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 usage goes below the threshold,
+ * it will start allowing queries again.
+ * </p>
+ *
+ * <p>
+ * The memory threshold is defined as a percentage of the maximum memory allocated -- see memoryCircuitBreakerThresholdPct
+ * in solrconfig.xml.
+ * </p>
+ */
+
+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 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<>();
+
+  public MemoryCircuitBreaker(SolrConfig solrConfig) {
+    super(solrConfig);
+
+    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;
+    double thresholdInFraction = thresholdValueInPercentage / (double) 100;
+    heapMemoryThreshold = (long) (currentMaxHeap * thresholdInFraction);
+
+    if (heapMemoryThreshold <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+  }
+
+  // TODO: An optimization can be to trip the circuit breaker for a duration of time
+  // after the circuit breaker condition is matched. This will optimize for per call
+  // overhead of calculating the condition parameters but can result in false positives.
+  @Override
+  public boolean isTripped() {
+    if (!isEnabled()) {
+      return false;
+    }
+
+    allowedMemory.set(getCurrentMemoryThreshold());
+
+    seenMemory.set(calculateLiveMemoryUsage());
+
+    return (seenMemory.get() >= allowedMemory.get());

Review comment:
       Should we store these to local values so that we don't have to call an extra `.get()` on the concurrent structure? Probably doesn't make a difference.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void register(CircuitBreaker.CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreaker.CircuitBreakerType circuitBreakerType) {
+    assert circuitBreakerType != null;

Review comment:
       Is this needed?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,11 @@ private SolrConfig(SolrResourceLoader loader, String name, boolean isConfigsetTr
     queryResultWindowSize = Math.max(1, getInt("query/queryResultWindowSize", 1));
     queryResultMaxDocsCached = getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
     enableLazyFieldLoading = getBool("query/enableLazyFieldLoading", false);
+
+    useCircuitBreakers = getBool("circuitBreaker/useCircuitBreakers", false);
+    memoryCircuitBreakerThresholdPct = getInt("circuitBreaker/memoryCircuitBreakerThresholdPct", 100);

Review comment:
       I don't think 100 is a safe default here, since later we check that the value is between 50-95.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void register(CircuitBreaker.CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreaker.CircuitBreakerType circuitBreakerType) {
+    assert circuitBreakerType != null;
+
+    return circuitBreakerMap.get(circuitBreakerType);
+  }
+
+  /**
+   * Check and return circuit breakers that have triggered
+   * @return CircuitBreakers which have triggered, null otherwise.
+   */
+  public Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> checkTripped() {
+    Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> triggeredCircuitBreakers = null;
+
+    for (Map.Entry<CircuitBreaker.CircuitBreakerType, CircuitBreaker> entry : circuitBreakerMap.entrySet()) {
+      CircuitBreaker circuitBreaker = entry.getValue();
+
+      if (circuitBreaker.isEnabled() &&
+          circuitBreaker.isTripped()) {
+        if (triggeredCircuitBreakers == null) {
+          triggeredCircuitBreakers = new HashMap<>();
+        }
+
+        triggeredCircuitBreakers.put(entry.getKey(), 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 is enabled and has triggered.
+   * </p>
+   */
+  public boolean checkAnyTripped() {
+    for (Map.Entry<CircuitBreaker.CircuitBreakerType, CircuitBreaker> entry : circuitBreakerMap.entrySet()) {

Review comment:
       circuitBreakerMap.values()

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrConfig;
+
+/**
+ * Manages all registered circuit breaker instances. Responsible for a holistic view
+ * of whether a circuit breaker has tripped or not.
+ *
+ * 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.
+ *
+ * It is a good practice to register new circuit breakers here if you want them checked for every
+ * request.
+ *
+ * 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.
+ */
+public class CircuitBreakerManager {
+  private final Map<CircuitBreaker.CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void register(CircuitBreaker.CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreaker.CircuitBreakerType circuitBreakerType) {

Review comment:
       This method is never used?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.MemoryMXBean;
+
+import org.apache.solr.core.SolrConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * 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 usage goes below the threshold,
+ * it will start allowing queries again.
+ * </p>
+ *
+ * <p>
+ * The memory threshold is defined as a percentage of the maximum memory allocated -- see memoryCircuitBreakerThresholdPct
+ * in solrconfig.xml.
+ * </p>
+ */
+
+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 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<>();
+
+  public MemoryCircuitBreaker(SolrConfig solrConfig) {
+    super(solrConfig);
+
+    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;
+    double thresholdInFraction = thresholdValueInPercentage / (double) 100;
+    heapMemoryThreshold = (long) (currentMaxHeap * thresholdInFraction);
+
+    if (heapMemoryThreshold <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+  }
+
+  // TODO: An optimization can be to trip the circuit breaker for a duration of time
+  // after the circuit breaker condition is matched. This will optimize for per call
+  // overhead of calculating the condition parameters but can result in false positives.
+  @Override
+  public boolean isTripped() {
+    if (!isEnabled()) {
+      return false;
+    }
+
+    allowedMemory.set(getCurrentMemoryThreshold());
+
+    seenMemory.set(calculateLiveMemoryUsage());
+
+    return (seenMemory.get() >= allowedMemory.get());
+  }
+
+  @Override
+  public String getDebugInfo() {
+    if (seenMemory.get() == 0.0 || allowedMemory.get() == 0.0) {

Review comment:
       Why are we comparing a long to a floating point?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreaker.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.solr.core.SolrConfig;
+
+/**
+ * Default class to define circuit breakers for Solr.
+ * <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).
+ *
+ * TODO: This class should be grown as the scope of circuit breakers grow.
+ * </p>
+ */
+public abstract class CircuitBreaker {
+  public static final String NAME = "circuitbreaker";
+
+  protected final SolrConfig solrConfig;
+
+  public CircuitBreaker(SolrConfig solrConfig) {
+    this.solrConfig = solrConfig;
+  }
+
+  // Global config for all circuit breakers. For specific circuit breaker configs, define
+  // your own config.
+  protected boolean isEnabled() {
+    return solrConfig.useCircuitBreakers;
+  }
+
+  /**
+   * Check if circuit breaker is tripped.
+   */
+  public abstract boolean isTripped();
+
+  /**
+   * Get debug useful info.
+   */
+  public abstract String getDebugInfo();
+
+  /**
+   * Types of circuit breakers.
+   */
+  public enum CircuitBreakerType {

Review comment:
       Does this need a "CUSTOM" type or something else for folks who will be implementing their own?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org