diff --git a/server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java b/server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java index 9ea8a3df29492..291c3f56d45bb 100644 --- a/server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java +++ b/server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java @@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** * CircuitBreakerService that attempts to redistribute space between breakers @@ -201,7 +202,7 @@ public AllCircuitBreakerStats stats() { } // Manually add the parent breaker settings since they aren't part of the breaker map allStats.add(new CircuitBreakerStats(CircuitBreaker.PARENT, parentSettings.getLimit(), - parentEstimated, 1.0, parentTripCount.get())); + parentEstimated, 1.0, parentTripCount.get())); return new AllCircuitBreakerStats(allStats.toArray(new CircuitBreakerStats[allStats.size()])); } @@ -223,11 +224,20 @@ public void checkParentLimit(String label) throws CircuitBreakingException { long parentLimit = this.parentSettings.getLimit(); if (totalUsed > parentLimit) { this.parentTripCount.incrementAndGet(); - final String message = "[parent] Data too large, data for [" + label + "]" + + final StringBuilder message = new StringBuilder("[parent] Data too large, data for [" + label + "]" + " would be [" + totalUsed + "/" + new ByteSizeValue(totalUsed) + "]" + ", which is larger than the limit of [" + - parentLimit + "/" + new ByteSizeValue(parentLimit) + "]"; - throw new CircuitBreakingException(message, totalUsed, parentLimit); + parentLimit + "/" + new ByteSizeValue(parentLimit) + "]"); + message.append(", usages ["); + message.append(String.join(", ", + this.breakers.entrySet().stream().map(e -> { + final CircuitBreaker breaker = e.getValue(); + final long breakerUsed = (long)(breaker.getUsed() * breaker.getOverhead()); + return e.getKey() + "=" + breakerUsed + "/" + new ByteSizeValue(breakerUsed); + }) + .collect(Collectors.toList()))); + message.append("]"); + throw new CircuitBreakingException(message.toString(), totalUsed, parentLimit); } } diff --git a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java b/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java index a03739b2d9a94..0ea96f44234a5 100644 --- a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java @@ -197,6 +197,8 @@ public void testBorrowingSiblingBreakerMemory() throws Exception { .addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "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(exception.getMessage(), + containsString("usages [request=157286400/150mb, fielddata=54001664/51.5mb, in_flight_requests=0/0b, accounting=0/0b]")); } } }