Introduce durability of circuit breaking exception

With this commit we differentiate between permanent circuit breaking
exceptions (which require intervention from an operator and should not
be automatically retried) and transient ones (which may heal themselves
eventually and should be retried). Furthermore, the parent circuit
breaker will categorize a circuit breaking exception as either transient
or permanent based on the categorization of memory usage of its child
circuit breakers.

Closes #31986
Relates #34460
This commit is contained in:
Daniel Mitterdorfer 2018-11-02 13:12:44 +01:00 committed by GitHub
parent 0aa66b077a
commit ccbe80c3a0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 183 additions and 62 deletions

View File

@ -16,6 +16,8 @@ Cross-Cluster-Search::
Rest API::
* The Clear Cache API only supports `POST` as HTTP method
* `CircuitBreakingException` was previously mapped to HTTP status code 503 and is now
mapped as HTTP status code 429.
Aggregations::
* The Percentiles and PercentileRanks aggregations now return `null` in the REST response,

View File

@ -24,6 +24,7 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -63,7 +64,7 @@ public class RankEvalResponseTests extends ESTestCase {
private static final Exception[] RANDOM_EXCEPTIONS = new Exception[] {
new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES)),
new CircuitBreakingException("Data too large", 123, 456),
new CircuitBreakingException("Data too large", 123, 456, CircuitBreaker.Durability.PERMANENT),
new SearchParseException(new TestSearchContext(null), "Parse failure", new XContentLocation(12, 98)),
new IllegalArgumentException("Closed resource", new RuntimeException("Resource")),
new SearchPhaseExecutionException("search", "all shards failed",

View File

@ -96,7 +96,7 @@ public class Netty4HttpRequestSizeLimitIT extends ESNetty4IntegTestCase {
Collection<FullHttpResponse> multipleResponses = nettyHttpClient.post(transportAddress.address(), requests);
try {
assertThat(multipleResponses, hasSize(requests.length));
assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.SERVICE_UNAVAILABLE);
assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.TOO_MANY_REQUESTS);
} finally {
multipleResponses.forEach(ReferenceCounted::release);
}

View File

@ -32,8 +32,8 @@ import java.util.concurrent.atomic.AtomicLong;
public class ChildMemoryCircuitBreaker implements CircuitBreaker {
private final long memoryBytesLimit;
private final BreakerSettings settings;
private final double overheadConstant;
private final Durability durability;
private final AtomicLong used;
private final AtomicLong trippedCount;
private final Logger logger;
@ -66,9 +66,9 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
public ChildMemoryCircuitBreaker(BreakerSettings settings, ChildMemoryCircuitBreaker oldBreaker,
Logger logger, HierarchyCircuitBreakerService parent, String name) {
this.name = name;
this.settings = settings;
this.memoryBytesLimit = settings.getLimit();
this.overheadConstant = settings.getOverhead();
this.durability = settings.getDurability();
if (oldBreaker == null) {
this.used = new AtomicLong(0);
this.trippedCount = new AtomicLong(0);
@ -78,7 +78,7 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
}
this.logger = logger;
if (logger.isTraceEnabled()) {
logger.trace("creating ChildCircuitBreaker with settings {}", this.settings);
logger.trace("creating ChildCircuitBreaker with settings {}", settings);
}
this.parent = parent;
}
@ -95,7 +95,7 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
", which is larger than the limit of [" +
memoryBytesLimit + "/" + new ByteSizeValue(memoryBytesLimit) + "]";
logger.debug("{}", message);
throw new CircuitBreakingException(message, bytesNeeded, memoryBytesLimit);
throw new CircuitBreakingException(message, bytesNeeded, memoryBytesLimit, durability);
}
/**
@ -234,4 +234,12 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
public String getName() {
return this.name;
}
/**
* @return whether a tripped circuit breaker will reset itself (transient) or requires manual intervention (permanent).
*/
@Override
public Durability getDurability() {
return this.durability;
}
}

