mirror of https://github.com/apache/lucene.git
SOLR-14588: Follow Up Fixes and Documentation (#1615)
This commit is a follow up to the original commit and adds more documentation and adds timing information for circuit breaker in query response only if circuit breakers are enabled. This commit also adds a test for ensuring that the query response is correct when timing is enabled and circuit breakers are being used.
This commit is contained in:
parent
ed025741d7
commit
74ac97e402
|
@ -12,7 +12,7 @@ New Features
|
|||
---------------------
|
||||
* SOLR-14440: Introduce new Certificate Authentication Plugin to load Principal from certificate subject. (Mike Drob)
|
||||
|
||||
* SOLR-14588: Implement Circuit Breakers Infrastructure and add JVM Circuit Breaker (Atri Sharma)
|
||||
* SOLR-14588: Implement Circuit Breakers Infrastructure and add max JVM usage based circuit breaker (Atri Sharma)
|
||||
|
||||
Improvements
|
||||
----------------------
|
||||
|
|
|
@ -303,27 +303,27 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware,
|
|||
|
||||
final RTimerTree timer = rb.isDebug() ? req.getRequestTimer() : null;
|
||||
|
||||
Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers;
|
||||
if (req.getCore().getSolrConfig().useCircuitBreakers) {
|
||||
Map<CircuitBreakerType, CircuitBreaker> trippedCircuitBreakers;
|
||||
if (timer != null) {
|
||||
RTimerTree subt = timer.sub("circuitbreaker");
|
||||
rb.setTimer(subt);
|
||||
|
||||
if (timer != null) {
|
||||
RTimerTree subt = timer.sub("circuitbreaker");
|
||||
rb.setTimer(subt.sub("circuitbreaker"));
|
||||
CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
|
||||
trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
|
||||
|
||||
CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
|
||||
trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
|
||||
rb.getTimer().stop();
|
||||
} else {
|
||||
CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
|
||||
trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
|
||||
}
|
||||
|
||||
rb.getTimer().stop();
|
||||
subt.stop();
|
||||
} else {
|
||||
CircuitBreakerManager circuitBreakerManager = req.getCore().getCircuitBreakerManager();
|
||||
trippedCircuitBreakers = circuitBreakerManager.checkAllCircuitBreakers();
|
||||
}
|
||||
|
||||
if (trippedCircuitBreakers != null) {
|
||||
String errorMessage = CircuitBreakerManager.constructFinalErrorMessageString(trippedCircuitBreakers);
|
||||
rsp.add(STATUS, FAILURE);
|
||||
rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Circuit Breakers tripped " + errorMessage));
|
||||
return;
|
||||
if (trippedCircuitBreakers != null) {
|
||||
String errorMessage = CircuitBreakerManager.constructFinalErrorMessageString(trippedCircuitBreakers);
|
||||
rsp.add(STATUS, FAILURE);
|
||||
rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Circuit Breakers tripped " + errorMessage));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
final ShardHandler shardHandler1 = getAndPrepShardHandler(req, rb); // creates a ShardHandler object only if it's needed
|
||||
|
|
|
@ -22,6 +22,11 @@ import org.apache.solr.core.SolrCore;
|
|||
/**
|
||||
* Default class to define circuit breakers for Solr.
|
||||
*
|
||||
* There are two (typical) ways to use circuit breakers:
|
||||
*
|
||||
* 1. Have them checked at admission control by default (use CircuitBreakerManager for the same)
|
||||
* 2. Use the circuit breaker in a specific code path(s)
|
||||
*
|
||||
* TODO: This class should be grown as the scope of circuit breakers grow.
|
||||
*/
|
||||
public abstract class CircuitBreaker {
|
||||
|
|
|
@ -22,6 +22,15 @@ import java.lang.management.MemoryMXBean;
|
|||
|
||||
import org.apache.solr.core.SolrCore;
|
||||
|
||||
/**
|
||||
* Tracks the current JVM heap usage and triggers if it exceeds the defined percentage of the maximum
|
||||
* heap size allocated to the JVM. This circuit breaker is a part of the default CircuitBreakerManager
|
||||
* so is checked for every request -- hence it is realtime. Once the memory usage goes below the threshold,
|
||||
* it will start allowing queries again.
|
||||
*
|
||||
* The memory threshold is defined as a percentage of the maximum memory allocated -- see memoryCircuitBreakerThreshold
|
||||
* in solrconfig.xml
|
||||
*/
|
||||
public class MemoryCircuitBreaker extends CircuitBreaker {
|
||||
private static final MemoryMXBean MEMORY_MX_BEAN = ManagementFactory.getMemoryMXBean();
|
||||
|
||||
|
|
|
@ -59,10 +59,8 @@ public class DebugComponentTest extends SolrTestCaseJ4 {
|
|||
"//lst[@name='explain']/str[@name='2']",
|
||||
"//lst[@name='explain']/str[@name='3']",
|
||||
"//str[@name='QParser']",// make sure the QParser is specified
|
||||
"count(//lst[@name='timing']/*)=4", //should be four pieces to timings
|
||||
"count(//lst[@name='timing']/*)=3", //should be three pieces to timings
|
||||
"//lst[@name='timing']/double[@name='time']", //make sure we have a time value, but don't specify its result
|
||||
"count(//lst[@name='circuitbreaker']/*)>0",
|
||||
"//lst[@name='circuitbreaker']/double[@name='time']",
|
||||
"count(//lst[@name='prepare']/*)>0",
|
||||
"//lst[@name='prepare']/double[@name='time']",
|
||||
"count(//lst[@name='process']/*)>0",
|
||||
|
@ -85,10 +83,8 @@ public class DebugComponentTest extends SolrTestCaseJ4 {
|
|||
"//lst[@name='explain']/str[@name='1']",
|
||||
"//lst[@name='explain']/str[@name='2']",
|
||||
"//lst[@name='explain']/str[@name='3']",
|
||||
"count(//lst[@name='timing']/*)=4", //should be four pieces to timings
|
||||
"count(//lst[@name='timing']/*)=3", //should be four pieces to timings
|
||||
"//lst[@name='timing']/double[@name='time']", //make sure we have a time value, but don't specify its result
|
||||
"count(//lst[@name='circuitbreaker']/*)>0",
|
||||
"//lst[@name='circuitbreaker']/double[@name='time']",
|
||||
"count(//lst[@name='prepare']/*)>0",
|
||||
"//lst[@name='prepare']/double[@name='time']",
|
||||
"count(//lst[@name='process']/*)>0",
|
||||
|
@ -102,10 +98,8 @@ public class DebugComponentTest extends SolrTestCaseJ4 {
|
|||
"count(//str[@name='parsedquery_toString'])=0",
|
||||
"count(//lst[@name='explain']/*)=0",
|
||||
"count(//str[@name='QParser'])=0",// make sure the QParser is specified
|
||||
"count(//lst[@name='timing']/*)=4", //should be four pieces to timings
|
||||
"count(//lst[@name='timing']/*)=3", //should be four pieces to timings
|
||||
"//lst[@name='timing']/double[@name='time']", //make sure we have a time value, but don't specify its result
|
||||
"count(//lst[@name='circuitbreaker']/*)>0",
|
||||
"//lst[@name='circuitbreaker']/double[@name='time']",
|
||||
"count(//lst[@name='prepare']/*)>0",
|
||||
"//lst[@name='prepare']/double[@name='time']",
|
||||
"count(//lst[@name='process']/*)>0",
|
||||
|
|
|
@ -50,6 +50,9 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
|
|||
assertU(adoc("name", "john smith", "id", "1"));
|
||||
assertU(adoc("name", "johathon smith", "id", "2"));
|
||||
assertU(adoc("name", "john percival smith", "id", "3"));
|
||||
assertU(adoc("id", "1", "title", "this is a title.", "inStock_b1", "true"));
|
||||
assertU(adoc("id", "2", "title", "this is another title.", "inStock_b1", "true"));
|
||||
assertU(adoc("id", "3", "title", "Mary had a little lamb.", "inStock_b1", "false"));
|
||||
|
||||
//commit inside the loop to get multiple segments to make search as realistic as possible
|
||||
assertU(commit());
|
||||
|
@ -68,6 +71,28 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
|
|||
System.clearProperty("documentCache.enabled");
|
||||
}
|
||||
|
||||
public void testResponseWithCBTiming() {
|
||||
assertQ(req("q", "*:*", CommonParams.DEBUG_QUERY, "true"),
|
||||
"//str[@name='rawquerystring']='*:*'",
|
||||
"//str[@name='querystring']='*:*'",
|
||||
"//str[@name='parsedquery']='MatchAllDocsQuery(*:*)'",
|
||||
"//str[@name='parsedquery_toString']='*:*'",
|
||||
"count(//lst[@name='explain']/*)=3",
|
||||
"//lst[@name='explain']/str[@name='1']",
|
||||
"//lst[@name='explain']/str[@name='2']",
|
||||
"//lst[@name='explain']/str[@name='3']",
|
||||
"//str[@name='QParser']",
|
||||
"count(//lst[@name='timing']/*)=4",
|
||||
"//lst[@name='timing']/double[@name='time']",
|
||||
"count(//lst[@name='circuitbreaker']/*)>0",
|
||||
"//lst[@name='circuitbreaker']/double[@name='time']",
|
||||
"count(//lst[@name='prepare']/*)>0",
|
||||
"//lst[@name='prepare']/double[@name='time']",
|
||||
"count(//lst[@name='process']/*)>0",
|
||||
"//lst[@name='process']/double[@name='time']"
|
||||
);
|
||||
}
|
||||
|
||||
public void testCBAlwaysTrips() throws IOException {
|
||||
HashMap<String, String> args = new HashMap<String, String>();
|
||||
|
||||
|
@ -81,6 +106,10 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
|
|||
expectThrows(SolrException.class, () -> {
|
||||
h.query(req("name:\"john smith\""));
|
||||
});
|
||||
|
||||
circuitBreaker = new MemoryCircuitBreaker(h.getCore());
|
||||
|
||||
h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
|
||||
}
|
||||
|
||||
public void testCBFakeMemoryPressure() throws IOException {
|
||||
|
@ -96,6 +125,10 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
|
|||
expectThrows(SolrException.class, () -> {
|
||||
h.query(req("name:\"john smith\""));
|
||||
});
|
||||
|
||||
circuitBreaker = new MemoryCircuitBreaker(h.getCore());
|
||||
|
||||
h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
|
||||
}
|
||||
|
||||
public void testBuildingMemoryPressure() {
|
||||
|
@ -133,6 +166,10 @@ public class TestCircuitBreaker extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
assertEquals("Number of failed queries is not correct", 1, failureCount.get());
|
||||
|
||||
circuitBreaker = new MemoryCircuitBreaker(h.getCore());
|
||||
|
||||
h.getCore().getCircuitBreakerManager().registerCircuitBreaker(CircuitBreakerType.MEMORY, circuitBreaker);
|
||||
} finally {
|
||||
if (!executor.isShutdown()) {
|
||||
executor.shutdown();
|
||||
|
|
|
@ -499,12 +499,32 @@
|
|||
<queryResultMaxDocsCached>200</queryResultMaxDocsCached>
|
||||
|
||||
<!-- Enable Circuit Breakers
|
||||
|
||||
Circuit breakers are designed to allow stability and predictable query
|
||||
execution. They prevent operations that can take down the node and cause
|
||||
noisy neighbour issues.
|
||||
|
||||
This flag is the uber control switch which controls the activation/deactivation of all circuit
|
||||
breakers. At the moment, the only circuit breaker (max JVM circuit breaker) does not have its
|
||||
own specific configuration. However, if a circuit breaker wishes to be independently configurable,
|
||||
they are free to add their specific configuration but need to ensure that this flag is always
|
||||
respected - this should have veto over all independent configuration flags.
|
||||
-->
|
||||
<useCircuitBreakers>false</useCircuitBreakers>
|
||||
|
||||
<!-- Memory Circuit Breaker Threshold In Percentage
|
||||
|
||||
Post this percentage usage of the heap, incoming queries will be rejected
|
||||
Specific configuration for max JVM heap usage circuit breaker. This configuration defines the
|
||||
threshold percentage of maximum heap allocated beyond which queries will be rejected until the
|
||||
current JVM usage goes below the threshold.
|
||||
|
||||
Consider a scenario where the max heap allocated is 4 GB and memoryCircuitBreakerThreshold is
|
||||
defined as 75. Threshold JVM usage will be 4 * 0.75 = 3 GB. If, at any point, the current JVM
|
||||
heap usage goes above 3 GB, queries will be rejected until the heap usage goes below 3 GB again.
|
||||
|
||||
If you see queries getting rejected with 503 error code, check for "Circuit Breakers tripped"
|
||||
in logs and the corresponding error message should tell you what transpired (if the failure
|
||||
was caused by tripped circuit breakers).
|
||||
-->
|
||||
<memoryCircuitBreakerThreshold>100</memoryCircuitBreakerThreshold>
|
||||
|
||||
|
|
Loading…
Reference in New Issue