Skip to content

Commit

Permalink
Remove return from breakers (backport f #66943) (#67028)
Browse files Browse the repository at this point in the history
This removes the return values from `CircuitBreaker`'s
`addEstimateBytesAndMaybeBreak` and `addWithoutBreaking` because they
are not supported by the new pre-allocating `CircuitBreaker` and they
aren't used for anything other than tests.

Co-authored-by: Elastic Machine <elasticmachine@users.noreply.github.com>
  • Loading branch information
nik9000 and elasticmachine committed Jan 11, 2021
1 parent 66564c5 commit 6dff748
Show file tree
Hide file tree
Showing 12 changed files with 40 additions and 61 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -82,10 +82,9 @@ public void circuitBreak(String fieldName, long bytesNeeded) {
* memory limit is set to 0. Will never trip the breaker if the limit is
* set &lt; 0, but can still be used to aggregate estimations.
* @param bytes number of bytes to add to the breaker
* @return number of "used" bytes so far
*/
@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
final LimitAndOverhead limitAndOverhead = this.limitAndOverhead;
final long memoryBytesLimit = limitAndOverhead.limit;
final double overheadConstant = limitAndOverhead.overhead;
Expand Down Expand Up @@ -115,7 +114,6 @@ public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws Cir
throw e;
}
assert newUsed >= 0 : "Used bytes: [" + newUsed + "] must be >= 0";
return newUsed;
}