View File

@ -83,6 +83,13 @@ public interface CircuitBreaker {
}
}
enum Durability {
// The condition that tripped the circuit breaker fixes itself eventually.
TRANSIENT,
// The condition that tripped the circuit breaker requires manual intervention.
PERMANENT
}
/**
* Trip the circuit breaker
* @param fieldName name of the field responsible for tripping the breaker
@ -127,4 +134,9 @@ public interface CircuitBreaker {
* @return the name of the breaker
*/
String getName();
/**
* @return whether a tripped circuit breaker will reset itself (transient) or requires manual intervention (permanent).
*/
Durability getDurability();
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common.breaker;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -33,23 +34,28 @@ public class CircuitBreakingException extends ElasticsearchException {
private final long bytesWanted;
private final long byteLimit;
public CircuitBreakingException(String message) {
super(message);
this.bytesWanted = 0;
this.byteLimit = 0;
}
private final CircuitBreaker.Durability durability;
public CircuitBreakingException(StreamInput in) throws IOException {
super(in);
byteLimit = in.readLong();
bytesWanted = in.readLong();
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
durability = in.readEnum(CircuitBreaker.Durability.class);
} else {
durability = CircuitBreaker.Durability.PERMANENT;
}
}
public CircuitBreakingException(String message, long bytesWanted, long byteLimit) {
public CircuitBreakingException(String message, CircuitBreaker.Durability durability) {
this(message, 0, 0, durability);
}
public CircuitBreakingException(String message, long bytesWanted, long byteLimit, CircuitBreaker.Durability durability) {
super(message);
this.bytesWanted = bytesWanted;
this.byteLimit = byteLimit;
this.durability = durability;
}
@Override
@ -57,6 +63,9 @@ public class CircuitBreakingException extends ElasticsearchException {
super.writeTo(out);
out.writeLong(byteLimit);
out.writeLong(bytesWanted);
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
out.writeEnum(durability);
}
}
public long getBytesWanted() {
@ -67,14 +76,19 @@ public class CircuitBreakingException extends ElasticsearchException {
return this.byteLimit;
}
public CircuitBreaker.Durability getDurability() {
return durability;
}
@Override
public RestStatus status() {
return RestStatus.SERVICE_UNAVAILABLE;
return RestStatus.TOO_MANY_REQUESTS;
}
@Override
protected void metadataToXContent(XContentBuilder builder, Params params) throws IOException {
builder.field("bytes_wanted", bytesWanted);
builder.field("bytes_limit", byteLimit);
builder.field("durability", durability);
}
}

View File

