Skip to content

Commit 11af873

Browse files
Cardinality request breaker leak (#62685) (#62718)
If HyperLogLogPlusPlus failed during construction, it would not release already allocated resources, causing the request circuit breaker to not be adjusted down. Closes #62439
1 parent e7fe325 commit 11af873

File tree

2 files changed

+51
-1
lines changed

2 files changed

+51
-1
lines changed

server/src/main/java/org/elasticsearch/search/aggregations/metrics/HyperLogLogPlusPlus.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -784,7 +784,12 @@ public HyperLogLogPlusPlus(int precision, BigArrays bigArrays, long initialBucke
784784
this.bigArrays = bigArrays;
785785
algorithm = new OpenBitSet();
786786
runLens = bigArrays.newByteArray(initialBucketCount << p);
787-
hashSet = new Hashset(initialBucketCount);
787+
try {
788+
hashSet = new Hashset(initialBucketCount);
789+
} catch (RuntimeException e) {
790+
runLens.close();
791+
throw e;
792+
}
788793
final double alpha;
789794
switch (p) {
790795
case 4:

server/src/test/java/org/elasticsearch/search/aggregations/metrics/HyperLogLogPlusPlusTests.java

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,13 +21,22 @@
2121

2222
import com.carrotsearch.hppc.BitMixer;
2323
import com.carrotsearch.hppc.IntHashSet;
24+
import org.elasticsearch.common.breaker.CircuitBreaker;
25+
import org.elasticsearch.common.breaker.CircuitBreakingException;
26+
import org.elasticsearch.common.breaker.NoopCircuitBreaker;
2427
import org.elasticsearch.common.util.BigArrays;
2528
import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus;
29+
import org.elasticsearch.indices.breaker.CircuitBreakerService;
2630
import org.elasticsearch.test.ESTestCase;
2731

32+
import java.util.concurrent.atomic.AtomicLong;
33+
2834
import static org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus.MAX_PRECISION;
2935
import static org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus.MIN_PRECISION;
3036
import static org.hamcrest.Matchers.closeTo;
37+
import static org.hamcrest.Matchers.equalTo;
38+
import static org.mockito.Mockito.mock;
39+
import static org.mockito.Mockito.when;
3140

3241
public class HyperLogLogPlusPlusTests extends ESTestCase {
3342
public void testEncodeDecode() {
@@ -127,4 +136,40 @@ public void testPrecisionFromThreshold() {
127136
assertEquals(18, HyperLogLogPlusPlus.precisionFromThreshold(100000));
128137
assertEquals(18, HyperLogLogPlusPlus.precisionFromThreshold(1000000));
129138
}
139+
140+
public void testCircuitBreakerOnConstruction() {
141+
int whenToBreak = randomInt(10);
142+
AtomicLong total = new AtomicLong();
143+
CircuitBreakerService breakerService = mock(CircuitBreakerService.class);
144+
when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(new NoopCircuitBreaker(CircuitBreaker.REQUEST) {
145+
private int countDown = whenToBreak;
146+
@Override
147+
public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException {
148+
if (countDown-- == 0) {
149+
throw new CircuitBreakingException("test error", bytes, Long.MAX_VALUE, Durability.TRANSIENT);
150+
}
151+
total.addAndGet(bytes);
152+
return total.get();
153+
}
154+
155+
@Override
156+
public long addWithoutBreaking(long bytes) {
157+
total.addAndGet(bytes);
158+
return total.get();
159+
}
160+
});
161+
BigArrays bigArrays = new BigArrays(null, breakerService, CircuitBreaker.REQUEST).withCircuitBreaking();
162+
final int p = randomIntBetween(HyperLogLogPlusPlus.MIN_PRECISION, HyperLogLogPlusPlus.MAX_PRECISION);
163+
try {
164+
for (int i = 0; i < whenToBreak + 1; ++i) {
165+
final HyperLogLogPlusPlus subject = new HyperLogLogPlusPlus(p, bigArrays, 0);
166+
subject.close();
167+
}
168+
fail("Must fail");
169+
} catch (CircuitBreakingException e) {
170+
// OK
171+
}
172+
173+
assertThat(total.get(), equalTo(0L));
174+
}
130175
}

0 commit comments

Comments
 (0)