private long noLimit(long bytes, String label) {
Expand Down Expand Up @@ -163,14 +161,12 @@ memoryBytesLimit, new ByteSizeValue(memoryBytesLimit),
* has been exceeded.
*
* @param bytes number of bytes to add to the breaker
* @return number of "used" bytes so far
*/
@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
long u = used.addAndGet(bytes);
logger.trace(() -> new ParameterizedMessage("[{}] Adjusted breaker by [{}] bytes, now [{}]", this.name, bytes, u));
assert u >= 0 : "Used bytes: [" + u + "] must be >= 0";
return u;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,17 +98,17 @@ enum Durability {
void circuitBreak(String fieldName, long bytesNeeded);

/**
* add bytes to the breaker and maybe trip
* Add bytes to the breaker and trip if the that puts breaker over the limit.
* @param bytes number of bytes to add
* @param label string label describing the bytes being added
* @return the number of "used" bytes for the circuit breaker
* @param label thing requesting the bytes being added that is included in
* the exception if the breaker is tripped
*/
double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException;
void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException;

/**
* Adjust the circuit breaker without tripping
* Add bytes to the circuit breaker without tripping.
*/
long addWithoutBreaking(long bytes);
void addWithoutBreaking(long bytes);

/**
* @return the currently used bytes the breaker is tracking
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,13 +38,11 @@ public void circuitBreak(String fieldName, long bytesNeeded) {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
return 0;
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
}

@Override
public long addWithoutBreaking(long bytes) {
return 0;
public void addWithoutBreaking(long bytes) {
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,45 +115,45 @@ public void circuitBreak(String fieldName, long bytesNeeded) {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
if (closed) {
throw new IllegalStateException("already closed");
}
if (preallocationUsed == preallocated) {
// Preallocation buffer was full before this request
return next.addEstimateBytesAndMaybeBreak(bytes, label);
next.addEstimateBytesAndMaybeBreak(bytes, label);
return;
}
long newUsed = preallocationUsed + bytes;
if (newUsed > preallocated) {
// This request filled up the buffer
preallocationUsed = preallocated;
return next.addEstimateBytesAndMaybeBreak(newUsed - preallocated, label);
next.addEstimateBytesAndMaybeBreak(newUsed - preallocated, label);
return;
}
// This is the fast case. No volatile reads or writes here, ma!
preallocationUsed = newUsed;
// We return garbage here but callers never use the result for anything interesting
return 0;
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
if (closed) {
throw new IllegalStateException("already closed");
}
if (preallocationUsed == preallocated) {
// Preallocation buffer was full before this request
return next.addWithoutBreaking(bytes);
next.addWithoutBreaking(bytes);
return;
}
long newUsed = preallocationUsed + bytes;
if (newUsed > preallocated) {
// This request filled up the buffer
preallocationUsed = preallocated;
return next.addWithoutBreaking(newUsed - preallocated);
next.addWithoutBreaking(newUsed - preallocated);
return;
}
// This is the fast case. No volatile reads or writes here, ma!
preallocationUsed = newUsed;
// We return garbage here but callers never use the result for anything interesting
return 0;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1045,19 +1045,17 @@ private static class AssertingCircuitBreaker extends NoopCircuitBreaker {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
assert bytes >= 0;
if (shouldBreak.get()) {
throw new CircuitBreakingException(label, getDurability());
}
allocated += bytes;
return allocated;
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
allocated += bytes;
return allocated;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -212,15 +212,12 @@ public void testBorrowingSiblingBreakerMemory() {
assertEquals(new ByteSizeValue(150, ByteSizeUnit.MB).getBytes(), requestCircuitBreaker.getLimit());
assertEquals(new ByteSizeValue(150, ByteSizeUnit.MB).getBytes(), fieldDataCircuitBreaker.getLimit());

double fieldDataUsedBytes = fieldDataCircuitBreaker
.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "should not break");
assertEquals(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), fieldDataUsedBytes, 0.0);
double requestUsedBytes = requestCircuitBreaker.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(),
"should not break");
assertEquals(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), requestUsedBytes, 0.0);
requestUsedBytes = requestCircuitBreaker.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(),
"should not break");
assertEquals(new ByteSizeValue(100, ByteSizeUnit.MB).getBytes(), requestUsedBytes, 0.0);
fieldDataCircuitBreaker.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "should not break");
assertEquals(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), fieldDataCircuitBreaker.getUsed(), 0.0);
requestCircuitBreaker.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "should not break");
assertEquals(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), requestCircuitBreaker.getUsed(), 0.0);
requestCircuitBreaker.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "should not break");
assertEquals(new ByteSizeValue(100, ByteSizeUnit.MB).getBytes(), requestCircuitBreaker.getUsed(), 0.0);
CircuitBreakingException exception = expectThrows(CircuitBreakingException.class, () -> requestCircuitBreaker
.addEstimateBytesAndMaybeBreak(new ByteSizeValue(50, ByteSizeUnit.MB).getBytes(), "should break"));
assertThat(exception.getMessage(), containsString("[parent] Data too large, data for [should break] would be"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,18 +94,16 @@ public void testCircuitBreakerOnConstruction() {
when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(new NoopCircuitBreaker(CircuitBreaker.REQUEST) {
private int countDown = whenToBreak;
@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
if (countDown-- == 0) {
throw new CircuitBreakingException("test error", bytes, Long.MAX_VALUE, Durability.TRANSIENT);
}
total.addAndGet(bytes);
return total.get();
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
total.addAndGet(bytes);
return total.get();
}
});
BigArrays bigArrays = new BigArrays(null, breakerService, CircuitBreaker.REQUEST).withCircuitBreaking();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,18 +147,16 @@ public void testCircuitBreakerOnConstruction() {
when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(new NoopCircuitBreaker(CircuitBreaker.REQUEST) {
private int countDown = whenToBreak;
@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
if (countDown-- == 0) {
throw new CircuitBreakingException("test error", bytes, Long.MAX_VALUE, Durability.TRANSIENT);
}
total.addAndGet(bytes);
return total.get();
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
total.addAndGet(bytes);
return total.get();
}
});
BigArrays bigArrays = new BigArrays(null, breakerService, CircuitBreaker.REQUEST).withCircuitBreaking();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -210,17 +210,15 @@ public void mockBreaker() {
private long total = 0;

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
logger.debug("Used {} grabbing {} for {}", total, bytes, label);
total += bytes;
return total;
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
logger.debug("Used {} grabbing {}", total, bytes);
total += bytes;
return total;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,10 @@ public TestCircuitBreaker() {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
if (shouldBreak.get()) {
throw new CircuitBreakingException("broken", getDurability());
}
return 0;
}

public void startBreaking() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -638,17 +638,16 @@ public LimitedBreaker(String name, ByteSizeValue max) {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
long total = used.addAndGet(bytes);
if (total > max.getBytes()) {
throw new CircuitBreakingException("test error", bytes, max.getBytes(), Durability.TRANSIENT);
}
return total;
}

@Override
public long addWithoutBreaking(long bytes) {
return used.addAndGet(bytes);
public void addWithoutBreaking(long bytes) {
used.addAndGet(bytes);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -654,22 +654,20 @@ public void circuitBreak(String fieldName, long bytesNeeded) {
}

@Override
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
synchronized (this) {
if (bytes + currentBytes >= maxBytes) {
trippedCount++;
circuitBreak(label, bytes);
}
currentBytes += bytes;
return currentBytes;
}
}

@Override
public long addWithoutBreaking(long bytes) {
public void addWithoutBreaking(long bytes) {
synchronized (this) {
currentBytes += bytes;
return currentBytes;
}
}

Expand Down

0 comments on commit 6dff748

Please sign in to comment.