@ -84,7 +84,7 @@ public class MemoryCircuitBreaker implements CircuitBreaker {
", which is larger than the limit of [" +
memoryBytesLimit + "/" + new ByteSizeValue(memoryBytesLimit) + "]";
logger.debug("{}", message);
throw new CircuitBreakingException(message, bytesNeeded, memoryBytesLimit);
throw new CircuitBreakingException(message, bytesNeeded, memoryBytesLimit, Durability.PERMANENT);
}
/**
@ -197,4 +197,9 @@ public class MemoryCircuitBreaker implements CircuitBreaker {
public String getName() {
return FIELDDATA;
}
@Override
public Durability getDurability() {
return Durability.PERMANENT;
}
}

View File

@ -71,4 +71,9 @@ public class NoopCircuitBreaker implements CircuitBreaker {
public String getName() {
return this.name;
}
@Override
public Durability getDurability() {
return Durability.PERMANENT;
}
}

View File

@ -25,22 +25,24 @@ import org.elasticsearch.common.unit.ByteSizeValue;
/**
* Settings for a {@link CircuitBreaker}
*/
public class BreakerSettings {
public final class BreakerSettings {
private final String name;
private final long limitBytes;
private final double overhead;
private final CircuitBreaker.Type type;
private final CircuitBreaker.Durability durability;
public BreakerSettings(String name, long limitBytes, double overhead) {
this(name, limitBytes, overhead, CircuitBreaker.Type.MEMORY);
this(name, limitBytes, overhead, CircuitBreaker.Type.MEMORY, CircuitBreaker.Durability.PERMANENT);
}
public BreakerSettings(String name, long limitBytes, double overhead, CircuitBreaker.Type type) {
public BreakerSettings(String name, long limitBytes, double overhead, CircuitBreaker.Type type, CircuitBreaker.Durability durability) {
this.name = name;
this.limitBytes = limitBytes;
this.overhead = overhead;
this.type = type;
this.durability = durability;
}
public String getName() {
@ -59,10 +61,15 @@ public class BreakerSettings {
return this.type;
}
public CircuitBreaker.Durability getDurability() {
return durability;
}
@Override
public String toString() {
return "[" + this.name +
",type=" + this.type.toString() +
",durability=" + this.durability.toString() +
",limit=" + this.limitBytes + "/" + new ByteSizeValue(this.limitBytes) +
",overhead=" + this.overhead + "]";
}

View File

@ -106,30 +106,34 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
this.fielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA,
FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING.get(settings),
CircuitBreaker.Durability.PERMANENT
);
this.inFlightRequestsSettings = new BreakerSettings(CircuitBreaker.IN_FLIGHT_REQUESTS,
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_TYPE_SETTING.get(settings),
CircuitBreaker.Durability.TRANSIENT
);
this.requestSettings = new BreakerSettings(CircuitBreaker.REQUEST,
REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.get(settings),
CircuitBreaker.Durability.TRANSIENT
);
this.accountingSettings = new BreakerSettings(CircuitBreaker.ACCOUNTING,
ACCOUNTING_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
ACCOUNTING_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
ACCOUNTING_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
ACCOUNTING_CIRCUIT_BREAKER_TYPE_SETTING.get(settings),
CircuitBreaker.Durability.PERMANENT
);
this.parentSettings = new BreakerSettings(CircuitBreaker.PARENT,
TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(), 1.0,
CircuitBreaker.Type.PARENT);
CircuitBreaker.Type.PARENT, null);
if (logger.isTraceEnabled()) {
logger.trace("parent circuit breaker with settings {}", this.parentSettings);
@ -151,17 +155,17 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
private void setRequestBreakerLimit(ByteSizeValue newRequestMax, Double newRequestOverhead) {
BreakerSettings newRequestSettings = new BreakerSettings(CircuitBreaker.REQUEST, newRequestMax.getBytes(), newRequestOverhead,
HierarchyCircuitBreakerService.this.requestSettings.getType());
this.requestSettings.getType(), this.requestSettings.getDurability());
registerBreaker(newRequestSettings);
HierarchyCircuitBreakerService.this.requestSettings = newRequestSettings;
this.requestSettings = newRequestSettings;
logger.info("Updated breaker settings request: {}", newRequestSettings);
}
private void setInFlightRequestsBreakerLimit(ByteSizeValue newInFlightRequestsMax, Double newInFlightRequestsOverhead) {
BreakerSettings newInFlightRequestsSettings = new BreakerSettings(CircuitBreaker.IN_FLIGHT_REQUESTS, newInFlightRequestsMax.getBytes(),
newInFlightRequestsOverhead, HierarchyCircuitBreakerService.this.inFlightRequestsSettings.getType());
newInFlightRequestsOverhead, this.inFlightRequestsSettings.getType(), this.inFlightRequestsSettings.getDurability());
registerBreaker(newInFlightRequestsSettings);
HierarchyCircuitBreakerService.this.inFlightRequestsSettings = newInFlightRequestsSettings;
this.inFlightRequestsSettings = newInFlightRequestsSettings;
logger.info("Updated breaker settings for in-flight requests: {}", newInFlightRequestsSettings);
}
@ -169,7 +173,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
long newFielddataLimitBytes = newFielddataMax == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getLimit() : newFielddataMax.getBytes();
newFielddataOverhead = newFielddataOverhead == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getOverhead() : newFielddataOverhead;
BreakerSettings newFielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA, newFielddataLimitBytes, newFielddataOverhead,
HierarchyCircuitBreakerService.this.fielddataSettings.getType());
this.fielddataSettings.getType(), this.fielddataSettings.getDurability());
registerBreaker(newFielddataSettings);
HierarchyCircuitBreakerService.this.fielddataSettings = newFielddataSettings;
logger.info("Updated breaker settings field data: {}", newFielddataSettings);
@ -177,20 +181,20 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
private void setAccountingBreakerLimit(ByteSizeValue newAccountingMax, Double newAccountingOverhead) {
BreakerSettings newAccountingSettings = new BreakerSettings(CircuitBreaker.ACCOUNTING, newAccountingMax.getBytes(),
newAccountingOverhead, HierarchyCircuitBreakerService.this.inFlightRequestsSettings.getType());
newAccountingOverhead, HierarchyCircuitBreakerService.this.accountingSettings.getType(), this.accountingSettings.getDurability());
registerBreaker(newAccountingSettings);
HierarchyCircuitBreakerService.this.accountingSettings = newAccountingSettings;
logger.info("Updated breaker settings for accounting requests: {}", newAccountingSettings);
}
private boolean validateTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT);
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT, null);
validateSettings(new BreakerSettings[]{newParentSettings});
return true;
}
private void setTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT);
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT, null);
this.parentSettings = newParentSettings;
}
@ -225,7 +229,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
}
// Manually add the parent breaker settings since they aren't part of the breaker map
allStats.add(new CircuitBreakerStats(CircuitBreaker.PARENT, parentSettings.getLimit(),
parentUsed(0L).totalUsage, 1.0, parentTripCount.get()));
memoryUsed(0L).totalUsage, 1.0, parentTripCount.get()));
return new AllCircuitBreakerStats(allStats.toArray(new CircuitBreakerStats[allStats.size()]));
}
@ -235,26 +239,38 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
return new CircuitBreakerStats(breaker.getName(), breaker.getLimit(), breaker.getUsed(), breaker.getOverhead(), breaker.getTrippedCount());
}
private static class ParentMemoryUsage {
private static class MemoryUsage {
final long baseUsage;
final long totalUsage;
final long transientChildUsage;
final long permanentChildUsage;
ParentMemoryUsage(final long baseUsage, final long totalUsage) {
MemoryUsage(final long baseUsage, final long totalUsage, final long transientChildUsage, final long permanentChildUsage) {
this.baseUsage = baseUsage;
this.totalUsage = totalUsage;
this.transientChildUsage = transientChildUsage;
this.permanentChildUsage = permanentChildUsage;
}
}
private ParentMemoryUsage parentUsed(long newBytesReserved) {
private MemoryUsage memoryUsed(long newBytesReserved) {
long transientUsage = 0;
long permanentUsage = 0;
for (CircuitBreaker breaker : this.breakers.values()) {
long breakerUsed = (long)(breaker.getUsed() * breaker.getOverhead());
if (breaker.getDurability() == CircuitBreaker.Durability.TRANSIENT) {
transientUsage += breakerUsed;
} else if (breaker.getDurability() == CircuitBreaker.Durability.PERMANENT) {
permanentUsage += breakerUsed;
}
}
if (this.trackRealMemoryUsage) {
final long current = currentMemoryUsage();
return new ParentMemoryUsage(current, current + newBytesReserved);
return new MemoryUsage(current, current + newBytesReserved, transientUsage, permanentUsage);
} else {
long parentEstimated = 0;
for (CircuitBreaker breaker : this.breakers.values()) {
parentEstimated += breaker.getUsed() * breaker.getOverhead();
}
return new ParentMemoryUsage(parentEstimated, parentEstimated);
long parentEstimated = transientUsage + permanentUsage;
return new MemoryUsage(parentEstimated, parentEstimated, transientUsage, permanentUsage);
}
}
@ -276,16 +292,16 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
* Checks whether the parent breaker has been tripped
*/
public void checkParentLimit(long newBytesReserved, String label) throws CircuitBreakingException {
final ParentMemoryUsage parentUsed = parentUsed(newBytesReserved);
final MemoryUsage memoryUsed = memoryUsed(newBytesReserved);
long parentLimit = this.parentSettings.getLimit();
if (parentUsed.totalUsage > parentLimit) {
if (memoryUsed.totalUsage > parentLimit) {
this.parentTripCount.incrementAndGet();
final StringBuilder message = new StringBuilder("[parent] Data too large, data for [" + label + "]" +
" would be [" + parentUsed.totalUsage + "/" + new ByteSizeValue(parentUsed.totalUsage) + "]" +
" would be [" + memoryUsed.totalUsage + "/" + new ByteSizeValue(memoryUsed.totalUsage) + "]" +
", which is larger than the limit of [" +
parentLimit + "/" + new ByteSizeValue(parentLimit) + "]");
if (this.trackRealMemoryUsage) {
final long realUsage = parentUsed.baseUsage;
final long realUsage = memoryUsed.baseUsage;
message.append(", real usage: [");
message.append(realUsage);
message.append("/");
@ -306,7 +322,11 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
.collect(Collectors.toList())));
message.append("]");
}
throw new CircuitBreakingException(message.toString(), parentUsed.totalUsage, parentLimit);
// derive durability of a tripped parent breaker depending on whether the majority of memory tracked by
// child circuit breakers is categorized as transient or permanent.
CircuitBreaker.Durability durability = memoryUsed.transientChildUsage >= memoryUsed.permanentChildUsage ?
CircuitBreaker.Durability.TRANSIENT : CircuitBreaker.Durability.PERMANENT;
throw new CircuitBreakingException(message.toString(), memoryUsed.totalUsage, parentLimit, durability);
}
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.ClusterStateApplier;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.cache.Cache;
import org.elasticsearch.common.cache.CacheBuilder;
@ -368,7 +369,8 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
// Otherwise reject the request
throw new CircuitBreakingException("[script] Too many dynamic script compilations within, max: [" +
rate.v1() + "/" + rate.v2() +"]; please use indexed, or scripts with parameters instead; " +
"this limit can be changed by the [" + SCRIPT_MAX_COMPILATIONS_RATE.getKey() + "] setting");
"this limit can be changed by the [" + SCRIPT_MAX_COMPILATIONS_RATE.getKey() + "] setting",
CircuitBreaker.Durability.TRANSIENT);
}
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
@ -735,11 +734,11 @@ public class ElasticsearchExceptionTests extends ESTestCase {
break;
case 1: // Simple elasticsearch exception with headers (other metadata of type number are not parsed)
failure = new CircuitBreakingException("B", 5_000, 2_000);
failure = new ParsingException(3, 2, "B", null);
((ElasticsearchException) failure).addHeader("header_name", "0", "1");
expected = new ElasticsearchException("Elasticsearch exception [type=circuit_breaking_exception, reason=B]");
expected = new ElasticsearchException("Elasticsearch exception [type=parsing_exception, reason=B]");
expected.addHeader("header_name", "0", "1");
suppressed = new ElasticsearchException("Elasticsearch exception [type=circuit_breaking_exception, reason=B]");
suppressed = new ElasticsearchException("Elasticsearch exception [type=parsing_exception, reason=B]");
suppressed.addHeader("header_name", "0", "1");
expected.addSuppressed(suppressed);
break;
@ -916,9 +915,9 @@ public class ElasticsearchExceptionTests extends ESTestCase {
expected = new ElasticsearchException("Elasticsearch exception [type=cluster_block_exception, " +
"reason=blocked by: [SERVICE_UNAVAILABLE/2/no master];]");
break;
case 1:
actual = new CircuitBreakingException("Data too large", 123, 456);
expected = new ElasticsearchException("Elasticsearch exception [type=circuit_breaking_exception, reason=Data too large]");
case 1: // Simple elasticsearch exception with headers (other metadata of type number are not parsed)
actual = new ParsingException(3, 2, "Unknown identifier", null);
expected = new ElasticsearchException("Elasticsearch exception [type=parsing_exception, reason=Unknown identifier]");
break;
case 2:
actual = new SearchParseException(new TestSearchContext(null), "Parse failure", new XContentLocation(12, 98));

View File

@ -40,6 +40,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.PathUtils;
@ -349,10 +350,12 @@ public class ExceptionSerializationTests extends ESTestCase {
}
public void testCircuitBreakingException() throws IOException {
CircuitBreakingException ex = serialize(new CircuitBreakingException("I hate to say I told you so...", 0, 100));
assertEquals("I hate to say I told you so...", ex.getMessage());
CircuitBreakingException ex = serialize(new CircuitBreakingException("Too large", 0, 100, CircuitBreaker.Durability.TRANSIENT),
Version.V_7_0_0_alpha1);
assertEquals("Too large", ex.getMessage());
assertEquals(100, ex.getByteLimit());
assertEquals(0, ex.getBytesWanted());
assertEquals(CircuitBreaker.Durability.TRANSIENT, ex.getDurability());
}
public void testTooManyBucketsException() throws IOException {

View File

@ -22,6 +22,7 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
@ -60,13 +61,14 @@ public class ReplicationResponseTests extends ESTestCase {
new ShardInfo.Failure(new ShardId("index", "_uuid", 3),
"_node_id", new IllegalArgumentException("Wrong"), RestStatus.BAD_REQUEST, false),
new ShardInfo.Failure(new ShardId("index", "_uuid", 1),
"_node_id", new CircuitBreakingException("Wrong", 12, 21), RestStatus.NOT_ACCEPTABLE, true));
"_node_id", new CircuitBreakingException("Wrong", 12, 21, CircuitBreaker.Durability.PERMANENT),
RestStatus.NOT_ACCEPTABLE, true));
String output = Strings.toString(shardInfo);
assertEquals("{\"total\":6,\"successful\":4,\"failed\":2,\"failures\":[{\"_index\":\"index\",\"_shard\":3," +
"\"_node\":\"_node_id\",\"reason\":{\"type\":\"illegal_argument_exception\",\"reason\":\"Wrong\"}," +
"\"status\":\"BAD_REQUEST\",\"primary\":false},{\"_index\":\"index\",\"_shard\":1,\"_node\":\"_node_id\"," +
"\"reason\":{\"type\":\"circuit_breaking_exception\",\"reason\":\"Wrong\",\"bytes_wanted\":12,\"bytes_limit\":21}," +
"\"status\":\"NOT_ACCEPTABLE\",\"primary\":true}]}", output);
"\"reason\":{\"type\":\"circuit_breaking_exception\",\"reason\":\"Wrong\",\"bytes_wanted\":12,\"bytes_limit\":21" +
",\"durability\":\"PERMANENT\"},\"status\":\"NOT_ACCEPTABLE\",\"primary\":true}]}", output);
}
}

