bookkeeper
bookkeeper copied to clipboard
CleanupLedgerManager' futurers leak when zk abnormallity
BUG REPORT
Describe the bug
pulsar broker can not read ledgers when zk abnormallity. after some time running, threr are some error log like:
2022-05-17 09:16:45,869 bookkeeper-ml-scheduler-OrderedScheduler-5-0 WARN org.apache.logging.slf4j.Log4jLogger caught java.lang.OutOfMemoryError logging ReusableSimpleMessage: Unexpected throwable caught java.lang.OutOfMemoryError: Java heap space
And a few hours ago, there are some error logs in pulsar broker:
2022-05-17 06:32:25,870Z [main-EventThread] ERROR org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManager - Could not read metadata for ledger: 840047: java.util.NoSuchElementException: No value present
2022-05-17 06:32:25,870Z [BookKeeperClientWorker-OrderedExecutor-1-0] WARN org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/biz%3Arouting_rule%3Adata%3Af676c1bd-33a4-4702-aece-86548001bda5-partition-1] Ledger not found: 840047
this is pulsar broker memory head dump
num #instances #bytes class name (module)
-------------------------------------------------------
1: 1335679 361693392 [J ([email protected])
2: 2732947 258403480 [Ljava.lang.Object; ([email protected])
3: 1339824 193053928 [I ([email protected])
4: 1320603 105648112 [S ([email protected])
5: 1318302 52732080 org.apache.bookkeeper.client.BKException$BKMetadataVersionException
6: 1380358 44171456 java.util.concurrent.ConcurrentHashMap$Node ([email protected])
7: 224134 37567960 [B ([email protected])
8: 1405956 33742944 java.util.concurrent.CompletableFuture ([email protected])
9: 1320792 21132672 java.util.concurrent.CompletableFuture$AltResult ([email protected])
10: 1768 9055616 [Ljava.util.concurrent.ConcurrentHashMap$Node; ([email protected])
11: 1216 6009448 [C ([email protected])
12: 183789 5881248 java.util.concurrent.atomic.LongAdder ([email protected])
13: 5239 5415696 [D ([email protected])
14: 200293 4807032 java.lang.String ([email protected])
15: 89059 3562360 java.util.LinkedHashMap$Entry ([email protected])
16: 5040 3386880 io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueue
17: 36483 3016152 [Ljava.util.HashMap$Node; ([email protected])
18: 3151 2803568 [Ljava.lang.String; ([email protected])
19: 8689 2324392 [Z ([email protected])
20: 57111 2284440 org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$ZooWorker
21: 21583 1899304 java.lang.reflect.Method ([email protected])
22: 28986 1855104 org.apache.zookeeper.ClientCnxn$Packet
23: 14922 1770896 java.lang.Class ([email protected])
24: 51385 1644320 java.util.HashMap$Node ([email protected])
25: 67235 1613640 io.netty.util.Recycler$DefaultHandle
26: 31083 1491984 java.util.HashMap ([email protected])
27: 22234 1422976 java.util.LinkedHashMap ([email protected])
28: 28845 1384560 org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$15
29: 56646 1359504 java.util.ArrayList ([email protected])
30: 28125 1350000 org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$16
Additional context version: 4.14.4
is something went wrong in this method ? futurers add promise but didn't remove. org.apache.bookkeeper.meta.CleanupLedgerManager#recordPromise
private void recordPromise(CompletableFuture<?> promise) {
futures.add(promise);
promise.thenRun(() -> futures.remove(promise));
}