Skip to content

Commit

Permalink
Cardinality request breaker leak (#62685) (#62718)
Browse files Browse the repository at this point in the history
If HyperLogLogPlusPlus failed during construction, it would
not release already allocated resources, causing the request
circuit breaker to not be adjusted down.

Closes #62439
  • Loading branch information
henningandersen authored Sep 21, 2020
1 parent e7fe325 commit 11af873
Show file tree
Hide file tree
Showing 2 changed files with 51 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -784,7 +784,12 @@ public HyperLogLogPlusPlus(int precision, BigArrays bigArrays, long initialBucke
this.bigArrays = bigArrays;
algorithm = new OpenBitSet();
runLens = bigArrays.newByteArray(initialBucketCount << p);
hashSet = new Hashset(initialBucketCount);
try {
hashSet = new Hashset(initialBucketCount);
} catch (RuntimeException e) {
runLens.close();
throw e;
}
final double alpha;
switch (p) {
case 4:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,22 @@

import com.carrotsearch.hppc.BitMixer;
import com.carrotsearch.hppc.IntHashSet;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.test.ESTestCase;

import java.util.concurrent.atomic.AtomicLong;

import static org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus.MAX_PRECISION;
import static org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus.MIN_PRECISION;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

public class HyperLogLogPlusPlusTests extends ESTestCase {
public void testEncodeDecode() {
Expand Down Expand Up @@ -127,4 +136,40 @@ public void testPrecisionFromThreshold() {
assertEquals(18, HyperLogLogPlusPlus.precisionFromThreshold(100000));
assertEquals(18, HyperLogLogPlusPlus.precisionFromThreshold(1000000));
}

public void testCircuitBreakerOnConstruction() {
int whenToBreak = randomInt(10);
AtomicLong total = new AtomicLong();
CircuitBreakerService breakerService = mock(CircuitBreakerService.class);
when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(new NoopCircuitBreaker(CircuitBreaker.REQUEST) {
private int countDown = whenToBreak;
@Override
public double 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) {
total.addAndGet(bytes);
return total.get();
}
});
BigArrays bigArrays = new BigArrays(null, breakerService, CircuitBreaker.REQUEST).withCircuitBreaking();
final int p = randomIntBetween(HyperLogLogPlusPlus.MIN_PRECISION, HyperLogLogPlusPlus.MAX_PRECISION);
try {
for (int i = 0; i < whenToBreak + 1; ++i) {
final HyperLogLogPlusPlus subject = new HyperLogLogPlusPlus(p, bigArrays, 0);
subject.close();
}
fail("Must fail");
} catch (CircuitBreakingException e) {
// OK
}

assertThat(total.get(), equalTo(0L));
}
}

0 comments on commit 11af873

Please sign in to comment.