View File

@ -117,10 +117,12 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase {
@Override
public void checkParentLimit(long newBytesReserved, String label) throws CircuitBreakingException {
// Parent will trip right before regular breaker would trip
if (getBreaker(CircuitBreaker.REQUEST).getUsed() > parentLimit) {
long requestBreakerUsed = getBreaker(CircuitBreaker.REQUEST).getUsed();
if (requestBreakerUsed > parentLimit) {
parentTripped.incrementAndGet();
logger.info("--> parent tripped");
throw new CircuitBreakingException("parent tripped");
throw new CircuitBreakingException("parent tripped", requestBreakerUsed + newBytesReserved,
parentLimit, CircuitBreaker.Durability.PERMANENT);
}
}
};
@ -201,6 +203,7 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase {
assertThat(exception.getMessage(), containsString("which is larger than the limit of [209715200/200mb]"));
assertThat(exception.getMessage(),
containsString("usages [request=157286400/150mb, fielddata=54001664/51.5mb, in_flight_requests=0/0b, accounting=0/0b]"));
assertThat(exception.getDurability(), equalTo(CircuitBreaker.Durability.TRANSIENT));
}
}
@ -244,6 +247,7 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase {
assertThat(exception.getMessage(),
containsString("real usage: [181/181b], new bytes reserved: [" + (reservationInBytes * 2) +
"/" + new ByteSizeValue(reservationInBytes * 2) + "]"));
assertThat(exception.getDurability(), equalTo(CircuitBreaker.Durability.TRANSIENT));
assertEquals(0, requestBreaker.getTrippedCount());
assertEquals(1, service.stats().getStats(CircuitBreaker.PARENT).getTrippedCount());
@ -252,4 +256,41 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase {
requestBreaker.addEstimateBytesAndMaybeBreak(reservationInBytes, "request");
assertEquals(0, requestBreaker.getTrippedCount());
}
public void testTrippedCircuitBreakerDurability() {
Settings clusterSettings = Settings.builder()
.put(HierarchyCircuitBreakerService.USE_REAL_MEMORY_USAGE_SETTING.getKey(), Boolean.FALSE)
.put(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), "200mb")
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), "150mb")
.put(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), "150mb")
.build();
try (CircuitBreakerService service = new HierarchyCircuitBreakerService(clusterSettings,
new ClusterSettings(clusterSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))) {
CircuitBreaker requestCircuitBreaker = service.getBreaker(MemoryCircuitBreaker.REQUEST);
CircuitBreaker fieldDataCircuitBreaker = service.getBreaker(MemoryCircuitBreaker.FIELDDATA);
CircuitBreaker.Durability expectedDurability;
if (randomBoolean()) {
fieldDataCircuitBreaker.addEstimateBytesAndMaybeBreak(mb(100), "should not break");
requestCircuitBreaker.addEstimateBytesAndMaybeBreak(mb(70), "should not break");
expectedDurability = CircuitBreaker.Durability.PERMANENT;
} else {
fieldDataCircuitBreaker.addEstimateBytesAndMaybeBreak(mb(70), "should not break");
requestCircuitBreaker.addEstimateBytesAndMaybeBreak(mb(120), "should not break");
expectedDurability = CircuitBreaker.Durability.TRANSIENT;
}
CircuitBreakingException exception = expectThrows(CircuitBreakingException.class, () ->
fieldDataCircuitBreaker.addEstimateBytesAndMaybeBreak(mb(40), "should break"));
assertThat(exception.getMessage(), containsString("[parent] Data too large, data for [should break] would be"));
assertThat(exception.getMessage(), containsString("which is larger than the limit of [209715200/200mb]"));
assertThat("Expected [" + expectedDurability + "] due to [" + exception.getMessage() + "]",
exception.getDurability(), equalTo(expectedDurability));
}
}
private long mb(long size) {
return new ByteSizeValue(size, ByteSizeUnit.MB).getBytes();
}
}

View File

@ -277,7 +277,7 @@ public class RestControllerTests extends ESTestCase {
int contentLength = BREAKER_LIMIT.bytesAsInt() + 1;
String content = randomAlphaOfLength((int) Math.round(contentLength / inFlightRequestsBreaker.getOverhead()));
RestRequest request = testRestRequest("/", content, XContentType.JSON);
AssertingChannel channel = new AssertingChannel(request, true, RestStatus.SERVICE_UNAVAILABLE);
AssertingChannel channel = new AssertingChannel(request, true, RestStatus.TOO_MANY_REQUESTS);
restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY));