Skip to content

Commit 02739d0

Browse files
committed
Mute accounting circuit breaker check after test (#42448)
If we close an engine while a refresh is happening, then we might leak refCount of some SegmentReaders. We need to skip the ram accounting circuit breaker check until we have a new Lucene snapshot which includes the fix for LUCENE-8809. This also adds a test to the engine but left it muted so we won't forget to reenable this check. Closes #30290
1 parent 48dc0dc commit 02739d0

File tree

3 files changed

+57
-9
lines changed

3 files changed

+57
-9
lines changed

server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,7 @@
8080
import org.elasticsearch.common.Strings;
8181
import org.elasticsearch.common.TriFunction;
8282
import org.elasticsearch.common.UUIDs;
83+
import org.elasticsearch.common.breaker.CircuitBreaker;
8384
import org.elasticsearch.common.bytes.BytesArray;
8485
import org.elasticsearch.common.bytes.BytesReference;
8586
import org.elasticsearch.common.collect.Tuple;
@@ -153,6 +154,7 @@
153154
import java.util.concurrent.BrokenBarrierException;
154155
import java.util.concurrent.CountDownLatch;
155156
import java.util.concurrent.CyclicBarrier;
157+
import java.util.concurrent.Semaphore;
156158
import java.util.concurrent.TimeUnit;
157159
import java.util.concurrent.atomic.AtomicBoolean;
158160
import java.util.concurrent.atomic.AtomicInteger;
@@ -5635,4 +5637,44 @@ public void testMaxSeqNoInCommitUserData() throws Exception {
56355637
rollTranslog.join();
56365638
assertMaxSeqNoInCommitUserData(engine);
56375639
}
5640+
5641+
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-8809")
5642+
public void testRefreshAndFailEngineConcurrently() throws Exception {
5643+
AtomicBoolean stopped = new AtomicBoolean();
5644+
Semaphore indexedDocs = new Semaphore(0);
5645+
Thread indexer = new Thread(() -> {
5646+
while (stopped.get() == false) {
5647+
String id = Integer.toString(randomIntBetween(1, 100));
5648+
try {
5649+
engine.index(indexForDoc(createParsedDoc(id, null)));
5650+
indexedDocs.release();
5651+
} catch (IOException e) {
5652+
throw new AssertionError(e);
5653+
} catch (AlreadyClosedException e) {
5654+
return;
5655+
}
5656+
}
5657+
});
5658+
5659+
Thread refresher = new Thread(() -> {
5660+
while (stopped.get() == false) {
5661+
try {
5662+
engine.refresh("test", randomFrom(Engine.SearcherScope.values()), randomBoolean());
5663+
} catch (AlreadyClosedException e) {
5664+
return;
5665+
}
5666+
}
5667+
});
5668+
indexer.start();
5669+
refresher.start();
5670+
indexedDocs.acquire(randomIntBetween(1, 100));
5671+
try {
5672+
engine.failEngine("test", new IOException("simulated error"));
5673+
} finally {
5674+
stopped.set(true);
5675+
indexer.join();
5676+
refresher.join();
5677+
}
5678+
assertThat(engine.config().getCircuitBreakerService().getBreaker(CircuitBreaker.ACCOUNTING).getUsed(), equalTo(0L));
5679+
}
56385680
}

test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@
5555
import org.elasticsearch.common.Nullable;
5656
import org.elasticsearch.common.Randomness;
5757
import org.elasticsearch.common.Strings;
58+
import org.elasticsearch.common.breaker.CircuitBreaker;
5859
import org.elasticsearch.common.bytes.BytesArray;
5960
import org.elasticsearch.common.bytes.BytesReference;
6061
import org.elasticsearch.common.compress.CompressedXContent;
@@ -269,6 +270,8 @@ public void tearDown() throws Exception {
269270
assertConsistentHistoryBetweenTranslogAndLuceneIndex(replicaEngine, createMapperService("test"));
270271
assertMaxSeqNoInCommitUserData(replicaEngine);
271272
}
273+
assertThat(engine.config().getCircuitBreakerService().getBreaker(CircuitBreaker.ACCOUNTING).getUsed(), equalTo(0L));
274+
assertThat(replicaEngine.config().getCircuitBreakerService().getBreaker(CircuitBreaker.ACCOUNTING).getUsed(), equalTo(0L));
272275
} finally {
273276
IOUtils.close(replicaEngine, storeReplica, engine, store, () -> terminate(threadPool));
274277
}

test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

Lines changed: 12 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -2426,15 +2426,18 @@ public void ensureEstimatedStats() {
24262426
final CircuitBreakerService breakerService = getInstanceFromNode(CircuitBreakerService.class, nodeAndClient.node);
24272427
CircuitBreaker fdBreaker = breakerService.getBreaker(CircuitBreaker.FIELDDATA);
24282428
assertThat("Fielddata breaker not reset to 0 on node: " + name, fdBreaker.getUsed(), equalTo(0L));
2429-
try {
2430-
assertBusy(() -> {
2431-
CircuitBreaker acctBreaker = breakerService.getBreaker(CircuitBreaker.ACCOUNTING);
2432-
assertThat("Accounting breaker not reset to 0 on node: " + name + ", are there still Lucene indices around?",
2433-
acctBreaker.getUsed(), equalTo(0L));
2434-
});
2435-
} catch (Exception e) {
2436-
throw new AssertionError("Exception during check for accounting breaker reset to 0", e);
2437-
}
2429+
2430+
// Mute this assertion until we have a new Lucene snapshot with https://issues.apache.org/jira/browse/LUCENE-8809.
2431+
// try {
2432+
// assertBusy(() -> {
2433+
// CircuitBreaker acctBreaker = breakerService.getBreaker(CircuitBreaker.ACCOUNTING);
2434+
// assertThat("Accounting breaker not reset to 0 on node: " + name + ", are there still Lucene indices around?",
2435+
// acctBreaker.getUsed(), equalTo(0L));
2436+
// });
2437+
// } catch (Exception e) {
2438+
// throw new AssertionError("Exception during check for accounting breaker reset to 0", e);
2439+
// }
2440+
24382441
// Anything that uses transport or HTTP can increase the
24392442
// request breaker (because they use bigarrays), because of
24402443
// that the breaker can sometimes be incremented from ping

0 commit comments

Comments
 (0)