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/06/23 15:46:12 UTC

[GitHub] [lucene-solr] atris opened a new pull request #1606: SOLR-14588: Implement Circuit Breakers

atris opened a new pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606


   This commit consists of two parts: initial circuit breakers infrastructure and real JVM memory based circuit breaker which monitors incoming search requests and rejects them with SERVICE_TOO_BUSY error if the defined threshold is breached, thus giving headroom to existing indexing and search requests to complete.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
anshumg commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444453638



##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,13 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);
+
+    if (memoryCircuitBreakerThreshold > 100 || memoryCircuitBreakerThreshold < 0) {

Review comment:
       Perhaps you want to change this to early saying "Circuit breaker disabled, but incorrectly configured" or <current message> conditionally?




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-649613385


   @madrob I aim to close this PR out today if possible -- so happy to stay awake late tonight if you want me to iterate :)


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444334278



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every
+ * request.
+ */
+public class CircuitBreakerManager {
+
+  private final Map<CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void registerCircuitBreaker(CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    assert circuitBreakerType != null && circuitBreaker != null;
+
+    circuitBreakerMap.put(circuitBreakerType, circuitBreaker);
+  }
+
+  public CircuitBreaker getCircuitBreaker(CircuitBreakerType circuitBreakerType) {
+    assert circuitBreakerType != null;
+
+    return circuitBreakerMap.get(circuitBreakerType);
+  }
+
+  /**
+   * Check if any circuit breaker has triggered.
+   * @return CircuitBreakers which have triggered, null otherwise
+   */
+  public Map<CircuitBreakerType, CircuitBreaker> checkAllCircuitBreakers() {
+    Map<CircuitBreakerType, CircuitBreaker> triggeredCircuitBreakers = new HashMap<>();
+
+    for (CircuitBreakerType circuitBreakerType : circuitBreakerMap.keySet()) {

Review comment:
       prefer `entrySet`

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;

Review comment:
       this would be better as seenMemory=X allowedMemory=Y - a little bit less human readable but a lot easier to grep for or search in something like Splunk.

##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,9 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);

Review comment:
       Should we validate that this is between 0 and 100?

##########
File path: solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
##########
@@ -70,6 +70,10 @@
 
     <queryResultWindowSize>10</queryResultWindowSize>
 
+    <useCircuitBreakers>false</useCircuitBreakers>
+
+    <memoryCircuitBreakerThreshold>100</memoryCircuitBreakerThreshold>

Review comment:
       Does this have to be set when useCircuitBreakers is false? I would like our configs to be more tolerant.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");

Review comment:
       Is this checking for overflow? How does the error condition occur otherwise?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;

Review comment:
       I think this makes more sense to be a long. Returning fractional bytes is meaningless.

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    executor = null;

Review comment:
       Probably want to also call executor.shutdown in addition to null?

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    executor = null;
+    super.tearDown();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("filterCache.enabled");
+    System.clearProperty("queryResultCache.enabled");
+    System.clearProperty("documentCache.enabled");
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  public void testBuildingMemoryPressure() throws Exception {
+    HashMap<String, String> args = new HashMap<String, String>();
+
+    args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
+    args.put(CommonParams.FL, "id");
+
+    AtomicInteger failureCount = new AtomicInteger();
+
+    CircuitBreaker circuitBreaker = new BuildingUpMemoryPressureCircuitBreaker(h.getCore());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    for (int i = 0; i < 5; i++) {
+      System.out.println("i is " + i);

Review comment:
       Use logger instead of system.out

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();

Review comment:
       This should be constant, right? Get it once at constructor?

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());

Review comment:
       commit-optimize-commit in a loop won't get you multiple segments

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       Is this expensive to get? If we're doing it for every query, I'd want to make sure that it is an efficient call.

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    executor = null;
+    super.tearDown();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("filterCache.enabled");
+    System.clearProperty("queryResultCache.enabled");
+    System.clearProperty("documentCache.enabled");
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  public void testBuildingMemoryPressure() throws Exception {
+    HashMap<String, String> args = new HashMap<String, String>();
+
+    args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
+    args.put(CommonParams.FL, "id");
+
+    AtomicInteger failureCount = new AtomicInteger();
+
+    CircuitBreaker circuitBreaker = new BuildingUpMemoryPressureCircuitBreaker(h.getCore());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    for (int i = 0; i < 5; i++) {
+      System.out.println("i is " + i);
+      executor.submit(() -> {
+        try {
+          h.query(req("name:\"john smith\""));
+        } catch (SolrException e) {
+          failureCount.incrementAndGet();
+        } catch (Exception e) {
+          throw new RuntimeException(e.getMessage());
+        }
+      });
+    }
+
+    executor.shutdown();
+    try {
+      executor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e.getMessage());
+    }
+
+    assert failureCount.get() == 1;

Review comment:
       use unit asserts instead of java asserts please




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445046053



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();

Review comment:
       I mean in response we send timing info with `debug=true`(see `RTimerTree subt = timer.sub( "prepare" );`), maybe this is worthwhile to include there as well since otherwise the times may not add up




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445038316



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.SolrCore;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every
+ * request.
+ */
+public class CircuitBreakerManager {

Review comment:
       That is a good point. I was thinking of a schema API since I dont trust XMLs for class level detailing much. Your idea to combine with the package API is a great one. I will open a JIRA once this is committed and follow up with a SIP. For now, adding a comment for the same.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445321786



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -44,25 +48,20 @@ public boolean isCircuitBreakerGauntletTripped() {
       return false;
     }
 
-    allowedMemory = getCurrentMemoryThreshold();
+    allowedMemory.set(getCurrentMemoryThreshold());
 
-    seenMemory = calculateLiveMemoryUsage();
+    seenMemory.set(calculateLiveMemoryUsage());
 
-    return (seenMemory >= allowedMemory);
+    return (seenMemory.get() >= allowedMemory.get());
   }
 
   @Override
   public String printDebugInfo() {
-    return "seen memory=" + seenMemory + " allowed memory=" + allowedMemory;
+    return "seenMemory=" + seenMemory + " allowedMemory=" + allowedMemory;

Review comment:
       Interesting, I thought the default toString delegate should handle the right debug output. Added the get, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445188767



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -44,25 +48,20 @@ public boolean isCircuitBreakerGauntletTripped() {
       return false;
     }
 
-    allowedMemory = getCurrentMemoryThreshold();
+    allowedMemory.set(getCurrentMemoryThreshold());
 
-    seenMemory = calculateLiveMemoryUsage();
+    seenMemory.set(calculateLiveMemoryUsage());
 
-    return (seenMemory >= allowedMemory);
+    return (seenMemory.get() >= allowedMemory.get());
   }
 
   @Override
   public String printDebugInfo() {
-    return "seen memory=" + seenMemory + " allowed memory=" + allowedMemory;
+    return "seenMemory=" + seenMemory + " allowedMemory=" + allowedMemory;

Review comment:
       Need `.get()` here.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
+
+      if (timer != null) {
+        RTimerTree subt = timer.sub("circuitbreaker");
+        rb.setTimer(subt.sub("circuitbreaker"));
+      }
+      String errorMessage = CircuitBreakerManager.constructFinalErrorMessageString(trippedCircuitBreakers);
+      rsp.add(STATUS, FAILURE);
+      rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Circuit Breakers tripped " + errorMessage));

Review comment:
       I'm not sure this is the right error code. A 5xx code usually indicates a server error - and I'm not sure how we effectively convey to clients that this error is something that is ok to retry. They might log the message, but retry logic will typically look at the code returned as a first branch in the decision tree. Need to think about this and maybe look at some examples.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
+
+      if (timer != null) {
+        RTimerTree subt = timer.sub("circuitbreaker");

Review comment:
       We never stop this subtimer.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;

Review comment:
       I'd combine this with the later call to req.getRequestTimer so that we're not doing that twice. You're also checking this before we call rb.setDebug(), so it probably is always false at this point.
   
   We should be timing the circuitBreakerCheck as well (Lines 299-300).

##########
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("query/useCircuitBreakers", false);

Review comment:
       I'd like for all of this to be dynamically configurable at some point, but it doesn't have to be in this PR. Can add it to the future SIP or create a separate JIRA for it, as you think would be appropriate.




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648696657


   @madrob @anshumg Thanks for reviewing -- I have updated per comments. Please see and let me know if it looks fine.


----------------------------------------------------------------
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


[GitHub] [lucene-solr] madrob commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648271244


   This has been something that I've wanted for a long time, so I'm really excited to see somebody picking up the work! Thank you @atris !


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444417940



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {

Review comment:
       This will never be true.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444955220



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.SolrCore;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every
+ * request.
+ */
+public class CircuitBreakerManager {
+
+  private final Map<CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void registerCircuitBreaker(CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    assert circuitBreakerType != null && circuitBreaker != null;

Review comment:
       Why is there an assert here? It might be fine to leave it, but I'm curious what the intent is. If you want this check to happen in production, then you probably need to use something like `Objects.assertNotNull()`

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.SolrCore;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every

Review comment:
       nit: s/practise/practice

##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?

Review comment:
       I think we would want to have two separate controls for this.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.SolrCore;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every
+ * request.
+ */
+public class CircuitBreakerManager {

Review comment:
       A question about the design of this class... We have methods for registering a custom circuit breaker, and some notion of default circuit breakers, but I don't see an easy way to add additional ones. Ideally this would be through the XML or through some kind of schema API or could be very powerful combined with the Package API. I don't have a full idea of what this needs to look like, and this is fine as is to commit now, but I'd really like us to come back to this and develop the idea further. Possibly as a SIP?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory=" + seenMemory + " allowed memory=" + allowedMemory;
+  }
+
+  private long getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;

Review comment:
       Does this make sense? It indicates an error condition in the JVM, right? Can move this check to the constructor and throw an Exception since we'd effectively be blocking all queries at that point anyway?

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory=" + seenMemory + " allowed memory=" + allowedMemory;

Review comment:
       s/seen memory/seenMemory - one word for log processing

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {

Review comment:
       Is this thread safe? If there are multiple queries happening, concurrent with GC, would these values be stale?

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static 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 ++) {
+      assertU(adoc("name", "john smith", "id", "1"));
+      assertU(adoc("name", "johathon smith", "id", "2"));
+      assertU(adoc("name", "john percival smith", "id", "3"));
+
+      //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();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("filterCache.enabled");
+    System.clearProperty("queryResultCache.enabled");
+    System.clearProperty("documentCache.enabled");
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  public void testBuildingMemoryPressure() {
+    ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool(

Review comment:
       Use a try-finally to make sure this executor is shut down.

##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();

Review comment:
       Do we want to time this and include that in the response (useful for both passed and failed breakers)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444417129



##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,13 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);
+
+    if (memoryCircuitBreakerThreshold > 100 || memoryCircuitBreakerThreshold < 0) {

Review comment:
       We only want to do this validation if circuitBreakers are enabled, right?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445647072



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
+
+      if (timer != null) {
+        RTimerTree subt = timer.sub("circuitbreaker");
+        rb.setTimer(subt.sub("circuitbreaker"));
+      }
+      String errorMessage = CircuitBreakerManager.constructFinalErrorMessageString(trippedCircuitBreakers);
+      rsp.add(STATUS, FAILURE);
+      rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Circuit Breakers tripped " + errorMessage));

Review comment:
       Coming back to this, yea, I think 503 is best. I was thinking maybe 429, but this is probably good.




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-649322479


   @madrob Updated the timer usage -- let me know if this looks fine now.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444994179



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/CircuitBreakerManager.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.SolrCore;
+
+/**
+ * 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 practise to register new circuit breakers here if you want them checked for every
+ * request.
+ */
+public class CircuitBreakerManager {
+
+  private final Map<CircuitBreakerType, CircuitBreaker> circuitBreakerMap = new HashMap<>();
+
+  // Allows replacing of existing circuit breaker
+  public void registerCircuitBreaker(CircuitBreakerType circuitBreakerType, CircuitBreaker circuitBreaker) {
+    assert circuitBreakerType != null && circuitBreaker != null;

Review comment:
       A paranoid security check. Removed, thanks




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648911905


   > How does this work for distributed requests? When the coordinator node is under memory pressure? When one of the fan out nodes in under memory pressure?
   
   I believe the behaviour would be same -- if the node is not able to handle the request, it will error back with the transient error code. If it is coordinator node, the result will be sent back to the user and if it is a fan out node, the coordinator needs to deal with the regular way of a node erroring out?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444394541



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       I did not see any regression -- if we note anything otherwise, we can probably cache these values with a TTL. I would be surprised if it causes anything noticeable, though.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445082883



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {

Review comment:
       Moved to thread local




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444444004



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       Yes, I considered MemoryUsageGaugeSet and did not use it for the precise reason that you described. To be honest, the listener based implementation is harder to read than the current, and I was not sure if it is worth going that route for shaving off < 100 ns per query. However, if you have concerns or if we ever hear slowness reports due to this feature, I am happy to take that route. For now, I have added the comment as suggested, thanks.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
anshumg commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444455004



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -1164,6 +1171,16 @@ private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
     return coreMetricManager;
   }
 
+  private CircuitBreakerManager initCircuitBreakerManager() {
+    CircuitBreakerManager circuitBreakerManager = new CircuitBreakerManager();
+
+    // Install the default circuit breakers
+    CircuitBreaker memoryCircuitBreaker = new MemoryCircuitBreaker(this);
+    circuitBreakerManager.registerCircuitBreaker(CircuitBreakerType.MEMORY, memoryCircuitBreaker);

Review comment:
       I was thinking of moving this outside of SolrCore, so custom Circuit breakers could be implemented and plugged without any change to SolrCore? What do you think?




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris merged pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris merged pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606


   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444382798



##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,9 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);

Review comment:
       Good catch, thanks!




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648995835


   @madrob Updated, please see and let me know your thoughts and comments.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444424345



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {

Review comment:
       Another of my overtly cautious safety nets :) Removed, thanks.

##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,13 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);
+
+    if (memoryCircuitBreakerThreshold > 100 || memoryCircuitBreakerThreshold < 0) {

Review comment:
       I was being overtly cautious here -- it seems wrong if the parameter is incorrectly specified regardless of whether it is used or not?

##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());

Review comment:
       Removed the commit optimize.

##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");

Review comment:
       Yes, this is an overflow check.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445063185



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {

Review comment:
       Yes, these values can be stale. I was debating if we need a synchronization here for exact values. Would it make sense to move these to thread local state inside the class instead?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445015098



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?

Review comment:
       ++




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444384399



##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    executor = null;
+    super.tearDown();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("filterCache.enabled");
+    System.clearProperty("queryResultCache.enabled");
+    System.clearProperty("documentCache.enabled");
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  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());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    expectThrows(SolrException.class, () -> {
+      h.query(req("name:\"john smith\""));
+    });
+  }
+
+  public void testBuildingMemoryPressure() throws Exception {
+    HashMap<String, String> args = new HashMap<String, String>();
+
+    args.put(QueryParsing.DEFTYPE, CircuitBreaker.NAME);
+    args.put(CommonParams.FL, "id");
+
+    AtomicInteger failureCount = new AtomicInteger();
+
+    CircuitBreaker circuitBreaker = new BuildingUpMemoryPressureCircuitBreaker(h.getCore());
+
+    h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
+
+    for (int i = 0; i < 5; i++) {
+      System.out.println("i is " + i);

Review comment:
       Stray debugging output, removed, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444952555



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       Yea, add some comments to this effect to the code so that if people run into issues with it they can more easily figure out what's up without having to find the comments from this PR.




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-649628396


   @madrob @anshumg Thank you for the review!


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444750302



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -1164,6 +1171,16 @@ private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
     return coreMetricManager;
   }
 
+  private CircuitBreakerManager initCircuitBreakerManager() {
+    CircuitBreakerManager circuitBreakerManager = new CircuitBreakerManager();
+
+    // Install the default circuit breakers
+    CircuitBreaker memoryCircuitBreaker = new MemoryCircuitBreaker(this);
+    circuitBreakerManager.registerCircuitBreaker(CircuitBreakerType.MEMORY, memoryCircuitBreaker);

Review comment:
       Agreed. I have moved this to a method in CircuitBreakerManager that can be delegated to for handling this scenario.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444383850



##########
File path: solr/core/src/test/org/apache/solr/util/TestCircuitBreaker.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.util.NamedThreadFactory;
+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.SolrCore;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.circuitbreaker.CircuitBreaker;
+import org.apache.solr.util.circuitbreaker.CircuitBreakerType;
+import org.apache.solr.util.circuitbreaker.MemoryCircuitBreaker;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestCircuitBreaker extends SolrTestCaseJ4 {
+  private final static int NUM_DOCS = 20;
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true");
+
+    executor = ExecutorUtil.newMDCAwareCachedThreadPool(
+        new SolrNamedThreadFactory("TestCircuitBreaker"));
+    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(commit());
+      assertU(optimize());
+
+      //commit inside the loop to get multiple segments to make search as realistic as possible
+      assertU(commit());
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    executor = null;

Review comment:
       This should be shutdown in the test itself. I have added a check to check if it is shut down before making the reference null.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444392412



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();

Review comment:
       Good point, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445011734



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  private final long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private long seenMemory;
+  private long allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory=" + seenMemory + " allowed memory=" + allowedMemory;
+  }
+
+  private long getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;

Review comment:
       My idea was to avoid blocking errors as much as possible but I agree with you. Fixed, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445094485



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();

Review comment:
       Added, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444749060



##########
File path: solr/core/src/java/org/apache/solr/core/SolrConfig.java
##########
@@ -224,6 +224,13 @@ 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("query/useCircuitBreakers", false);
+    memoryCircuitBreakerThreshold = getInt("query/memoryCircuitBreakerThreshold", 100);
+
+    if (memoryCircuitBreakerThreshold > 100 || memoryCircuitBreakerThreshold < 0) {

Review comment:
       I have changed the check to be only performed when circuit breakers are enabled.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445332160



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
+
+      if (timer != null) {
+        RTimerTree subt = timer.sub("circuitbreaker");
+        rb.setTimer(subt.sub("circuitbreaker"));
+      }
+      String errorMessage = CircuitBreakerManager.constructFinalErrorMessageString(trippedCircuitBreakers);
+      rsp.add(STATUS, FAILURE);
+      rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Circuit Breakers tripped " + errorMessage));

Review comment:
       I looked at a couple of HTTP codes and SERVICE_UNAVAILABLE seems to be the closest one matching the scenario (https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/503). Did you have anything else in mind?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444416519



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       This article claims that each call is about 100ns, https://www.inoneo.com/en/blog/8/java/monitoring-jvm-memory-usage-inside-a-java-application and suggest using a listener instead. I'm not sure how true this is for more modern JVMs anyway.
   
   If you look into drop wizard metrics that we're reporting in JMX, MemoryUsageGuageSet reports memory as the combination of heap and non-heap. I'm not sure we need to consider non-heap here, but at least a comment highlighting that would be useful.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444375970



##########
File path: solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
##########
@@ -70,6 +70,10 @@
 
     <queryResultWindowSize>10</queryResultWindowSize>
 
+    <useCircuitBreakers>false</useCircuitBreakers>
+
+    <memoryCircuitBreakerThreshold>100</memoryCircuitBreakerThreshold>

Review comment:
       I dont think so, since it is checked when circuit breakers are enabled. I put this config here for completion.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444988588



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       Added a comment, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445332978



##########
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("query/useCircuitBreakers", false);

Review comment:
       I would ideally want this to be a part of the API we were discussing upstream so will add it to the SIP.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r444798477



##########
File path: solr/core/src/java/org/apache/solr/util/circuitbreaker/MemoryCircuitBreaker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+import org.apache.solr.core.SolrCore;
+
+public class MemoryCircuitBreaker extends CircuitBreaker {
+  private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
+
+  // Assumption -- the value of these parameters will be set correctly before invoking printDebugInfo()
+  private double seenMemory;
+  private double allowedMemory;
+
+  public MemoryCircuitBreaker(SolrCore solrCore) {
+    super(solrCore);
+  }
+
+  // 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 isCircuitBreakerGauntletTripped() {
+    if (!isCircuitBreakerEnabled()) {
+      return false;
+    }
+
+    allowedMemory = getCurrentMemoryThreshold();
+
+    if (allowedMemory < 0) {
+      // No threshold
+      return false;
+    }
+
+    seenMemory = calculateLiveMemoryUsage();
+
+    return (seenMemory >= allowedMemory);
+  }
+
+  @Override
+  public String printDebugInfo() {
+    return "seen memory " + seenMemory + " allowed memory " + allowedMemory;
+  }
+
+  private double getCurrentMemoryThreshold() {
+    int thresholdValueInPercentage = solrCore.getSolrConfig().memoryCircuitBreakerThreshold;
+    long currentMaxHeap = MEMORY_MX_BEAN.getHeapMemoryUsage().getMax();
+
+    if (currentMaxHeap <= 0) {
+      return Long.MIN_VALUE;
+    }
+
+    double thresholdInFraction = (double) thresholdValueInPercentage / 100;
+    double actualLimit = currentMaxHeap * thresholdInFraction;
+
+    if (actualLimit <= 0) {
+      throw new IllegalStateException("Memory limit cannot be less than or equal to zero");
+    }
+
+    return actualLimit;
+  }
+
+  /**
+   * Calculate the live memory usage for the system. This method has package visibility
+   * to allow using for testing
+   * @return Memory usage in bytes
+   */
+  protected long calculateLiveMemoryUsage() {
+    return MEMORY_MX_BEAN.getHeapMemoryUsage().getUsed();

Review comment:
       @madrob Just verified that Elasticsearch also uses getHeapMemoryUsage() to trigger similar checks per call. Also, I am not sure if NotificationListener will allow us to be real time since it triggers a notification when the threshold is breached. but I could not find a way to know when the threshold has gone back to normal. I am proposing sticking to this for now, if that sounds fine?




----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648362568


   @madrob Updated, please see and share your thoughts.


----------------------------------------------------------------
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


[GitHub] [lucene-solr] atris commented on pull request #1606: SOLR-14588: Implement Circuit Breakers

Posted by GitBox <gi...@apache.org>.
atris commented on pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#issuecomment-648326325


   To provide some context, I ran this against large facets with the memory circuit breaker tripping in the right place.
   
   @madrob Thanks for reviewing, I have raised another iteration fixing your comments. Please see and let me know your thoughts and comments.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445356885



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,23 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
+
+    if (trippedCircuitBreakers != null) {
+      final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;

Review comment:
       Fixed, thanks




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1606:
URL: https://github.com/apache/lucene-solr/pull/1606#discussion_r445038817



##########
File path: solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
##########
@@ -289,6 +295,19 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
       rb.requestInfo.setResponseBuilder(rb);
     }
 
+    //TODO: Should this be for indexing requests as well?
+    CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
+    Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();

Review comment:
       Not sure if I understood. Would you mean the epoch timestamp when the circuit breaker triggered?




----------------------------------------------------------------
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